From 98850ab5cc3863cfb724b28b2f3919c97ac4bdda Mon Sep 17 00:00:00 2001 From: mgorsk1 Date: Tue, 12 Mar 2024 08:39:25 +0100 Subject: [PATCH] feat: OpenLineage integration (#15317) * :tada: Init OpenLineage connector Co-authored-by: dechoma * MLH - make linter happy * review fixes * :bug: Fix path for ol event in tests * :bug: Fix path for ol event in tests * Update ingestion/setup.py Co-authored-by: Mayur Singal <39544459+ulixius9@users.noreply.github.com> * Update ingestion/src/metadata/ingestion/source/pipeline/openlineage/metadata.py Co-authored-by: Mayur Singal <39544459+ulixius9@users.noreply.github.com> * Update ingestion/src/metadata/ingestion/source/pipeline/openlineage/models.py Co-authored-by: Mayur Singal <39544459+ulixius9@users.noreply.github.com> * review fixes 2 * linter * review * review * make linter happy * fix test_yield_pipeline_lineage_details test * make linter happy * fix tests * fix tests 2 --------- Co-authored-by: dechoma Co-authored-by: Mayur Singal <39544459+ulixius9@users.noreply.github.com> --- ingestion/setup.py | 1 + .../source/pipeline/openlineage/__init__.py | 0 .../source/pipeline/openlineage/connection.py | 88 +++ .../source/pipeline/openlineage/metadata.py | 520 +++++++++++++++++ .../source/pipeline/openlineage/models.py | 88 +++ .../source/pipeline/openlineage/utils.py | 59 ++ ingestion/src/metadata/utils/fqn.py | 68 ++- .../resources/datasets/openlineage_event.json | 330 +++++++++++ .../topology/pipeline/test_openlineage.py | 533 ++++++++++++++++++ .../connectors/pipeline/openlineage/index.md | 151 +++++ .../openmetadata/add-new-service.png | Bin 0 -> 37523 bytes .../openmetadata/select-service.png | Bin 0 -> 65102 bytes .../openmetadata/service-connection.png | Bin 0 -> 41087 bytes .../testConnections/pipeline/openlineage.json | 14 + .../pipeline/openLineageConnection.json | 99 ++++ .../entity/services/pipelineService.json | 9 +- .../json/schema/type/entityLineage.json | 2 +- .../locales/en-US/Pipeline/OpenLineage.md | 87 +++ .../assets/img/service-icon-openlineage.svg | 17 + .../ui/src/constants/Lineage.constants.ts | 1 + .../ui/src/constants/Services.constant.ts | 3 + .../ui/src/utils/PipelineServiceUtils.ts | 6 + .../ui/src/utils/ServiceUtilClassBase.ts | 4 + 23 files changed, 2071 insertions(+), 9 deletions(-) create mode 100644 ingestion/src/metadata/ingestion/source/pipeline/openlineage/__init__.py create mode 100644 ingestion/src/metadata/ingestion/source/pipeline/openlineage/connection.py create mode 100644 ingestion/src/metadata/ingestion/source/pipeline/openlineage/metadata.py create mode 100644 ingestion/src/metadata/ingestion/source/pipeline/openlineage/models.py create mode 100644 ingestion/src/metadata/ingestion/source/pipeline/openlineage/utils.py create mode 100644 ingestion/tests/unit/resources/datasets/openlineage_event.json create mode 100644 ingestion/tests/unit/topology/pipeline/test_openlineage.py create mode 100644 openmetadata-docs/content/v1.3.x/connectors/pipeline/openlineage/index.md create mode 100644 openmetadata-docs/images/v1.3/connectors/openmetadata/add-new-service.png create mode 100644 openmetadata-docs/images/v1.3/connectors/openmetadata/select-service.png create mode 100644 openmetadata-docs/images/v1.3/connectors/openmetadata/service-connection.png create mode 100644 openmetadata-service/src/main/resources/json/data/testConnections/pipeline/openlineage.json create mode 100644 openmetadata-spec/src/main/resources/json/schema/entity/services/connections/pipeline/openLineageConnection.json create mode 100644 openmetadata-ui/src/main/resources/ui/public/locales/en-US/Pipeline/OpenLineage.md create mode 100644 openmetadata-ui/src/main/resources/ui/src/assets/img/service-icon-openlineage.svg diff --git a/ingestion/setup.py b/ingestion/setup.py index 887ee1a1508..91ea70c8b7b 100644 --- a/ingestion/setup.py +++ b/ingestion/setup.py @@ -234,6 +234,7 @@ plugins: Dict[str, Set[str]] = { "mysql": {VERSIONS["pymysql"]}, "nifi": {}, # uses requests "okta": {"okta~=2.3"}, + "openlineage": {*COMMONS["kafka"]}, "oracle": {"cx_Oracle>=8.3.0,<9", "oracledb~=1.2"}, "pgspider": {"psycopg2-binary", "sqlalchemy-pgspider"}, "pinotdb": {"pinotdb~=0.3"}, diff --git a/ingestion/src/metadata/ingestion/source/pipeline/openlineage/__init__.py b/ingestion/src/metadata/ingestion/source/pipeline/openlineage/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/ingestion/src/metadata/ingestion/source/pipeline/openlineage/connection.py b/ingestion/src/metadata/ingestion/source/pipeline/openlineage/connection.py new file mode 100644 index 00000000000..fb95b913be8 --- /dev/null +++ b/ingestion/src/metadata/ingestion/source/pipeline/openlineage/connection.py @@ -0,0 +1,88 @@ +# Copyright 2021 Collate +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +""" +Source connection handler +""" +from typing import Optional + +from confluent_kafka import Consumer as KafkaConsumer +from confluent_kafka import TopicPartition + +from metadata.generated.schema.entity.automations.workflow import ( + Workflow as AutomationWorkflow, +) +from metadata.generated.schema.entity.services.connections.pipeline.openLineageConnection import ( + OpenLineageConnection, +) +from metadata.generated.schema.entity.services.connections.pipeline.openLineageConnection import ( + SecurityProtocol as KafkaSecProtocol, +) +from metadata.ingestion.connections.test_connections import ( + SourceConnectionException, + test_connection_steps, +) +from metadata.ingestion.ometa.ometa_api import OpenMetadata + + +def get_connection(connection: OpenLineageConnection) -> KafkaConsumer: + """ + Create connection + """ + try: + config = { + "bootstrap.servers": connection.brokersUrl, + "group.id": connection.consumerGroupName, + "auto.offset.reset": connection.consumerOffsets.value, + } + if connection.securityProtocol.value == KafkaSecProtocol.SSL.value: + config.update( + { + "security.protocol": connection.securityProtocol.value, + "ssl.ca.location": connection.SSLCALocation, + "ssl.certificate.location": connection.SSLCertificateLocation, + "ssl.key.location": connection.SSLKeyLocation, + } + ) + + kafka_consumer = KafkaConsumer(config) + kafka_consumer.subscribe([connection.topicName]) + + return kafka_consumer + except Exception as exc: + msg = f"Unknown error connecting with {connection}: {exc}." + raise SourceConnectionException(msg) + + +def test_connection( + metadata: OpenMetadata, + client: KafkaConsumer, + service_connection: OpenLineageConnection, + automation_workflow: Optional[AutomationWorkflow] = None, +) -> None: + """ + Test connection. This can be executed either as part + of a metadata workflow or during an Automation Workflow + """ + + def custom_executor(): + _ = client.get_watermark_offsets( + TopicPartition(service_connection.topicName, 0) + ) + + test_fn = {"GetWatermarkOffsets": custom_executor} + + test_connection_steps( + metadata=metadata, + test_fn=test_fn, + service_type=service_connection.type.value, + automation_workflow=automation_workflow, + ) diff --git a/ingestion/src/metadata/ingestion/source/pipeline/openlineage/metadata.py b/ingestion/src/metadata/ingestion/source/pipeline/openlineage/metadata.py new file mode 100644 index 00000000000..c2acf8be8ac --- /dev/null +++ b/ingestion/src/metadata/ingestion/source/pipeline/openlineage/metadata.py @@ -0,0 +1,520 @@ +# Copyright 2021 Collate +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +""" +OpenLineage source to extract metadata from Kafka events +""" +import json +import traceback +from collections import defaultdict +from itertools import groupby, product +from typing import Any, Dict, Iterable, List, Optional, Tuple + +from metadata.generated.schema.api.data.createPipeline import CreatePipelineRequest +from metadata.generated.schema.api.data.createTable import CreateTableRequest +from metadata.generated.schema.api.lineage.addLineage import AddLineageRequest +from metadata.generated.schema.entity.data.databaseSchema import DatabaseSchema +from metadata.generated.schema.entity.data.pipeline import Pipeline +from metadata.generated.schema.entity.data.table import Column, Table +from metadata.generated.schema.entity.services.connections.pipeline.openLineageConnection import ( + OpenLineageConnection, +) +from metadata.generated.schema.entity.services.ingestionPipelines.status import ( + StackTraceError, +) +from metadata.generated.schema.metadataIngestion.workflow import ( + Source as WorkflowSource, +) +from metadata.generated.schema.type.entityLineage import ( + ColumnLineage, + EntitiesEdge, + LineageDetails, + Source, +) +from metadata.generated.schema.type.entityReference import EntityReference +from metadata.ingestion.api.models import Either +from metadata.ingestion.api.steps import InvalidSourceException +from metadata.ingestion.models.pipeline_status import OMetaPipelineStatus +from metadata.ingestion.ometa.ometa_api import OpenMetadata +from metadata.ingestion.source.pipeline.openlineage.models import ( + EventType, + LineageEdge, + LineageNode, + OpenLineageEvent, + TableDetails, + TableFQN, +) +from metadata.ingestion.source.pipeline.openlineage.utils import ( + FQNNotFoundException, + message_to_open_lineage_event, +) +from metadata.ingestion.source.pipeline.pipeline_service import PipelineServiceSource +from metadata.utils import fqn +from metadata.utils.logger import ingestion_logger + +logger = ingestion_logger() + + +class OpenlineageSource(PipelineServiceSource): + """ + Implements the necessary methods of PipelineServiceSource to facilitate registering OpenLineage pipelines with + metadata into Open Metadata. + + Works under the assumption that OpenLineage integrations produce events to Kafka topic, which is a source of events + for this connector. + + Only 'SUCCESS' OpenLineage events are taken into account in this connector. + + Configuring OpenLineage integrations: https://openlineage.io/docs/integrations/about + """ + + @classmethod + def create(cls, config_dict, metadata: OpenMetadata): + """Create class instance""" + config: WorkflowSource = WorkflowSource.parse_obj(config_dict) + connection: OpenLineageConnection = config.serviceConnection.__root__.config + if not isinstance(connection, OpenLineageConnection): + raise InvalidSourceException( + f"Expected OpenLineageConnection, but got {connection}" + ) + return cls(config, metadata) + + def prepare(self): + """Nothing to prepare""" + + def close(self) -> None: + self.metadata.close() + + @classmethod + def _get_table_details(cls, data: Dict) -> TableDetails: + """ + extracts table entity schema and name from input/output entry collected from Open Lineage. + + :param data: single entry from inputs/outputs objects + :return: TableDetails object with schema and name + """ + symlinks = data.get("facets", {}).get("symlinks", {}).get("identifiers", []) + + # for some OL events name can be extracted from dataset facet but symlinks is preferred so - if present - we + # use it instead + if len(symlinks) > 0: + try: + # @todo verify if table can have multiple identifiers pointing at it + name = symlinks[0]["name"] + except (KeyError, IndexError): + raise ValueError( + "input table name cannot be retrieved from symlinks.identifiers facet." + ) + else: + try: + name = data["name"] + except KeyError: + raise ValueError( + "input table name cannot be retrieved from name attribute." + ) + + name_parts = name.split(".") + + if len(name_parts) < 2: + raise ValueError( + f"input table name should be of 'schema.table' format! Received: {name}" + ) + + # we take last two elements to explicitly collect schema and table names + # in BigQuery Open Lineage events name_parts would be list of 3 elements as first one is GCP Project ID + # however, concept of GCP Project ID is not represented in Open Metadata and hence - we need to skip this part + return TableDetails(name=name_parts[-1], schema=name_parts[-2]) + + def _get_table_fqn(self, table_details: TableDetails) -> Optional[str]: + try: + return self._get_table_fqn_from_om(table_details) + except FQNNotFoundException: + try: + schema_fqn = self._get_schema_fqn_from_om(table_details.schema) + + return f"{schema_fqn}.{table_details.name}" + except FQNNotFoundException: + return None + + def _get_table_fqn_from_om(self, table_details: TableDetails) -> Optional[str]: + """ + Based on partial schema and table names look for matching table object in open metadata. + :param schema: schema name + :param table: table name + :return: fully qualified name of a Table in Open Metadata + """ + result = None + services = self.source_config.dbServiceNames + for db_service in services: + result = fqn.build( + metadata=self.metadata, + entity_type=Table, + service_name=db_service, + database_name=None, + schema_name=table_details.schema, + table_name=table_details.name, + ) + if not result: + raise FQNNotFoundException( + f"Table FQN not found for table: {table_details} within services: {services}" + ) + return result + + def _get_schema_fqn_from_om(self, schema: str) -> Optional[str]: + """ + Based on partial schema name look for any matching DatabaseSchema object in open metadata. + + :param schema: schema name + :return: fully qualified name of a DatabaseSchema in Open Metadata + """ + result = None + services = self.source_config.dbServiceNames + + for db_service in services: + result = fqn.build( + metadata=self.metadata, + entity_type=DatabaseSchema, + service_name=db_service, + database_name=None, + schema_name=schema, + skip_es_search=False, + ) + + if result: + return result + + if not result: + raise FQNNotFoundException( + f"Schema FQN not found within services: {services}" + ) + + return result + + @classmethod + def _render_pipeline_name(cls, pipeline_details: OpenLineageEvent) -> str: + """ + Renders pipeline name from parent facet of run facet. It is our expectation that every OL event contains parent + run facet so we can always create pipeline entities and link them to lineage events. + + :param run_facet: Open Lineage run facet + :return: pipeline name (not fully qualified name) + """ + run_facet = pipeline_details.run_facet + + namespace = run_facet["facets"]["parent"]["job"]["namespace"] + name = run_facet["facets"]["parent"]["job"]["name"] + + return f"{namespace}-{name}" + + @classmethod + def _filter_event_by_type( + cls, event: OpenLineageEvent, event_type: EventType + ) -> Optional[Dict]: + """ + returns event if it's of particular event_type. + for example - for lineage events we will be only looking for EventType.COMPLETE event type. + + :param event: Open Lineage raw event. + :param event_type: type of event we are looking for. + :return: Open Lineage event if matches event_type, otherwise None + """ + return event if event.event_type == event_type else {} + + @classmethod + def _get_om_table_columns(cls, table_input: Dict) -> Optional[List]: + """ + + :param table_input: + :return: + """ + try: + fields = table_input["facets"]["schema"]["fields"] + + # @todo check if this way of passing type is ok + columns = [ + Column(name=f.get("name"), dataType=f.get("type").upper()) + for f in fields + ] + return columns + except KeyError: + return None + + def get_create_table_request(self, table: Dict) -> Optional[Either]: + """ + If certain table from Open Lineage events doesn't already exist in Open Metadata, register appropriate entity. + This makes sense especially for output facet of OpenLineage event - as database service ingestion is a scheduled + process we might fall into situation where we received Open Lineage event about creation of a table that is yet + to be ingested by database service ingestion process. To avoid missing on such lineage scenarios, we will create + table entity beforehand. + + :param table: single object from inputs/outputs facet + :return: request to create the entity (if needed) + """ + om_table_fqn = None + + try: + table_details = OpenlineageSource._get_table_details(table) + except ValueError as e: + return Either( + left=StackTraceError( + name="", + error=f"Failed to get partial table name: {e}", + stackTrace=traceback.format_exc(), + ) + ) + try: + om_table_fqn = self._get_table_fqn_from_om(table_details) + + # if fqn found then it means table is already registered and we don't need to render create table request + return None + except FQNNotFoundException: + pass + + # If OM Table FQN was not found based on OL Partial Name - we need to register it. + if not om_table_fqn: + try: + om_schema_fqn = self._get_schema_fqn_from_om(table_details.schema) + except FQNNotFoundException as e: + return Either( + left=StackTraceError( + name="", + error=f"Failed to get fully qualified schema name: {e}", + stackTrace=traceback.format_exc(), + ) + ) + + # After finding schema fqn (based on partial schema name) we know where we can create table + # and we move forward with creating request. + if om_schema_fqn: + columns = OpenlineageSource._get_om_table_columns(table) or [] + + request = CreateTableRequest( + name=table_details.name, + columns=columns, + databaseSchema=om_schema_fqn, + ) + + return Either(right=request) + + return None + + @classmethod + def _get_ol_table_name(cls, table: Dict) -> str: + return "/".join(table.get(f) for f in ["namespace", "name"]).replace("//", "/") + + def _build_ol_name_to_fqn_map(self, tables: List): + result = {} + + for table in tables: + table_fqn = self._get_table_fqn(OpenlineageSource._get_table_details(table)) + + if table_fqn: + result[OpenlineageSource._get_ol_table_name(table)] = table_fqn + + return result + + @classmethod + def _create_output_lineage_dict( + cls, lineage_info: List[Tuple[str, str, str, str]] + ) -> Dict[str, Dict[str, List[ColumnLineage]]]: + result = defaultdict(lambda: defaultdict(list)) + for (output_table, input_table, output_column), group in groupby( + lineage_info, lambda x: x[:3] + ): + input_columns = [input_col for _, _, _, input_col in group] + + result[output_table][input_table] += [ + ColumnLineage(toColumn=output_column, fromColumns=input_columns) + ] + + return result + + def _get_column_lineage( + self, inputs: List, outputs: List + ) -> Dict[str, Dict[str, List[ColumnLineage]]]: + _result: List = [] + + ol_name_to_fqn_map = self._build_ol_name_to_fqn_map(inputs + outputs) + + for table in outputs: + output_table_fqn = self._get_table_fqn( + OpenlineageSource._get_table_details(table) + ) + for field_name, field_spec in ( + table.get("facets", {}) + .get("columnLineage", {}) + .get("fields", {}) + .items() + ): + for input_field in field_spec.get("inputFields", []): + input_table_ol_name = OpenlineageSource._get_ol_table_name( + input_field + ) + + _result.append( # output table, input table, output column, input column + ( + output_table_fqn, + ol_name_to_fqn_map.get(input_table_ol_name), + f"{output_table_fqn}.{field_name}", + f'{ol_name_to_fqn_map.get(input_table_ol_name)}.{input_field.get("field")}', + ) + ) + + return OpenlineageSource._create_output_lineage_dict(_result) + + def yield_pipeline( + self, pipeline_details: OpenLineageEvent + ) -> Iterable[Either[CreatePipelineRequest]]: + pipeline_name = self.get_pipeline_name(pipeline_details) + try: + description = f"""```json + {json.dumps(pipeline_details.run_facet, indent=4).strip()}```""" + request = CreatePipelineRequest( + name=pipeline_name, + service=self.context.pipeline_service, + description=description, + ) + + yield Either(right=request) + self.register_record(pipeline_request=request) + except ValueError: + yield Either( + left=StackTraceError( + name=pipeline_name, + message="Failed to collect metadata required for pipeline creation.", + ), + stackTrace=traceback.format_exc(), + ) + + def yield_pipeline_lineage_details( + self, pipeline_details: OpenLineageEvent + ) -> Iterable[Either[AddLineageRequest]]: + inputs, outputs = pipeline_details.inputs, pipeline_details.outputs + + input_edges: List[LineageNode] = [] + output_edges: List[LineageNode] = [] + + for spec in [(inputs, input_edges), (outputs, output_edges)]: + tables, tables_list = spec + + for table in tables: + create_table_request = self.get_create_table_request(table) + + if create_table_request: + yield create_table_request + + table_fqn = self._get_table_fqn( + OpenlineageSource._get_table_details(table) + ) + + if table_fqn: + tables_list.append( + LineageNode( + fqn=TableFQN(value=table_fqn), + uuid=self.metadata.get_by_name(Table, table_fqn).id, + ) + ) + + edges = [ + LineageEdge(from_node=n[0], to_node=n[1]) + for n in product(input_edges, output_edges) + ] + + column_lineage = self._get_column_lineage(inputs, outputs) + + pipeline_fqn = fqn.build( + metadata=self.metadata, + entity_type=Pipeline, + service_name=self.context.pipeline_service, + pipeline_name=self.context.pipeline, + ) + + pipeline_entity = self.metadata.get_by_name(entity=Pipeline, fqn=pipeline_fqn) + for edge in edges: + yield Either( + right=AddLineageRequest( + edge=EntitiesEdge( + fromEntity=EntityReference( + id=edge.from_node.uuid, type=edge.from_node.node_type + ), + toEntity=EntityReference( + id=edge.to_node.uuid, type=edge.to_node.node_type + ), + lineageDetails=LineageDetails( + pipeline=EntityReference( + id=pipeline_entity.id.__root__, + type="pipeline", + ), + description=f"Lineage extracted from OpenLineage job: {pipeline_details.job['name']}", + source=Source.OpenLineage, + columnsLineage=column_lineage.get( + edge.to_node.fqn.value, {} + ).get(edge.from_node.fqn.value, []), + ), + ), + ) + ) + + def get_pipelines_list(self) -> Optional[List[Any]]: + """Get List of all pipelines""" + try: + consumer = self.client + session_active = True + empty_msg_cnt = 0 + pool_timeout = self.service_connection.poolTimeout + while session_active: + message = consumer.poll(timeout=pool_timeout) + if message is None: + logger.debug("no new messages") + empty_msg_cnt += 1 + if ( + empty_msg_cnt * pool_timeout + > self.service_connection.sessionTimeout + ): + # There is no new messages, timeout is passed + session_active = False + else: + logger.debug(f"new message {message.value()}") + empty_msg_cnt = 0 + try: + _result = message_to_open_lineage_event( + json.loads(message.value()) + ) + result = self._filter_event_by_type(_result, EventType.COMPLETE) + if result: + yield result + except Exception as e: + logger.debug(e) + + except Exception as e: + traceback.print_exc() + + raise InvalidSourceException(f"Failed to read from Kafka: {str(e)}") + + finally: + # Close down consumer to commit final offsets. + # @todo address this + consumer.close() + + def get_pipeline_name(self, pipeline_details: OpenLineageEvent) -> str: + return OpenlineageSource._render_pipeline_name(pipeline_details) + + def yield_pipeline_status( + self, pipeline_details: OpenLineageEvent + ) -> Iterable[Either[OMetaPipelineStatus]]: + pass + + def mark_pipelines_as_deleted(self): + """ + OpenLineage pipelines are coming from streaming data and hence subsequent executions of ingestion processes + can cause deletion of a pipeline. Because of this we turn off pipeline deletion by overwriting this method + and leaving it blank. Setting 'Mark Deleted Pipelines' in ingestion process will have no effect! + """ diff --git a/ingestion/src/metadata/ingestion/source/pipeline/openlineage/models.py b/ingestion/src/metadata/ingestion/source/pipeline/openlineage/models.py new file mode 100644 index 00000000000..8a05dc3f568 --- /dev/null +++ b/ingestion/src/metadata/ingestion/source/pipeline/openlineage/models.py @@ -0,0 +1,88 @@ +# Copyright 2021 Collate +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +Openlineage Source Model module +""" + +from dataclasses import dataclass +from enum import Enum +from typing import Any, Dict, List + + +@dataclass +class OpenLineageEvent: + """ + An object containing data extracted from raw OpenLineage event. Used as a basis for all abstract methods of + OpenlineageSource connector. + """ + + run_facet: Dict + job: Dict + event_type: str + inputs: List[Any] + outputs: List[Any] + + +@dataclass +class TableFQN: + """ + Fully Qualified Name of a Table. + """ + + value: str + + +@dataclass +class ColumnFQN: + """ + Fully Qualified Name of a Column. + """ + + value: str + + +@dataclass +class LineageNode: + """ + A node being a part of Lineage information. + """ + + uuid: str + fqn: TableFQN + node_type: str = "table" + + +@dataclass +class LineageEdge: + """ + An object describing connection of two nodes in the Lineage information. + """ + + from_node: LineageNode + to_node: LineageNode + + +@dataclass +class TableDetails: + """ + Minimal table information. + """ + + schema: str + name: str + + +class EventType(str, Enum): + """ + List of used OpenLineage event types. + """ + + COMPLETE = "COMPLETE" diff --git a/ingestion/src/metadata/ingestion/source/pipeline/openlineage/utils.py b/ingestion/src/metadata/ingestion/source/pipeline/openlineage/utils.py new file mode 100644 index 00000000000..4be9b16594b --- /dev/null +++ b/ingestion/src/metadata/ingestion/source/pipeline/openlineage/utils.py @@ -0,0 +1,59 @@ +""" +Utils used by OpenlineageSource connector. +""" + +from functools import reduce +from typing import Dict + +from metadata.ingestion.source.pipeline.openlineage.models import OpenLineageEvent + + +def message_to_open_lineage_event(incoming_event: Dict) -> OpenLineageEvent: + """ + Method that takes raw Open Lineage event and parses is to shape into OpenLineageEvent. + + We check whether received event (from Kafka) adheres to expected form and contains all the fields that are required + for successful processing by OpenMetadata OpenLineage connector. + + :param incoming_event: raw event received from kafka topic by OpenlineageSource + :return: OpenLineageEvent + """ + fields_to_verify = [ + "run.facets.parent.job.name", + "run.facets.parent.job.namespace", + "inputs", + "outputs", + "eventType", + "job.name", + "job.namespace", + ] + + for field in fields_to_verify: + try: + reduce(lambda x, y: x[y], field.split("."), incoming_event) + except KeyError: + raise ValueError("Event malformed!") + + run_facet = incoming_event["run"] + inputs = incoming_event["inputs"] + outputs = incoming_event["outputs"] + event_type = incoming_event["eventType"] + job = incoming_event["job"] + + result = OpenLineageEvent( + run_facet=run_facet, + event_type=event_type, + job=job, + inputs=inputs, + outputs=outputs, + ) + + return result + + +class FQNNotFoundException(Exception): + """ + Error raised when, while searching for an entity (Table, DatabaseSchema) there is no match in OM. + """ + + pass diff --git a/ingestion/src/metadata/utils/fqn.py b/ingestion/src/metadata/utils/fqn.py index 0140cd2a4d1..d0684ab3304 100644 --- a/ingestion/src/metadata/utils/fqn.py +++ b/ingestion/src/metadata/utils/fqn.py @@ -192,17 +192,34 @@ def _( @fqn_build_registry.add(DatabaseSchema) def _( - _: Optional[OpenMetadata], # ES Search not enabled for Schemas + metadata: Optional[OpenMetadata], # ES Search not enabled for Schemas *, service_name: str, - database_name: str, + database_name: Optional[str], schema_name: str, -) -> str: - if not service_name or not database_name or not schema_name: - raise FQNBuildingException( - f"Args should be informed, but got service=`{service_name}`, db=`{database_name}`, schema=`{schema_name}`" + skip_es_search: bool = True, + fetch_multiple_entities: bool = False, +) -> Union[Optional[str], Optional[List[str]]]: + entity: Optional[Union[DatabaseSchema, List[DatabaseSchema]]] = None + + if not skip_es_search: + entity = search_database_schema_from_es( + metadata=metadata, + database_name=database_name, + schema_name=schema_name, + fetch_multiple_entities=fetch_multiple_entities, + service_name=service_name, ) - return _build(service_name, database_name, schema_name) + + if not entity and database_name: + fqn = _build(service_name, database_name, schema_name) + return [fqn] if fetch_multiple_entities else fqn + if entity and fetch_multiple_entities: + return [str(table.fullyQualifiedName.__root__) for table in entity] + if entity: + return str(entity.fullyQualifiedName.__root__) + + return None @fqn_build_registry.add(Database) @@ -574,6 +591,43 @@ def build_es_fqn_search_string( return fqn_search_string +def search_database_schema_from_es( + metadata: OpenMetadata, + database_name: str, + schema_name: str, + service_name: str, + fetch_multiple_entities: bool = False, + fields: Optional[str] = None, +): + """ + Find database schema entity in elasticsearch index. + + :param metadata: OM Client + :param database_name: name of database in which we are searching for database schema + :param schema_name: name of schema we are searching for + :param service_name: name of service in which we are searching for database schema + :param fetch_multiple_entities: should single match be returned or all matches + :param fields: additional fields to return + :return: entity / entities matching search criteria + """ + if not schema_name: + raise FQNBuildingException( + f"Schema Name should be informed, but got schema_name=`{schema_name}`" + ) + + fqn_search_string = _build(service_name or "*", database_name or "*", schema_name) + + es_result = metadata.es_search_from_fqn( + entity_type=DatabaseSchema, + fqn_search_string=fqn_search_string, + fields=fields, + ) + + return get_entity_from_es_result( + entity_list=es_result, fetch_multiple_entities=fetch_multiple_entities + ) + + def search_table_from_es( metadata: OpenMetadata, database_name: str, diff --git a/ingestion/tests/unit/resources/datasets/openlineage_event.json b/ingestion/tests/unit/resources/datasets/openlineage_event.json new file mode 100644 index 00000000000..a04bd7502ff --- /dev/null +++ b/ingestion/tests/unit/resources/datasets/openlineage_event.json @@ -0,0 +1,330 @@ + { + "eventTime": "2023-12-16T14:22:11.949Z", + "producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "schemaURL": "https://openlineage.io/spec/2-0-2/OpenLineage.json#/$defs/RunEvent", + "eventType": "COMPLETE", + "run": { + "runId": "59fc8906-4a4a-45ab-9a54-9cc2d399e10e", + "facets": { + "parent": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/facets/1-0-0/ParentRunFacet.json#/$defs/ParentRunFacet", + "run": { + "runId": "daf8bcc1-cc3c-41bb-9251-334cacf698fa" + }, + "job": { + "namespace": "TESTSchedulerID", + "name": "TESTParentJobName4" + } + }, + "spark.logicalPlan": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/2-0-2/OpenLineage.json#/$defs/RunFacet", + "plan": [ + { + "class": "org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand", + "num-children": 1, + "table": { + "product-class": "org.apache.spark.sql.catalyst.catalog.CatalogTable", + "identifier": { + "product-class": "org.apache.spark.sql.catalyst.TableIdentifier", + "table": "dst_table_test_from_src_table_df", + "database": "test_db" + }, + "tableType": { + "product-class": "org.apache.spark.sql.catalyst.catalog.CatalogTableType", + "name": "MANAGED" + }, + "storage": { + "product-class": "org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat", + "compressed": false, + "properties": null + }, + "schema": { + "type": "struct", + "fields": [] + }, + "provider": "parquet", + "partitionColumnNames": [], + "owner": "", + "createTime": 1702736481797, + "lastAccessTime": -1, + "createVersion": "", + "properties": null, + "unsupportedFeatures": [], + "tracksPartitionsInCatalog": false, + "schemaPreservesCase": true, + "ignoredProperties": null + }, + "mode": null, + "query": 0, + "outputColumnNames": "[id, randomid, zip]" + }, + { + "class": "org.apache.spark.sql.execution.datasources.LogicalRelation", + "num-children": 0, + "relation": null, + "output": [ + [ + { + "class": "org.apache.spark.sql.catalyst.expressions.AttributeReference", + "num-children": 0, + "name": "id", + "dataType": "long", + "nullable": true, + "metadata": {}, + "exprId": { + "product-class": "org.apache.spark.sql.catalyst.expressions.ExprId", + "id": 9, + "jvmId": "78343417-b80d-45a2-b489-e7d8920e61dd" + }, + "qualifier": [] + } + ], + [ + { + "class": "org.apache.spark.sql.catalyst.expressions.AttributeReference", + "num-children": 0, + "name": "randomid", + "dataType": "string", + "nullable": true, + "metadata": {}, + "exprId": { + "product-class": "org.apache.spark.sql.catalyst.expressions.ExprId", + "id": 10, + "jvmId": "78343417-b80d-45a2-b489-e7d8920e61dd" + }, + "qualifier": [] + } + ], + [ + { + "class": "org.apache.spark.sql.catalyst.expressions.AttributeReference", + "num-children": 0, + "name": "zip", + "dataType": "string", + "nullable": true, + "metadata": {}, + "exprId": { + "product-class": "org.apache.spark.sql.catalyst.expressions.ExprId", + "id": 11, + "jvmId": "78343417-b80d-45a2-b489-e7d8920e61dd" + }, + "qualifier": [] + } + ] + ], + "catalogTable": { + "product-class": "org.apache.spark.sql.catalyst.catalog.CatalogTable", + "identifier": { + "product-class": "org.apache.spark.sql.catalyst.TableIdentifier", + "table": "src_table_test", + "database": "test_db" + }, + "tableType": { + "product-class": "org.apache.spark.sql.catalyst.catalog.CatalogTableType", + "name": "MANAGED" + }, + "storage": { + "product-class": "org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat", + "locationUri": null, + "inputFormat": "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat", + "outputFormat": "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat", + "serde": "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe", + "compressed": false, + "properties": null + }, + "schema": { + "type": "struct", + "fields": [ + { + "name": "id", + "type": "long", + "nullable": true, + "metadata": {} + }, + { + "name": "randomid", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "zip", + "type": "string", + "nullable": true, + "metadata": { + "__CHAR_VARCHAR_TYPE_STRING": "varchar(10)" + } + } + ] + }, + "provider": "parquet", + "partitionColumnNames": [], + "owner": "test_user@ad.test.net", + "createTime": 1682523315000, + "lastAccessTime": 0, + "createVersion": "3.3.1", + "properties": null, + "stats": null, + "unsupportedFeatures": [], + "tracksPartitionsInCatalog": false, + "schemaPreservesCase": true, + "ignoredProperties": null + }, + "isStreaming": false + } + ] + }, + "spark_version": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/2-0-2/OpenLineage.json#/$defs/RunFacet", + "spark-version": "3.3.1", + "openlineage-spark-version": "1.5.0" + }, + "processing_engine": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/facets/1-1-0/ProcessingEngineRunFacet.json#/$defs/ProcessingEngineRunFacet", + "version": "3.3.1", + "name": "spark", + "openlineageAdapterVersion": "1.5.0" + }, + "environment-properties": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/2-0-2/OpenLineage.json#/$defs/RunFacet", + "environment-properties": {} + } + } + }, + "job": { + "namespace": "TESTSchedulerID", + "name": "test_user_spark.execute_create_data_source_table_as_select_command.dst_table_test_from_src_table_df", + "facets": {} + }, + "inputs": [ + { + "namespace": "s3a://test_db-db", + "name": "src_table_test", + "facets": { + "dataSource": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/facets/1-0-0/DatasourceDatasetFacet.json#/$defs/DatasourceDatasetFacet", + "name": "s3a://test_db-db", + "uri": "s3a://test_db-db" + }, + "schema": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/facets/1-0-0/SchemaDatasetFacet.json#/$defs/SchemaDatasetFacet", + "fields": [ + { + "name": "id", + "type": "long" + }, + { + "name": "randomid", + "type": "string" + }, + { + "name": "zip", + "type": "string" + } + ] + }, + "symlinks": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/facets/1-0-0/SymlinksDatasetFacet.json#/$defs/SymlinksDatasetFacet", + "identifiers": [ + { + "namespace": "hive://hive-metastore.hive.svc.cluster.local:9083", + "name": "shopify.raw_product_catalog", + "type": "TABLE" + } + ] + } + }, + "inputFacets": {} + } + ], + "outputs": [ + { + "namespace": "s3a://test_db-db", + "name": "dst_table_test_from_src_table_df", + "facets": { + "dataSource": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/facets/1-0-0/DatasourceDatasetFacet.json#/$defs/DatasourceDatasetFacet", + "name": "s3a://test_db-db", + "uri": "s3a://test_db-db" + }, + "schema": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/facets/1-0-0/SchemaDatasetFacet.json#/$defs/SchemaDatasetFacet", + "fields": [ + { + "name": "id", + "type": "long" + }, + { + "name": "randomid", + "type": "string" + }, + { + "name": "zip", + "type": "string" + } + ] + }, + "columnLineage": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/facets/1-0-1/ColumnLineageDatasetFacet.json#/$defs/ColumnLineageDatasetFacet", + "fields": { + "id": { + "inputFields": [ + { + "namespace": "s3a://test_db-db", + "name": "/src_table_test", + "field": "comments" + } + ] + }, + "randomid": { + "inputFields": [ + { + "namespace": "s3a://test_db-db", + "name": "/src_table_test", + "field": "products" + } + ] + }, + "zip": { + "inputFields": [ + { + "namespace": "s3a://test_db-db", + "name": "/src_table_test", + "field": "platform" + } + ] + } + } + }, + "symlinks": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/facets/1-0-0/SymlinksDatasetFacet.json#/$defs/SymlinksDatasetFacet", + "identifiers": [ + { + "namespace": "hive://hive-metastore.hive.svc.cluster.local:9083", + "name": "shopify.fact_order_new5", + "type": "TABLE" + } + ] + }, + "lifecycleStateChange": { + "_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.5.0/integration/spark", + "_schemaURL": "https://openlineage.io/spec/facets/1-0-0/LifecycleStateChangeDatasetFacet.json#/$defs/LifecycleStateChangeDatasetFacet", + "lifecycleStateChange": "CREATE" + } + }, + "outputFacets": {} + } + ] + } \ No newline at end of file diff --git a/ingestion/tests/unit/topology/pipeline/test_openlineage.py b/ingestion/tests/unit/topology/pipeline/test_openlineage.py new file mode 100644 index 00000000000..f1c74668bae --- /dev/null +++ b/ingestion/tests/unit/topology/pipeline/test_openlineage.py @@ -0,0 +1,533 @@ +import copy +import json +import unittest +from pathlib import Path +from unittest.mock import MagicMock, Mock, patch +from uuid import UUID + +from metadata.generated.schema.entity.data.pipeline import Pipeline, Task +from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import ( + OpenMetadataConnection, +) +from metadata.generated.schema.entity.services.connections.pipeline.openLineageConnection import ( + ConsumerOffsets, + SecurityProtocol, +) +from metadata.generated.schema.entity.services.pipelineService import ( + PipelineConnection, + PipelineService, + PipelineServiceType, +) +from metadata.generated.schema.metadataIngestion.workflow import ( + OpenMetadataWorkflowConfig, +) +from metadata.generated.schema.type.basic import FullyQualifiedEntityName +from metadata.generated.schema.type.entityLineage import ColumnLineage +from metadata.generated.schema.type.entityReference import EntityReference +from metadata.ingestion.source.pipeline.openlineage.metadata import OpenlineageSource +from metadata.ingestion.source.pipeline.openlineage.models import OpenLineageEvent +from metadata.ingestion.source.pipeline.openlineage.utils import ( + message_to_open_lineage_event, +) + +# Global constants +MOCK_OL_CONFIG = { + "source": { + "type": "openlineage", + "serviceName": "openlineage_source", + "serviceConnection": { + "config": { + "type": "OpenLineage", + "brokersUrl": "testbroker:9092", + "topicName": "test-topic", + "consumerGroupName": "test-consumergroup", + "consumerOffsets": ConsumerOffsets.earliest, + "securityProtocol": SecurityProtocol.PLAINTEXT, + "SSLCertificateLocation": "", + "SSLKeyLocation": "", + "SSLCALocation": "", + "poolTimeout": 0.3, + "sessionTimeout": 1, + } + }, + "sourceConfig": {"config": {"type": "PipelineMetadata"}}, + }, + "sink": {"type": "metadata-rest", "config": {}}, + "workflowConfig": { + "openMetadataServerConfig": { + "hostPort": "http://localhost:8585/api", + "authProvider": "openmetadata", + "securityConfig": { + "jwtToken": "eyJraWQiOiJHYjM4OWEtOWY3Ni1nZGpzLWE5MmotMDI0MmJrOTQzNTYiLCJ0eXAiOiJKV1QiLCJhbGciOiJSUzI1NiJ9.eyJzdWIiOiJhZG1pbiIsImlzQm90IjpmYWxzZSwiaXNzIjoib3Blbi1tZXRhZGF0YS5vcmciLCJpYXQiOjE2NjM5Mzg0NjIsImVtYWlsIjoiYWRtaW5Ab3Blbm1ldGFkYXRhLm9yZyJ9.tS8um_5DKu7HgzGBzS1VTA5uUjKWOCU0B_j08WXBiEC0mr0zNREkqVfwFDD-d24HlNEbrqioLsBuFRiwIWKc1m_ZlVQbG7P36RUxhuv2vbSp80FKyNM-Tj93FDzq91jsyNmsQhyNv_fNr3TXfzzSPjHt8Go0FMMP66weoKMgW2PbXlhVKwEuXUHyakLLzewm9UMeQaEiRzhiTMU3UkLXcKbYEJJvfNFcLwSl9W8JCO_l0Yj3ud-qt_nQYEZwqW6u5nfdQllN133iikV4fM5QZsMCnm8Rq1mvLR0y9bmJiD7fwM1tmJ791TUWqmKaTnP49U493VanKpUAfzIiOiIbhg" + }, + } + }, +} +MOCK_SPLINE_UI_URL = "http://localhost:9090" +PIPELINE_ID = "3f784e72-5bf7-5704-8828-ae8464fe915b:lhq160w0" +MOCK_PIPELINE_URL = f"{MOCK_SPLINE_UI_URL}/app/events/overview/{PIPELINE_ID}" +MOCK_PIPELINE_SERVICE = PipelineService( + id="85811038-099a-11ed-861d-0242ac120002", + name="openlineage_source", + fullyQualifiedName=FullyQualifiedEntityName(__root__="openlineage_source"), + connection=PipelineConnection(), + serviceType=PipelineServiceType.Airflow, +) + +MOCK_PIPELINE = Pipeline( + id="2aaa012e-099a-11ed-861d-0242ac120002", + name=PIPELINE_ID, + fullyQualifiedName=f"openlineage_source.{PIPELINE_ID}", + displayName="MSSQL <> Postgres", + sourceUrl=MOCK_PIPELINE_URL, + tasks=[ + Task( + name=PIPELINE_ID, + displayName="jdbc postgres ssl app", + sourceUrl=MOCK_PIPELINE_URL, + ) + ], + service=EntityReference( + id="85811038-099a-11ed-861d-0242ac120002", type="pipelineService" + ), +) + +VALID_EVENT = { + "run": { + "facets": { + "parent": {"job": {"name": "test-job", "namespace": "test-namespace"}} + } + }, + "inputs": [], + "outputs": [], + "eventType": "START", + "job": {"name": "test-job", "namespace": "test-namespace"}, +} + +MISSING_RUN_FACETS_PARENT_JOB_NAME_EVENT = copy.deepcopy(VALID_EVENT) +del MISSING_RUN_FACETS_PARENT_JOB_NAME_EVENT["run"]["facets"]["parent"]["job"]["name"] + +MALFORMED_NESTED_STRUCTURE_EVENT = copy.deepcopy(VALID_EVENT) +MALFORMED_NESTED_STRUCTURE_EVENT["run"]["facets"]["parent"]["job"] = "Not a dict" + +with open( + f"{Path(__file__).parent}/../../resources/datasets/openlineage_event.json" +) as ol_file: + FULL_OL_KAFKA_EVENT = json.load(ol_file) + +EXPECTED_OL_EVENT = OpenLineageEvent( + run_facet=FULL_OL_KAFKA_EVENT["run"], + job=FULL_OL_KAFKA_EVENT["job"], + event_type=FULL_OL_KAFKA_EVENT["eventType"], + inputs=FULL_OL_KAFKA_EVENT["inputs"], + outputs=FULL_OL_KAFKA_EVENT["outputs"], +) + + +class OpenLineageUnitTest(unittest.TestCase): + @patch( + "metadata.ingestion.source.pipeline.pipeline_service.PipelineServiceSource.test_connection" + ) + def __init__(self, methodName, test_connection) -> None: + super().__init__(methodName) + test_connection.return_value = False + config = OpenMetadataWorkflowConfig.parse_obj(MOCK_OL_CONFIG) + self.open_lineage_source = OpenlineageSource.create( + MOCK_OL_CONFIG["source"], + config.workflowConfig.openMetadataServerConfig, + ) + self.open_lineage_source.context.__dict__[ + "pipeline" + ] = MOCK_PIPELINE.name.__root__ + self.open_lineage_source.context.__dict__[ + "pipeline_service" + ] = MOCK_PIPELINE_SERVICE.name.__root__ + self.open_lineage_source.source_config.dbServiceNames = ["skun"] + + @patch( + "metadata.ingestion.source.pipeline.pipeline_service.PipelineServiceSource.test_connection" + ) + @patch("confluent_kafka.Consumer") + def setUp(self, mock_consumer, mock_test_connection): + mock_test_connection.return_value = False + self.mock_consumer = mock_consumer + + def setup_mock_consumer_with_kafka_event(self, event): + mock_msg = MagicMock() + mock_msg.error.return_value = None + mock_msg.value.return_value = json.dumps(event).encode() + self.mock_consumer.poll.side_effect = [ + mock_msg, + None, + None, + None, + None, + None, + None, + ] + self.open_lineage_source.client = self.mock_consumer + + def test_message_to_ol_event_valid_event(self): + """Test conversion with a valid event.""" + result = message_to_open_lineage_event(VALID_EVENT) + self.assertIsInstance(result, OpenLineageEvent) + + def test_message_to_ol_event_missing_run_facets_parent_job_name(self): + """Test conversion with missing 'run.facets.parent.job.name' field.""" + with self.assertRaises(ValueError): + message_to_open_lineage_event(MISSING_RUN_FACETS_PARENT_JOB_NAME_EVENT) + + def test_message_to_ol_event_malformed_nested_structure(self): + """Test conversion with a malformed nested structure.""" + with self.assertRaises(TypeError): + message_to_open_lineage_event(MALFORMED_NESTED_STRUCTURE_EVENT) + + def test_poll_message_receives_message(self): + """Test if poll_message receives a kafka message.""" + self.setup_mock_consumer_with_kafka_event(VALID_EVENT) + result = self.open_lineage_source.client.poll(timeout=1) + self.assertIsNotNone(result) + self.assertEqual(json.loads(result.value().decode()), VALID_EVENT) + + def read_openlineage_event_from_kafka(self, kafka_event): + self.setup_mock_consumer_with_kafka_event(kafka_event) + result_generator = self.open_lineage_source.get_pipelines_list() + results = [] + try: + while True: + results.append(next(result_generator)) + except StopIteration: + pass + return results[0] + + def test_create_output_lineage_dict_empty_input(self): + """Test with an empty input list.""" + result = self.open_lineage_source._create_output_lineage_dict([]) + self.assertEqual(result, {}) + + def test_create_output_lineage_dict_single_lineage_entry(self): + """Test with a single lineage entry.""" + lineage_info = [ + ("output_table", "input_table", "output_column", "input_column") + ] + result = self.open_lineage_source._create_output_lineage_dict(lineage_info) + expected = { + "output_table": { + "input_table": [ + ColumnLineage( + toColumn="output_column", fromColumns=["input_column"] + ) + ] + } + } + self.assertEqual(result, expected) + + def test_create_output_lineage_dict_multiple_entries_different_outputs(self): + """Test with multiple entries having different output tables.""" + lineage_info = [ + ("output_table1", "input_table", "output_column1", "input_column"), + ("output_table2", "input_table", "output_column2", "input_column"), + ] + result = self.open_lineage_source._create_output_lineage_dict(lineage_info) + expected = { + "output_table1": { + "input_table": [ + ColumnLineage( + toColumn="output_column1", fromColumns=["input_column"] + ) + ] + }, + "output_table2": { + "input_table": [ + ColumnLineage( + toColumn="output_column2", fromColumns=["input_column"] + ) + ] + }, + } + self.assertEqual(result, expected) + + def test_create_output_lineage_dict_multiple_entries_same_output(self): + """Test with multiple entries sharing the same output table.""" + lineage_info = [ + ("output_table", "input_table1", "output_column", "input_column1"), + ("output_table", "input_table2", "output_column", "input_column2"), + ] + result = self.open_lineage_source._create_output_lineage_dict(lineage_info) + expected = { + "output_table": { + "input_table1": [ + ColumnLineage( + toColumn="output_column", fromColumns=["input_column1"] + ) + ], + "input_table2": [ + ColumnLineage( + toColumn="output_column", fromColumns=["input_column2"] + ) + ], + } + } + self.assertEqual(result, expected) + + def test_get_column_lineage_empty_inputs_outputs(self): + """Test with empty input and output lists.""" + inputs = [] + outputs = [] + result = self.open_lineage_source._get_column_lineage(inputs, outputs) + self.assertEqual(result, {}) + + @patch( + "metadata.ingestion.source.pipeline.openlineage.metadata.OpenlineageSource._get_table_fqn" + ) + def test_build_ol_name_to_fqn_map_with_valid_data(self, mock_get_table_fqn): + # Mock _get_table_fqn to return a constructed FQN based on the provided table details + mock_get_table_fqn.side_effect = ( + lambda table_details: f"database.schema.{table_details.name}" + ) + + tables = [ + {"name": "schema.table1", "facets": {}, "namespace": "ns://"}, + {"name": "schema.table2", "facets": {}, "namespace": "ns://"}, + ] + + expected_map = { + "ns://schema.table1": "database.schema.table1", + "ns://schema.table2": "database.schema.table2", + } + + result = self.open_lineage_source._build_ol_name_to_fqn_map(tables) + + self.assertEqual(result, expected_map) + self.assertEqual(mock_get_table_fqn.call_count, 2) + + @patch( + "metadata.ingestion.source.pipeline.openlineage.metadata.OpenlineageSource._get_table_fqn" + ) + def test_build_ol_name_to_fqn_map_with_missing_fqn(self, mock_get_table_fqn): + # Mock _get_table_fqn to return None for missing FQN + mock_get_table_fqn.return_value = None + + tables = [{"name": "schema.table1", "facets": {}, "namespace": "ns://"}] + + expected_map = {} # Expect an empty map since FQN is missing + + result = self.open_lineage_source._build_ol_name_to_fqn_map(tables) + + self.assertEqual(result, expected_map) + + @patch( + "metadata.ingestion.source.pipeline.openlineage.metadata.OpenlineageSource._get_table_fqn" + ) + def test_build_ol_name_to_fqn_map_with_empty_tables(self, mock_get_table_fqn): + # No need to set up the mock specifically since it won't be called with empty input + + tables = [] # No tables provided + + expected_map = {} # Expect an empty map + + result = self.open_lineage_source._build_ol_name_to_fqn_map(tables) + + self.assertEqual(result, expected_map) + mock_get_table_fqn.assert_not_called() + + @patch( + "metadata.ingestion.source.pipeline.openlineage.metadata.OpenlineageSource._get_table_fqn" + ) + @patch( + "metadata.ingestion.source.pipeline.openlineage.metadata.OpenlineageSource._build_ol_name_to_fqn_map" + ) + def test_get_column_lineage_valid_inputs_outputs( + self, mock_build_map, mock_get_table_fqn + ): + """Test with valid input and output lists.""" + # Setup + mock_get_table_fqn.side_effect = ( + lambda table_details: f"database.schema.{table_details.name}" + ) + mock_build_map.return_value = { + "s3a:/project-db/src_test1": "database.schema.input_table_1", + "s3a:/project-db/src_test2": "database.schema.input_table_2", + } + + inputs = [ + {"name": "schema.input_table1", "facets": {}, "namespace": "hive://"}, + {"name": "schema.input_table2", "facets": {}, "namespace": "hive://"}, + ] + outputs = [ + { + "name": "schema.output_table", + "facets": { + "columnLineage": { + "fields": { + "output_column1": { + "inputFields": [ + { + "field": "input_column1", + "namespace": "s3a://project-db", + "name": "/src_test1", + } + ] + }, + "output_column2": { + "inputFields": [ + { + "field": "input_column2", + "namespace": "s3a://project-db", + "name": "/src_test2", + } + ] + }, + } + } + }, + } + ] + result = self.open_lineage_source._get_column_lineage(inputs, outputs) + + expected = { + "database.schema.output_table": { + "database.schema.input_table_1": [ + ColumnLineage( + toColumn="database.schema.output_table.output_column1", + fromColumns=["database.schema.input_table_1.input_column1"], + ) + ], + "database.schema.input_table_2": [ + ColumnLineage( + toColumn="database.schema.output_table.output_column2", + fromColumns=["database.schema.input_table_2.input_column2"], + ) + ], + } + } + self.assertEqual(result, expected) + + def test_get_column_lineage__invalid_inputs_outputs_structure(self): + """Test with invalid input and output structure.""" + inputs = [{"invalid": "data"}] + outputs = [{"invalid": "data"}] + with self.assertRaises(ValueError): + self.open_lineage_source._get_column_lineage(inputs, outputs) + + def test_get_table_details_with_symlinks(self): + """Test with valid data where symlinks are present.""" + data = { + "facets": {"symlinks": {"identifiers": [{"name": "project.schema.table"}]}} + } + result = self.open_lineage_source._get_table_details(data) + self.assertEqual(result.name, "table") + self.assertEqual(result.schema, "schema") + + def test_get_table_details_without_symlinks(self): + """Test with valid data but without symlinks.""" + data = {"name": "schema.table"} + result = self.open_lineage_source._get_table_details(data) + self.assertEqual(result.name, "table") + self.assertEqual(result.schema, "schema") + + def test_get_table_details_invalid_data_missing_symlinks_and_name(self): + """Test with invalid data missing both symlinks and name.""" + data = {} + with self.assertRaises(ValueError): + self.open_lineage_source._get_table_details(data) + + def test_get_table_details_invalid_symlinks_structure(self): + """Test with invalid symlinks structure.""" + data = {"facets": {"symlinks": {"identifiers": [{}]}}} + with self.assertRaises(ValueError): + self.open_lineage_source._get_table_details(data) + + def test_get_table_details_invalid_name_structure(self): + """Test with invalid name structure.""" + data = {"name": "invalidname"} + with self.assertRaises(ValueError): + self.open_lineage_source._get_table_details(data) + + def test_get_pipelines_list(self): + """Test get_pipelines_list method""" + ol_event = self.read_openlineage_event_from_kafka(FULL_OL_KAFKA_EVENT) + self.assertIsInstance(ol_event, OpenLineageEvent) + self.assertEqual(ol_event, EXPECTED_OL_EVENT) + + @patch( + "metadata.ingestion.source.pipeline.openlineage.metadata.OpenlineageSource._get_table_fqn_from_om" + ) + def test_yield_pipeline_lineage_details(self, mock_get_entity): + def t_fqn_build_side_effect( + table_details, + ): + return f"testService.shopify.{table_details.name}" + + def mock_get_uuid_by_name(entity, fqn): + if fqn == "testService.shopify.raw_product_catalog": + # source of table lineage + return Mock(id="69fc8906-4a4a-45ab-9a54-9cc2d399e10e") + elif fqn == "testService.shopify.fact_order_new5": + # dst of table lineage + return Mock(id="59fc8906-4a4a-45ab-9a54-9cc2d399e10e") + else: + # pipeline + z = Mock() + z.id.__root__ = "79fc8906-4a4a-45ab-9a54-9cc2d399e10e" + return z + + def extract_lineage_details(pip_results): + table_lineage = [] + col_lineage = [] + for r in pip_results: + table_lineage.append( + ( + r.right.edge.fromEntity.id.__root__, + r.right.edge.toEntity.id.__root__, + ) + ) + for col in r.right.edge.lineageDetails.columnsLineage: + col_lineage.append( + (col.fromColumns[0].__root__, col.toColumn.__root__) + ) + return table_lineage, col_lineage + + # Set up the side effect for the mock entity FQN builder + mock_get_entity.side_effect = t_fqn_build_side_effect + + ol_event = self.read_openlineage_event_from_kafka(FULL_OL_KAFKA_EVENT) + + with patch.object( + OpenMetadataConnection, + "get_by_name", + create=True, + side_effect=mock_get_uuid_by_name, + ): + pip_results = self.open_lineage_source.yield_pipeline_lineage_details( + ol_event + ) + table_lineage, col_lineage = extract_lineage_details(pip_results) + + expected_table_lineage = [ + ( + UUID("69fc8906-4a4a-45ab-9a54-9cc2d399e10e"), + UUID("59fc8906-4a4a-45ab-9a54-9cc2d399e10e"), + ) + ] + expected_col_lineage = [ + ( + "testService.shopify.raw_product_catalog.comments", + "testService.shopify.fact_order_new5.id", + ), + ( + "testService.shopify.raw_product_catalog.products", + "testService.shopify.fact_order_new5.randomid", + ), + ( + "testService.shopify.raw_product_catalog.platform", + "testService.shopify.fact_order_new5.zip", + ), + ] + + self.assertEqual(col_lineage, expected_col_lineage) + self.assertEqual(table_lineage, expected_table_lineage) + + +if __name__ == "__main__": + unittest.main() diff --git a/openmetadata-docs/content/v1.3.x/connectors/pipeline/openlineage/index.md b/openmetadata-docs/content/v1.3.x/connectors/pipeline/openlineage/index.md new file mode 100644 index 00000000000..e113020e5bd --- /dev/null +++ b/openmetadata-docs/content/v1.3.x/connectors/pipeline/openlineage/index.md @@ -0,0 +1,151 @@ +--- +title: OpenLineage +slug: /connectors/pipeline/openlineage +--- + +# OpenLineage + +In this section, we provide guides and references to use the OpenLineage connector. + +## What is OpenLineage? + +According to [documentation](https://openlineage.io/docs/): + +``` +OpenLineage is an open framework for data lineage collection and analysis. At its core is an extensible specification that systems can use to interoperate with lineage metadata. +``` + +OpenLineage, apart from being a specification, is also a set of integrations collecting lineage from various systems such as Apache Airflow and Spark. + + +## OpenMetadata Openlineage connector + +OpenMetadata OpenLineage connector consumes open lineage events from kafka broker and translates it to OpenMetadata Lineage information. + +{% image + src="/images/v1.3/connectors/pipeline/openlineage/connector-flow.svg" + alt="OpenLineage Connector" /%} + + +### Airflow OpenLineage events + +Configure your Airflow instance + +1. Install appropriate provider in Airflow: [apache-airflow-providers-openlineage](https://airflow.apache.org/docs/apache-airflow-providers-openlineage/stable/index.html) +2. Configure OpenLineage Provider in Airflow - [documentation](https://airflow.apache.org/docs/apache-airflow-providers-openlineage/stable/guides/user.html#using-openlineage-integration) + 1. remember to use `kafka` transport mode as this connector works under assumption OL events are collected from kafka topic + 2. detailed list of configuration options for OpenLineage can be found [here](https://openlineage.io/docs/client/python/#configuration) + +### Spark OpenLineage events + +Configure Spark Session to produce OpenLineage events compatible with OpenLineage connector available in OpenMetadata. + +# @todo complete kafka config +```shell +from pyspark.sql import SparkSession +from uuid import uuid4 + +spark = SparkSession.builder\ +.config('spark.openlineage.namespace', 'mynamespace')\ +.config('spark.openlineage.parentJobName', 'hello-world')\ +.config('spark.openlineage.parentRunId', str(uuid4()))\ +.config('spark.jars.packages', 'io.openlineage:openlineage-spark:1.7.0')\ +.config('spark.extraListeners', 'io.openlineage.spark.agent.OpenLineageSparkListener')\ +.config('spark.openlineage.transport.type', 'kafka')\ +.getOrCreate() +``` + +## Requirements + +{% note %} +We support OpenLineage events created by OpenLineage versions starting from OpenLineage 1.7.0 +{% /note %} + +## Metadata Ingestion + +#### Connection Details + +##### Providing connection details via UI + +{% partial + file="/v1.3/connectors/metadata-ingestion-ui.md" + variables={ + connector: "Openlineage", + selectServicePath: "/images/v1.3/connectors/openlineage/select-service.png", + addNewServicePath: "/images/v1.3/connectors/openlineage/add-new-service.png", + serviceConnectionPath: "/images/v1.3/connectors/openlineage/service-connection.png", +} +/%} + +{% stepsContainer %} +{% extraContent parentTagNme="stepsContainer" %} + +##### Providing connection details programmatically via API +###### 1. Preparing the Client + +```python +from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import ( + OpenMetadataConnection, +) +from metadata.generated.schema.security.client.openMetadataJWTClientConfig import ( + OpenMetadataJWTClientConfig, +) +from metadata.ingestion.ometa.ometa_api import OpenMetadata + +server_config = OpenMetadataConnection( + hostPort="http://localhost:8585/api", + authProvider="openmetadata", + securityConfig=OpenMetadataJWTClientConfig( + jwtToken="" + ), +) +metadata = OpenMetadata(server_config) + +assert metadata.health_check() # Will fail if we cannot reach the server +``` +###### 2. Creating the OpenLineage Pipeline service +```python + +from metadata.generated.schema.api.services.createPipelineService import CreatePipelineServiceRequest +from metadata.generated.schema.entity.services.pipelineService import ( + PipelineServiceType, + PipelineConnection, +) +from metadata.generated.schema.entity.services.connections.pipeline.openLineageConnection import ( + OpenLineageConnection, + SecurityProtocol as KafkaSecurityProtocol, + ConsumerOffsets +) + + +openlineage_service_request = CreatePipelineServiceRequest( + name='openlineage-service', + displayName='OpenLineage Service', + serviceType=PipelineServiceType.OpenLineage, + connection=PipelineConnection( + config=OpenLineageConnection( + brokersUrl='broker1:9092,broker2:9092', + topicName='openlineage-events', + consumerGroupName='openmetadata-consumer', + consumerOffsets=ConsumerOffsets.earliest, + poolTimeout=3.0, + sessionTimeout=60, + securityProtocol=KafkaSecurityProtocol.SSL, + # below ssl confing in optional and used only when securityProtocol=KafkaSecurityProtocol.SSL + SSLCertificateLocation='/path/to/kafka/certs/Certificate.pem', + SSLKeyLocation='/path/to/kafka/certs/Key.pem', + SSLCALocation='/path/to/kafka/certs/RootCA.pem', + ) + ), +) + +metadata.create_or_update(openlineage_service_request) + + +``` +{% /extraContent %} +{% partial file="/v1.3/connectors/test-connection.md" /%} +{% partial file="/v1.3/connectors/pipeline/configure-ingestion.md" /%} +{% partial file="/v1.3/connectors/ingestion-schedule-and-deploy.md" /%} +{% /stepsContainer %} +{% partial file="/v1.3/connectors/troubleshooting.md" /%} \ No newline at end of file diff --git a/openmetadata-docs/images/v1.3/connectors/openmetadata/add-new-service.png b/openmetadata-docs/images/v1.3/connectors/openmetadata/add-new-service.png new file mode 100644 index 0000000000000000000000000000000000000000..aa2b482e2c334cb439fb76be768460b6fe07c050 GIT binary patch literal 37523 zcmeFZWl){n(l&@}oM0iiySqEV-Q6{~1a}P*TtjeocMGs_5AN=+!6D4fIq!L&aAs<% z=Fj~4DmxU_`@Wa-THSs1wR#b*C@=929tR!_4D6khq^L3&7$hPX7(^H>G)S^a?E?-5 zMg%4$Dy-@be)8kV9$n(`b)+h8oUltlB7A%}zJMQ1DW9qaomx0c(g4G!zMTBypdFWK z9UWW~nRR-v5Ny*seWHj2lqyK9Ik*%uj&FUjSP%>S%#AGzC?>exom*{D?-oQN?|gp! zGAnHT)Srp{Go_`SKB}mw?Wm+kUv2fey{**KJiw1k2;$9W<4YGE<5No-K+^xsM+n#@ z|KE=$fk6dnu<`wu81$sTe~C8!KRoHb9%$t4y;4@=k1QK$;!U!#62rND&-GEL_narn zZ8o=&7LV_8`6g_f$=8pKZb>kgc*+Bf2>f54fNF{jVW!CJf?9|V{be0aGap9Xc1^rK z-cJ{#tHIMp%;3q15)}6186S*rmi4A5PeCKInPIG+!^)lW(o_ zapiQd`PBIO4p=w!mu6(2p&hP#(**iktIPQ9DxNpRdGs+~>jNi^@8r#IHEj#(&{Mtd4$avj6K8oFvY4wNB4S%m%}6NY z11odjW5j^VrXv~hhnLErQbVkgaS7iB-*1iE${f(p@fqmcrGR-=E;tnxHF^X3+6Q4T^G|K;h=b`0QeO)Ju z_gaDhiIb)@lR4CXX&qAC%XP$PO8WHiM?%@fPX=e7xnbL}p5&d=!E=^&i!q6%M*6`H zquWkLMAeL*%TczIXBGqxGX~W1#ity2`XrA*KHb3aC2vzg>YA{q%T}kso_I7+~#SEHzOf|ZwQ?c=>QCD-|V|UwG1zZyB{bhAEj?)^037$7i*}9si z0x%VWZR86H@geOzLmJVeazlgd=Fm;0+xm^Pb=AA35#NF`-DEO?tz%%zkZTnS!oE$G z(%F`d-Tv^PYE22PQdWt54zLzAEZt56 z(Qw+)iq}*pLGiuXAevzM?<3chLV#tU5ov9uduGVowF*wW;6ikH*09oV6%5zHX7Q~0 z-*>_+${aSp^v`lMGNW2M2w@r(yzXL|jOkL_PwqX9_YrxT9 zSt0ZRL1AD%XMPVQRn^So?>}g7*kU5y9@3FgJ&s~U6_1NweG;2%TSMc+0d~@$&Dg>% zOfk?jJrO38vU(s(o6e(iRMVx4OT*DL%n{d`W8pLS?`2=Ys1z&G${QU=PO;FpmN9L z?a+|q8qxIec+N21D$B8#m{R=MKuw=!S{@v<2;vd@Bvjzy+QehtWOAt?(}}OzUM=qu zwKbJ#)F8S`J@K~0U+Sd%qt2EPVq~$Tn*-*e%Pt12(Mf2Hq|q*&wK47;mKa?x+c=M1 z?pQdcQc1-oJ@95r6~dv|wpW!!HYd%s0SV>_?VK1@EVqh=Y{fI8G~c18LUg%}!3-NW zLQ&Z7;GFh4tj;eTUXDQZ0E(T*t@23O%Wp#Q$u7Wl6@(djj%#>^w@_Xt`Zn@FK z=@1XhlMJTqqc*~2(f+8bEt!~ioXJvsT}D4uRE6g0wuqLKO7OOU&E&k*V6Fq{VAD79 ztOCS}OpdhedEGoFQo(cKT52j)V7BeKbF{M7>i5#Jj1mm9q@k$KYL=&TJiBJn=U+my2H!ST3!zSWlp-T(radWW(p|OX?SQxM zZ`3<`B}jj$0H~lP7zz&~e0QUpSUzvbmaU#PLY=+e#=?&)l9ifh!yh=dp%WL+ZVd}} zM66FHv*-OOlGdNER5xOVpBTwOwthRg9k2pcL$>W=li2j8dQWCu^tqv75 zy8X@HO9f}j6xcu^=lQmti`; zazo$!PA$nhy{Q|^tDGEex^36M5p{@5Tb4n1A{;JTglgcnmp7m(F>%@Bx=zTdlLR&5 zOn?ADCLE>a7DT23_iSn8=+Z?X0{6UvsVy`HY;WgTwA?2(76QGcr&I^%3zh(o*)Xe1 z_`cgEGO7HQF+fK{Z@>+uSyTW}JaR}iU!l>YjoM0-%%>WExv53mlg`ZQ7%Bdy5oK~N zwW@-BV*U%(?)kVxL~%nXKb2rJ>QLI}`UXw{C`d5dv4JzT6Sm$m^-LP9QP0zHiBQ;# zQ2fF5j1!wz>SK!%Fv0&(CrGJ6_b4EZ4t|unaXeXx%~d#*S`6RrpN`J|nm4w8F`JC; zLld1pMCSQz3@FgFzRYy z>>c%jNBP{tf`S{NbAfX2@nUf&J;p^+w27sd~0%Ynz<&onZYqux5JU+W9ut zr)5#;gCvLOC)~4JHU^Q68!P(X3-c9%(AIXvc|G1EF7aSU*|Wrdqze@0`9vS>m%3Hk zcFkoar)1UG*~GI%+x7Dw9F0N&;Dd!W;7<@*OV0#E3aS>qF39mA0;`B&hW7G(myb7( zKRxVPRdMw*pXt6g=Hyyr_R@Z!3#j8x8l1kgJWH!5Ai<+3`$S~+r1L5?tat@j_3OF4 zFLo*YuAKj@m~(gAM7HyHBtn1?evfFRl&0yl<;z{$K{rJ>hGXsW$w%rh5N{p~2GtX3 zXiogkI6*KSfe^-kNoF5@9J@~H?Igmpvw6N66dDHjQ?YW zf3}SUoFY?jGGoPCsp$J`!?1(Byz4iOplCy&Bj)VN9KV%L0;sS^S|1Mh-i#zT84OCq z;$e#CjZ~Bk96lt>PJl<@ANL4qEr(nPh@#z5FaJjRe}?>zqEY@|wJdMBdfCNnhC=}h z4ju{_EkRr@lknHOW&j8bL>E0sO|OIWfqB2mwM=f=7(6U&yh#I6BP?&+-yZPt6243R z+?P((4P=NfNM;D7wzouyP#!h>pD}PV!w1rB&BB?K=Wlp(2z)oZSd>g8E9dop08Q!nN&HjVjo!l7 zhs#S%Z!*~>T4}{PoWJx#2cZwulNS!BTWkm14dJCKw-gh0LBR0}N#9LXi639F=uOV+ zBPgQqiSvVc|N2tJs!+{g)sY%=|0zChe;3;Esb)7NZ*8{I7c3lsbHj#kLgO>okUhoL zmv4Kdmt&@QRF{T3ldx9ur(LOE!c$(S5F~Y*p?T$J4MkfJ)+pVEF$sirCEtDKWbAtm zcIV^uqmE}CnJCKdF%1;Hlrb^!J&T=8V3+i^MA~al3}ffLR@^q4FwN&c=2fR8ar}+o zzKN_qMeQm> z60F@iSWLLxD(%p99#1`6f@RUj*we&<OE>8$KHZH!fBqpH~bGpbLY(~Y?2q`0m(w+vx+B*}p{*dw-Dr&=}yd&GBxz}IiP>`m| zw9prP`(QSj^$L+(xNpfZB9Gsd`fwSRl@%gw*6p)i!oDQ=m38%t?uom~vwoheBg@g1 zEJ-}8i!{Ue5Np-xlmHJbK`aH$spW%@GOs(tbr(H(qyiq}|${C$d)*IP~~N=+sy^ z!11k7br}zQZdqT@m*O7G;#0WoX)#N6|SeYeMWbrO`=r_Y1{Ha@dfU%*HT2l7Q_ylbhEDwX^+f2qEwDDv21Y;Htwy!1)K;*# z7YS7mE`l9gW!mam2JJzUll34bD=PrUMfB4SUA&sQHjs4CYrd~<)EXYT%2uD;#}>&| zciI}w;m6I%^IuT_RjMn`(BcTYyKjjj=ELI>3R(v|G`872*FFheXQ@(##B|4LJ!8aZTJIPq<9W{G7zEF;~znczfw zQmz}16Aa2zsGkg+WMza@2v-OY`q7TxVs{D}-8m{2K4Q6Ca%n@<`*o^f8(mOV2rSjy zud`RKcUFHpH!6&)qaonHdCdsB|1?$KH_&;UtDj$P za=DRj$r^TZs|tS?_irTh2i-KIUQl864uzZv<8^-4LAWK~V5~Urn!M7%@?*=H1!@2t z>V9w@xFq-0Ib~`^>zb?vG=Ra-xDBf$Hs%hdn^Zwze;mE7!$p!gw7M90;L^tHI4>?;E@|Ad!l4tHv(F~7J5jwC8D%ZDk_a8 zFcF@T8!2AQdOJO_qW0KG>K+?ls+~8D@f_q|n2`_8TO}7atT4B$IrxwKB2mEzkN5f) zwbAR9zI+LtERK@=9+5F7I>&Lfv{%uxlu+xcJG1W#rEOoQ`gc-AhIT=NmCth5Pe5|6 zox!o>&@l9PYz)c@xZLE@I>=N~cE~%gY<@%hLNy6+z5u4=3S}y)hV(PM$a@)>pt9^p z3I8M`k<&XUg9{O*^S#j`E8N3Ae4214ez%n1Fk|qXgc|v9nQ}eAme>&8wdr7(-+*Z8 zmu}*r9Op-hDC5{X$Agpoa#iOe3jDnYH_p>d)OI5=!XU0K#Vbz0wSi?RvABOVQ}iI& zX1B7%d4wkD2Z>~0Jud;!9*Mo&L(uH#z713Hn?lMIyJEde>14oU*W=0SDAlz=vD;%~ zDmot)*C0ryeb~WlU6%R9p@`}5q{`XGr1Eu=BgzFK!oUanGVL@EMjc3`ASiu3w^zyp zU%%$Me*gU+K8_@;D?iYv)-xp#hz@h>_Fx-QV-UpkVJL`o$)2YDvHbX3W|jwdgdmqc zOIf4i>h8Snx{&X}Sc?7YD1v6D-P0<7dS93n6sKJaBC>*d8$y*z=vZzQn40(Qj-6hk z9&TI%3|~*m>)H%ssU@NY8jT$TM{d&v$+jR>S~cL$nr{??*f)neOa8GlVSN@x%eBx{*k(W7mLLERy+o_51JbGwp zlxag{)th>Bo~@(tf$iwDj8<0SADq4^L}Un@@Ux#``7o3oPj;?>a~>?Rsw=Pn6?6>S zBnzPiQ|-Qq6Lb8rzU8Twpi6ed?H_ML6p%uUQf(MU^${IWh4WjhYhI5ofZ zL70(OlixH~GleOE|4d4f$P-qPNmM~czLSDQ&;TPU&6=;C9hs5nAhIPE ziBNC@f%6`Rag*jbcUM5zM;uxuQibqZK_2#B1$BVGpJgHeJxd(vKmFk^DL7gf87Sw8 zo6-K2^w%K#0p=;7#0(MgljA=q@=szW^(Pf7X*i($$NPg)^id{I(t`~>zV#0q56ba~ z;{VJ<*?LmrO@}E)pbl9P)BFB$V}Cpu87e3v%b&}wdeb3O52!#g&*#WL2T-gM8)-5*c|@25g+x?Q9Z)RB-|Ke7d|QB^F4DO6+kDnhc6J zs6Rvj6>UU*kfJA8qg&7*TCh?{0&|J?S`ldRj3HcWh)7|xGOQb5N+%8HSJiHtj_f~wT6aq3`kLI! z^HZ0$!P0UD0Zgspcs#4O6k{k*+FOVqK28;TaPbe}(w|X+zrrqb5}xxEjx$=$oGbeZ z%8x}-e-r_!nN8PZuJk23%rocOMaSkM1|Vtp2$(^IM1|=1U6L#OzmjtagWf=`TpFDF z7YpUhC)q!1VY#KGoB47!lnAkThGlodU%SH3e0{+njqZl7ZYvI3oQHt+YvdrH$}e91 z11_G~MH1e0oc||?qdOBK;Z7Qk`bB>ZjO(tH;#ST zVw(5g>-Nsk(=UDi z9Wgz5hmpUK5}X$N=1EC!!$xlRC1HoTQvsy*jQ26Y(IE2RV4*+V?An;);fZvG%1XHV zE3^%D+j%>f=kR+cFfAiASQ79DB@Wh&yH7mvhk-lH*Aeu;K}G|R7kY_Cn2~0(0cdkv z$ALjXJ_$|07GttjE%QT4E~|zH|7>&It~`zopa0d*D00oERfBv#4--g_HMlPLdq+%v z?NKP(ptbaekS!A-96%bhY>J;p_r-GRUDl&3&YZ8O* zQ35xbgReIufY{WgY9KJ^rFqvVr|z+@TTCx*>gX4%E}X-pQ*$HY%U3bbGLmBQ4;Ukd zJ%bV}QD58;xpy-I{fEF;ze(NYFSF;fo3E=bF#BJ%nHfv4VQ_sjuI$JOaWOf@q$Nla z!3fK6a^6Wa<;D9yzF-aJnz;b7)?<_w#kxC`(GiW-MdNm}J(Z@#dPrAVS3;o5j!>Gq zn;H-TbY)2+(0-5yd?(Hppbi&Y;b0xrf`@Y_&Gr?%3SDkj+WPJAhq$Svf1l%9;--|Y z5-EPR?Q9O2>0U-)9dYL9O#E><*-nB0+$B*s)155WsFspqi1|h%lN{Nvp8jengCLX= z{OK-_F<1B{>C=P!?!ia*Mq4^}F(3*;1z(Q~p6@}&;PH4Df8spS_T1{LJPj5Z<15IE z21;z$uix?z#?U|riR14`ESeJ6!vZ`&9!N!kjI3g3_g{YEj}@BD2+y$eoTl9GKRZxRvm_>|>8AlK4AQK1u<^i&_DkP#UR z5wy^B3;C|d>HkY^Tq{#O6%z;6Nk~ZQjd6+z4iDYwz|8z6y7XdzHrIlc{ao^!2=xq# z0UMYM6#rzJeJInQx(%)^5FmhE;zZ%-C}H-2TnE=6yKde=za|-oTzsu2HX{xL3(`L zU5trLDcMB`5#!KQ!YH~A6w&=zD)G#FGuRAAbg&;JcvD~S?qJEZj`?uF$ookNKl!)2 z{gf)7B*Z}(umEA-@prd}YQ4l@lD7@4h}ArUe#~NBQx3`C7`JMf6Dr5PjP*o8@}Y>9 ziHHc+waB3EqXmb#U+#SFu_;=-yz7G^e1a$hXooP_71ZEXxI4N)Dl52ci@K8TRTN7c zMGZl?*Rl9W`jmYz+ZzyCr`5UaEtGfm8d`rbA>CTH7wvDc^KNSbzof-+Bw3gb3HK`g zak~FuJfidQ42Ft#@@sui#X@}rfkL_cT;sI-ug$ak2lrS29}3o)*wK?q%3TZw9^d?z zAnY;C>Q;hteXQG9u2{X#44hlN*>Ri)`yxaF-~k3{&$RaCK44>czjuv}G)b6(WwJ*5 zR2YAR-@rs~9R%EHa-UQ4bw0ACcH@a?xRRLLw&c5W2d9=2?>aefFxr0qvi{=-LYwQL z%zdxPnU}=mP6F&f_vV+n>WUd;gGVdH>7t@;u!#GakHz9@^G) zPSjKFRBa<#)$2frbY|7{{+!)-8mNxfe%YnoB8(?nG9bLK<^j=wsU7FIhJZB9AGkb)YgB&&isC4Xx&DtU~!M_D$~3ptZ%Jq^_bL_+^1FF-Qw z(+aL|!d__u)^E2iyc7d(;8;+GiOv0^If8hH@Z@8ivTy&PfV1_qC}P$N-TGi(Qy4i@ zpgrfTiOToW6|A%VP}-w7b(73b@0*vWPWd#DF_5YEOV_Hb8k}072z@8v_qkzFo5nOR z)FIc{e+oZ6Y_vnyZ!Qm`I*$G}n}wBy!|U1OyxspETxazD%+=X}pPpA;8@m?{$F zLAFle51t;XyL=W*^19Lc=5X)CZ?Tr(1E(ilz2f{CQ(nVm^Ep6IQd+*d5^`T}Ae6!A zH9*Fql;yIsc&-m(ec_{niCGUHgmZ(nrwkoR(w^V<#Fg&T^ZDRtvs z4$buldIe84$N2<^h)HJyzm-@0)tC^0Lt9;zY~OPN#|(TC4M48QXqn?Z)v`_~l-5gS zg7&3aTTfFXHVkd3W0M=o0}L)o=V>%EPBtvG1flZ$-1cgoj|D+T=+c%to#rVH+=Mus zfR&i5@2N~jPql+`{pPzoLo&o*l zj$O%}G${3bst6Do>o=CSgjev}gPJtzv%8Gc9X_UoT;8@kd1^Z)_UM0eqEFc`AAyy( zDeDOgS}i{H@!uOpYHxGBLd9+~>~g?dQ#;@LcnR5E4=(dM85TJbbYMr=1!wfz7q`;d zG3RwRtzp+I7_Hc{$H{y>Js;jJA{Es>y=+zPN30XEZ<}Z39I>w#?(ou>uTH2N;|dJX z^ghqUt6et3)U9^)@P%)anpN{6 z-)!x~-Uq;+q$Y&*NUvXNwhi;AJf*E0EhDoZSXFjSD)@#L=`VeOH`$8^%+&c^Q`{zU z7!^OD-GZve>edA+V1A#5#&LOZ>=ojP$!TC(`7VU4kNFa`MGA1cn&D>p*^%GMtX1Nq zV=Pw@eyK!gb@+8tbkU|DpI2xND_7ukv1gw-&`4_%ORXv)m>Quqa%(YNT+e%@JlEah zxpsJO6STpAMFu3dN&A6TL|#c)*44P5IUtQHE+UTaca0O>2r1GaQS*CO%biuPkp0kD zvh4Zlok^h;k%h&m64Q~Pq~mUqrlEUCi%v&J`< zEM;(Z_bIny-*(R>EHl(BjkJ&ADNEni2f|O$R`|gns0Q zVh#&BHUoT|lIg6?q&4}LdKp}|-a2kJ)_U#j?*FX!dA#_#;pX=-9{aV!?^1!0+k|1* zGy@-Z?fF7zZ|gP7B&FcIcPZaY^yxyJIr&ijM5L$ns@iL8AphVL8ya<*DP?>$Eo%Fr7>-B@bkh%d$n`Lx7%F&8s{&=$YnC zoWy$gnjD9l7d=NUrSC2C{j@xS22FQ4-aa{jn<+RI;X`E{UfR!Rtjs?svQDpDKWa_n z)U3sX0RfCWC5b2(Nukpq#)G#2jOnWP1myim-X>QFkl|dTefiy zDAHx*A}b(gc(TS+nPuGv;S}}7ya|bsrbDVlp{iIUm2Slun*VD1Gb@V_MS4b8J$VQ+!RsbidCl)L88fGq$i1LzSW9@y)1mfDn0;-SQh1*btco zju9X9kMHJIA+5+`Si{@`oOY9(>rh{(bVq#_Z#;im{j{FQL)>;kKQC!{#kzvs&_YX7Kn|M*@)e=3*+r+{}E z)WeJVdb(?daLfn}#jg^$b=RS1i(UDS_}j-E&t3WaIJaNm`3y^Xi;S%+$J|3InKK(h zyWt+8*97u58qXIFl$uMDnTDjo8brHo!#nDtXt5m+`n1F8Du<)mkvR|IjFoOL@D}@l zr+7iF&|NN-sVx=z8K=$H-qlsaQ%)lf0*kOg*Ldu`Uudwr&wD;67(UKZeDcYOnJG}0 z7kZvAB*B5n2oz3Zu-HCm|6?xBNxc;oB-!g@>*}S^1P3>`)*ZpuNi31G!7q$I`+T)X zv!92kfGa;9)4lHewc);I2)EPL(K=u*clKls6C=LUGPqLB$w{DPrhRBoBxn@Nk|> z0vTH;Y~*JN@OzuG4u8gE-GMx!yrr;jk)tAIc10_XBmV#Y}I-4QSTq|x`cg$flCo? zXYrZPav_9*9JTKH%44y=P{@&fFYj~p&9S3=?q_(>y1lq|$Z>O;VQkmxyb~1_$iAa; zVwmQ*?ZZ+ebU_D({J-2QShU>R2~y|o38Db~R>-7>e%s7XoIg#@>_2Unc+l1qEkybj z^otbV^Pwt?MGt@gNcYKlM+m=$?!YN{LnF~bdVfCPl7jbmunUFZRI*n^`F7+dslJUy z>UvL?=BaP{!?HWy>y<9wYg$okvj{5YX;Iv-DVxz$Grqn-$PZCJEg z^M~-1X4_e-#Or%Xr*+>v+izd)&C89m*f3n1r2KIX(;=6u6%tt2H!#z9x0LwQ1+skg z@_x!la@MS>uAp4XbilhLa#9pNu)UoEILOoP=uV&M5c4Iqlg_aN3}t-aM-{kwRZQ2c?Dj zFD@j&ff&>q1HG_dH;9@2FnsFBWpu{68UY6p&HQp`*dZ?$2}e~<_Ai1&hzUe(0TL)| z-<}^N{-IoIq0L<0o@83#q4BMB_|M6NR!}X^TRuqk57nD5rSn#525KU=Tv!YL z4Q&HU0h*}cR6hM1X&4Kr38lI0a-=t84j=&(jhQT`Fy2T-u|Z8xSu3iCzG0dIr9d1{ z3Y%rb8)#_!#Ihv#DCL70fv9}6@}aq$r~vZ zDX0mvcf&{j6kGl|koiAq{_Dcf|Nm+ZQv^3imlLoFU}+XLlYRn!hcf{>IQeTkr3irk1RXxZ-@18u_Y4l4$^BI$P-9>pCrn-hf5kCGE)g8X*EuCmD&=x5Oyg^>FU{^35KzcJ}vTf zZAaT8R3{FG%`7nA_uthp)5(gevTsDvXQj#Z>2uQys}|4mv_0e54%Xr`3keq`B#utBP%-S;2`4ooeg>on zeXT%pTvt-oCI}z1wzvzgWvadn?D7bm5Ii}u1CJ@%cLM~fL@!AIx(70tf~ZrAV!ohX zO=z*P-q3{<7C&V^c8`?BdNksk;?Gizu11PEM#SpxF1pDvd20o95GF8jKIB_l#p^Ly z+)R%!(9w!nDj_Kw7zBYT~D-lJ}Hj~fuXDFH@`b2wzgMM}_=@HLCn z9J(owSL(!PjNA;&8$9_jOcvmNgyUpC#gsFZU?L1Bj+=-oy0JH`DF{>z@6NYv0*R5* zB6M%S@u*gCLb=+QtoVY-9i_NPr76KWlV}_g-gv@7xvo0HY6USCK`&anYVZP&(Ad;k z$@j=*9gWch!%t(fgU3f+luESK_BT|nuv=N3Vf@rb&W<#Yu z#p9CIaL{)y$f<@>b>A2`qUyo#PtV})-uliB04Q^wUQ_)Dlr8FT+c8vIfiRsLl!x=( zH=2!>H9d3k`{0?XZ@OeP1gvdVO&pfBqvZmhA)II1q8sf?8?-hh)FkI7Qr+mByRPz!!gVlQGPw&EIp( z{xuC)cC2BI45?LS3$stXUx@66DfQ`^REwG_JGG05Bu_F679N?4PGDiDCe`zhj@R!r zYiXp`n7GrKX95smL0l|ri}=2cGl!@tnTA5_EN0->-wLpN^LbfxlcUAp(v&R1!Zxqd z;x2~G)0zEDPn8Ja1@(fP+4nvgv)?zV_g^~EY$+?pJwB258-KcPrdN9GQlS<8r*^9fbF2vr;DN1RE+mkJ@Q=4`hW@!bJqHQ0$k|#L-0dC2ja`2pTR4*YM z-eqT7iFf=&Go&RX%MtE6c1PDb2cy*93mvg`@Tr$3!Z3Cw`4QU{*HpNH8k+8yK-bLD z8f%nod{vqA8&N=(FYlPcUw7{QoDK>ZJ_Z7jDe-!G=8E7R;^z1BkwZ3_#CiGl9Sz+V zXe{`OuKlV=MO|GKKJGWYvYR3O<)Z^MiJ9^OiPf`fGv96(`ad|QfLsgtuZKZL<=`NY zf+AEP^+pRKM172W>5Vasr{$iLKFeigoUNl+}l}v6as-1fTCsm z8|%*f0&1eZo(z;q{yjY6KX4SHm`&=9bn+vp32p88o4-R4Q2+`G2LdQT z3<0YBfs-6%ta(-SvkMeXgqFZ4?C#(GTMJ|;ivY5{Zwr2X{Do4;P+kDu z;V;x!xt)a7A#c9f0M!WuNzkpl)gKuVXw4Up-weVZ&<=u$7hgkQ{!0vc^1nsxXknj$ zOkst%o!OQ9IT~R9eeHT9qiYl{#1U?7L`hjUa%;;tQNQgA@14jq?PN5CE;W^q)M>UG z#5CWOV;)xDZqtN))vzM|aKwnxzshU?9+P*LvujS z1U*d!RD*Ar=ZWSwD`+P`R66fTc zA54?Yp+6jA(@JlWGIkDiUs4)9mikx7=j&yC{0E3c)w!I!Dop$vP9@(OFtHD|6B*KK+Xw?CkxFN#x( z+vU_}V{tZM>Hc)8n5A;9hni==$iBalWBd7OBJ4l404{gAZ52rKz6?UZ0-XsM2#N^< zmB7GmG~LeCfVB=JwiP-FC1KRc1}}Y>9Jml`daQDKVPFBy{0v!|A=5FX87CWij?@vQ zO5L!uqC|7z6HHspl2xd7A6-9f0a^@ha za-pUMf@Aeo$Usq*Fc(s20fLyxkF6q5H*gmvbQ}tbyC|^W74@s&%b3}8864h=zt`)` zZq5cYnMR3%`SQ|s{Z(}ga!)D2U5Iedt3j7>c5JiHfwTo;kL|Cx_~Kv{Ft-JrY&lYA z9lQoj4!g$Szfd9g$C5CtnX1D5Y2`q0nbGdIuKyH|9s?NYwz{a?5IRD(PE}5JHhL%5 z2&Z?+kHFkOYo>m1Ixf9H@Ck8Lco{`f+H6o^^+n!lyxP-*x)CPg*)ahd@N+{Sd18IK z@w6na|N2>`1M(Z2Xnvof1kA<|TyuL*4kgySM(uUu-?}sGqFd$ccc}6_yJ1i>c zrnsmLLuSJfj3kn=#?WR^{42T{tDqwKKAy7m{NEHc3JYz*Y(ElqB-QCU(AoYwppi>U z9J*Y^5sd3`a&9K@>?vO1-ETGCJBPA*u!5$uoVHS|WNMjYKIBQ~f-YBG<7cB4ORJAf z_vT+?lx77k9J1fZRji^YocbNX2D#eg60)$4!DQZVw>jku}*sE8FZ~UEBHN5^+K+c&Nn>c<75oDV6YLAIUNfv<5*P zg-SG2mtUvcviIMvk|=(?$!H*`F!usU%leq#}&h3~@m4gZ4qikYS zj+G77iKRXlWLWhjQnTbs(^0ZOq8R>$IfW)K2@KV=>8O7J7&&?@u=~TN5uTG_pTloS!SG4X%o>0ZDPc$bnnz zOFs8s;3FW62qh$tn522~zaR-52$C>Qj{Xl#^dIm6_(O#-*^Ry>NSK8|Qi{~?NpB%S z7YGXg`lDLj44Q}nB<&Stt&4xNFHo=o`~f0*diWF+6cDs=Z{P6;kOcboKBGM;+;EO$ zX4lod7ZDM;U44FA%|nJF_D?ux;@~Kxe#hJu7gBO@$@RuGMKQp`FUjY=O+tmg6g@h< zM~oL{`MQKc%2E~n{|HK={|!ny!k{Lh37~pZX@Umc6e9TpjDtWZ%ch>y`Yj#+2a#y7 zu(+&msWlrAoC=F_HK%{m2oVuT-{3|j6o0J^sGRs6NLn*FUQYgI27o^`xI{Lie_a{} zDYNF!DsU`)Z+N>3{(n{>IO_Y`Rp9uu3gWzV$!}Ky1%xgAtC`=w>8=+VBweuhH1`%; zLi{NXL*X!go0ly7ftI+ZquOr~pb!WeL3H=azU>YdBsI><(2{vGYuGT5I)xTe!r$Pz z|D)vpC58ePh)^5@fl5wj-&;OlWtu+$M#kAz4PD=A(HepPAi|FHqXr<8UZu8>o*cx= z?qj4Bce`R1CL<})Ah?F8LG;N0i3U9hH5T7**i`$?`(Y4&-iMmn-l}f1!0PGg!8$7z zKU@^iyz_+ED@+&mB2#ZYCR6XL=pL$+bj}obOM`9yx#yR!v6NvoFth`p8!iqJ;hxd( zK>JQuH6>0qcRDCsVoQ`;b%MfC6^9xDqL1js)=9}({fDlQHTEo~V;*_n8(5Ik!1n}v zs*uJ(o6HS;t$; zAMa7%brOl;u*}$z@<=#89Ge2n$IS6)>eLpr`Tr}E4x`uzH3U;k2#0bkp<+iD@zO^_ z)7SqpS~qKPUMy~OW={82C-@f)Kk(vVOw!$q_46>!grXDlp*=a#=CpFrMhcCe!5U5P zi!Un1n86oS5iE6D4W~|D|ZDO`xb1rkf9MaRjZ7?S71Zv#R4CoBy_3pCH^Lt1 zgRBxy=i2)%7;3>s4WAbrPs7bbxZ6&I=+KOQ$j*TqQb-436P+npz8PUg&xZx?%Ozt;gCzM1#r0I^#TI6P@xDIZj?$^5D z-{hhGQT~2`!U*jN4&u2EaEY%m0%}xFS>^8(4f?49(K!~f7-YT*)3e}6_rB=AuZ3v8`OlmKRTZBYE>8indnFGQHQ z6fbv4$h>unU!gzBASL(Nis!U6FA!}Xl!_nDy3$99WY*@dJy%H&1Om40+LPCLRwZS* zcxV*0XN#i5H3NNnI$W{{dKni!`3;sr7Vx6QUKB9nTa}_hGj<3q+Y9K02NEyw2p<1^ zQo+Oyo&^ZXs{;RmJra9y#gl%>6or&`u??VG@`OGuq&lGOJ=55#J_;82s`$em94%fU zch#L`%HexC9O!1$!9&A#@-v*|qmsdt%?B9ImN}mg!}}K&8-e9Y1rJZQ$fFgP6xtXi zcAwcf89HDNP94!QEC4YeEy-22HnbN}`seYm~(hr=GB4n;!LS61DUT7ND{^oPUD6PWOJ8lU~{X52_{P}|n z*Rg{CS9{+b)a3Rwtbm|`RFOzWR8){6Ra#I`K!vD?AiXKQNhgE|N>RE6lq%Ac7Li_( zfB|BmBQ+Euw9sN81qeyL`1@V&z5d>J=KJHF_nY}__)0EOMnY%qE-mazSm38cHF>!HH$=!Chs=elS8#KlieH{MI7so%M7P}(iu^5%!i@c#W13M( z)}p_=UrZB!^5-qSWtRW`OwIO+X#-QE21I|mv4OzP*pG+FaLjL}G5Q+Q@_*8EPyVg< z&0{7CmD-alzt8W&Nv45zd9Ct$u%rK{OOa{yEa>+fmn5^4WdFJAnZFkdx&5p`)(}O$ zsrKaU_J+UpY(WVfeb4is&A?b96=3fKdAtqT{^@4jNSu&>>_?*v0oVktQI7v?Z%TQL z$`zE53UAs>6ywJm7~dUl?ag1x|Ka>!0!8o1dmnF~Am=Ujnkar#5At`3I+z)q26^yX zll($`&~@_ZpW&Uii1OtG?N1MX8}tAoDNKFPPB)|7P&8<~pV*n)tx1UP)@}V;J)j84CHgft{2o6 z@GyC;^jUvLFTgsbQ_u3pah@Jy)8x5NeV9^*Eb}ml?Fb=JT()>}PK*(+#9t_;zqF zRQ9A?rd5+v)cx{=ebw}1*)BCPGU0xvf%lIa{%xI6del~DKCW|uGU3 zmvH>r#UX-whLAZq3XS22#V5U~%s(I&(iE1vNtjw?X5m&@9j7>$rFnW>TQ}MEvF>AK z9Yu?@Ugl)P79Yvp#%a&#*sKIrFB~DSV^S!^ zvKy__mn1oh+m(!RI8`$xXp6tQS5Nq0?5D$AdC)2j3)xE0QC=fqV`_tZ4gc%4(MC=Y z-JMSxJE4OoBVN%J4j;B_f9t$kwY`{hb)IDZ-XDuTl2^o?@~mjmV&=A0@X={VUIGkx z{!xn=@cU(RlC$%xrysa|TL|oPlSK2-F-+u#-uwmdLP@}AZ1<45 zy-6&hC66v8;rZ-Y)xB=5t?v(csyv3ycujo}yiebLn&*B@fmZ||+*UsgNoi|79P7c* zv9OFy!VJ7W<6bQRGqK!OpSF`=(3Fjidc2z8Ufb4hQ)G2&%{ay^p)^x$+4v%>v9T6n zW><4OsSsi#R8wzJ-uf|yQwPxwyOeOmadY8ps3LjAd9z<2tEW;t2`>aje2ZW#0tfw; ze(+@@uLF)ma!-l1QSs2%1Z#MBZ5=!ice7<@xoot?ebc$Ixy60+Ueb$}tZOkDew!pg z55%z#g)j)Q|FU`;uMZ-Vt0a879%#10VZSyR*DTHB2PH~bVKh}|ek^49zG)6=9TPVj zn(YO$Nr+x%e7A*pXFLO6^Q34^?wHYtM3mhi)vONY4&Gr`&8KA`clT}Z_g9eX zzgoz?cbdN4Qb{7O8X?7m>`<-GJnOtEN^Ps{+`6#Pz7{ z`xHgCujFfEd^egsM@p<-2cN^&Sc5|-L@Wq~OlC^TYOq&v1lQvIxt@sa`Lgsq7Nn3; ztjD;6Pr_^X?J)gj-FxC1OI2$wV#$e5y+O3N&4$7ko~3g+k3|%N4s}~1BF;0{tFx=M z;^_jR9ipc{e{}b2`pobed$$4~pNLw-BZJ-3$OMCf(|2TDP?piOG5o_22__dF;xg>+O*XLd+)LgxrkrVAWUuy1%wuyMIp!e$In;m*XeFtpB9xBGceJ|#P zeM(jWaNe-V!Og>c@UpRs@IY^dHYg=Ea)XTV)xZ3C*HM7e9Zcnrl2(S&R_@=jVRR(* z-a_kl7#UDNiwbV)tidIa*vi~U_36UZHwi$`0?y@{ZaM&gZWl4|X@-|n_hTJufD zp3_IZ4Xn$R)WQYa-SgZBrHi~k(`NG-Fn8+J79s|>Q+|;Bz1Nt3XUZg7#_j`oDGs1&_x5G!L38;JJHngKp;uMBHWBGm-=nAMjoyYGnR;+G&>l(!0+M;Nr5b^WHBTuXs26sJTY7T<=&^Z$ch7$ zg;^WreGh4Kn}JewC274gDj#3($h39!o+vVHlZ;9V*A}hw9a43(|1#^UvE{rYIp~qD z0OiEg(Zr-F=#O922ho1e-klxZIl=YF@J;+PP#t!X!aVR?5njQ~-#Rw7M7cUh9kANH0zI;0sU>j3{Kg}oj!CpL7(H5vZZ~rb)(a29B zZ@zYA^V-$04Ebq4aq6NtB~5fpW_uOgev+x1AM(tEawJ#U#mD3w#v-0Jc}#nTe|H`O zKH5z=;xy9~VL2c3^oAOG9ZQFGBBmnB?HhQhzM}(=eqf(R=jZd@0rOzXR?RfhL)g(9 zrZd-v9SiYaI~rqoIW}}`-PV^5RVn(zSPcQuAxfzV?3<(>E&Rubim*kowX5CQN0QsJ zu06AAA|Cqm16iU_jBW(wKH5YB>Bb`C8K?KB3<_0y5u~wY@bTyS1MC|RJY0wL?Pivz zT_Q<`CWfMG3(%%+^gM-A1JP)$Nl4GmC#5@s8?Z!}B7ZC`h-%S}%4jFJA z6r_ENF1AY<*&58VLsQNhDyc+FkEN40d~QzhCy>>aA530RVV-4)$0nR=ONo(2*ftIunt*K%~3Yu)oFQP(1pVIWBl5dKXT)FK*!7N}Eyj%R`}6@?&Xsr8CjLdEnAOL8 z&gS>`N%GmgZ$I{bqg1U#C!U_@OdFD|I!9IOgNirbmh1536q>EQD_hH#AYFU;Xv%s4 z?j!R3oqv2C&+T1Pz_aFdMJ>ur+@ibKuXYtpwuP%H%7_1Vv&GuMH&XrbvjzSqxct0r*I z_RW2?oa){e>mgn>bM?-kv2=)62S=Ft+O_%S}T`ad3Sq(tB#FxZ7G~eFWP07)Zl`##`=##fEy@G^cmx@A+=N@(H;^AuxsNhDgz~ zG|g93-zh#JT{AI@qwJp6;-*rIrtAIAJpt&fwVGaCbRJ{ZwL&8Fsx-ADeN=m6jbb*Q z-IZM8+^kD0se3Rss1%eJX7Us-752II%DDAqWFfxP{bB9dS-nTmqt8OjqWWlbxQSdRVlBJBVl z_s03=>-Nw3AV;T-!}SA~Jo$aFXc!aFsOm`HN-$Kf@yJ9-{%$*8MRvK(QJls@JhyS9 zVJf(m6FxhLK#hjfb@HkuX=GGT=lQi>Xw=~qxpxCGMp*s>%bD>;d|dAsqxm`kw(DwO zu(5EX8(8yzt5e={Z^WEZ%JS9RThjIs3mI4FV|n6PdBI2S3xT42!@k7PMz2BB&ks+H zTad5nNnS9$Ibh|zx4dRMW!v>^n0EIdcd(Aj3^wlcid@w4Ed*vo&;xk)pkaV#bKR-x z3R?0Bw*%_--DG#4wbp6iShAq(?bnF>J1Z&Y)?9ERzIw&o#+r=Zj{X#pfNjy`V2c%`%V$QM`t&X*JtxQj{F&^U2c6OGEzztTKR8^coqzQR zCw}{!*s@XDBtkAwNGWv5uWryDEW9nJ=Cbx=wB`DB6?ZGpcFDL*$`?9j`e|{E>lfJHHWrs5?pZ)EBB9=LAxAu!IvA zMdO>ZQ(7nU5N8*iTt*J`F!?L$!eKS3IvQqr_{>l-O6|H zVXB1dap7=VBUI3#hg-5CB-0#v|LIdMO;78?yNHo@cu#}&96;lYrYu!dYGI^R$}7RiX3sf|WL9fpzb; zx#-K|m~q`#&zOvSnyM3a@;U>hl$Y4*4VsJ@?vYwMf!PR@8XS3-oP~>~$#cG%rzq2_ zoJ&r6=!^lgX=m;1Kd@D-O-Xb~MO%8spRK}u?g^yO7;BT08MlUV)PFQ`sc))x%|Ana z50ej2_3X}3uT=edr+46US*A;8SK&pYb2^V4+h6sx2fN2BNa-dXNOdsiIF)>y`waVR zVcP59)yrW#3egu6TK3OT$|0qu=0W}5f`>##zm<%fDc$*YzPSfq%`-%`UMz}i@(3d7 zZeGa4KQM~Qr{HgCqNQTu1DhvEFTI(?8=!|fM4KiUMXnHil>rH(QtZ9Q@83A}GCagy z^=wScdh2xty=R@(JnQuHF7kNs`~m)X_Sh|ry2PPHrR>BEJmS0$73yo)^)gaJEn;P% z;l$c`H<}77s@@ru7Jj7(F98VN)yR<`%r{We7VP3`7NsWF5i1M#Bym1I;wc=5xQ4_Z z)d)#V-qsdRLGA>LdWj7yJ3H8$ly8u%ew_vIjC`6$mRIQoyRGwuI?>F`CEf7mp|tJL zrI{hZx0P;G{L~gK54~ZS``Ch8m4i6Bw!Vu;XMu%s-(ozbfQU0|L6$-a>sovqb=rfG z5p&SoqXFiihLn;%ud->za>`&GN6vp zwg6l09M-hmQLr7Mdmzd42)lzCP)Z0iWYTsFcYmen=GPKb(11)OrVjsBzHHnr0^7uV zeZ#EBjzQzpv%a7j}3aer0gUi?g!CVaud$Y_<6Y0Nj- zkYx77)~v(4!PT@W4QmIZ&0Tu;yDhZu*)sCw2pWdco&{a92{w0lNn;GXU?BU#a^A4& zZ}#P5W`Pn{7nLW3tWE1Ic(yPC{Y0zZAz8gjfUt4zT#Oye+>5!#^XqhlL2SDi<*AszVsA{>4JW4vfmxHq&3mSMb73&ux9`MT4? z$80D#fj-~JfE$8mStPcRg}b7OLVVo<2ViA&IGM)K`kskA`0_R*{Y_Qk67;_E=S zqGm|y_vp6#fJ!>!@dsA@8hKcz=~A};>7mAA9snR&O`$9tGdie52Vx06O{oJoF+u^x zH`#i9*KyZlWJV)wZZdQmJsofDL%+D|1|A7JlmR%rJhsUeE8y>dsV00v0h0`ey_2Vr z)#@jwcTKbjqaf$)g5J*IN@f9djXY-OQ$d;a;;}&X4P_Qo)v$3_N_m27BqextoD%E> znR&OYdYfJvV27&A80pxMfF;K@A&~i+-%gu&ie=u)D^nbS9Eh43w)Ra30;wtRjnLRf zsT$qS7S?i@KW_no8VyJZR>#CFytslm%-B%4iXEEvg5O(tDL0iHQL@v%GEJvVmkDeP zNFazBIRbvQved>hhJplW$m{;hI(-b zA9b;*G}B~JV)_CsquU?YxH-8ziVMsR7k2Xn({976d3K4u%MK?>XX zfC`NEYC1MBf*|t@JzhJXT@yy)xCCE7mw+xe72$AKEJLQY=a<3K;frNZd@eFKovND? zNB*|sTKXeX{#`br`#j{wP2~FAw`TJt;|o%g;*nd1Lt&nio8LC=-chGoQ`xsj6%^HF z91cNgBvR8EWB&a_6?{ex$}jD!SEPl9)l5`AqGh?;pHlqlA$j4hg>SMiP-_z-#6FJV zmBFs5PA6l0-0kiZ(;!@iXe^NNF6{G=$9EDG$;UXaJ5t{^bi?62)h24@Qx%WXqGvDq zi(tj3_&_5MFibbHx-4L9aNz;Ox22)!d@o3CZ7sVmqI|J%F>tCaN{p~8&@E7zWpQ^W ztbAuEc@zUNy%@CrW|OAbNaMi@xB$sKXW`&M5|N)pUz1xaXJQx+bjg%Ko1fc(`_LlB zY1zR=KsY@^i0S{ql7_ykLG;~q7`08Hyl5D7VJu;oU^PyvHd=%hIuiyZ?|&U0nWgx) zsA5xskn=F&rUG)qBWRKmXkVXGhg6^5LABOjsKYg_ZBGEJ);}9XEmStIm-aX^<88Sb z;j5vwh!ITFFsX>44=M+j9^_6yQYX(il$7|Tnbw;DydB{H@Y0BkS-cW=%r**@+B8z?>#!uA*t!yoM&@m zA*%s*_H#`ZkUh7yJlJAoRa22xYx}vjr}x~4SGieCkFk5P)XQy|jVK-uMAo$W5yTHh%`C(DLI>}9i@ZNH`V+-o zOsfs%L2I9m2QoHgAA6KzC8Mwy%v%5J>I3iZRz9pOC{?ZllL}`Q_j8SGB(-`{^4DAb$o%s1ifUPMId+HvQ)cHvO08wwVX( zwzrnLNEn{YcfIF!2Gta^|32@wtRH@e11ljkpFbuxLRFJgv_Ylz(FRx^w)(-5cPxR> zzJIOwfp_aq=++ii6c}CXjHth0R|8hm;Y!&TJI0ENs8{;h&0xrA1v82BbEpFoW1uhV z!^BBOc}T17r;G5ffsDwxrN`^`I?3@N6wxPh%fm{eaJ&wNeJ=~!eqKqv2d%0{04}T0 z4px+E<=9sNX3w5~Us8PipjA=w`Pjck{i~0wfZ3A>ue|pMD6^CG;C?RFi?5RYfMKFj zWcHv&S3FMsgXkxD{|7W`<=q#HQlhsvjkYXK_>ndVlS;b&8kO!tt|d;`Wl|M(>+XR4 zyI|(ym7M7xOaq*ZTG3?w3`?*|x^|Olu`BV@x+Spbc)GT(lC#G(-z34ShR&$mSI;oW z9I+HZP7x($Ql#6Ms)jBd%5sK+V?~H<$uIf=lYltb-7t<}^oPsRx8Y%@WtGGPdMlqC z@(m`@TgDrokEkg85+6NSYmgY=*r8yd%0w}trrY4&9m@eX17{cSYe6Q--ql?gS_B^Z zE!g?)>QYW*0WDI8EjH|xcydxf5&$xjc{wMnqA})|h^aqqC5Zuge3Ar{(k!*cq=360 z-Bi}0lHzW1k4d&*fnsrq$DPg%=SG?qIGk-QC85geS1DuSOuJMDfpuZjeZ$b26P!1a}w zSGl`Fc>5YP$jzjxS;UDd7u-_-*volU!wM+GM`|LgDAN{VElD1p3`3pX#PD(1fw0tJ z+G7qUl?p0E)_Jf8_x9pQq7K*M>}YaQ)iOsxH|b85ZfEm|%&{IZv6P?yUB>6H7Av$> z15vHiL_=rWH?R?;IhJl=q3Q{S9#hj*yIp}^;u-E3W-YO*ErfH*I(ZfHNmDB0&C9I8 z!r|lZ?&an_F?p9_O{?|!N7VExO>I5i)L^ZnYt#Dg}|QBuXu zn>@l*tJ^zf3k!HzJ1rX%e8f(R*06$i`;!r?f>xA>bBz+ebb?UF(v#uuD#7hvm<%tz zmX^E%-H38i5sweTbUH_1))yQ>CCrN*NtPQwb^z1Wz+hzK>D!Zo@tyE$iGgaytGoRT zU|JU{b3i;bM%HQ9>y56Umbl2{&elX-BuuwMcpu@e`$R5)=#HK}giP?DCor-5^zTqok$ zaKYCiC4O1ii~#>oCuVYTJpP2Ny19=4NAD$KjR1#w=jg}Fa@LJXb;X(Wfvp{=HU(_XB9Qkaw`u;z}9xZY2T6!6=sH2-YE;OC|Yp} zY0iy*+Y{5EQP_Q$B=YCN`5V?*^=~c8mB&}$1gPs+LK$cyYf(KL?0w1gdiKv%e+_3} zrm^S;Jg|bk$sx0Us}G`d2g%VI0nnQM1N(;_4}Eo_MJ;riAIil~0xAZ}wXAJwBb}=V zZtzMW9y z6L?ofwoo7W5N?Bi=Z?JHTi2JnzW2`s!dgiZ0H-s<$tu0hMtN^Sxb>H>fgXOm+Z$+$ zf9v{ST&~GAb#&)AJYXO|R&gS)$Gu@o4eOyZ`9#3exiIKcPvdk&d?zGpJg({1L;4su z1ZRg_N87I1i2|0R0!OZ`#dQZkzrR@5InymuVjt#k&%yGW_N}E^;pR#}kdJ1tue zJ_-G8<_@=U>B5A>3f^=PB${kO>DCT>caT@w)AN`owBuq? zv3SyH$Y^CE#bY}cm2^{wNzuov(T3Af&LiHS*{kUEVTy{9E*i{FAdHI)p zvw0ogdR^>(joS@5t0vuhT#JJ;Q|{sTlBsI$_h1&(+&I8p`*2or$f4xHnwYOnYst8U zR_e#VYdShdlG0u%O(}m`KbafpE+?E|=u{ZOJ0aYX%runT7IEb41l(tmQr#(tgaITy zVAEi|6E5Ne)#**H9ikkmf$gYx>P&j3a?4*OCARA;TgbkZ zpu6$Cc-XT@_Lz5)V1kH}Ri35Yn$ppPLmXD}%xDBl>9Y>&uOY*=)e*M6-=iFlr`L`Y zUH-{dmsU}iIFlU zo>NE8+VW+AO|2^Jiy_lHHye(|T^a)Aj*E#%*bVr=s@#mrLmeqy0zoA*PLoS-$l6!M zrvTP@qVPvLT-`!hjq3?cpPzL5Ye0aP;Bivj!9GwgkCDmv&X1Rd?z(D)E!|p;oXNNR z@Lp{xjQ(amqg~&Y#3c$47fGQhRK%Z1u!05jhZ|&P*-Wn|c%-z+(ZanjKqH|#F3yA_kbZDnx;W(5nJtQWo0LVOoMZSrZ^X8 zZz0)xbA)$QT_I@;?ExjGLAo#UJP|dzlX2qR=cXH<<@NRK`Pw_+qWX?v##jzo$P8=6 zu?}fFSl^TWwKtab@8=Rr2m_WPQu0?^7Q@t4j&qXPW2H`8zxua_yeI0|KDOAeYjw=u zI0>%AyEjbK`S#84ixsJHesSsFGo46fVL{yuUTM_Ha#~yNRz?C=&tY2ng{B!Ev^9)+ zc+v8&+dWs1RD3=icjlL!=9Nq}(R288xyZ$FGk`k$iJ@AKT%C`TX z>HI?-rcnJOJOAvgy)6Ieoqx3QKStso6Z+FT_{YKcf9QUlawXUA*~5Rv*x}r~ z9Myl^_p{@~C|KFF*dm zwwDDZC8@||2rPU0=Ob2Y3o}#^R&^)-uO2^U5C+xXTYCIgR7O%!VQ-Y9o!5~+W0;bP zqRb$RAF=w(U*U_ct9lMEg|hxkAhGS|iesiAMQP<%{tAq2^78@~I_^pR)$`Ak{_j+Z z61m|J(fexaD;}Q!WNh4$Yk+LTwiJ!uOnNtbA(Acj!t*ZuU$GNE6ldXHVv^|LaMtUk zL8*O`iW5Jw5G;T4iJi|VuZhB4>Pwt9>!vY*BQNI4{Ngm=V3gx{^L38=TLDOksQkRt z#XsRTkP3W-$>*YoBs-+_rD5InaK{or5t^m2*v;&YgX^`c%OYJbrv76HBm-**9#9L5Vs?;h>K|FwndrBYu5Fk;5`eW z%t-xS=X1Zdo8DPJ6>%FCLFiCWGnRQ^at<|AhYqIK?gsN*9jX_Y$n&%_X0U1jp?;5RNrL8VuuGuz z)F_y=NB6>yNCn% zP%#?xnY?4%a3S>|u}l?d#Ry!|mi<>s=d)+7*sWK<4qQr4M>`qfzoK4ekw2phf&yT8 zhGe8=T(W{`yAHMlOAef7wwTY=nruR zgh!3Bpm3up~;#O+nZI0YF z-R-al=^X8JuJTOR5o8`pi(>W1j2jP`yZybGp z|J-`0k7ivVF=)5Ivih#(uy%-%L&F*CI3cNDTW*Uj>YhUb4h(i$?O*UFAlDMJ*0li5 z@+v}Lnv9vb;#(-J*iM7b^9U%I1@-u$|Hzt|68W-c{=zlF&Ug;Nf=9g6G_aBtRViRt zy61lF$BR}k$gPa~Oc9d7!t8WKN`Sh8DRpH2Ko*pQlNZRKB}Ch2=VP+v_33p8@T9!K z>UdLqqqr>S_M-o4_+9&!Eywo!wH${&ch7||{!LrPJ)RyKOouA~6B_$#%fIGt&`?aA zL@Mi6yMern^Agd4f?Ph}byPwNx6VjyoFgXJ(r+<0l=h}kQ=qi<{ZdZD+R8KkK~S>Y zP}zy@ke|ezLjBlONd@wzi0_(Z)3N!>eT6-e45OB<8|^rtc#0|v0mPDb;`q=Ezd|va z_0h=60|C3Jh6364z;S+nrggI69|?iIEF;$)>hzz=w^!C`VHMIP9uHQu`i;O85FP^r zd;IdMqfyk1C$sJ^7G~1nx}N<6TxUC(P*%r5w|%&IHB(Jm@mgo}e2)Gz&4;_U9W)|O zp5)HX3By27*0n^u7X+NNiaT~GdkP|{OY1PzSgIL%6>PjRW@e8(^i}-ts*|~`y`rRM ziDf|m4G{(TJg=BIpYjDF`R+~1T1u7=R3gjZVAj}1#I z?c5r&FND;3`8qT}&iRDbhPe&2r|q0EraE`e#1G=>O1iWdV9m!i38%KjsU)!7`6E2^ zmLQ{ndh#vsv0ym(WHLG?6r#U)=1O;~-(C6s%;Dzgl-54br_EnUlYF72LtH|$#C~VO z(!l4fE>N;+(A2oK?LAVZy?6o^7kybGFOgY=Bun2}=?)OdoG$ZYz75y+9Nh^QH5ybL zB#uhcui9er5EC$Ht`+2$H(~Qq-gk`TW^3^L~O2Y@|A$vbrLe| zht=xTk0rw;>Y5{gq)^$0!zv4Ee&9e7CGsipUBJoF#@u+2kW36@_( zNTmLqmwQ?CrTor6Tz{IeHqywV50vWt)2HN$(fj<6{`C$eGDlXCD6sLT=hu3DzfI5U z=DEAnpvwQmb$#PC%IN_C1?-x O&-jw5LFGk<=>G!(du+M@ literal 0 HcmV?d00001 diff --git a/openmetadata-docs/images/v1.3/connectors/openmetadata/select-service.png b/openmetadata-docs/images/v1.3/connectors/openmetadata/select-service.png new file mode 100644 index 0000000000000000000000000000000000000000..a80a0b58bf2400af5f1701d4903f82a20edf330c GIT binary patch literal 65102 zcmeFYbx>SQw>}DlAi)VXcn0_2?izHE5Zv9}HE8fja19I&!8N#BAUFhfcMX0g?>Xm7 z^3^?m+*|iob?Vlsy^EUKtlhhN^;+G}v)1YlAVq1k7lbchU|`VR$VjNbz&r!Mz`!XW zAwZw#Mm7|~z>vedkq}k$fIVpObx_;R*dEHz_O@1Ox^C01RXZf(KDb@dZb{H)(BHd# z@19%>VCR?m)UDnbRY}uiG8;_L*NuE+@cf&>C;!cmI+b#q~Bm_e9Qfw0)*f88LH3amI7__rUIBJP5FO&Kc#>AFfgFX68Hh`~?6p)%WV z0^nu-xv^6NB*Cka)(V`(weH5-8X7PzS$%%)=*(A4>Ech@_Unh=?_l9Uz6)sA1GgbW zoZ3hpV_vG!d9A}lVIwPo>B1{gBc>mWE%l?H-<|q(E%=<&A*ZQ(Fr};ed$C+)#$-T} z`@*!8byZ_ySdFP*7fQ;dT+7bmYIO*KMM@H7)i(2xK~b z+k%h%^e=t1k7HNRve~UPcndozWws-Gk_||0EnO=MGmpe(L$aS?J@k0?x*Si~nD~1s z33`9Vihg#^fwNz8>XknG>BGUYu0&4)d;j8o@6o5rI+WpHN2^9x-=Np_)dOjS z{`X*hXYMPr!=58~?ghjbV|+A1cvh%aklP8PnmI>Njlvko|4{L-zwIC3x5(LIx@I%t z!3dXjwurJyi8|^jg9Bn}*C7~$lA+gIp6P8x zOK3+8UT}F##lsZRF)Q6atcJLvN1r7~uL!AN;o*vNgFLDTq9Zgu!p2uD(z6WmR6#Vnii)~qDZ1FrX8|2&lP_eOCt55yO zhaQEb3MS#nynf=x(MtG$rK=Y>q~9g4_N5)RCcG{fY!mm3*SEpk+Usgv?2ygo&% znZ{X@@uTD0|GwTmFpd<6ZP5_^ay;>N-|=w+IX?7^Y`)9`Td)_C&hEDxiHKoq#Px#; z;0I{Km_FWk9ues1`g90>+V3?m<;NH#d?64(;*qp%)O0Fk50wH>trD17;M-EGlAe1K&d0DCu8G5n*@2RwR`%RZP;=2Da&@w{;Tt&$vtg)W@OakO6C2l7oMAbU zfCH{1>SKT?ft#)^(HHoy)_)e@7XG~8sbcSf=*Zs9uS#u|wanfr(%n+cU_k_7)rcLt zd`k%fz45_^TpP zGkQv+aj49v;BXrC=QW>O^gcCo7ljy!`5-B5eiY0Inaz5O)I!8UqW^H@uVKQm_ON)T zJZpI(MkHv7cCo%@ZKgghy7Z+C5~mze%5E{~62iFFI%a2J@ETz2JhIRF{?fz0o=BSh zS-S^k%)#r3RCJtHE5gOm!IFtQGk&`>&`SXRv8VEFzL4!8Et{J6T{O_L6wbbBk;n06 z0FWT&vzXN9U{yC}CwJ=w6ZpQ2ptW#i>LUH2mU*;m=9F7^Eba?s4#6eapj#W>pq4o; z3P}?SIF)D#TwJf#>>6pNBq?zA58oyM8~0Mx>EQmb)@7wtj@0)32DUThC2HL{7LZwx(e z^)JT&i|;$-H9xV=J3gQuvQx6b^oI|lr*KVKX?)6_`{YWrRZo90Sxd3@y+!GYfr4u- zl2|pr+teB&ns1d*zFrktNMP<)`%c#zh_cRUXiz5CDn}xlPMsE5d}0xMeXi5sie-cY z7<6;9;9wlj_+b>qIBfri*~$DeJF`8;{l46FjtAx{`7UHvnArH7<^7PEpRK}oE=B$5 zWs?xMrAb%3N?R#PV;|Hd)~uLBVbQ7i}kjUHYe{mhwow6>_SK4C16y_ zDP|cYZ7im`9R(DxC%Qz|ukI`K9#s0nb~xZ1F#&c5f|?|$p*TANvFRjpi#jPnA5((P zGqj|IcO)-Q7qjW!^p2sdyA@85<`Cfl1}8HCHld1OwB3c!X&9Itp;()TqZWDpIcMUj z+o)U-#1ZrT$azS#iAaWeyeE8L>~RbxjG=!Y$dU3TZ^A zN~A6BhxuG;=*fimBX#BN9(x3;VA;$FM7(><197Bn3w_gnnt%ds1^*9~68}TkqO5GcnCwfN zvMOq0`)NCqqru*k+Y_Ig$JA?ZM#_i9v+xPslzt&yGOzAE7c#X-S!q>#dM#5d(>^Wb zpC$4pN6H2IreUYBqg7 zPTq7by|?XBD{?TT;_?lq_Q1%Kz5=SIb;jS6h@}rqNRz^Ra}%&p#|8DqG(!DQ0dQz( zb3v!tm6)!a{i7^UB9vpqEB$8sTgBo5d*Mt-Y37p#YwEKV_dhg<35O>cI%Sc2 zUaph@2e%$t^v3Mvy5rBcHfc5^JN3NiCCi)m-;CFg$T79njktIwSe-R*m@{mBQ&nr$ zQ5Lw3iGf^Z5za!c-#H&|CJlM|+@3h+11tM29YX0h`)G`}g>%T;{zM6w;U)btNN@vN z8xJpt{i`*6QSGXFdOd#Qw*YwT_b`%DV!0m_KmH4sLVt*?VOD6GSSTsVrvKcf`wPIw z1q{R6-~qV2rJaATLH|H^Wa$7P#j1)_*uPNtKM?T;JS|AP-iUx@#Ih5yvv|9ou_ zCzCxDE(hx61<3gc-+!XRq!Fk57uO5qd0Fw+gu~@}y~mwJ%Gdwr<~v999s|E|jN^cJ zzTPCWhBNn()7h7PUr{W}I1U(GKQAM6*Vz*~b6GWA{y_6$F_@9XAFmn&8*P}IcVYk5 z%xh`5NNP3-pGo(t;jtE8ysQJs8*S|hvpf{$cRwFSRqsHc;|xe84_-MRQj#^wi?5K$ zF=eD{1nw6Ql1LQ+fLm?Wa!6EodfqGJ>*qBrTZNq_{4d;1+I{P|0dZ^yzg<&91ul}F zP4BVGoFHOG?el;ej*UXl{;DmH))Z;x7290QSv}Y4s~PdxE5o~wo#=0cKJ_Ev%7UXq zAu{fHhC^hjV!Zn;%^17P_*|JTGOa3#!xK3}i2re%a;j~=2M3yEteJtf+baQO z1oP(05T2ht2yEjs2E(6N2LVRPMt+avFHCc#i}q=)qTZya{?Lmjb2Od^)zlz7xXxUH zOWHRu$vEjcH$-<*oNgyrJE)vm*$G8G;9HJa8W*tfc0zM1p2UiXn#qw5I!mH37e~1A zrN5c}&bhHf>>Z!Zkll|oN4(BOzC!(bxuGNpV0}?M@K^S!cFD{`7ta0bjzXgzB=D+n!{^v6jCoI{46_>Wz#9r%hTRU(7TKiDnQ|i*~VF@&x$N1i>wX z5j|m^*7>16bZD5C?HioLO)L8Du64!k7qK3*n;($b#H05Nf{2`<&*59l>**-CvYnDp z92}~DwhX_1kc~7oMOctl^R{B#!pfyf+c18MO3G#?ImufJ?E+6Ta$i^Ug2i{=a0z7O0SD8 zt@(#Z&I!yF@BynMM%{FC{r%Kmy7c_opU*_%1M`mxH@y?kn;En-400o)cHO_Nvh)fW z)3{Un5BJ3%kHM?GJmV&iG8;5yZ6v~vI*+XR9Hsg(Uc)6D7mH)`!cHo41*-qEaDv z8Er93M%Z{;hlM2zC2HOxV*dBi7$*VG7#TGJ%VP3HvL;DB<>QQwL~rHyN#~p&Y3Hr^ zJ|+axFN_F4d@;8*RL!y5bBxN4f8DRWLf} z8n}S7_@c_wb`I_HrR?&04~7NPbThd$r#0r10VTYJBsfU+~? z@7;A@q=3a1Lm-jN?(S!f&9h~^b^NLeOk(Kj+aaX0Gv!CteXt%NFFtPcfsg)K&6D9L zLrP<2(<$YI=(f{f|Ez8u|9DMMHYmr2v4`6@<0tZ z>3XBrNF>_6Qw~qjd=2Z7MXdwbAHyO1H5|AmUw0UD`Zg8#_u;`+z3A!14I%9&pbzuK z{e{-Yl^LHH$rZI5pFf}Di|BuGUpSChpmZo+*=i=T+uT%f)P7NbUbK(YscIP?($(wi zy8aBmg5|;m=*uBWhCGTgP*C{{rQ4YD+2BRi?!G7t3v=_wKvb+ZqxhoInihgTa$TCE zt=T4@PfQPbEomXnsNPtN7x3(^&XD!wqY*iBa4@|Q{_I!WY4(nXWzNuaZ`ycHCa7mN z-!Z22#o$g)s1FAnh5qe(`xcN%jL}x}7ZK0S*$#%i=9-({FEf8pDJeK-Yyg}=H+M{D zF@mYu`p{Ote$<4d)KU+Q9 zFB8?x$om>*HL(DI*`Jf*{XC$;gk#BxO=4OlOt#0f>ZthKrnoD4^uhV&le>qLx&(^t zl$vlFfni%p~Ke?UFGj{dSkEtwdneIrQT~GCzC%!R5FD(>#g- zbt2hOBgedBF~7G>d}pKTG0nr)6{aP-j#bWd{0^{33Cy`qIWU=a3ChROa0l3Sw>Y(N z$Dz?jVm7SlG;Nbhdc|Aq4dII_HfF5mg;GBzrwwgSYI{Xx0=zFc8PhN~o?4KE{5*yx`8O-cb|4jHQPr&FI}21=1tmwHK;b<4wb_g?+yYpPI+TNr zgU1;J&X1T}tttW3z8vzxo|%sO1?PX@n2*uzIT5HScC{AeZ-kq-Mm|?MIdAI@GVJ3; zh2+>UY&K`>7sRM^Vm00Cu{48QE_VGiqcul365h_dNrwNR7gSW*IzL$8pPz1tz}RD! z$5vumy5$4wsO=iE`J0oFVelPdru!vYtAj(5q_Cq(jKRE;J$Rr!qn%@!y;f*`IFskKgq<)|YK;K}%IkcN+xy(9JDU+JF%T z^=F_rL}bSIl4-t5DdrTKZmRHr_YhPejQ zPw3QEWVN4ocuaOO(i{0E{pS;ya_PI2@wM@npZhg#iF1Py%>rLh;6B|Cz}UIP7#lgs zdJfecu*bT9EOI4Y8pcJ+v@c|X!4Y6&a!geISMych1;$0QvCO)euq%R~ZSm8NhrUDx zneebOyxICB@68sYV&i+A38Xg)q012;B>28MS)@_ZkOjXbC6W)_xamXhtFu^4qqZIP zCJ(-8yoWEvT)DIe!5*TdhmDAm7}kNiu_aEyJTcyg%%b<)I0R;$PzRL&DMNm z*#-xgjvN$x#z$1JB~Wc^QS~@|fF|oGl-ZNW{e~zkFD;d-mW;|+c%_B)ZGKJ4Yk#;i z87<$rPo?$A`*I^=-|@A5p!WYQ3#Pp9rg}Z)gDCWE*0cBuJwj=qiQ%=5zEcb{8;_|~ z*@Z?XJP?^c8WzvW38Cl7znka$o5mX`-aRwF@`%|P~M zJ9-h=^%F5+ezj@PL=?~2_SyM^G5d^{MyBm2TZ1bugT6~1eIkKa_%%srU-e8Z-@P)} zb}4!~QE|2?MtGV?eGq11142^f+QkawhE3A?X0K)uwx$vAyab(Yf&99P(8T3xcPS-U zN#|#V#DEDcvQhww!b_nLRPql{yOYl`VT&(>#y@h5pz$!I6RbKna4yNOnGs@hUPZMG zgfheS{W0c`HS20nyH*^Q$G$O5XX_zbm&@s9iw~wW@X0~*k79YHDpT+&9I94IS!(eV zDd_S+h<#L%7U0Mso{2*xW}Hu>Fu=NA?mNVL|4OK-TFkphw%y}eZEIK?i?CP!F44Ig z;g@rv(6sM6LO>VV19fVOe)EcBPzwc5q7oV{#rt$ zbk0f}cnCNt7cVFzkj=3(3nke8FmIlqhZ7&!jD#ith2WY=4~^f*jR2vVDD|PNh^usqKUMs4rP;Eo4q7zhmtuR^@iyd zLPehBp970CAPKb+dtGC zq;-DO=$Cb3Y+>RPTK)dS^$H?$T2AKv;E4H5zh-S;dmfKejyA$^;mga+eD}rIKG|#w zkpTG6+ zT3422Kg-eyEqEFw!C0?Jk%t$vJ;g4bw{3x0mmc5_@4yPW;nAHBx;%4#~eyD0sQ@V=W?5&qprw^rlD* zMUK_eFVQ>>w)u)}qy{YL_l0Piw=*|>(>1Hs-h7VSy*KVpv+y(D)8gDunU}n`xpcJC zP68MGmEKbnK3T>1;~M`gIU<$3LA34XoemY$jQ%+{f@jLV9i)dk>_Lk0Q3MAr=pFcS z=SAs5ha$0%?~5>fKlFVvjp%J5s~8mzp{6pR^$)Z37p`lFtBtHYWa4$1>m}bh_Ym3Q zT-S&dTPgw8N##=Nc@G)<`f{A9ht2o4c_PMgN(Pwk^mQX~W9@oJAO#M4JBk9=KVOvk zJrW;$gvaG)3Sx zpFg+^w&RnD0?-^B;e6YIiDkisZd!Rj;c~A&<%<%8W*1II`p{y&IOl9(pT3p?#J-p! zlC9>*2Z9%9l*tmk!T6E|lTN%eUqQme^ho64+2vp33V!gaAH8iMIqIntuRpEL;xhbR z$wl16Wffrr)5k{S$as+?5Pmyk+kYo^jMZJipx%9j*kWOMAf~5>UpD0m+&&Y@eb}x+Qs<(CF^iz$iMWk)R~vIEeI_Z$9(%ta=uV{XO*82obX>x`aX;4nPGIjybm>lz zpp$%rY+oue`?1-%rGekj!8_)XXk^1o3_I>AfUJ9DC|~huh1W*El`zbzQxuIWv&(Q( z_(q)P-jTi2oMU~V!}-KTzjHNczH-qQDU40R0GsHZZ5P6tU= zG?h0Hah@51GM|%Px{W-OzKid` zPoZ;TBZ7{&i=pUs*HhMvqwuY;?gW3vDP!%)iq5?^~QHYjXb7!H|%vy1T(7Z zya)G7K^j*-on^Q6E!QW?4n1#eWGlm?WWXyy+f1u&oIWiGlRp#oAEQ-IEc`#G3eSBg zm~a6bVcKOU7d$O0Ka~|CKzUbS&zN(wr@^2qn@?W9Od%tJ@}uIjOr9nDM{$D_eZZzi zC4eWJk+BuKpN2#iBUy( z(p}k9cnpw~Q$W=7aNGiuRrj|$aO?e4ekp{sOAc^H&Oj#UVtt^xP zG@AB0P8P?i7#Z1b%$vQ7T}IWv^i6o5JC^C~?e`WPe`&vcCwZ~cDnXXr@YS7n`l*WV zc9|NmIl4xocCQtSyUgC(L}<-zCv{|*qNN@!Zn^I`OG~L}-y4KDg`QIxH2Dx#P`Q?- z$@6YGx+Afq|KSn3p1i#jHN>h`6lZHlk$}%VG$C*EO^ma0s=Nxk?7y82I!d%k_i`0f z7w|%wN|j3Fs=E&y?q{-ApG(lm8p;wZc!A%zl*rOI_f7T0#l7p?5pSBvj{Mug(YF~@ zW-j5+x=Jmi>0X{pi4WxOQU-J7xjA&^HNwAB6Zcxv`s~nGB?^Z*4`!uUnS9ZT-j30e zJpYWh{!D3IjsLZV{|o*Zrmn4eldlP0)dqEK{8EO73uTX+R;42X1(V7TBNG~rKMnh0 zxk5Z0hh9JlBSU<;npr7^r#I2c?SO5wL+MM&FSiz*NJ9Nq5{em)5@LK0(f!JHGNF>8 zE?4hJjw8cm`DRL%4iQIV)3cU(S=Y&S1NDgcr$f9N#f67l=v>^^^S&W8=-G726?|1D zBIMoaflcAV-76gm_;6s|abu`pVq+`r+5g~((_%_cdwOjU>(`2etzd$xYu7QeyvOQd zg9@RUnLD|B&;_ezT0wt^FM{t-Df2&|7WxUeDiRH2Wo6Cu#J=WaZ;Qpa;$8_Eg0u_l zaW`}h!&4d@RP?#|d(jYD^se@iO-k{7o5+S)4ms4wcXuN1tHw-K!`cf+U0bLgqL6wS zj}`iD6B-g4ik{%4udcYiD*zt9NHPuK*XbpfE(SM^h2Sp7DJS~qSg9TB6pKm5DhP{` zoGMGZQD*kX7@m(^YTU&LQCgWi3+9(m=AnnYDp*05F}KU-a~1xXq6WXA^;(ZeIVjRG zqrvr6d0pjj!Q~ah+#_}!%LLzwiu7D|I45ZMxU?r|kamFaP|;2-nQ#0yJgb54@VL)r zrtWQ)0&(X|+^L8MVSKo|?&LmIH@9GK|E>zF2jSxpL5QS>_K0@=q5xP6VHJF79g0QO zV&_NA{12~MoJHDei{n2>o4t_Ug^>%ZhsAho#)>r$U5*u44lrj~n29N|W;5Z~*cV1l zQXXuZ4JdyTro(r_{$(*)!Gu;I=p-5ya%Du6IXV+X>qc}E-oagT-SGvjzi2`tRERHo zJNF1(-4fXGZs|o5Gc_tu+z1&N=0V_46$|xy+S0rZU5`Qv zww7w<0x1_A;6dRLAU2F2Ce#?Cw0F+cy6ae4SL0E{ghr0(%LD&7+ILge3rXG2OnyvZ z`^rJluGW>efugY z&pjp^%-R&iY#CizSJOu?@%Stndd=+Id##y#+&rx3PC>e%w_QDGmxMj_dgQmfPA*

S?XYDn*Z!%ht8sfF__j#^fIu}*?GHKLi z)YS{;Ufu?ZBast?cbj87&C}1HBunZrwWq}wq%zHM#Jj-uwLT5v3uk^5nLmA>YMjK4 zKY)A0f?C9m%M6AOyiV01n**&>fZfY~B{zQ5>TsCd`{Mn>j| z(eV|!0M!IIUdThknW+Q`5?jpq$0b0(Zdao`Zybxt{W@l3tcOIf?k026&~U@)xvqMP5= z^SJ|H2MRMw*)Rn89qSR>m@;{)19d#Fpj zTAqKmq9EJ@?Cp%L_eFi<+AWvNp6NF3FvaERI*@ifs^MOaPF$)uK!Fd&xu+>CKyG zTp6z!sb}^)P9$s5XEM3suPq~~?jQQ+Cj!bJXnQ>AO%2mkwBTuM1nYziiu+ifkP* zvVl2nSt`7JJVy{n{tf*D zX)?CqXn1NU#_e`1ed={8)u_cQ+hl+;CLJ9#;TlvVj`c~U8KXvd$&XI4_I*VlX(8Ey z`JDbZ&*66%W(GY@_|rmr@2$m|6i5~-@<`P+3qT^uuP11D-fSIa4I9W2~;Ja?QF}>rT zmeXA_o4|=bBqkv@$Jbo`T_ID0@g2DRJQkSChuako zW1AXv@-N2`z?I}#3U3mB=AvJ=z?E>@^cjlZzsUY5-tS=$vbW8x<8d2(B2MyEcqyiK zXw%2H+Z8kEsx0>Am+ka4e34HYf2=Eg3`I%vpC`U;o8Mq-qo#%K)9?xl-|Un+%`c@- z?zdE^auVaNA0x(A89bpSb zYW_CJj}mcA|GpDch}|g^>6Rpni)T!7Rl3-5)(_jCa8hv z&Ha6T2-^YaU!q88azJslXthCT@c)k_Tb#%lcAQY##RL9r6(yU*m;=0VcA=-xUot#t z0T(xU$8DoYRNzK*#p6E#tM=U;n9)haOg9qsRMQ>Gsc?6=KlG zFQ^h<{oT(0Q)%)*Nhy|ySmK@#>b6Y-Cftj9NPZ2%Vd{@125H9&pO-deJs%lPnQ&+dGzW_~?C|@QSFU#WB`4k>kd~kDzt!t?J{8{_p zMmaM9VP;xvR!1(k`cn(7@LDf5)3oi6kP{Dz$k|6Gy?Yz?c!a-Nj}CxH_KGH<+%bEv zlAPq!7a@e$+jfdG)PDwH!8nSMykc8ZlMdE|RwN{cUg&Q?@`q9Ewb5c)PcU_MJx}DN za5nxZ8-{ASB11$@4&`XMjclTf_!FD6C$F9faVcBqI<$s|*`#?M{xb>wQh!GX*JVRJ zJsB^J^$Lltbfpme$-OAfq#{|+&KG)-kr)Li->Z9frfi49L=7H$v_y9jvVBKCwr^oi zi<(7dJJ~TGy!Pp?LP3g}s|oBp{mh=RV^lkwsZ4=(=XEzPDCnO13{={aPrO?u+CemD z(RY3qr*9QL#ISj;JTV_*MCZJL;xV#CzTND=*4kXin-T;WYy{uOtKi*df=L3F#vu;& ziCBfUv+h;IEtP9rUwdr-u$4e63Wx0x5}hVDCbI`8`N40S2?oHf*tK^$8rOA*B4{SL z;`pyx3hLWy4N>X@QUY~pB9-;DV%9!2#X)Ppv=vK$qU|+VOYdvKxL+NYA|Be3=E;9$ zAmOc@D@OU$UpU~_k)R8C{dUiTkpz(D4AR&PEGWkGOjCfMLpXVoGee^mZ)okUq*dR( za>{%ou2o9+?$PML`Ab*F89?3F1;=1$Fzi>+n#bJSYGZn=?k;m~n=+&fSJ_7|?BarR z_9B8l;y&*#?kd@m!{jsOybck#JUmIpuVUre=Ae0dQATb#}higx@9?zv|H%|C(lm} z!m9)+eC$IqyDssvusCK49H1XCV!(wSpNk&C9qNTE%-QkTrm5nv)bLebE}~~XAG98? z<5KL^qzd@H`}l97e<`6KXeN0+H?~2EY}H&8ip2~mb>Rd$FnCRB_w|fo6Lw2>!C&x3@yr& zmK#a{qXP~>OJorqyhx_aV!IXlC~bk36aZny*GzE7k{uYjvrbz1EJgDG#1|BYNqFl! z=UyoUbc7aSH@lG8YoD(`9xjqo3o*%P=Xra1p(H$`<@y>hagX^#JOd1ncwCGE`LN%A z@cH$P#eEHdnU;ioU`87k;%-@31zi)&rb5e!bFo^Ki8J@_C6FgJAUvX~sw&R7$`*$t z`xzeBKhp!6F#Z5!$AA=%_it2mA(-jMgUc3(Ke1h7a|X9eBUw*(0briKgJ z-psCPnG;lw<%=quYifToI^#7dVIm{ay#q0>sn%U$k>mvgMpDOg!cm&MYBmhXh$kJL zh6pqwSTe9z?E82WYt0hW|GCko%Y1HZ1aR9gsV!9rF28g7fI7p4gPsiHcq6S%lmg;g z%->Z!`#cvz7@x#lG!ZUOa@81ZQW)Dj{4?59peHha+{xoB1eJ3)vX0{3PP&5{-_4gk zM}j19*7b)=*_k)#ro~4fbygSrWXKs?bn)&YYN@x}bkzo;y%@qnwXt&O`D(;J{-Bh?POFh2u9TxTChdU$@C z5Kfbf;*;&iMlwn4aNmE8tYldXVe%h6YDnWdaDEAZ**lMGb)q=CP>y`VB5-UcI%kZi zY^O7nZLTn_*bh89I2(Og*qnz5ZXJ(C(IUQmzbCf1JD`VSp4WBR1y>`7xU^AirzQ08 z?yBRmllj1lv8x8Jhn9__O?Q_^qd2;5$FE@fvlJG3b@h33ExM&PXBLbMF%F>D#v$!b z*EF!PK|Ry!XK0k@#Xwmp5{zJq%`e*L(ZMiWXTYC~X;8GRA_*zC=@t*ICDj(&C(C!A z-DYIVbtYL2LctKM#lVk>Eu8Wl6ZReg*Sn&Pb7YN^F209wM~TCU-2L#K{aH~b)RPd( zsU6!1iA;o6(bbSGI`5*)U-BuSY@1$lv~h1|FP!A+eCxHVL4MxEF6P)ky2Qa01G@Y# zQ2fs}Dx(F5jt6iyE@QmsVjd*D9BDrB7{{>hUdU{mGB3KY5z=78)|APo?C}-7wDJ4i zYge}pesxOp+sflaK4Qt}fx#vpjCg*}-!I%t`wKWzsb2h1$pp1T>|l}ca8&!ZLeP3; z3t^5V@{8R+N{^u!gYs2iE*9WTGEu0+U-di4Z=kgTuO_4l{@gTW+W#Y|PUQ9X$!6p? z24fOZ;^NQVcNmC?QMXpMW1tSjc_utjNzjSDh6(`hs0j&uU|J9-pj7HShpJSr{1^85R~6?eFw`d$<7=X#ajuz9IU@81uf{_YHmQj~mQg z>*x@L4-)0y%R~DEIQ_EQFwFlF4I8b1g7_(!ykC_kf5Lo97TDc$xn=)s7AOnAL6d%s z7m>RC`zSqBIG`H&KV1oG_*DqMilY96=YbnE2GYw}K#H4rWQxD>{19|`V1yWn{`~?h z1N3nz$O!OnkB8u~rA(q?VvHOdL_9GtFdVTOH*u=T|LQFxGi+ZZswez(QUY#+me%|I z87qeeEvDkud$_zT{f-QN+eo>{+J773H`(?vAf&AggX*1-xU=&_4LC9|&`;8XgNt%V zwM5FHz7gQwQoY4kCE`HfE-<64y(=ocf#pZ>*C?r8t!RK*d3irnm1M0eYFo>{);q2M za!5pJcs)lL_4xvqM@20Ftm>UND5dG-u!a0nk|G5xC>=P!V|-ES;wk$Rdo#WXd<@E3I>Bot31Igv({8DHWtC9+?@lvU!37nsh@_N71q*+W#H1QoIEnvkUn}C%w9;yl9W14Q0Z}#HV?CEDnv3x z8>1GBhLk_x(aS}E&Ni7ixUDDhYP=$b${^W|fZ<`ZLMA3L0&rzB~_6f43MI?d=8(P786`x8a7b_<*ffrzik zJ0bA0{@%H;rpmz)<+a#7p_e$a<~2M{go#ydIS1++Uy+Y()e_@#j>NRGxo3SjfR%Q~bT8$?(F(-~uu-NDA+BjHZLbF^fvmThi@l$=$02 zXOuP7OOay6tx2@{R1P;1b(5N&das$(^v0y)(YcME21TzNR67kq$P2*oizFCFO^os)QSUfy9&$&h57QmWFnO;)c7zg z1_%ief-~$_ZV4A~jSgW|bXLngo=7J2MN3fO!cm{Ha;bKX6i8bBO&-YK{$to2w4Ptp zw5{d7E%e8p4IG=hxV!f-vQGqx*ptYLXIR!kG_~l?FACZcF7d{B6R;vIrT9VGHmS^S zO?A{u-5w!I&?nNv;a+-rl^+&RN)G0wd z+P1XI6;nL;EGr?0P1f(CK$6Zeky|2$08p&PMX{G4sN(LTY-(bl{EUbr?@!ctJI-mv(!k?CfTM6|~hsZdmZ>BBUpa0k6p4V$h26o~?m3oVGB zj|P4H9tj#FU^PTIxn5OtW;j|(0=m+U6)1)PtnSRQ0qYiJjAD)oVTLZxpc{g2_A>O` zzyQ~DkxiPURFv0B*C7kk%=2Q(GX8}8^3*gms1L9@8nR27 zZiPRlw-xjEO2W09Q0XT5jeFwW!xJ;HQIefOyc6`}8H^y!rD~p~UyveM+V0_=Jv#Pwhy2dnQZ~btjrf^Fy5%(6GA%%w>mjEs|_$i_p||0EXc3 zTUv0>DXFh=J+(pEbaoemLCA+vPFCjRdUEm_S0eVla^cXQct=+ImT*D!3A;9V+?@XT zo0^$Ec`+Ixcwr2Ug(5*~<%6oesp@old{g)EeRmGpq%%Z=v66PZ7k{!cmH2>l$y099uSTQz=^7wTDiy3Ds~`JQj#2Q;R3R(;@p6Z&aptC_8kSI`$E0)Nlb#f2*Oyw_ z=>8T~VH|+67LxmKbR(h$hfKki<7GcfH&Gxf-1a{E)uA*UYTs5Xp!?@eAs;s8{>BQA zly4|mR`^@y!N&l5_;Ufyh}zkv_v?^I6{XoA7$3+SEy%;%r!M%(#`U=mK?>qdfPLaZ z9}Q58>RLRZk0nO`75Lx_}>ac4>J=I z5;l;}JMI>U|J9NK3#=#7r(a=gEwdgGb8{L5#KWIxC=9ig|3(JeRiJ43zvA8hNd|?C z+_L-sJ?I@7aB-Sf&H^DAzt-RR1;J%=*_1g|D!)yvrJ>Fu{QLD0f+jEH-CAbRvC9vuNhtM3)=s$-4+9QG=5Xg>|G5Gl{pAH7NeS-K z5+y=i7>}1)BFDy*q4a9~X2I<>>_EbXypws@r~$u&LzTHh^q~O)51mW+A$iD0LGeNx zME3SK+2S^cbnyoOvDzk*}>Llj-L`H+(-U_RWW@VL=>uqn(b;5MkfD) z$q~1eczMPzWHyL$L(6ov)g(MrFXi^bbPwB!L!fz;F1j4IpR)4*Z(en-|0!z&=e9A5 zPV_;6^|Ym)BFt@8#^>V)l0XvPnl33LRe7J|!~Sd%F6=B5BWVL^^ur49EY8d2JCBVx zAjRo?NFLIGg@D^^#RelvPjtXPFD=038p^>R{CbO#cSy{U;-A;B>R~Krv0?e0EW*&_ zMyfyyNZk}+H~>evq-xapF^_|aRYv-v?dX46*aN$(cmLC?k(`fSOS52cew$oUPjg9| zUmf^IP(i>+1`Gxs_Q&)ZhehLb9L~P}r=P?qw35Wt{|p0bAqH(ChIr1ft1>{PoAuY1 z|3mQqV(+cL+WMkz(NZYTV#OT_w79!dyn+^Yr@>u=6n7}a-HN*vCqQs0gyJs2-CZ8Z z=iM>x_l*OCbIIO0fyPRPqxTie>-YUPcRoh#G$#1WqMC01o?hZq z!LG0RzrF1TTfa$#)+flRD>sFZ^?UmE-wyMz!DuXg&;s7UC$G@W$HNglAjCp+v%z%y zIIbY~huhe4;Ydn@a6b;V&|ZJUFr$^}gf{zsLcwWY+u=7b84&;7hiuAqDt>#_kYs*K_1nIN%|FEY)|bqoL20VU%z(1@H8o=my3#F8qW&8vXxzK zP|tZvc?uuP14-Be-W7aRp#c+#co01->%=VCm3KkdGJCrJYRlbtbBsFN{zn~mhur=u znJKt!QX81@*kP@jmoB)G*+;C z8vWHt3@3#1l16{lFZjZ<%pC)3m%t z%50}qhT0V2TFNG$TIn|PdX(szlDN@#-9^r|PK$skf*4rV40mbcn+C`O9`O|F9E@0+ zXNmc1((fn)e)orJ{2?=+hS?S)#OSkt8~ne&VYY%_{(+6J$;=GR%II5(d6Y>Z$uDcU z&PA%3lF)MFFrX_aVCUft4M>W6;k{vjfhClWuB|4R#SIdey*M+V1u#!cX2e zs`@J0aFEX&SU=b=qle)i@KADK_*R8x$s_95%;5q$GMqQ&_7379&t0fhxRg|FvV84{ z_z066>`H5a;R-m5Fu}hZ5C799GWU>IaBafoV^-N!$y;}0^`tZiTg?!Y$*$I?pu=xq zbOCohQe2C&bRUbgzIdXed>J?5{U!nKjw2(7is9&1Qt*+>JchoVP``~K`)1nBq=Q*& zB@3+v2ZO7ArwhT?#DKD7gfI_xYK)-?yqA1d9ChBcoZ%eTwMV5f8Es-XAVzPKV}lz! za7#lDn@JS4?IGom%elWj+&SGp1W|)YwEoXPVzV({iFsjpi_HhN4}a&V8jZ@CzB0PD z$2=swCuq-&;n3Fz_^2VY9#ZGOL?w6gfR+|R{^4ztthV=Z2-m|By=+GH(0~49A*Y~_ zQ%Uz@j;-%_9{k|GLzi)_FN4;8rXhcQX~4ZY;3Dt7Td8v#@O{pHg|gY<^8BqWUnE#a zxGCG>{wsNdJ_wG^p`xy6K-}PYe)afK?vc=w^v*Tb(JcpUT6;}f^FW2GyTPJg`_Uue zxsMf(j6U#{x8fr)J*Ty-2wlK4P3>nLSmuqusSTG&jCN>ivnm~lSCdz@g#{3vvZ--5 zUmP(}E_}huyq6v0BKc(TCZm~{l(?il^1B>&ZIPfJc$j=DFk{=&Hw5=3h5Yh@LG!O?sDu}aiA{ihb71VboZ!6qm(M8w zl79Axm=RgM#;bFKT;D6+NWEKkaWsj&-g`}z>By(1R!lLggJ_*(G#sY@m^Rz43Zv-8u*(%UQYCw$#8?51)zVWrg<+&@p!d;JMRQt0vZecznS{bw!m#DqyBH{htQJhQe3-1nu?h&tBaE1hi+`84NyPmP5Ga;c}< z>85yA%cniZW;tT*Rf_(Dc?xsA09?<;;d##@_lHMXl}7Q$slg*nu&cFl0Ut)w6N(pQ zR&RXJ1<9!?<5RxVAa1GYZ{zKtC*R$u%4g7k8sHU$Q+8#eI5R-(2qWiuJ zJsDt7b&Xu>Pik{-D4;aBxaAPw3A%%d*I~&U8d~GTsVE`IiZA}4CLp=J4-aKWL$6kL z*lPc_srvk$DlKgxyC8*BUzO!o3Bt~k5YlR2d7sxO^lI}k4>u|1N1`n}4!EL`egAks zv2tcUnP*~vbX>?LFC7{tnN=*2o6GX!$>sYxeSdUq+#r4bw;rx#PiQPSo`BfL;H_EK z^~s%q{0cz*#Z!&dOFMJq6Z6gYr}~&HGo=W4_d`e};SWStiX(h>(fp(E1g7T)HW`W3 zXbf*Qbd(}FZAP>0rHD>em+AMk4}2Sn!y4@l_k~ss_zqMrC6{J1+7d?$!X@B-9D(1m zF#Nc`hRSOL$f)hOJ~ZHgrqF4>%i;*?HIcV}YQ)te$Bn@M#c}};E;BfC+%%n^`{{$^ zqIP|6**IKJr;|x|B8BB&VUulxXHe)MPkb>YdDtnV#$8S?$h`+TgSdI7-m<>E zB$1i!viO9Z$5HgeEW~#!?{7g36m2_LM(5o|a<9_odEpk%&4*NYPq*~F%fq04>!UMX z7-`+FN=Q6L4{|ctLFc`^k<23vw+CokHK1RQ@s%hlLQHrltr?Tz;pt{pcG#kAv>aCEKn*?r=^%Bx74tNhR7LZJ&u{_bZV zZsJXLw+5#B#=Pl%+Pg7dM2x|cj>Rl@UxlSTiKnkRkejWi>GU9(r$66+T=utQnfq>x z_FX7bPI+jb&~217LZo{uk4~Z!Ql`IoVIdXRKOgN)eo2*R|5JMT>Dtll6aV>A+Q&n& zD?=!sj`7^D9hGp%r=L$6&!2KXsKRb=t zyWrEzvgO$Yghq*h9~z#t@Uf^dUm_sg0TZyXQ;a{nl+*-J)EwdHJLg%gQ!(Pp?+>HoI`%YV$)df81<1c3b(d36 z-up2`qD(|+2Zo_yq!4_NSU;P>-e_FtKUP#=IHFp9e_C&$F2nI{k!N?iJpi5=3D0+0 z|M7;#UEeGcP#PBq8d$+|^~L!SisLQcm(`Ax-k$mH ziI}M4Ielkf;B3FQehKpV@Ro}8)K?%{@fjtD9IPJ7C}_+L^!lLF!Aw;WpG2Gt=5)6BzQu4 zeD0nDX@K4+IJ*g6+2+SK!SjYOI-LfW*b|K>tlpu@mB&>w^!pW|!E<`M%c0~V))>4O zem|%`wrbf*e}ZM#9I!G+=t2nH`lf>O;`7a7#IK0tRMai^!6R^Y6k}xTNzC@$d_B>% zcyLY8!Y-hwoMRN63>&CdShv9|T+5W~p#}@1tzD_mpxd7nH8QJrM?Mc%ybdEOL`KB2 zrS~nWg18LbZC@9bU!#K^b4j_F4^LU>+aU!uNAdsE855}O5)$sEY5o@NX=v21#I+uD zRUPR}6dhL7|DY?PzQFD0dlqr*T=KLdz0#y29vPC>Ti2Z4NJAkYClGsLc0&K?d%1)p zT*9eHXg9x9u(gzuDa5tN<#d!G#C3Hkw>1K~D@e1gQ`2o0w}Jlajw&j5u*fS4{qlmt z|LubRS1D22b8z%@<6WpQDUeX|J8`*Ku<&&9 zY(hnCEB)xfXZg-Y;AZK}<|~CVkiNC+6WP2N($-HbiUzE@TlBV_JID{hyLPg!nkLgv z@f6W!XAAt7^2b?`#7tirlld93P-Gfs(1kY7I_DjvwAc z?%Hz(9?w4vzPGEtlDKQii}96RH4^Xk_#r8Y8OaZh5hP~e0S?t)U;`vVci;?MZM z&H|h^ZeMISUwm=a6+!T36MHq?NiueqvoSZkEby2pz#h3yuOC;f&J~h7z1lg4s8_V- z7)m#fXLt8QXSg=IiDGxJVfmBFYkr>@zvQmZimy-X_y!=wc&(KaT~}r z;v=qQHn?-TB9=9Ym2!K-C0aLEe(`ZG6tux%#+jyV#_G$n@6RsaAJ(!`}0#Pf1>*za=YKxMb5 ziY}%vU@F>m*w=%@$8kyPdX$N^XXN3|zPaa>_Mm#2%w0yX_Q5UV$%j}dv**MBKlge^ zsNjkw8#)KEr6bAIx~St@5v=8?%c0SiDN%m^G!P0m?Gh;zAnV8BSlxm4B&_*7)O7(3 z+pGO;FAp@*gEF<;x?J2rzAl2kdH(Fpf-46H%C)>3&o(0?lKF*e$q8u0!?4Ot_}wqe zJ+DW^t81wt6kUfB@^#E5K3Y9r(B1Qs+Z;QARd7w$?d6+7f?5f(5L~>oSTT z30`r0wh6lA8^Tvb4TWfWL0lh!X51MMY3f z#+`_{Qi|7x-F*^(!6;GF$ab@j+cG1kIDZ{;DJ1{a&eDF>Sr)|>Xv$jwG$eW4lHOYF z+lubna9c$1G``p(l{utpqA8@vP!?_tZTW%z8g?H;`Y-ONl_-*=>Uj?}1`!m7UFB_!`)(`+yNL+~sbm=-Tdhn2?Eg^D)3jqvV zoP>y_282WFm}Tz7qDj3Sfib*hvak`@f4wLl&{P}u`{R?e9W|pgPeW@P$5OWEY#8L5 zqMd4Y;u6PslIF<@SZjE;A3r`ZPB{ZroH#DXqKmIB8aL55Y+>51KH`J5MSV#Dg%(!i zL<$;kAc@uJttyQ;I5+V+MB3YFFbd?MYA0dqqr-v2%?H&)v2T?3ptL~@)~%s48BUIr zNke2CRo*mWHrSRp3tJT-;-2;omXb&F#BmEyhw#_p-yrI4|w=syJ2 zZZd=^KZp9z0%+Xs)wSdnzh{jH6zCUjH)U^FS#Vw{JmLJ|4ojnTS3!*3)KjWNzHo&6 z`eRA~2*qO&E9D`c^B_If;~}nFI9PQJ(WG{RY~c63j?m|J3-A;jzFli9bTUl#%X(n3 zy@z@|eEBwAhIE_pary@#A*h6p_{jtzMb#=L?2|BQ&T29{alnIqnde9t;dfvU5Spd{ z8RVMNP{}}>Ov{+LZO0N4b;wyIDu1633ig!zlI4bEkGao%6!+R@c3VhxW3>Y{ib)If zc|dXQN0#qyZwznqG3fbuAFC;&D>FC5MnE29yB5vwoXFpoLy7I$2^^B+ExBz)r;vmW zmQ4N;!o_0DX}=1QPd-{$0LUFDzV%(vN@8+Lukmq;?<5T^++pff<;pana0uyOR>~OKqa=g!{8%qJc5_F=?@tx(er@WP!_3EKSVo?^V+n_lYb&N?q8hiN2ioDDc zduJ-e^viELR$OU$BCe{^>@6wgMfQ7BF45Fvy?NgE!yPBlRMq?YIy{5nD%`^1o^KjQAECJ?;*qzIzhuVt z{Rj&tbH-NZn${rC`Dc(tK0_)}g|~NozSxgi51Y5Lz$jvn6Fkn?OTIbWM}9m~UjcdH zc|9!#iw}Emf1aVew@w~ARsd;#=Om$MLxhT!5RI9pS3{ql4`>_0)SX_CI~@C8u=(Z{ zM!^Pb5wd3M{eauveFM^pC_}pp5~<{&oWPBB+IS5y2hKscm;Dn<@|Q3VPVATMQlGyJ z#Cp>kwG-xdxt_8KOJaS-$6!w9C9f2~{gv=fro{Bgk@{>ARzSPV(Sk)TVXo7ll=G&X z6mZH8BE*DsIOm-``P-i@zaenb%~{u5boKXG-tw5q0Fp0h z^V)g>eibgdvo4dmXO&=8AUmvul78Lv=x*9NYjgK0AU*#)4Z03Rk-dCX`5~XgS`s0> zM&n{DSDo_}AuL)%ni6JYey}@u;Zw~5V6BsZQchp3BnTDk$jQv-gCN9_nIU$)ZFUKf z0QMp}D9GCzm9%+(fYn&;#sSdphphtP(A7PL9KT-H&{d6Cemv#5-`pBC>T3L^d(&c) zml|l_OEqOe>1(+383Yf5?`-yynis9%Z~1UniLBv-AR>6xB_BL-`za*woe;v(cW-nS_K!q&4& z>fi{P4545O3ylYSH+M%ox>WBWgFI!HQ}Ul!w`;Ucx9@i6knr)AsYWZO#;6PRIQ}Ww zW|4nDNOwzgUzvE(!sRj6D^-9J8wM#s07`PUA6YacgdpnyM`VPI|(A+lc*u5 zHkvJ1q;#p{s^l`tlC$@yA;%T6(53^!gh&nRF zcXDLzt8y89E+bt`3;}i=)=%$WWMNe-3PG+0bB&ys0X66uwPGBk8PiV`#)^?OeBxgh zyGXsrcJ_AsD_i1RKPx+)c#1pm2M<@(g zR{RxR8U?KNqAi}`_zs4Fi0!n3OsmE0qu!RlhoHm(e{E_&Wr2b+{r&y%ledjGs2)~AKUw{cp{Fi@*POCj0tPWlmL$&}y zlwu)HybkSJxl2wciP>WM7@OgZ?@zvA+pIu|H{2Ydm$6MckU)qxPkcoC>v{gFBA7PbfmQncb@&Itu1HmCu=0BV{JLN)(})25CeGiW0x!Nb~f z`U|G_f%7(!@q2C~Y~1(Sm?P(t7Zmr4!VgHtEkp6-9ptQRKI*@-UWgugu#thMao5lX z?3rn}<#gi2hsN74e&H&LY~vq&H*b2m>}IXp`(^wo7;D*KWdNm(K6SNo^GX+B<7#wA zOGxndJalh5p>bu3S@Mqy)bPdkaX}}s`l+c;(Su;WoT!2RBb2$;Yu}-L{=96a{X3mk z8P}K4w(BLUfG+!cNiDX-1y?Ku18>Y`^QYH)nLxa!?`N0FK(>k3%VJ9|L|GS@cIiY~ zz3x@;B$DnD@DBiF7=mFXv+B&I;HxabEJwPX>zvzc*qXz9Q#4 zi+C>b(DV4ekzSeKaF?WqtvNnRSS&z*f4t!e=+Mb@8+G#tWysFa*_lnRR~8dUPi%f@ z9chj5mI6mT$FK5%nObmAp}G6AGmfw)m(n5w8CkFY z3L0_tTX#W$n+`=sXuTl9IOTzM&D2Pvm-*X2oMb;diz+s`#v_4V!;>=l z_tT53S3s&4?urfBb|=tWlw9awtq~jJkB6okYWY4FP=L2Gf^Z>sn@XKq??*9RplfIw zR}o)eVuOoUvvNpUF12g|OW!glii~P!uS2ZXa1ei<>GsJ8TvL03(ngEeP-XOu&%o{ z_j?4k_Unl-KOeSC--!>=EU5>bt9J1|EW0O~wF^Ogl*ge?@iN|^R!CGPYkq0kT#Q;4 zL$dQ@(@VN^f?ZN*BZ9qQp!&zdNekG|B*&fGd8mvHj>bStLmmd= zQ9Qjlw8Y8KfFE~c_ke0>)^lY}C2Fbfd)Nmni;-aOPDy)pjhBZj$C|r2Z_fk?M8cy> zN5rS~ri~3ZaLjO;UQzHCwK_c$Hm4W-P|SPy_jEu{lvNluLgOGVTJ-c5`0r7p{Rt=1 z5}UO~q?CLd?s6uv?&saekQXlfs#)L=8(vCL!N86*-nH7(AykafRU0u+3dCGRFgAGFR?%9-~ zKns4mg%V{?p6ELuM?3{O39DN54yrH(n{7tTyd8ZsbV*1QT+Y`TRfYz7og5i4`uvcV zkvq=gyWOw(11tX$F#4d_XSB#m+Tj&Ctb~$9`=cYcl4&pXrY1_qzK1yV@tP4HFWKr{ z9-4$e8r-P>?rK|6N!h+0)ia8cRuayy}dCKMco3*+w&_YZ{stO zxwVyj!VqVs&kyrZ(Q_d@TpT+ebNTPjaq>H-;Cx1x;c&|+nx}6Q4*)Nfbiw`BD288Q zi`Vz9?%Bf5@oL7jo}bfE>mlTEwDXU=g0M{TmFLN=6D#_;PU$ZPSZz=9)aV)F)wp>; z59FY*>TJ-c=u6>oTM?=k;IG}(_S$QUaOeeUU9+LIcPvl1y;?Hme>}D4@~L`y-wvw8 zT%Okmp`@_xFq7BgU*kObJz}p5V-z+kZ?wtA(Y*bQ^c^CG_pV+xMbvAMk8E-znh)3M z&%)Vl-A5))XU(Pednkb;{k zrvNVY+*-m)trQpX*SUZdyTQu$#jv!5L~TUM0p^~c>c~zC4E}(Fjx0Ty0rf>{EeX6! z!HAOlDo%%y#2u+5R#*|?-Eq%ioiJc8;O*Yf9pn%6UTbrX#br(UM(m|;I8SFg+_KCx z(x$ZSKP1K$fGYNU_`WMdcx^!i3kXpnBztsh?uqIBLE7JVZG#46C z_TYbRznRGfkcu%y8?*M_n|lFmUbmn9{o7^ubAUhZl+nv-v1iSDv(Ig@;|Jo(ezNIlKoFsU#;d;RU)UbDexz86 z7hz|lOAeg}>bhm~8$DJ%BOOiC;pFy$<&zQO2^o2(&mEGB5Lw?Nrc3@4!k8cfc}DhC zpe<>8+aZtFR);)7fWu}d!}SzO60#g}O4N}JOLvU5yipV~aby6&UZ;}B)^l1V85MFD z7tB+xUylV6YVk@QL(SmAz6Cy?94C5;Ae3h7y*3aINx_N_AOHd^TutW4qq9iitM8!O^CWuPjE(xF4v#!(G%u{tL)61_;+v1*?AgWKQtFEzE7+@$as2} zZs5fjEH|5_>L~=Ui}Y(Sn7nd*pKMnW2lFc&{(7rZ#j>q4$~!oU z@4{3or$cHQg4yqEL$8=d-Mgji1!WZ(_V_%f=|($Y{o) zv{gj?rg2e1I1JpP6MJ9GW-s?`Xs7a)vDaOK&2_YIDBgu3d9#5)kPJJt6A4Pqk#PD(1A25pL%VUik!p@vZa5Spg6GetBM-AY+u(dAnxlxEOH;4HeGEOe zU?Rqg9@C6Yr4adylckMo@kyNf3kTB2&2EeI=>i8t=Ovo__+2N73efALZ8auC+SY8QqUYNMrSCJso+-wVAM3V%2p) zPjbAxu_{{16o^6?lCnCI<`deKMCTw8`w{q1W!0HCeLt}-Tv!)kRoE&r929Yz1^)^e zguq6osZ+#L8J-3R`?YUvf*;Rm?6dgIcRUDbPGCKc-cKB~jtVvlX&YRzHAE?i@GFM+ zNiK^i_#UnNd<-H7?C1~rH=loVihW#YV!MUR5n6wb4z>$AdoHcjX=&0!A0G6XJ*EOe z4g=jmwVeDpX4PeD%mq(>LTi*vuk%M_@ zM~+xr*?exjMngr2>vkt5vktnRL{ce<7(d88SJ#0&#No4zNXxU%l#jW)(xyj1T#SGz z{k&+{L=_HbjGA;QtY^LOwqgpYltxYN*$ALz2CYQyR!PPAOA)nPS$(yNHbHIERG#br zie8>1M&tlbv2X(VUK7I(G)^+-{Zz4u)w6PM>Oi-peH;;ri9O|-Iu%2yaF)qFxKQ$7 znTYtpNY~qpSkT1YFb1JhC%N9UYJbEPq*|&3?cZ`52N!O|C2%&vBVR7DnTYbArtLvLWc>Q>IEZ&^8{-1)xxOi(xQfT^C;4_QIA< zhMyDG!iFSJ$G*L13`{Ycd5FJNxaFbjHhfzj3PtYl!`4P<@~NCXLbu{<7pp|$E89U4 zDMs}^AQO&Wp7`6DGVeb$U2?`DbJuzvApjuW^;gtPR5Cq#+=KxJ2%ghk^c$F)I0lK9 zVk)u$I#=hY94M=1vAPPj(3K($9jM@v z0N3#lm$R@Aw=PEkkB(PQnrz7+a+inu$7=(9!pWYb-8V6OyYk(oY%SeQ#SMsdtZpcY zwLDthp$uejX2H6k_iLDa>~SYaXDtr%w?598jC`T?_T0|9Wr$pPhXYD~``o6U5T;cL z*8m1_`H{%58L!r@2+J^uuq2QB??(hUG4@+Syk2u6asLim;@*-tNXb6i1>R;v6?)FBE*1f{$mm*bfpJMQ2w{*pzz7jO6xr%2}Fx)dY^FjWu71^-{LqD|U^YAt? zjJY(op&JP@>>aRd8ruvl_83Mtb;QhALf?|_= z+#|_B8g&Fp3K|fikjx9B4S-i8pV3C3-mYxzCA(Pr4ml*Dv7$%KO%yte``f$D5gX+( zl;k)&bc%0M6Yz)d0vuM-sO}l=OgoYxr}dJOdDDC&oV^k7owC-GA!lK9SozVZLK23< z26v=xQeWYn#G{GP>g>#@7%YTKDmEXzvSlvlR)iN|8yPuWR8A{j>Eg&pM277 z63h)pfaaH>P2pm)PYjB0`>RGS!kJqwh?KJ%kOS(|z%RS?b@_MoT+L~bE{QUN+P;(+ z<@G0!Z10BMMP9j>=Rx#UR@c#=zUe%N!#^mHoqhSb=g-8#x%}&S`#s0$G#yt(2vS{f#3t#8GZ#S-dXi0VXOizat9+(A2Np3JWR z-6Jh&FMaUjy*eWA%Cb}PWbdz&8@0lbto)@_A$1Dl1Whbwe_a?Cj0iGoA7W9%;GB4V zv}Y)fjbsDiQ>&645iRRQ=K(@^1J;hGK)Vnwy;xBlUE-1RPO$b^)7n9M-3%A_c`rJC zAUmUtQ0%DdfDh5PeKMk9$^I3s^Z0KF#!-{~9ZLs!lW6O?KU&?rO>-+ejw^d@CSF8j zMjnkEbmcSMeM~#+xy{|MaB^emT23?wu&V^UFUX9lZBUTZ^NrXEQGo9(l=A9M14Z7- z7-ug>Jc_afBe&TLdK+inup-*`o~v+|EPRm+l-vV-Nq_$xbsX009%rFK_%G` zwD~`mYun1&bYhn>mYS1tjz8hGjI`|Kymz*lWu?|)h|1;l-yHB>R|t#2>Yv3?{NT1SQMDKs; zm~)f?i4~LWUo@P}rb@#CE8`CA(q8g4QtTAJB}8Ri>!(;@G= zasRKV9P0si43llhQ_+;SeWx6JyUBKZz1rXa9Vz_ShW~VKgbDc9Ac&xQce8lm+tEGx zJ#1j*e=cQier;;c--%JxNxawC(l#J@zs+vca3+Wn@Ja39k*Zn(UzT~G7p_{f8;LhN zZ~>E~Ot??5MusI0pIn<_+!{}yeXY$o${Nfl`GUFOmT6W6turtB^I4!@ia8ugaW|{DfQR*@*DkuB>e6D8*2TJE^8CfC8J}2m5np&3@LiNsg`hDpX6;GKC5TXhs0*J@rNTiFZ;&w2eD_j%OCjU zTCSk%LzWPNNF&{oG<*9WJa2w!{|62W8mAkK)xU9y&ETvH=P1fl* zwDRDK1#dfkxsQdv{J*}-4qJ7k`rr50up5BiJ4ep4!!ys?m&w0Z;6@CL<{*U!VH5Qi zad_!4P~spQ=3w}xy`NnEf584HV84H1%BoG2s zPXC##p58z=Jhei(BElY4)kdPOYOPg4l?1{yrhjPY30bR6R^7Pp(8j!LgBbvc=GXGK z&L7w$U*clld{FuM-zKW~m{}qToJEkp_n?9w&oT||1yp|Qte@@2Y%R>O_aH8@IpG;6 z;Umm*cQP4f{!=+B*_&Yoh!My~oJ8Y_<)|0lt>cN{{8?Q@yP5trx^yxDKz1xSnM6%T zgWCf%2`RvHuqqm06^$f`M>;QT{uyy!_(i#0%$0{bhv>B4yy;JcIgOh-$KWM0)LJ(q zAkv{HRB{n}tz2AL207f7DTydfmi_LU8@-^=j*Q4t>V|0+ohT=Vn|MH-LmIA@k$Ct9 z=6l~^9*EBH$?}=3?6#4Koa<)mx#vI0FIk1=YjoNC0+PGWDNTiZM*CjEKAbaVx>#ev7ds9A6*!Q%ETww@hxE#LktJF=>k?DbsUV z%S?ztO1|LCL%g66gBpdm_)Lc;~-`cir79ZV)ErsgH9!pRyxqh|hNathJJQ zS%R~^tYGnMZWo@pNf}rE3gLQ9K@P;AdGa4^$obs5U*a_0ozu5>m**f0iAJJE0TgKl z3(TpRx~=CdKK(0O$WCn5XH&~_RSlWKe-WK$Ij6hM#u_4j@ao&piiPaOu7n(}2-CPy zT{~^R*bk)x`V?;{x2dkq8&dB=L3|?CjhK82z_Ly-b-gI_^tov9&9?{LvZ(`!cy$Z; zh|H`ij~Q-<<;)lyJx`vvZBgjBQp{h!9NBo#ufPBC*&1IZo)Ug2K>5V%j}M&BKmC6v z!r6>EZGSnQW78_09E~0c@@5z;Em77}i_ULQ{P-AY;b)}4J)L9fiOgA$l>O{XTO!;E zRCcx9P@*^^xyI*449bFJ=`pl^9+63>vSnQjb znb81cl(RXC?c4>GYU$BZ;?R_|%_Q^6kvAfoehRQ+ZTQgSw=Gq-plT6o(a)7TOV99a z&~SbIY&CBXy++Cv+f=2!GxmBY0V99f{GD$c`4>f5A~1p3MYxV6hL0D~txO#<<_rig zj`NN3&QY<2ZdS;!g8;pD}J z?_~GQp5~PIhg3mcij=qrWhftw_Q3={`8--IhXaEp0@82^e{npMoh1eB_U?hZsVV*| zcB!{=eeP*!Z;<}X( zQB;wXwPznSi7Pi<$4 z_n1i`f(*hxF%tOl=qp}tpI8v}Yk949KxhUH2F&50eEJj4Fc<>^<#*9Fr;ZrBjQk}2 zZRok2a4?CSU|-S;l&k(0^X~;Z-+e$Zp@l}!Ki+G6^BRUntVWC#qk|Gtjl%IfuN+=3H(M9|_CScEd*`If`%=dAeFgmq zU<;8r@Cm0On56NVz3Yt8@WG6+km1dQ{Uz4df2PCslM#{ zfBK|FLR1Cu73li1W?4TLZh8iVsZ+oH_Rx@;jiZv6;pbVemLdv0vh!Fc%O4<0$*L0= zf+czXf|Vi&0%3tG;$IT4krRN3Pv&=Uwcw5hx}lt{=bw{vw=VvyRFK%gDE)tR#@K6M;P*Bk|^r7Fdn5p3m(%HU*!pK76B z{*oDlEMm5}c26xCKRZw$NmoR7f_Bl^Na?inb>y?Y=z9^Zdc3+n(@`C#P29#H4=2cw z$xB1eXAQy($|GXj`c6G=CsVI&6;6Il!j0wpiD9}w`HKkiOfG(gc@bZ3@MZ7Yxa*7i zz#+-h$R9|jXt;>a8_zN6P29vZXK}pW;c;*7t>u_Md}mVm_8+Y^EbgJ&5CO@mrl+%= zG37Hc6&ibT@{Lbz{RFvtOk@vkU#^=oK({AdJy|XkhKo_r4X9$fO?RI)ufkF_m{Nd5 zxs9K2R_mGnxO7x|z4?BB_sQfF=u^{+Q$};__KWd9P9OiC%mhR}H>y9PUrOd115{7+ z^Qnki^(~~V=n$q^Qv!#JJ2~zz8B=jNFXe*sdV;^Cvipv3$D?=x?=c>W^NZB@3eK$X zbp04xVQ@^AndaC>fHIZHbwn_rG;(tq7t=n$Trc8iH?s}%|O%nr;8@!;g z=}l+ ze3vO;lQ`7j5S)bhq5S~(^{1P1fv-ZB`}%O@M-!J=+eSQXMkRi;R>vrFa7%AgP`^N5 z)@bAFejR-@(y01r%%tX8^!n9P)blUfq(Dn@)KugrY#_#$0)>wSg04^Ly3FP3Ha~qZ zQSLFmaH>w8Pk43&bI#pK#QiY^fli1m`zKyZjQ$U9ys7kSq9rCK%1Vg1-$;0*yGca{ zgw1|vBC0%<)Mg!HJ}V_%D)=lbSG*Suahn0WJWCm_w__6@pq=m62F@wh zwj=eN4Rr@LIagA`Qs4Kr6#cEi@J>wcaX&Q~an&B3lx!pOf>DSq^S4H(`#zk0njc@< zIQ@c+R{+ii^fq zOWRji?Nx1F9LljoK9|3V+{Ay>>KGKOeZ$0~a*glJuhpa{O3xLMCi1F%o@dugl-~kf zKwy~KC#u+jm5J!mW0GVn_@plQA7-!P;ohSd4JNJ56M7@0r+-F(OXW3E_k^Z zYcDQ`_|}+H4tKiW59Wp(j`6D_Q`u4vzS$4D`jT?i^N=V*cQf+M{iHZtVq@qj5 zzJ2{)gY2a=yVf@g&RIW9>-@)X8F?0M9PS67h4v$D$C>}P7~kR=|$DAui#Y(MuldPZD}OIgLR zVItdlj5|l`unb3aO6c8ghxuQMSI^HV(fRVzSIDAtc2e_U0^PbVLRzm`n=0RXh3o}Rl4 ziCo0bB_dT~2wFp(UVof4hY=GK+HBmJ^12;TXz*aYM}Jy+l>cOcF<1KFFA>G~52#83>(A^*%(%pGy#=ZAB=bnA;bAR0*_us7b%v#_3yqa|7Li!anCYuMsSmj`gN6tq6kAsiv{b0y);>t*PuRy3M10GPjUrRzFdw6` zhI=CCq{PXo8weKHLK!gYA&f$L>!KGP5+gbRmDyNHJkUZnEE@wjjgt%%g}IA>D`2E0 z(BI1U#0tKqcCpjkczIh8CA>D_nP9p>e9k#J{{}1r`c7cwJg?AVm^LTYDr%WmQis>M zJo;J{0ldg=n`B%OFpcUEE|?LuDteIh%cUaaeI34@2DrWlf^u}DRh zV0+su+d1(lngyH^o2^Fv3KX7U#6d}n{7m&0uj{y^$QbTjq8srOsfK3mAwe;zk^8R| zKU5MEIFHw7{eh<0=4G_a>%t1B-#4DiUM8MK) z+fpu;l@zW?uVk`!{xZ}zMkln_W}AaAh8a*4pCplq^?A{^Utv$;C{6!3WC1{NVA=O5 zzA4(c@m8^Pzmjr;nVP9UeroXBy^^FI@^e2lXSVh_5ihb2mdFMmP|M^0%W2o(d@(E8 zcb1=}JIJ?!3uVu^^GOF{9P-xleMU>5Aa(FA^v=bVKnbTX58e*JWB737EvH^rt6`*{ z$URVnZtP-$%z`&hdl3Z&PjjVN7=HZ}izRMqN*zy{g)u4cRUjg7!Zf+$-EA>rw~tiY zWszwKJiaeJ7c@fY^KH7Sd(S;)N{&-rYU&*jUz{0FO0&XBg{tNT)RaYa79p(#hIbE$ zrTvL}N`yMm)gQDk=**li4{2uk{E9s5IB;|CtCzea*zw4s?yy7%8r!nMh|KudsPc3? z&E6z7nvGXiXwqet5a*v8u*XlVlb`ZHwZbWu+oyPRUWsSNPR1vNADk>>@#u<+SN5B) zWe~<%N#zgPGuoO7>n}++mV}QuzEJpiSj9f?s+{mE{NlZVr~N-FATo3e16cNSZZIb$ zGD9OxXA-U}_QJ}yJwU%EpZooss50n#-Rr4w>bye=_m4D+_Ie1ftXd=~L}o`We$uid zlU#{D%li+~(AJb$jR?+nZK892J;4F6)_v<&$Wkz&FGG?$sYxW0M>aTPAvu%RTkQK% zj~?Y~5GBh^K-OqlH*rFRAE6dbxD?;J7NuNsItp)0C0B!6Ra|-zkyztlj$C@P3ANJN z#yBG1Dhp)mruuc+6P8O?w$#des?onk={H6%>~~+=c4Y)lv-n(7@bz5j$uCvsB}YY@ zMOCxkV|hZ`qySlY7WGK7@_dr15oTiOcRVvG^20&(7mlBQO9*pXiNzE&E&hm0Nn&Hr z%82<>ewwqU;&*u@viDMl-Y_GBaoZf_H`>kft=WDdMRS4o7W9p!K}o47|H`rke*JKt z|D}uP-Vse$zk4~hVv#tGHc+j(Nxgu`)Ppe%a8U}uwx=RP_YZ+ zFiFtkqws;jnZhK&qgi0^3^QKdxB#)ng7HA*u=osDTFo5k^F>GmiP>Ca(|K`&9eVk< zDSc#Bus-?QMlJ47+o4#2$nh__Ti;C&%<<*Z_DRB$z(xhQ`2xj}9VFR>GGJ6XGpQ&+ zS;j=w=M$8vSXuGa&mj$ilLE(N%1g1y-fADht!EZg=9?&qQ(0&fep7I`(%rqncP9VG zcudW~FS-dM>FD@GO9tk&9#$VMaMUX);1c&f`)`=s@E8#s-&*YG>0j;9qBMN z90m-OB~dr{I4032P`bICAV3CnSl?LK`*d`+fHKfPdN9H?EpD z(|d$G%*j!cP)=1LCpE<{l_wNj2UPsXxzW1v?eMiyEQJJ$D28INmx=`AK#JB zhUb!$>uiH`#xQ_I`_dQ`DKYZ!_VE*uU)3hhpV(K?A{`fgjl5A*+1UZ7czAD*AYl>! zkOxldl!wM#oK&H_%u;SjA5}@-aZm!rWJ|@F!~1gv5iF_F)1zcmCLQb*8u=Y@?2PSy zHsA!QGUg$z+!>j-mfpioQxhWu_A5B$jM`y-Ov+y7^E|NF01IzsfiQbGTL(z%*6WWhQTvV$Kcv-R&-n7pNOx4q@Z z>+{LRz{=YT!EAUhF4UjcO`Kv`@nDEU!U=oXi*V6J>c|xu*1UkU()1(N$Ca>paaf<; zpA=erWrcU8pB~8@P5?T7`Fq^ z5Hc$FOw#C#t2#ak<_yH?Sf^%Z!Nl@Hv2ZmxAE9S0**)|h^!O2lo3d@6kwh-c71bNI_&YrLGn{WBUnNyw8Rfc>~RA>5sD<^;C?fy`x9s=#Pn|b@b{ns z&OW50yZ{3fnTBoqaE~?GM4?*8$rA-{O(rMiMj;Up6W$GJc0&7gByD)`T-U z@S#aQT`%06d3=RLHniXLk9Oqx@DmrCIk7Bz!elMDZeh(!tZN@K8E-L332Rw`d6Ue? zj>}wDy&k%>eg&eoKKA&xeAtmE+XOMg?on9Q9KDS=NeUL)(1~f{BS7gbj;~}lzaB^zA8t#Rx7dANbV&ZW``_HCbn7^X#4c)RBr@t9%UQ zLi@*L53v4HkXqO?gmcW~IQyUsBySpe1~&x)6GNR{M=>Jc6IAENOv{9B*9k$%3MA+#LX`18P+ zEOy8>$TW%e3Ppfq2_|(Yh9KH1tX0Kf1S5ed|y87SY+$ zLq&X5845Y?`Zk`KO90uM8B!H&mr-VcBJB;0akNb*NL^F}6%4&zrHbed&Hh(F_U5BY zjDw1ZdUzcxGPDCVlC@w)NetTD5Rl0FY+GtZAd+@UOdwSZ60}Uy)C7{weCyx-XNRrBhc#6GqamMpusD%*b2SnQdvvX2Yp*U^)V=nxAb?d+s18K^Ug{wL2`7$^>Z?lkfK3q zUz##%I(J20Ax4k}UZ2*6o)C>t{L6)IHd;)sL390U0u-3JN)OC4XH%Yu>qUVIulS$W zOf#p1Yj6`#SYcE6XkKDG^o%_YA9AV zZx$iUI7-Ty88gKGi;a~$&s^>Q{p2c4{3U7#3pFH9`#1UxV1g|f?Myuy1S>XVgfbJu z0zD(`-~^m(=IEK#3qDJ%)<} zLnX8DPqPS!gFOLCQMF!fho#T{pBg%#Pa8&Q>5f7RAB4Egg>j}ez<>1n?k`paOC4et zGFo*)`#%9zZ|Y5jpwj$*?EHv=YLdEAVgODcbia{=g}fH5FoNOg$b|_2VL$&L??QJS zJpMfuJ{x94|4%I)FoX@g)sORsBFq4T8*GUyZi<`O&*`*>)vSA*LcYBBn(|PxOH{9x z_|t&C`Q)JcH_fNvxZmZ7CW>VBc76ZrrR{XUn3AY5kei2xnUfPYP4%ha4O{_{S;~&| zOn@k!jFr0vn$lpKhCO7e1D#X>JL>=KveNy%W_XEA@ZgrPg+&&(gWCmeEQ#CDQYN~7AZN@xAEUVqKzyWkB(ynjQww38TJz5 z@h}5or2ltF?3h2wifh;Pi~cvS{Qtq`-q-ir{?gt#IOr}_E*|^yhi(OX)whmpSdynQ zxg2q(>|T5-Ab_4hMr$!h52t?huY5+F%{yB`%~+EHqFL!>w9QA~y*Mam*e)r5l5=N% zpV*&6AZcMii-3Tzdg(qH;m68=VkVXr>s(-v$EbmZVa&v~`)mItn^I}V-N?~PWKOSA z>VzRS69g#Z)on20pLxIk0beB9?j`W;vK^}yObGA77s~Y_RbPe)`tss6$j8x){J%NU zeKM24r?`u|{HTbaoSs!iJT_@-UOte_hW8OB1!vyhr~*o5N1Y(=`B=|Qk;bFOC&`MM z%Fl-vIsh88eP`ctW-N;!$uq$;4%uoLFlFIF`_eZnaF951$~;x_=$}k3LqV?J{dEM{ zV}WX~sKejyh+$yT$zn=4#Z=_908+gf6PUrZ_+k3E!^K*_g?naf1y$W8(laLGcjGXv z;YCrQ^Qttr{*uMv0-`ZBh`@84)?|dZkz#QQcF;T$5!MdX!YEU3*801cnG|$50x9_qy7|iFjFy8@J-zAf zdbnUaV8&9@HGlC>ttzl0;laTvb>a5KC4;*mvdcVfycEF&-NuYD6CW8QFO=_>Mr>JB zgr%2lyV{wD@UXLN#CaX&{;*q*{a~#KczSx;xLL|WsGH{_tELY@I641C$+U^zI>ZD4 z#)+B@c|cnQX7m&`VzWyP1kF?MmwSaPCqR*}6JOT28Rm&4a^yi&q?bFyn&6j|e(g*{ zZ`N6gcy=v7{h3tP&R?K+T2(U`lxnd!{`Vwz@8{wzNIdpZI4$ z&0Z(r1sMyP<0fIdee;9JKdxlmEBId7K)9~A+Jz305Nxt4Or{#;4>;F$MGf`Jtrw1p zFK-y`uL(UI&pWw-yj%l>P9cu2VshlV)Vg>t%&d)zrO&D8JQ9B)#v?p^yd4+1;JVo6 ztI@8AU7*3&43NbuOoA*`XIXIS0;2`KvSnBvmIC7%I>)|dC&7Z{S5n#Q;W<<=OFxbZJ}l+bd{A8sj=BI0;r?7dt*$$ zgT-gi!mQ{ZN*Q;s%4`hA4@IP)NCZgKApz$UIYHu5V|I2GQ%oXZQ3Whb_vPmri}6X< z2`T2^gGV{P>1mEn*&rdS-|v18W-O0MA%RiK(DzJK2}Y@|zE1DPDuF#>WGkCqKZ=l8 zgVeW+2l#2>ba*l{P@RscGt-i?LDA`Fpo1_8u!ph6ZimU4)cEnqdiK{jTTLS%)wR3@ z;>q(cAN965Zv*LE+38GjOXJNvIP@jNUsOycL!grX6a)*kBk2whJ?__mcs8EnEb9+Z zw&y(f;yZk-wUJ?l={jAagOroS(GVgk^?E5eY{O=T)Q+aI8JqIDJH1lt%jB6dDaZZ8wrY4y;lOHdAb5Q0@gOQvM)5 zyw4`C^;>qSv?RI`Z0`e7gj+eIVA5yHWJ4uilIZ&La4K}8u~Wi>L&41L(9XdwQ`v!+ znhjSAYH1R~5y7gmo<)irdH_~*7#dj_Zh3ye@+}6S(hWAed;AT^Y&nmCFb`?@4N5e^ z-n%6)aqK7*2tsQ=e0fEl3E5bjoSQiiHVmbp=mMyg2*nD1i8=2Xi#q};JJ0F1_#%sA%iS=knnn3WQzG zicVKG7+DP_enfGOZ4xp&Kw#RN2o)NL|5phQ9)De!IA0gvBys-~@;6;wIIpl^)T8!g z9ezftp6b}x&_n+m84JXcgRCd^kfBlzsF1AJH-IQP(;Urj+>?QSrVhc=(rI>A8<#g5 z5weZ9m)IFC)x=ZZ+?|ziw6n&wSYh%o`Hfo~4jR>TnckVnr&0>+b(4=q@wWL7_0(GN zRimAj>n;8{#NG}LR!F@yevL9blpRSAgfp52-FTjG5}ESdu$k+ohJj~~1B#tlJVl&S zo@zuue1NPqC?|;`GxbpJd{*A=)mpoZUKW(LD?u`b9Q1%#hg{*lZhyZ=S-Iv54Gv=W zpC|;^mqzV)UQ3V^LOw&P;=t2(zUhi$@Jt2mn3q1SHe46wVmkAYt06Sp0c6cp#BGxh zb4Ek7t_)G-)Y#OSI(H+p8IX5`w2p8ttYTEAH=Qe0-KA^Pmi0Lq*K?1MQ+4U}C@GNO zB8L3VXThZEH6aH&D)NuBy1yrOyf=R==CoUS%SmgyA%|ro;1Ua&Fg{uTGhTK$?Y@DK zPeal#t_XES%;p=MI{97qs~vaif89)MLViPT_6>qc{ZAF`)O|bI=Tx~E;D7yAf+Z$R4ZI(`G>7F;_d$ic_;9_DzTm^2m8XFh`Y-~4?l2qfeU;-~blEHvO z?6pO20NieF#?bwNmLuyc3gcHKEFlVGeTZd_Zt7ORVh?>v;%045!DPbG%pX_hPNyef zc?r!@7*NOw8|Ai`?huVi6ES>{UjP+-c0q~nFC3B{ZO3KBvnG7BBwB9V767D2_V#ww z-*VmDEnHY?+~B}Wg(X(Fff9)oI^^glaOd&PBfDTDluL*Ys%pRdm=37HMuW;lKV37L z^UpQVZkGW>cShYM+mEd{Ezqsi_P`QmT7D^h-=k61ZH{;-mM{2bX1n$mgrw!u9}`FL z(=zWJW8~8!1)f-$v}wkm{a17D+0h(%8&SoM2#|??-lz6+X_{U3`j-JPlT|U2KfM7p z#SDdU64|qGzpc>lS3EE-BZ@^?am{1I?`vB8Y+H1*!a82IV|l^ibFm&0W-Z_cAD%;X zeHEHJ;)wAaa?Ac4R_9+5L%qn9lPV%CwmdUNkj9_1$dYG-i9_)z4yMq~Sv52kJP@l1 z?=!di{l-37OxhThb{~vcUyCJcm_xO9K7*OC)SjAzJi(soFe)?l-7nnlzZBeo*6j*< zF`+f*?ap($zfcZWEd53?-5BEvu&G9dpkZGqi)UAymkP(HtHw7vdsoND0@0aH`J+`i zEtXV=4=6;zJ$*G&y;RO;bj_f znjCbJ3Vz9Mcj<>gs8!ODdhJ4l*eci+mkx)J?|ph{s(uacF6S8#oNWDua5gP++bPIT zA$@S0cDC7-Z*Rr(4MT_%i8!=sjsB72v!3ba`Jc`gh6ZlE)f{+m3!;ou`Xz0k3X2B|;-?{#G&4|oR+6^!JW$8aLVAO` zwSG_RieK1GGw}Z0;~UayqRE3yx~th8)H$6n9&m&aRrWKg@4$k)XYDBa`u*f1C0!dQ z)ZK8RF1>93+qJugQsQ$<(H7fC9JMuSP+g%LD1?U9gk~-ANc)RBf4tFzKcnT)Pr<{= z!`6O28*Y}hY9OR*`2(j-AtOu@BYs?8Uy$an!ao@op>VV~o2PVq%zq!%pJUO9ho!01 zWT{3X=bYk`Y(xK7bwYvQ=YzF3!o&cG^Ui(DfX5e1MAA`j1K{l~Tjg_l-{{4_sGJdp$Bg2aA>>lRm+0Us)g zuD0Wzu`=aVz6xHq$XQTh2mab{VNtebpYD_RZTtpmUHrH;GfKHhD*AJlqi^>c@>dKh4A5z+ zARt2hLx6x8?w78nUO1DS`}EPd+ViF*Q*}1w4sW}+ck&u#A+WJo>_TKOj6ZO!#a?!h z+MqKKIH$;nkx9>?!m;kYq%}2F+5@TQGeOKV7G_(23gL=wbd?%I=QiaLc%d}j_AG30 z^-tsbqb1{f1+OA;Wq8A5u~910}_ zz$DlZss-++(s;TdaTvr^pHk{M;y+V?IJwDus5U9T-1qA1SgqvcInoIBY`@o_5?NK8 zxQcr>GL1>4-%v}hX1RLN@G|Tpr$f!C$ks}Ud|J8J_pcx#z*fsI=NH*TwMUyzl3NqR zKmCIjT(?~IVY{H{IwTanEw`xgJZpL8Vg50B5ue`er~ic#DTR+9$fwdHQlAXX#Wc4Y|-?>sI{B!_~d| z<0*Ud<7q=9aTWHO=|if!>vvDX{5ulak8h1u-MZ)}C@ zzT`g&ni6ZN`3)o;+cP3u&}z!i{#2YrbDF$#QNE~S3#exv}ig;bf~g$H*iEf z5Fh2v>tgjNR6M_}5EZ2k>u7y{toC*~^fu)X9}0aKK?*ABFHggX>s{Tfw$T<8)FF-8Xz6T= z?w(O`Z@n{Z0I}y!6jr}K2aG=~Or4)Vp0Rk6wDC3G;_On#zmN;v+PY?6$Dha=V9PPI zzFgWT2}q_egEQ>!d^&QTyZT}mxrr0+Z?Quov_w@WQA*L<3B2Z zot#8b?r!yRz(hMB(M7Ji>}N*AfCkQnsUrJA(jTl;9n4IaH~T3)jx zR9d(N_xq`j5rk0%wSYAKx#PzA zanPpMDf0YXQ`cj&@slc_t)$SF_r2!VqE9QmA)LONK9wOxo-6N>01K zm9vG$E^)*CYKYORTWni5n}bWP?qZuuEh=#6hm@D$>ZdTCm^M9%Zg33G3Lihq<-*l@ z?D(*b!3=SqS{pR-?I7h32_1adY|)X)C+thwB5I0_C^LiH47smU2qK5WJLa?3-lgpU zRw$;fig>_jqaWfnuu=<6h`3k7wc80A8exnX(xPq6*yicDj9BBj$%hr zILM?L1a8guqaM{iJ{)CD`tptZymc~3oinZ{h@ITRmL9eK(~3#ZaQH&@PxOpvr1KJh zu`003tYu3YrbzK0DSr$)zb(`w#AQoQ7ML$wJQmKpc?7c?+jHWDs4t_n{c!4Dj#q?} zns^PP6(B;IBHvD>l?iWnKl@F(iiNbp5Jmp!W=*wvZKjk1-;ZBjoq*{%?mhI=%O^Jl0q7Dc@j`aW~n9>jpKQ>!p?Dojr$cn4!z@cx2EO@8J(2T2+n1 z{~*Qk)ih)*UC_AhsF!+0dR?ZeJop{SabUlM_!}vh=aIh~zvC8T^a9T6@&q}^W;2s} z8Wh<-ryOg#^Is?L-}%aPGHz#LnQW+?%SUq-ll}3GaL5X^DoI!gMmFiTII`e$NsSoS+LZH*4DHN91+L| z)EC=;G>%=i|45)JC}68~{9-%VilnAPVADRpowyFO0h1Z;2jm;u64|$uZWv z%u6?DRR24l9Y5syP|!!O3N=7AZm*(DXt03;4zmq$pW z+qJL;>me_wjlKQOFk=_&YF!qN)&+i`HU3E^I$?V_RjCh8b_=VbT}<^cRS!(^!BXx({9FI_hB z6LaC?K=f0$f8R>zyw3FN+4+J(j+z7*>6_E7)Ky{Uh}+Q0w0u z`g@L)UtQ5aoK0f=;xGn%w=ga8K_%Ppa}NVu>CN*Z&*B_=CCiGB4&1$)z~9MN&A#tBsrM(jPo^EWCWSImA`9N;HCfOP{v>0zt@Ds zSpuVj7ZiNcM6JU-a+$xd)cdvPiIYQfMRn}ojvX#4BDbA)CA<q{7_I)vX5!}tq2tmlP2G;F)>R!n@d?nE}a28qD&N`TF z{oU;lqhnA3-BE%%#p%;It6WLyxUqv!t$=S>*OXV^i6i+hxotV&4L= zJ{ZL(IWBtNq9)}lS}xwNES%1Ads__G4GDbiR>%v`tW1gARWd=3J!(m#Pzgf|!Wpr!${Sij0l;QuQ;x-^DjtJU=B9$s8kfZ9&KZhXSkc6**LuzwiQwQo~R`!BD_#+)!A*dhE*N63L`2s z5Tz^Iu$lP2OQYH0K-r4bXz&OQyXITPYTqZ|)J$XI#Wx705bL#5x5$85JdOkQfHr%2 zWtz56hE$5lO1BhtmbWt)l~1*hrdForV)Q+J{2<<7`A2uLd#NcIpUV*8Asq4wQDkyU ztitLckl3rpwQy6{Ji%8g)W8FoshIwXbc4XZMa(>o==pkC;8n^@&U~5nPz%k=^Psdhl;OXg~?2C?lR6EPlG49>N-1 zVbvHUR>%6$V1YrG5(+Xpot#M-4e z{tl9-pA2yZU-KpaV{$o&J1Pn#8MdYDVX!9@oD8wb`7%f5uM!x%Vj~3}3=QRNEcAW! z*G0od;0u-0osgihW_U#%JaOnuv-?97KxUAiGBic>OUv6yVG(?Vb4)Zy1?1;jjOkCc zXygTA`*^ePgfKE_hYNThR+>8e@<3#;=-sYNzMctmUWRb?PR_z3P4eh`cw~b-jRk#l zW)%d7dvQyQ{iEZBZ(e26%9y!3z-c7p{;=- zy{vhNk`2NgnA>y&}b_8I1@b}Gg4bji18w%^o2PK~;1CpfY3vBC5Z@9r^{;tEmOX1rjS@$q; z(N6IizY?(p0k1O5V?qI>dT+8N&B#PiaAJEP`4S^fj0MkdbKUcM)Q6)DL!8o_2MBwQ z3(=Rnm(gZ=36AA%=g07F8X{5EEWFZFFY-EMiQcc7|3~PmU-BQ&AfMbgIy>CgV~#Et_|Q(9|Wwzr(LfJ zaPM6Oz&>pRegUHfn`$xL$G?~#j?g0hmUnSM^jBwaS5Lt$GCXb*SBiZ2(6tx1z9pD4QUL@w+PiPxEs% z2>UpdCTJ&PZ!9o}V?c^+EJv;`S)tFM@Z#hPZ8ahe(!DOBnrrkkiCDx>KOs_*njzz@CWm+`$m!Q%m1Om)Z1 z$dBs=$d3;mxC{4Qk%X=o8zIl{Rx;cPW+^K*t=H91pK#U#U}C3)P(+{c!bqJ{L{m~W zhe8az6)$lz0|)sGd|CH^L|GvzetH_%=W%b=6>QZal#PLR?B%%t9LUA?F7s%hsmaaS zGto(W_@{c~@KghiAt)-5-WpJ<7*^M9RjE7*yCp{#K_W1GotN7T5ezp2qddAdP{EMF zHleV4vL)aq(JPZp1;&8rTCaSnFBEC55xqXRbhS91CYg$UvMe~SN%=eFf0BCpzV&7= zuEF(w>uP27U^al-B{?nY-P&z~%V}pTDlU)ohe@^mW|%l6(qUR{$q7P9MC~$~Djeqv z-K%6$Sc5%fJMQ*KBy}x7A)ZVmS5b>#IR(G#w##NU{`ujnw%b~^?RmF6`r|h6eMPHJ z^(TCEB!YRB!3mwh8PI{(GbgH#JAxOES7R$K*m8&4+hfl`HX&hzB{c(L0LO(AsZlC6 zrT?v~_nayh3|qG^Az1Z{9I~!-^ezy+L?bp9Nv1wC8M?VI1hZ1KF%aOmOA^5q3lE7< zMf9D$GEtQnLgl#!aeYgko+T* z6*m9t_FUf%6{Bs!3!@#XOPnbK!|J?;w zV@+8A$#|kFrq+sEZ`|5_%!FE?j#d9w69wwt~Yv?P9{!mox zDNzrM@M6M42x0(OQ#?8rhSM?is(p!qwlgZr!!yni`#ZCaI{5e6@=_FJf<4VOsMJXP z*XE8J#Vahl)Hf>qZZM^vxLQ7E2-W5Rnl&%Vy6qs~PZiR1Xw>vT)(eb!cp*Oa^Y=?9 z7mnP+6db?#_x}87+%Bwn)=qGLT4EoDiln9&-zJxR@G2cq>8P3LseZd!^TrHG0aVbV z%tr}>ep;jsC`|_AAmfQL;wp%wDJFjpF*QySg>ukYE)Lg{f73Mm4ib4WYV#~|GWXA% zk=J7_i%~}gi%#Z_>-r??{UW-Khkpv9LRtCt#b*T%BAbrp#T&Jw=9(9+h{n}XI~}TD zr`IVa$eQv^gsj?=Ww-ZQHws*l?wIv%Uz!*6I1c-x>HOx%-}OTKNiB}mrIYEOrPo!Xh@%}9B`$1vB1rKXB)m3O{)6>M zhhKS5GER3pLAhuZmkd(^^U(~4t~d)bRYFUT))!{$2s?|r{$;JiknX6Qzy3JO@kDdq zCk_q-wUhyWdhb?S#zHEWqDqu+Z!P(;ca6*p)eAbA)8jC5`lzU%){PmUJ{6U2SgJA& zRLFL6iq4{X!zKR7o-4VDTq)FYcZ$=jOpqvd=As@wU8wDT{qY_TGt<`7%#S4x7mRn=53uJ(TLh}o}Ik-`fcEwjHo^ZN!BIl9g3h~~+CtIo;5X|;u7GA;6r@Qn_L!z0Z z%LVP2oUa~4dxiiEEIfAJ2XzN{DxmqNO&DP4HX-&qud+!wANCm9l~##s&94Ggh4~j2JvOY z674S76CxUUf{8_&$+HzvsJO96w}1>Bp|gfMGaXeY3gtZI-pfJ(3=s z$zF0XTIT(dDOVojvNU(L8bhJPE%rGM{5v zJUpp&8ESrkjE<(M`um$XNB9B#wof56umn6~U?tVs_`u<39ReR&KKv-2j;h$1dAgbS zJCDy3u;WquTcPTTbC#RF;XLTqm$)em=Ap6xZmy_+Lx8L8?6)h=Ca+g|GHKUNbin9R zYTh{sUi^45)503~H|8}n6Nb1c z2(>uo^+^u*)%@jz0$z4ECVdwtP&e&BI@7|MS>~`pRHs&**i__kPrOn7?tKPIr%U+< zl(PNWKSERn{bZ0+UPNw`zYqZNl8SDh1)z3Zq)EFjnE(<*bLcj6d%Qvoi=0D!N5B*R z%oV*Qa`zBl7oX*i;FCDc_I-SQ5}J<&N7! z%?3wf=JGqH9Q1lAC>uLlPlyYoWm!qY)HDvQ{wDN41|e$J`7EbZYb?=EZRV?hN`Q^y z6-OW63XXP#AMXaD|{FnL1TQwC#)APX_B0w-Q#3-R+klS!217jt{!yjS1l zn&%?xE9*_YnH4D^({|VEJt-iJP#UnO41nvJ`zxz@j(*(ee3ZlZ{R8WiKp0=I$@?ba z&RxJu-aK)>tLXJUHzShI0lh%~y#=qXQVFrj;nmgL&lfQx8#lRunHxA=!7ug6V+{&g ziuAbM-H|=yMgcd;NX^Y^QJu&}gRy4tHWV^1@g5YD#k)=(?FhAW)!4;P_QEva^GQAP z9wN62H8L_fVo|-i|HgW4aPndJfw$^P4aO}~AGj`15Gr?(hjj}*vXzn5m_8{c4;9@z z&vw*z8S%rfAaVRTzT9GrO|433 za+$$31n%TJdYn7Xp361)_C)U_h0CW)JE23f7kc;24raaqzeY(M(`P5P9F=LmIYOH*xp6|A5lOP(c^8HJ zo#T^@%AZBbK8!VijLT;mm8f1XMNhh!j{dOuQ#$YF6YFJUMn}2coOUA_bpyi3F#(PH zg$4&aTp{CgC4Egvqz7DW1(Or--S1}(@xfW~EVw5lt2em@p1+OX+&9;Z+(?miIcwkg z{2d|XJ==zt^Z3hj(u>5~HW>4Jp~!G$lI}qW{*g$eQTf~r|GV};wvYL{RkL%`ft^oT zuKh8HD!N~bPlJ+X-#65uY>eOnoWANC?^o))B;@6Z86R|?KXNyN?{$D2AmI?^$ShzpaTl;c!Pt$ zE?%cU76=Y|L9Y$+GcEj?b!b=#0G|?PAfvlvdt12b; z1THB>gKIC9UKFDY7rNCIyo>qJlOaTXn&yR!X?Ae@X)ms4TbSg}UQE(4yW#nfoU3Qi zIDu2{m6xNzA!>_PRI$!U&gC+9UB>u^)orw9YgbdBsq5g~B0A?>%G?0!RMc>#xT8Ab z`0=`B-D6E;e34YD7`Wl19KE>R-lI?>Dx<#rDFrG~X1g-Uvaio)fv1X=*)=`nE-(F8 z`?}**=~|kdVQ9r;j;w)xVlN`z%^70gO(++^;yhyMR%)b|;Ob9v7G%nQMZqmrl7EVd z&<>M=W-k|1Z7W*B?ZSO-qU3aW;Z0(5_1bpny#j@zfIFiMbq{(tj_Id?cjJweF|t*mFMnxHI2I{S=U>2r8f}Rt)AH zFteY2QiY~346e)6v+0S@*n}NjWJHU?ThMV9xwguB__e>HLVBNOX(MS(Q6|~=*<;RyBq?)C2A^-ksoO)WT|ksSxGT4S!s2Q z1S+FI-4f`?(kxuJ`CIo9Hmsz^>SMaat;`kg#;uApZI^whNP(r!gA-%_~)PLa%$p+U}9y!?Nz9Zscw^)q(&gQiQ&gNUVt|9&Wf{GxEh@sJNMQhMy z>Wrx2X{P&}eZ^GPqYde>!FlOi=-r-9ScSC8LIJ^{Mv?SO2R{q-p9b#{!M-m?;nPre zM(OqFd&POTne{Qy5y95^&jtJgxl!K;(Eyg*RneuqYf_VM&G{GQR z)K=tEhqLZ^2Rn!Q?=iPrZ<^YVzcURa z52{dp9$?btXAS22JW4$&DSnn`-76}N;8W+hFJ zyDf@=`D9H$pDwO1FIB`C_F@!a;Dzlfc{hewclkJDAyWAMDow}5mFB%4{cfhh9?Oj0 zQ`%L;hI`DYA58`zrf9W`_T5-9{;OTQ1*BY!>=S&S8S_h_jb$1??9CLKgyErI{n~YF z4K~9)mA@Ih5dHGLYhO?W<*+ho&Sa zJ~PaxJ0{`oks)D;&K!+eOtb&t$^Vx_V>ShdeoEb?U2O^Ll*k z*CQix{saR{8y&s6$=r{HN2356o}kZ<&p!KgO3K7GfDgpce}8Uw0vV*5@^i5(envpu zOY9{;(Yd0EghEZ9arLZe?=tU3y~2I_p;!G#WhxHb!h*=8;(cZ|{1pv+T$&B!BVZ+$ z-plV8+&3Czm@R1=+bMz*jpD2eRWMD35}Wh`_Oa*ibhX{;6C}aOqH&4Y6IN{MF_^Iw z!U6VuvI4{os;GX~k4RRR%u6X-B5=P#x4gnKg#|iqf{u12n~Y1JipH}>vlHsd5k$-@ zfB1oFR~yopu0!*g`yEJk@p-7>`Q=gaoN9}SB45B0&~-eo=L~HN={63ltpV@~K)G%9 zwAr<;hUg^sE`4%Z8HUz|@vcFc>}Fl9dyoKk*94D})~s4Lv#%+$E!5E^PKiYGOQ}9m zyT=pqLFVB^!1B3VKh`yveber%SNbSW_wW~ynTfF_mhHDh!-3|N4cAni1Bqra;Yj>v z*Kdwlwm-Q$EgleC+Xkh`j39TP&snVDSu3;vHVkm(EWRXOyUoUazo;3O=LQn15%Z#d!TU$No+?_XmciwuhO@qykeOr;0t7 zR2hr5<$m}$yjOT@aAiN!d*N=r&Lf@vM!G>~){0a6OjZSJgo3yZ#V0X#bUh((!u7K# zXThw;*r-nfVaEFM5WdAQ@(8y-8Wr~7q`nILVrIXG0Crd0;ZT}(I0^4_j8UogK?#@s znA$86>cGghE3(o)TsM>Jry7Izr&ndR!Vgb2 z)1&5JY1NFiOO@EsX5UiW-BAs5hKq|3rzbVRLxjnC z=DM75Gvjd8)1)U}Q5veF!Q0>_;}c`!e*t~mH7ua{U+&F}gXe=9t`!ngCh|7YweU@q zaf%=AB4^X@!(s(-sdNbyVS%eZB_#k2Pupw zJeaWgMZoCQTYnBI&KUH7mvoZfvJZ_n27ctql_&oN3{ekDeJ8dS>~%eyXibXy2;eeUItJ zbVaz&CTd93ndmy=$3u4JN=e^)J-P9*y^pJ69zz{0>1Jp$Nw_`B=x@e;pbmNSZbCT| z^}0oq9aAfQ$bV~SK`uY8!zXMbx_&L=c^zG6?a$1a>o!xp{I~;EFNBguni|udfURGA zS5hpmBj2)48Q77%Bm&=7n%22G;LvxVZ@tkhTJVE_$nECP_C{ri0(2efy^7xR_Jjb~ zlJs(v1M@4Uh*s@5zTlw4Pa{hj01kR$!9Jgfz9xmY0>UpvFG#K*d|2e+7ceLpms%2y z)^HO0xDBU~x*7tsfPr+XG`@V$xCvri{j$|+2v-EHd%$t${j7v%XMIcXpI4_G{G``S z=^FvS$9WB~D{+oQ4XRI&AEyllJoqv?Ve71~5n}5xZ4?u{Z|}12iQ3$hpE|nZIGR%1 zZn~;q=gn}#X9zonAIsE%dV01<;cI8>>N?s1uh zl>m9WiUn7_P_k+~8!l_0!U8f5<6Vw=&zh!JZ1NQczH+E3D$;o4)fRVISV)LOb&8;2 zo@!1{n_D_1{40dP@cBM{lS45$^%V9HAzeLV6cR6_zGeCFwb)^ei=f)iAjQH6z?NM* zo^}i=p@}6um)Kk3@)sa?7Daqgu@VMne+nw&!ZZlj$xW>oeaXD)q~9BFrjOkX|ZaVEJ_yBi@u@Od8o?|us>8aF#!LL-_~EwOr#H`Fn(nT}SKstG0Dv=f3wV+BHQL z>b5wk`CVHUKp|d4pXPN8p@%2xF==)&>k`Il$^EhzQ{W0nk=n#Gm`otc67@(CPue_Z zNbqOQ+=O4$T|Y96zGE{z6V+15wGUwCQ*;%Ok{_WZ`!+M%QDL$2il99&c`7Dj{U4fF z8x-$LQyjwA%=H)jL5*4HHIVo|WXXvkp6;(D75$4kYh!AT$6Wt6?Z+zp#@Emqm75(b zEEWN2>RE?A0gE{m_0PTpHi97fwC2~{_>CIZy-Q!yNa%VcyT)RHg999UyQ?uP`zCQD ziyspj$5@Y@xvMbbr?_AN*c2V-vS~Je6e0*JiIw8M8BFQ_%03y0@re(0e;8rr+?C?OFNzp~)7HxP$|JPDMwypqA@!4-bJ`u3$ad7;p^3C{B*F1b^!1SNQGHs% zxDvw%{nzECRA)!Zub+KWSdfSIvq&n&Dg+FZEU5-;n2jakM7wqOkqAKf-+fUhK5eT}n@Fqiy(0^0G zE}BKS)*Vy|7@_9adSMF~p*O{D*V06qbp)4gFytXQk4d`4Fa{9&SN{aVh}WsL1TA{i z^Ho7PnQ>ldzi5Rtp9!TBVRvcc;Eg@#Ay&M~T5vbR^7R6UC{7?L^(?A0X zV&x40uS)dS>pexPSVqRgu-%9nTA#Dx0+YelbQ^;j*etneZtB1V-n@Kkcl z^UU>E1NJX|4kZ?8l8I1gx|+jF^t3<4_d-%Feh?WC+_T`P?|w$L6OZt-0#D6+CNvS5F1FQG-OdkcC>S1M6#>m!rC)Gyni(xE2*kY@AB)TLVd z4JKU}gXRZP^%FjtO&n9@7UtL*qyX7Y9Ile@KaL?%U{w`ER{l8c%mLV!5Qu{ro9y>P zNzs>;je#MJ?VqQSNkDo7`mO#sv=TD|KrXwuxJUm(q7t(;Kpl#+S>XI*Xl6ebfb|?2 zM2Y`#`VCNlEk;IO`9s71DgM71|DUP+e{@dUpccEYRR_m#pOgJgx#CGpZw)9PR2r(@ zjSmiHzdy;DirC+4>^A*}9wC#HNsTyFJ@#?)f;mrR%G>6~$8lj2$Aha=!F#ohN?oIa#Q zV=uy>F+>DkhPQ2DoK}Cz(WUp}T;_|i=(?TYG#j(GvT)ovmt@E{^-_)TgFlo%(yWqz z(dsBcrtyc+DzPGITGGd=7~^UJ-@GF>f)MI~_=Nlcuxx9H6Y#+67TImlcqy`#&2j5r z?=<@;XoJWsJ^zl{`J6ar?$?bw88l#i$FFD)R;UF$TnR%aOXs4a5-PIGXe=(jyHq5F z5RCBC7c4WO=`38nx8l?5OZL=^ULYcJ`Ktc@CE_!cC?Da|FTx-v(juRX=p=}Aa{v1H z-T96}CL+`bj_BLbPxE-=ZTA7=HEdM@t>o*N-aZvrI^%mJox0bVOogt|6uaw1; zt6c@)g%J?Pai=*OA51wE;S`V*TF}#hR zD`lv56pK_TL>;zchhzPD-aqZZbZ45@8&FiSZY2`S9yL-%KAijVUT1dGL08?aL7B!# zNA~cYR2lx2hp39mS}iD{X)m?#9p30 ztaDZ6A*6R*l8(=!>s^W=gz`41`CVp=aA0Xi-hBgbC)QXhY+yW@htKTc1_xfQt0Sty zp?s@Cu3kFKW~D7b5#_l^7_E*S8dza5g?rr?=Fc(4TDMp2JxoDsFk%mr|IKTdA4bF-Ly4Ym~4Xvrqw z4=jJwo1nkH{q2x)&n|0+9=a}m;)rjr5EZ)jq6D`a?jC(?VQ$VXVxnS&+os=Zb{#6c zEpEciWUHp|x5|{tvSE?Ej?b$cn;mG1M6PGctPL@{+2F{_AR2y@#Ux6(Na4!9nB7C| zv#ai2;x4-;U7eab1W_Bw5BquINHF_FfJh@Kg6Jk$t+DpH0AL~?e+Xie8ohE_m8f>g z2_nG5i3nR+kTSX;!GpvEeWlVqYRv`TWNBi%fRyOC!MAJMan0@lRcjeJ~Uk0lGAR_4k)^3)saTC%bYfmHz3_zNEEI267T!0iSn}NA(5J|Jd%GC+VU7@mrH3Z zN!hU0tY2X37{R1?3kw%gVc$kcPcR`h6urUqlhXdyn>$R_g)sz1-_9@J!tsC|{!0M; z;<(vV8nyAz_QpD6Qy`Qs*HPt4%#wimsTo;FbWBIPqIvJMS0JRAZ1Gecfe@A8VE-#K z`VrpDkT~oLDc_PIW6&CpIkAznGZL+Exu*EbIIREt!-9;L&9SZCRAx`*j^w5bR|jYK zyQge6pp{>(0#X$wuZ6>F-|KOG{IR}vz|eeK!?JJvS9fA!o!il9H5VLla(~zx)H&B& z9a#R@g-fal>lchNU65Y8QJEJ!rFP8ZM)=MN+YlQ@6<;C=J21%f4sqZS5(8KKshtBr{uqv53abjuEGlD6bewh@W{er zm2%nNYlI-OdWjm`QB-Lh={2Z26b7{q1L`GQ@Z|3`UcL&B?YG!(UJusI9nnQq7?<`k zp)A5I_v5eM#{xcIMngR`EZ6?7Z!pD5BCwlEe7y5}0fMeUJ_36$LWt6(KL)V^c4}5P zynZi&fPzd@0NB_0lMZ19{Cj*~U>6vigg5(pRRnxXp$4Fg4+DjNtK`s99s)be|0im< aoZ}BM-w*3=4y48c{*=M$_e$j-`u`u6+Ii~$ literal 0 HcmV?d00001 diff --git a/openmetadata-docs/images/v1.3/connectors/openmetadata/service-connection.png b/openmetadata-docs/images/v1.3/connectors/openmetadata/service-connection.png new file mode 100644 index 0000000000000000000000000000000000000000..64d25c67bf395569cbcc15181d7f683da54ef840 GIT binary patch literal 41087 zcmeFZcTkhv+b(KX@C6GX3IZYm($!Fe&=e8rNEI-kB8hY)gwPQYQKYEU013T!LJv(4 z5eOi?B_Ik3BoKO#ejZWqaNrQ~z=6XICyoJ69!H&q9XN35fcl+VhVBPviMzV-CMolq>xf>RHF=_W`c!2ZA0Kv)+{O5qm_$CzzBwb$M*NXpX01JLHumJWH1D73 zC^JlmXJ?P2yHQyBV6$#!seNNppDF^4c29Zaw$^L0>T7$pBT}4OA@yx0{-Q`liz^9j zfmp?SYaZ&YUzhNEt}m!4V*X>~Nu(InW!yvCsaOr#3xU)--(@d!Fe+qVI_9>XSTy5*ax?TDw{^_z$-rGQ*D@Ecj~C)$XIiaq~Cr2As9+-Yr*e zqQL>N!{c#8jV@7*k)ly4DO`dApYiwAvM3MjmJBJ&Hql{<(JJ{!ffmjMBX*|`elzQP z_dT6TQv(+_jlo^H)JZd)O7q&>nIVL>M1hcHCD!8PF~sFsQTs5SNX~d1s)jt_e?qC1np)JZUwX0e$cWRAVEoClZgD?g8)fQwAKZWMq z1if>tf7)>)&Vf`jwj59E%uh6mlFAp=qI%k!A&JXVdP{tNPF9)dLhOW2&yP2rT9;7g zU7;_vE=}K`2X6=8DYTm%tfhF{X$@@k{ICuT$CO&sd(Na)C;S!`6N`ZIvHl$iveY?T z&Q~u8rTRM_#&(sDD8$LaA@6e6pbG1EGz>(h%~JZ`embK~yjj?3FxY$0iZZAmr^GLE z!W8r_*Fes%)C5-WaBx)w*5|cSlcDwK0qPsdyKr2-u)xxyuYV}EbHz>Q$7{T+H5@~b zFWhl0Y8#GT{)umaP^u&gzH*}F{CbI%DR0-nY_oF7pRn5$g^_F>B&oA>*BR%z+4XE; zP8`8CLy&~S0&#V>n7Cz=7dJDsDwINQJ(x4D3XtpWFDOE_)_9rLI#x_vyn5Fqof;m)Um88! z@Y;)k&Q~Fb< zn9f~?lg3L1DA*J^yPT1Xo`*NRU;sXk!AIC?f3$Z5qo52k)l4Z<_p(_O5SZ`4$_ z7r&ERyc^>=CY##sR~BP^k_&c7d$5JpI97AHCE-GPs1SO-4Ax9osM<0eVrJ`XH9m(I zjecsObfM$b>&d7hsm(ICd`ktGV5DUGz}T>oleIz@Oi>YKgC(za=xV!D?mzuhPTg9K ztbK>-pkC~I#hG(XtTL||r-r3m48}((30v#){#o#4)2nZ6%yG4SNDTXn+DAo# z+^Sk_Al`ggAU{MSWZY;tXF0g0JN;|kb4kSWm#OOkjjzB|sjgU@Og_~BSq7CKIS!~3 z$o!NNUF27LJ4R!X^^d|EVZVBvuzJ;{-M2|gqSm^DU>&{m4~ju8fJHsDp2g71B&IqH zS|{!a`$D)j!N4S4#c(DA-)KEK7uW9*x>cgd?!>Ox$>c~|6o_5j_0TrH25Rrn+3vs- zSDZ7C5v4TN4zzmKZr{t$&hQbezn1Kxy4?L{V_X`9I-_m9lt_NIW~2Xeo8Q7o#u{28 zZaZ=$fB4L*8)h();l1O~^LM549cQZ*SSa~88zfB%i5i?;7~Cc2(o6DC$tTNmdwZQA zY7%XC+SWA{VBYDE z^OXc=K{#>LoUhuOa>tY2O>ANGo-tLlApttE`V#nrr$s$yZC3HH6!lo(bI|Bh?&4cr zc+_&-+UOSq6SKuyo&LglUm>`Eec;hXMl_4lk-2ZEi-YQ*rt(LHJ_3bI88Ox{an~I9 zLbbxXXOMU236}||hn_~;bGMG>+tHw|i*HbfvUUinIuFF7`!$Y53hU}4i0ConM&F;D zW#R3Nfc(v*`;_LJ-CH3oK*KnVVLONM;IN2vmNN zLx#}WCjlbmZTQ?sXZzVbR+!t#bveVb)ABjY=HybzUkS#Yy=qVi&R*t8)SCwjveGEs ztr7`fIT;#dX%t%Js<*w{lg<(Y&9jz~k+&{byz#h`^Yo1gouh3F^hTnDP7XPgh2oc~ zrZ69BKIqIQCk}V4IO3IkZMu`$14>IvE2tk&#Z=5~tXagmL^GL4E>gJ>+nV*qI%JO* zjv!CR+bxbwcfbAQyUNXAhkxwc6}jrOS^zx9y}zKvslN6D-g#8!qp|4etS)~K*z)EYXUH>gw`fSHNObIz24 z%$*V*nn4k@g1~?j3ADbhhnZy28FOx~ISR$rWV8DgWbL8&dvjIayMN5`xubwtZjM7& zov0te*#Ctpz#(-nKlBO*qiW9l0X&ypG?X(&@)Dw!1Dq<#<`Vw-_Z$O%vSj^lyu!tS zl5g$3sNBzZnwC>e&ARy)EL|!xec@a&{L18CeDoU$<*;2M7yfBr`WpH_Z^Juu;j zWhmT7SVkhsmANAYsXz95ZGawZBtls%{-6n9?m5OUoK4CgYRzYUqpsh*b^tT(5U6>% z2ht4l4@?x;;=+RY_HbML(G$pcZTF-5lQ%oq2G%c4y{YS!0iEW| z$q9$;^=g0a62U=wf^Bc=Nwz?zMxzMBi+jBei7Ho)+Bb^qP5u8)Am6i_h^agSuG1@1 z+LnS6mV@ImiG|XZrWcLYCJt(6*kV1b9gJ-C4+r6-svivK3+>7KqCe~$P3t>n)fow% z70=tj$WXWc&&jZ{yXbY62_J?^t)P1x9Q{oyq73#T9vv+l z+wA@EJJlKVh_h`P3IzLT1C1S?cTo_$uwQmf@V^_NOk{{;rD+wtdx*=f2m8~gPW8R0 z#rg;B1&)=QP4zxeGK5X++ATWYPyy%mZJ;HaKqxVW53aU9G3LiAln%Gko-6?e(RmsX5YrNXWyDmxJEB@`+T zlFt$Q^#+XOic z>XgxSZSmmTO2y3XLGx0$TEv!Dd|zd*Dv=NFsDiDxJemD?x2Wd2eALRT@yphx0`(lm z?oO#jojl{jR0W;HN4Y_{pu%J?y zT1ARZoroC0;r`uXxd)@1Yzw8l7noY>Q?IkoXraQDO__Rt*2II5Nu(q+~>P*Z)Ua~>g6XO|XBJg7u`+Ac`2k}J5+RskRmObqvlan|c=*%oc%{mjMOe1+ zzHkml2g6^lJuveRpEK3g2yz8$oJ8ig6%=NMJV9I=MHf*VRP$t5XnAgRiqUZG1g@9) z)PsXojB{V}10<0JMxiYrtl!v@0oID+CLtw*E7W#FUGMNuode>@qEY-el?nWzBe^VIn=&^fG|1v92?imN-*N=rG8O1Pzh5bUVFxn`PR%0iDhIJ5RKgpXt(yxe(BD>0~ z{!zjQ)soVjVUCz+?w$JsuIpL-t|HWQkSahMg>3Snt--_ZmJ@BBd zue~sNqF;7s+LoVsA<-P0Vf(LdPjiwv1D|S1w3Vj=2;jdeO-pbkM!#lS3Ykjr{gk;7 zD$m|ey!gJ5H;H%6ad=ghY0k;8C6g}Y6JJ=>E-q5|RzH~A!O$EQM3+3}5#5LlowK9e z1q~hpL|8WTxG2_WBu4PB-TBTaZ(IpKwd5l+DQJP+HBOo+i{! z2S+OC2$JTgpK4$scPy^!JRv1-3n03866unXS#$cNFX7q~NQ2>g$~&l|yO8awYie-n zAIlCPj7(;vQeHl+W2j0KyF1smo4Fb~x9w->UZ@FDh|@0EVs23@b_o z_+@jF*7I)8bA3LQ%2@lox1pm1^S1FXYDsE40@(S1$BqcH;*CT*vdO}O2NOSuw#5Fd z=!ghY8P&?oC9c0}iK3R2(Fn0s%$U}mUxi4GeKf#0-^Vo+Ljc!ga_M6K;+iCB0AG|17nU;ab^gEXh5BkQlPx!w8kRF_$GP${w`r}h4lQ-0o`-I`U z16DXQ&tOl*um(Rl#YARA;f_GE9Y`p@V4L{wAdU;5%_ZfA6Q^mEXBQ)HqnSb5|4 zZb2b?M77-VRXfXqPq`7d^wT>;ZR`aNHkL?-w?~)8FN_+EbgddYz`4+1z6zrvI0Uaq z+j$s3&gq*+HbT%VKI$OOckLUhgJRn&$Z<+JJR*?cC4Q{G^-^{Z^c>Hq(sXj^sABz$;#%-RV*;Y7V%}BK((z|$whDm4h>J#_- zCUB?>zF7WrmZ5)&l^2P#cHqpQ#Lmh!ks;Lh;V*9Kbm7$-$i_I-O*jOSeF|GB`{iPL zY8`N95Iwu=hau2xIz3-?yL7o+hxm}Va_m{yl)rL1qrGf-M7ZmEee-IfF8_*CE8cyL z+aP-`#bQ9fdMZ61EVn`_wV)&nXzrFRl(4a*m3ZZX9Nsl#6gxmux;~dt z7sP7aD;Ls_L8IRk*NQ0UD@Ok^_EC^ot>5`!2Z3~oBC4!LwmXLk=wpS%jph4S%SJ0kyKgiS z-L9H^m6h&rj}c3v!Cw zX+YQhR0$DDQ_0EVc!d+D&VOMQF##f{1Jvi8uG@15UK;j%(2&BxiDp5#*5frBuzO)}$KzB^sp z`%PY^@Hv-0nmL%mM%oH@SMDS9DMPH?g4PQVvOBJ;7-UJR@(A4}J~+pwbni0Z`ak=W zjpOVkLcboYhjowlyl`$g|Cf0kJYstfP+BTzb%VY44>odH{Y3--N8|-)78621dtb?& zK!)tGCFP{ofMS*ELrLz_vP*>P$M!n%|9?F&gg?~3kB_M9V5lGJLMch!JbB8s{$p;e zft7EbzCVwCwZgJ;cxeF`-YV?yD((0wSMtI7YyG4CY-Y?@R zLe(p!PB+GFCi5#59}Kys^jOB48vg3bBir5TKAz6giHQ;}DgbwTY}dd>IH(ij;tI_? z2iqx|rVo7vwJOc^EjRO0la$+~5)NrxHbZgn7qYujka9 zM<`ywlh%(@h+eNK5&bDl9sg0dax3rE8s<0ynq#v(OSRg!*mS$e>ut&=R%IQ~2vahv zk+I*3RY1W7_ckWX(Q-HP>ug|H6e@P>t_(?kW*gf?e9bLJG+1f(8pe8X=2q`kyh`!g z)CteaD^FG%Cw71E!wmKXsCC-ue6Z}KwD^pF-GJFRT66@}^+bA=M;)(T%(N!z>>63O zd8DZ+=*IUu5r8|c&#Cn zZJxKQD|QU+LMWL1GDg<5qmsXJH4UgUh75aK!-Si$bLSqYPY->#5GoqGC^$1#qQiwA zD3(AcqIcRtOtBFP1KjC#y?rqZ+G4Vs=e&vHX7M{*;tQUX4|m6E(+zfazHw7&^`X$U zmYi^dx^)thg_3t6XMJrxY3(e1GjO=ujUYVZ+gkD)+-|6+n!_S#Y zRyh8X5U@L&c$S7qUwrL>jqML5m<9zFu4-&Lpf{RiIMzEHYWu9}8K8|sK2=?!{&Kde z&rYT)7uZGf@$xB6;biAcF)rLzsA9c*E{SeRyq8buv?Kw|Lln1qKhoUCj(14RhwF+> zl{%|b`0)&C?#dN@P$G6``_-nK-0iTmSNCy#Y9`@E95@3yd2>C=cdLNqpR#1mfC|=~ z_QO8TlhKL~Bu>^W8bZ7YjV2A>(UM2 zfjeBg76z%nL1tgx*wS5dwWP=T&WL4pIDP3KajaKL8*lF+|FYn7(xHBJLi-&atB!!B zw;IS`6w3}w(6SjSgdeG_?XE;I*Fnb~L03P;&`V}R4J!I%(3qW)fZnui=;Q51-{ms1 z-BlfuM3$E&HA;pZvAjm-eC_o`4bGrV1Sz5FCE0Iauh5t|$AvseE z_P1Ls;s*^kRx}yw=~0e1Cun`+gn;_~Y(GplSwPfLRLJ{tjYs@}>8+)6S(?e^Rq55^ zAg2aWl#$O4t+dhadenvmPn}KT;Ow$)0LGbelOXXO>tQ;gz&W0n-7smgktr5!;g}%p zcfRtl&W%sx@32#IGFjn7{&ptS@pxl*{hl8$<9dG%Ga*T*_*r%c(cXopyfhe)6?4}` zvwscYcQ-xBYtOm1RG_Av=DX}&=JhW+&%2GC?>J8@?uz$~09lojsd&zjwu<1LIWIQh zQ3w9<3iI#DcgI|JUUa~1*G_eIB;_O?H>vJxCgYcG8kHlZ~Cj)U#2ufL{;;%0c z{bE3z^Vw&0+Xa;?MG~h5_|xn9kE?Z*>wcqSEnsShf?$86Uj<;7c<UhxrK3yvhrldK^HPH8yesF)Bj0aZ<5ZV?Hjio*wQhZg_u0|jrKLB8 zZK}P&)o(hv_z#}k1}DH0aQvi`$Z$)ZiqZsYQ?hk#jYP#Sq&(3=J^qU;X`5Gp)(b*@ zjE-LUls#(eBUWJL=e(P>+!vH-4LF6PqClC~;9?KMYmZo$6Lx+@vqd}Ze(}c2O?d3( z0w7_BxM#jNk=YV{)GQZ93YeAl$Rfv_8ceIcl+o%!^`hwgEu_7UMjk;l`ED8w{9(&`Ag_R z7|X;U8Ov9;?_$Otb+OW!#>!)k;%^X~oMw2&x6F_igVg-uierLL{;Euwx$IG*eTe?C zy*M`W_&Gce3#sIw5K`;T=B#z$3g-)h_LbPL?6jx5Q@ax~osWJ!R-aJOsoWf`vC!&r zjn_OuP?9ehe*mXW?2M&yQIni5M81cb$_!6W~Ss=pZ~#0 z9x>#e^r6DhbnqgV_gu8^4r+PuV-@|YXg|FpYZdk!9VhM7hWZ{Y^hq>ktWPPUuy&bd zkR*O|BwDD~H`WC`_WNP#T_Lc?T@2urd15LVb7C&z$chokwy&Dl;e}O2KZca&@t^54Zz?(`Dwi zK3=yBXf1Ts4M=f8nKX6?TGrx|Se+g~^28zQdp`Jyv(lvV>}vDJlPw87ngJM28F`~0 zHtPWNp2DW3WwJUKTD!bkth-)axL9j(R37=m^ay7#o#LNMI1qyI+{h`n z+xTv-x~s6Mf|>WLN;0ri%$sxTt!_E}U=Tg@azm#hXTE}g*t-2Dx#j9x2;lkfHv_^f z{Vwx8(tr6z78u|VM1(BUpSackdEx)a;s2ZmfJ6VUx))3c4~hk4xD2Y0{&eX=biGjr z%YD_9xB2u(oPT4bHxH!;;U?2A*E%e1#`Tp}7=7Bg-l32u-g2i@*Fj>luXMCDAM5MO)6+O8mXJtr(?dkJT$#3bah%y4_;N713ppjABIKh8>eRI3pU(#)(&_pb~8x(@s)Y*)ny9j8Q!M3Ryj$a4sV?ADW9OY4on9`KhaVpWVG z?TD^0%Fq_l)09?5^mN28%{f_slG}+ShDR9*PwR%W6MU9CAEBI=Zr{6~$pD8Pww33I zs@M=RSR*X)$RWC9)r?~@<>y?ePHxWO+OpP_$33|4#TjR*LfRNdBkD}+tXw6zLv?k- z^zP9{h|X2b<{;Lg?))rxXK7iBtiybBt60lQ;M#eoc!FE?IU16TXqUD0!qb6>(+V^Q zoU1qn_o(Fxs&R8v`1L2etjxs_d71{e_=(vxkujtvUt|v3`4>t7Zg>vOr8_KmmRjm9J*$9CV@QZL!+R@v-Q_D=(t&exi%`FXt- zT`g0!X|9209e8 zdy1A7R*{A)v6kAK)oju}RilO9*mNyD+jX1!$T#DCSL)@eJp`^SN3IO)OJ=5 z6I1-N4J~l4<9~yP{&Fl=c?r#|W8L}JWwfRa$8qUvF2DEzb?cNLU}XuPd;;3 z>@BZ?A1q5id)2tb>(O2AeOSy&`Dw+Wcuuc@X)GFX?>^GlKG)Z^k(e-(OLPdnxvJqi z!kd^G>UrMiD;fU}DcZ?bkpsaIX*j#`{t4>w<9bJWE4oUv9{NT9-D@cN@{d$TKU;a1 zxr8}ym4E$tNzA{a%_OseEY5pNmr!~=o}IoC2$}fz*7fS~GLtV}vlGSEqsQbmgJ)!1 zkh>i%nJVuYF6KA2$t`jv{r+f{;}Qn@jVvjfjqD{92Qk?X|&L1sz;R z-HbE4(IoMOSy;>L1uZKdQG-uno87{KNeyGO4K^c_N|VLlvHXUUOhD{)W7wds2ZU4! z*1`{Kt~$!##${Ih=@e5D7&O#Zl`Qu+z-=1@WVe+oQZig+u;acld9+CK>yV;9H%zU$zMwM@8 zI>Nqm7!-2CHGt)5Uu~~G(})+R-5)n}dsd=TWu#O+(MdL#HiL2umS*SsiMDhySggN{ zLqX7J!!)IToIdQ!pXmW72V#9WBv8j;!tM^)@EJekG@6->D{i3knD$9 zT1}yRG{>e5+QX9WcFex@__;>NOZpvpTpK?4!}K*ccUFNh6gEJ|hn{6xZ~Ps0c8__f zU5UM|t5Ix`c!>SRvLBr1XunGjhIr#(esYy=1g*!qM-(il8Q?KWd~acl2Z+5{nZjn! z4b7U2Job_pYY@m^fYmF&P@@=W&Q|l3VpAS9L?bv?a1SbD4R#6NA}bVhl&YuXLxf5z zRDD?78n6ZR%i`u;YOGu8i9V^?I_s?>yz5>>YYMPCVBX%%+!%OBnb5Y`LzT@@$=Qtc z?1Ssfp=P}D<9VqS4={fcfb=8|r3ZpRqb~3V#14{`-WI(j`QmiMT58KjnPYv`xYmuY zOz?nPYi~HCF%x+7O#0{8+a2JtUUH0|(>uAluTP4$XRs=98umISZjpZK(cPZ))neuz zKWsvaEuqTiR$Gx*;XhxpeG?BAT4CAzd8cGlfWP2XUx{6&H!1b5Z%=YJT(-Rgg95`@ zf{lV8rM;Ymd@a4jb>` zbpKBpRT6;V?`58*1s^E>V|sxkcMZVbp9VgGRDCfN@;5VzjUV6#S+t}ZdwEX{y$q*m zn9lIOaa;h@2LnL_BSWO#g}orc_ZfhI=#pEy_u43{0&RqIz*+zNMHQD8To3UF;-22f zZf;f;6BS(uJZSaedc*lg|6DtE{&0xD1NUKpAe6juTk5k)&`~i72|EO$EOKe+_r%YG9sAiPAG7PNi)S5PM+ zuK@0%w{Ow=F%HUUKrU)Z_cizAl5_=NJL+1qulAZ41_9&BZM-$RyqC+y4+u)*$rHi- zCUU^I{oT5KXZKccRUMexSr+i$Q1gL9{k*^qh54pL?@L=9nA*qKA%?vsST^efUJIJk-Us zxFtMz?!A^kSxt>rgUGRFx4l(;zrYatsG_v=%HJI_cp>!liYkchEBmpKgI^BpABA${ zKd~2$HHtu0`DxB3qP!)0q4?9vNjCt&M0KC)JG8gTtM33|iiWWM=ZL=s9^BaQH`~uY zv_1nYJMhEp{Vi(yy+wDG^iJ;0EJ+lQUCA~56Z>+w06b`$K5py3w`q~8zy_SV%eWu* z68?QyE^I4{?RQ%ONVBc+9`C+*t$_!^sXSRn_q#m@Y>DZ4p}mx+(^>xj53E-Xm+eR2 z+Aag<=_Fs^{_LZ87ZMsF8jD?DSp6PcIPssAXR-PJm=OO1w_!erXOfGOM;KI@D{QAp zR*}!Gly?%DMe1m>U;6UlhPp;a zpgu5?b54>C!NYX52Ev044c4RQWXt?}0#6+5_()ptH@4l#i<_V;bYBgnARXz&Ibg$hhcRd1>#~QmGYFS?`!3$49M(7 z$2ob{`I`>kTl#|XL!CWQ`x0l|R(K5|{Yd38zScokEuMZ zN0k*}cAT4d4Q4spaUJS0z(RXc%fzUPeHRNRqpRBStY%&wZP@TOQE+_8R!}2=(dw(& z+>>u5+YL`G|3|mk9MWZ@HG-?9qNU=kgK%$5KS@r^e9O84x2DfNu%i}{Y9?r@FSf?0 z(~JxrO+yJb<^I^kynF_rE}0M9 z-10)W)6VT(#TRPdg&NUSd~P})bl@xjmcEt|amV`^m?b7GeP<({jXW zFZWY9=_3%>jUs^inD(@ryOuwUoyLv5{9l``0L?OG(vIwFuP1;~Wb0)!+*d2^o&Oq4Y^0vtoH((00%M{r}O5{?7mw{w1$)0F=rvD2R5w{U022+FkmPKsf55 z@$6#i6Y!M?bpT4SECk;w-Z!M~cMk~+0XVAl?+&p(V!WZ=t#x$ie?Vk&TmT}AIyS?g9{|0{vuWXjdTCS-z7NoJNq zapZAlh4jhVw2xzhSJFSKrw{NeXqk=XlQB(^BU_#u=aEVV<(vXMa#K^>WXw3ROJTqp zN$<#sA0GGMjVk*jrwp(-??Ps~B@9H*)>kXJ7R>hR>f`$F1FhSR^}chuIr#*WcPrn6!LNSi{Tj*MS5x0*<&R zjPpR=uw(i1IftXk{#vKabg#NR%dBo8CJp>H&QthbGw0@HWoKSho@45e3v5SFYP`ii4|+0CsQ1+#-pSds(Ksflp|bu%c$YZoQlTGsjtr4Tmc85X z=p>K=56R+9%i;`MVvYW7*{fzvGy0+&VHpn8BvL0KP~r}o?r~Ip_;nC6*$xvN`@Y<7 z2(%n6JJy$PV_DJI(lk@}2p({V{p?36xJ=$`+~^dv!bT@`<|mHL{S$+Y0Ds zkZl(m=8QUDChL+kM3D`dbfI@;=07yi{w&~5zF!v}gQJ{BBffB!jxfjfBr*}1i3oEm<4@%yvCB1SM6q)|z zP2^a9csQn}J2RT=R%qO?H0UriT96nu_^)$A_A$S%o@P)=i;ZXGhauLK+jjlUJmCJkUaX`TA-E&6t2O>_L}3Of;A@ zsz1b@P*vv&PWV~wI+UaRCF@pb5T8aRIZjHZ_$F_Ux!7MX1Yp0+Peh)h;k5`Ij{(P7 z_^riJo*n%&9)MtLu5y>)v9Fkmtg6)|C;Z7`+wZi#Ruw+j?IqIO!VJp&&{dLk9_L{s9(bKQg6o@c;_^_|B@sQ>De;I$5-+IMfF2EiG z93owRJ^Jbg3js%g`iDm*B<_@( zd$;KO^`lzcpsKu~-(CbwAlr5ydtmC7g{nfF*S0p_a_sY~{|@*5z8;9{XUn%=>nQ8` znZS=wu>t4jUnwgw9;urR{L2tmg%${nM53-&|K<$Q!AN2|VVOe-T(Azaz_oz8K5yP) z(ve@*cz)>bG_#)gJ1`-kBq0>=Re;x}vd{8u_)&&SUS77gb6#Fb4tX}sJ{@rHY?3*q zH`1ZgMmq>a7P$el+}z382-vQU(}H%17Zf~5>X%N;2u{bgW?HS1RhA6Yt%FCkcf^}~Dyze<JdFj5GUe-}hJX57YjwpXM;oSJ&RAg>)vY zdT)1IXc0FOeiuMzFB_mv4jBV==t*oeEhYiUSYL4)-V_WUIcnHP)nqe3rxLU}-}gw= zpnj5PvmYd%E)}&buP;DAxp=uQx@GLNo9JY1_aXAL-@x2<_c>HwY(x1gut?vR{-x9@XN#y66s-T{^sXuj7DGaoN~g ztGlwG_NS*|iZmA}EyI;m#9(VNazQoB6H1fovKzq-Gdvmg|Aq@Fosx|a7fIke-#hU& zGJ=a1t#mS%`g7L33JTrGvmW+dMN;$!mfag6$u6f7yCawSTYjQC_DTyc=~*2bJxd!9 zIMzQ5PVtZ#=P$G?0%1kLi@uqjelku({3Gv(!DIcXGEvSI(a{2%orogr5z|VUaz!!( zLYwRRF69-i3q81stc{Ph?w#nyM`L;?}bh}b_6&Q>LPLLD{0juqy30dO~|gxeJdVMMMi6DpeT2= zf2=#bpg?@p;(TwjMHe0d9OYWq#*qO;8f0$wN0mp?BSGLG*XDA_j}*}PNC^sZ>&R& zmG453SqBrGEKChoz0raggp63Ax~JBvm%PrcJlWr531GtdixKjJLMB9D7(`U0)Je9de(rF z7N!<`%)u4sQR`_&IMZVl(ZyZVjoOc4Pxm~DSuXxTC4jm%{f+}<~38Ayl*cs z#Xwr(g~JB&dt|W)4Rk%%M|!xYycc@n4X#(pOjs}*J#Xef^&KzB15^_X8*GX+rlQLPEWe9cKnDQH5->|88 z?R4cGcbP{}+7nn1tJg;r8Fu6EQLkRF9haO!oQq4 zlHqkf&|3#hgyY7ERvlec4(>OzlLw5MwE1NQ2WR68KRL>pJG8sBW#^jn{0BYgUIM~d znpa&K3jO2PHFSl13$(Sp0vNgGjUOQ+MmnL=|$rtskCpm z61UK)8nKCMr$l%`oaEll*XX6%S(BD~wAk{%Q>U>P9ll+llN%6kL_GyX|%u9wy)IOGs9jBKTpe!@SN1)Ou%g! z$XxNY!U+hu>!HK5{tbEm@nn8_+>E|tGJUblz~t0Fx_fuTzvVZcR?oKMU3V)Jc73cB zv5c|Fj6b|I$1{-NRr}kM1{6F!1Lma|}VVe>WnIlIojvj)~(ksOU1?_`@+AFJarx>&X*ezq%acmg7Yz1>r8* z0ClQjrnh9C3uU3zQZ-CR`JN_41)BbaewTkQMZrn3P5KpQ#R>O{o6p1mLh zGLQQByT%%Dnl3ClZ-@*Lk{SYEmF2HK^w0en+(}@9737Wz6NIwAegXjeHK_!{#6t7? zA^=Mu;VS{Lk5Uf@6I^d+%dQna8)+-$t-xf95IgZtbSkk>HU$JIJq@q_{9dv5XaZJ# zK(>rf)uyRa)9jAmCKAZvtxt_ zR@+8mSkaU9@~SeA1h_l3ZLexq89saC>Iz)`Dn`lUoJ~L;K3%Dz1-*gS8e2c=j zq+J1`l^@3jDC;NK;Cy%Wv~hDk&Ez&&g>6XU*~M0X`+SyDm;jBpo8yol98)l^jEJnc zo-{zzHs*|TtH|=Yf=#%nG@6q^#Bd6Kyv{V+{-Cr*F|LaA&1K0{#`~9#Vp$!hQ6oc2 zU{|r~X$0rj|M)QP`1dqI-UccZ67di$J(RU8_gKRI3CsG&pK=PBomGvjlF>IjCj6hc z2PQ_jKM40ZlW4mg<;B;${A99b&-^G`pD^^P%>C!?f1)vl|C53||Jz3+M|o%&;yKi3 zG0XkAEU*@-dT3F!)XtcCsKiuI{H|nP!OnAQAJ2C})W%3eLtmhHpc1N8&VBt|1ORnP zA`v=k1a|8%$rfu$ul0PrbpBTf-Dk6o8I>sAMn*sR{QAjRCoXCqFIJ{&k45>D`HHqD z9G~cYS`(ZdaLoG+??!YXgK;#9i&LSv*CC_VPl4||f%`qKz;Ky)Z`T4CxoP*GO2hm9 zrAFDA4Y-*8Q7==B$E@CU9vTVXV!2x(d5Q)lr8yKOJ4rgKE;(sH}LHKQac)Oiwo?g zb&6a59y=5hKZq}XCK=~T^jY$5B2r2+l$djux8I7vXaZwSCc9-);1hLc?#qo$y~Z)H zR5w!Ahb%Be)&|B`I2Im@!PVIHxE@H?Yytyjf}S)&ydZB9>P-IdUFfftlRtS71YY~X z>6CUDvp(!y^{LZaMv1J5(XF(bFD-;q6_P(`4wl(>l(NNEnb#KV(0w5lw@vh|oDkO0 z7R#@|4ut+G%aL}hE*H&MP;%&kA$h^IfD?p_zvKSr!#V&{7IWU+*>;Vk&dhvRZx2sR+4)XICUMhMoEI$1#qgme~@ z0ui7*uFM-cn(;lh|D5kWpKjw@n5!7iP(gFfYE6r*WGy&XWaMt&V%izTye`e)V;2`8 zcvu)f%M0}?v5h7V=7Iury;TCCe3TikZU`I$d}?h*5qX6H8TgvD@GtaGt~#m}3-(;e zD$~Ll6@ke4Mb7MoUNM{9SJ8F%DA!rs|??X{mZpE>9AtYd45?GHcMkMuoq%4)~Esa2Tc zw2qNSJRg_2I{dcc?W4n<`&?OhMKn}*p@2G;|1@my54K$T3ZUIfnG*_o{K3cNU5$C* zQZFNIT{H*e6t*50-o52X-4i&bd7jO7FSlmARiL0(iyI$L)zPimu?xPG&%!`c_Ze?} zdeLOTAII*!l|6WLdS#}SP=2t|T;mU!o*ec|OYz)^dsc3BXxb1AlHDTjq-HoxJ3nHn*D(_`xe0FOpy+Y{{i<;fewaW-Oj$=)3LRZ z0TO6=^VJ!elRynsnYRZlNd&ekD?fOHIYT3v3#`Qn#Suh8p zqsLRi1hzX1JZ@(F2;3jmuB3F2{SANesT{APt=F@YLo_viU)bd+1)DJ%=5nZrszkDXy-a*j=@2?EKIRi~YYNBVPm+QJgG(o0o#1C-8;dFu7uGt|;`3j^)WwiL9=Pw0NP)yP%d` zKdc2TC1ng%d#Gb6{RCjk|9fQEcd4@9*jxM;_WsuT8fa6q1bAC7iMJ&&XaBU;64>nv z6qr9%UOl6!_H)R+Q^k)L2H!#WFle0R-$;n#F$yFAA zduf&U8hde_h^|*u)Q}{MjaM_%XXfTwQ9lHQ9GYH8QHpbjlZrJ!`hWrT|E`n2^UQ+N zT>0$JPDeTcQa+^_x#gNf;!F8ZoNvW#;<3{UHG_!n(L=+v_Q@`3FzK zkl}h}SFIi=JI86P{!j&*G%21|vLQauiVB@3MUT$_zEjEU&RsP?a#sPkia*%Y|LzXZ zz6cU7Xa;I@m&|o3HQA11IH3u&y*Y5kSlPO$Co73rX3BdACc7Gh2)-V;CXquTeZCWH z0GulQhin#k}_V+t?1g%X|+#J*I3S(YWLC^E3wTEE8Tw{9eZ;kw8yOFpmNHcE@i&R6nFU4v(|gc%;J<@$@c>C;0%MX+2_xjM{@ILS3H|{ zJ*NYWHy+GK&8-YMJPw1jKw6W37KN8d<`Pp@^-wSn_HR!rzYDXtWIEQU)71TuaNB{9 ztOy_l+m%REuwmK(!&l4qYlNgYb|pte!H%r#^{~p-*zFNhd__E~>G9n;xz6-V=z;jw z$qRfFPnda1trDf>ik%FBf+^+iAr)p9SmF)~_Am7P{9N0v+sS;RzZK`)FYZJipz_7p zilL|y{7n^Vwj@^&QCV4t)wyjRF*5?Y zjTU3mAxxP&vAVZMXSF+W(E@(o1$8gL;UbVxpv$>@1*uda#kMMiY` zq)qV=%NZh=uJKePRWgq|t0>Wfg1LcHz>9gZ99! z+XC*DE>SRcqMXuYyhQ0yAcy$WxuE&2=IN$dLh`p;d9ZI2nc}a|Z;$-OP1(Z!$7_=g zTaKEV*BYfkQC|V$r5KB4?6z{5E8XBUOh!=?y1Y>#Uh?6#@}7GW?8INQaR+8s&E3#j zDw6$LFGQG?cmt(5S`*3jHTtN5K)o0Ah~c+Z>$W5&>kc$Xs{3qpqbnlenIo&sx5JQG zy=uf70l5OMXV0Z(vb`8D}pb_mL(Bu{W* z>D{z;aTop(!YV9*t5rAnL6SgeFQAN|EnyLNr8+NlT!Me|DN=%WM9;+SCMxVRHMho8 zf1WEQIyRSos_o=B0*T~q5O@1_DaEG6I7MGtlm~BK>Y8P=ST!9KY}Qd|uVDe0c$POT z@#1u(l14II6tCB@K$~w@x>wjkzxmI*Nn$Tyo{KSsJB7S)gIOAErvrU zHezP<>s}txSZ;VRF`@fQl=%)UQY|8vNxv4q?=JK}o2Uo_K?;otVZ6lwEdH)$GwFNF zRt*a9lgV=GD($sNI+zT2(1_)qT{UP+-4GyxQ&@ey;U>{pE9F*F_e!NTe0LtiS zh|!=em^0`9dlJe25?N$vrxfB>?SO+M?|4g%R-1~fJulWhN@sm&Kcm(QdY^y;f7#PD zy#987gKKd>`7bRnhxys8T(jVYeiK1791Iav+7C4MJY}U3F8Ri=SH19P^l8XV9C)@C zG4c(>uB$N>A32ezb3NwUioVa$j&m%2uWuFFY@8)2z5w1(SGgG|cS_XqREJb>z&<*E zV3_o+vGqaX%cR@bRZxirv32Vfbx z>s>U=_!`Jiub!E>7qd5Q6nKGO<>mi_ySe}u;(d$6Y?{ZD5wManeZ5BmOuT1+%d&pU zrcIkg|AE0!P^-lOnpywAzyXJMwCmkR7xD0KYixxqrXS(&fCcP4eBZ$V=2A51^D7Bx z8X9#59&qjTqgpg?=IXr{&I4=@q-o})|H2F52AyrDS7C2vLGjPiE;&M*_i34dkbvL$ zoOK1|{%tdr^_ME;*8qoOtoZ>KKD9}0eR#jYEw+KlzJ_5=d3g4^zEa%`NZZE#XYq@C``(rYVn#d^%gOB1H$s@zqm zUh_+P_BO^fuy^{stOR}{*q zkneU#5f1UUqqaPI9)A6B0D=7BMLkg_4C_n;_I&joFA~Wq^IxOL#Ip zTa*cHn>U!`qP@01==^JwP_X8gAd$&jlr2aiFZunmxFgLeszTRU2YxX0aWd-%#66RH z%w_8;-Jdub#upS&VHi>RWQo3_t=0%F_qds$CQ!Y=;y>EZ+|1(^=gIT)&s0$7^(cco6s5xA zDA5~(al}eX*Qk)ij+|%H6%tj``KjX{_*(p`j69Wcs#~q+p5Lb5S($t3+FbfvN|XHX zRqCD>API_=`1I8dHQKLVYPJa=e><(~D9Z`eGEP6~G?;{?9g1X&0w)Bc`{m$cj?K>u zcV-Q;cx#Ou3_XA)x` z22^#*_f z2>JadBM|H&n?;+C)}O#~Kzxt#c{WfsB14pBx%v&D^&G=eKWfm z#%ael_27ygNS>^xbQZqj_881Vfr~Fbwq7+Kh3NP9x8{_Y>m+arN}Ln}=T%&BtUn<$ ziS`e0Uh@do_xcTK{8i{*2BKAT@fiB)e7(g7Aq1?T!JpNUk2TC$SK#uBGW9`-^)XH1 zy~+3)O7|Y9L1($4wEfxlLOuECXTH5Ri&ua5tFH4I`g?!*-UO_e$%(QCzl)#uAN0Da zP&O|mz(rf`)Kme`G`TQzWA6t#U4iwaLvfas#>_n5o&LabW^CNw-F;l`ubYSd`6<~r zbxo>uvVb=tf2`fDsG=g5L?ZE0yAJIgukf)EC@4YXtc=Kxg<#A!~S={0RN`|BLC!9=&3171QN+G%u54sKp9WO z-+UR^+XrEA09yi7nkd!GAF6*&n>yEb3rPwK&quXX$W{c&^!U&IDkNb$5WfE(AAf=@ zASpBL+EbJ)5|m?&D)xKI7Wf=7`P>X3T7>HljVmR8 zgBR%-Y5k2>^ii9f-a48Vbs+}V>&bjEa-wPUkR-1(`l;&Ey3+^H+wap=DE8KUq4M_Z z9aCDX64kddR{ax|s(!@F(T1_{GVA3L_C|Lhu;VBFAR~2mwL9eok+CaEgr?l@K_3HM z1qgC_&Y-do3uv>kO|H*%WUr{eRDQXT{b0PTykN^2-7a}ycj%pfSiX5vW6~*sVnPq@ zUVVuro&*`klH>J2@WL@ltHENvFg8EmYSVc*uT3ycSrCGYb$B`RAfrFC|89m#3dWt? zq`DYPTQL}Kf&77Ilb3VXMWwwsXL7F0z4}%U+0D|96`t)}Xab8!IPmm7lP*)dQNEuX zW%y$^M`$y1`MGYKn_Ws|KS~`Lv;WFUFpR zm&?JM>^!jE-k3;xeCty?Rw3K1*|#7ccq#6ESZZO%Gp+1LQ!4^|m8b8z>X%qbbTYq< zt6a&7%vbq#rn%`WqIv3Vo@O8Ho|OS4E5_6DPQ4PiYLy;eX)2I)&qRDg(Q=B#YY}%^ zNpb`xM^S6-T?1R_u*Ev@qnw^C0@)}mSVQUw2Grh`Wy?K(nC@m23M zr`44U4n)P10>#i;>f3H$ko{_at*I@u`F%@Dq2M~5DZ0v~Hp<&p7m_7bS?!B3_`C}~ zAXeX%PhBa(DxvjHMnkNK(t$ErL78%oof=}KO&sIL;Etum)9$3D)3il{)BpQeeo0jb z$e3v7>ZRR1l0NQu%8psQB=g}RaTi4Y6GE74RU+HTwztEMFwOLdtTxfKXL)ifeZyAh zp&Hw=1i{DkJ!tGePTPQJmZeM(S^_FPMwA8MhJxoF+)z^n;D+d@R-VXTV54?q{aUnU zQ$iT9p^GeZ=*-uPMk((sEyX4dX3xoCbuUGGNF!`q1yMKBT6^@`J|r9)nW^0{W5!6# zi#ZZdspEqHlwkPUy$6UeyV=T~LfyhM zw?4qvY#gr@jBG*)mB$OvngYFBktmcFvXfS}${t2+>!2N6S zwCHUH=n;XQZrQXtnZZ^23PJXeb2|urZjSM)Z zyFTqTrBN^q1r`9|mt8d7krAcv(b)=H5*rs+TvYV+-!3i=a=_vu>Urf`nq8SF7f^#? z3x&!)@++ccp{zd&09i*!NXVbgc)-~z^8AWf0f33ePQv>kEwSAAjQ2rC=-PRa&gDQ@2B=2 z>^{2xtyca268OO_Jc5W3bInO_P9wI&Bbpo1+?QX^J83<`MpRxx++U}NFAnG@*pn5@ zuVvivL}mEU-ov6J2bmy1&Z2gao#%DauzcQO5tl1!0#M#84ZCWC&Gf%H3`Cwr|7tQ-&?XTe{AQbvzgy-X;apffje zr|-)iuR;K2v|2$3WKk`4tlo5M_Xemyzcy*0F;mnASOYVoP27X09dGNkAeuOZcS7jF#_3_9mbTaYOeh`o?Lo89(7+`;q2kETY^Tje5WBiw#5%mxMHv zE&EuhKBDo(+x;B!id^|SIM%SS z1DLcqUi!qqq+x<|LO{#E@*A!PKBWh)1+jBJx(pSBueu7TR7&V4)Gq^_(*Ko>QO$_e zgsF_FYE(spfI2Q(GH1kygP#KEV|DJw=hkWwj1j`HJ$vV?a!bxkv-y)iyQ#SuyZd2_ zp-r2h2LR*=iFhlN;8vGajTa6=Z_2@_VM5gBN^dUcSw0+}!%ODZf<;~9b;w%!Z>+9_ z`F?Mc85T?o5pw5Eb3xhj<}Wmk79vi2h@a^A%I@t|YEBssNwInTQBHVt?>?hW0{5BT z9Fv!471^Ae(zTH46p}4fwgBtQgAhiB-`-Hm5_eFY0QeWPjo6%27#kW3RKxQlqL)L# zv4Ic`B@a|kaRg_b(NzE28=xx7CMJu~D+v-H@v-(fus;6|rnlq*yV|18v;|&{6042Z z(YiT=T%{Mer6@QR{bL?nHs7-pxGB}u%`!2^*_<=!DKZ^%V>@zz<$-0AVDEw1c-T;D zw)*TsPkRj$L?g|U=>X5~c&>2yARPcvx_>h?HVpjYcFW=Nti`QLeRT>OK1xlH${|rm zH4|yR!*+k|D#v3%S1-ef(UnJcpuz*SFbqyFo$A7z@}alaQ2d1yl>LmuFRYW_PIM^D zGcy4I-F!#sVC!k=EKMfkZ_iI&y^LRh$>43QQDG++H*JhGQ83xO)A6TUW&7sOU~c4`D81WtIB$V^ynQ7hb=f5oJe_|lqt^r^ z1}^{ja>o<=Nsei%x4^U0TC!*QRoq&$gVk82CE;b*P&D*9;>$p*&(oedL)5gPiEi(D z&*?TXWnxS;7C#S2YOl~eXmKAi^gzXl zLC=M4%_u-k${)p@xd!Y|VWXx)znC}P@bWnWMZYe|b*@8$91<5TD2R#G+Pu?~XA-RW zYWl?T_b-VjD`8@;2VTeYuj1xrr`lp2P7-soz`~YSH&oHX_f`HP=xnAAeF_P&2q1B) zD)Uw*_L+95wH#uBwA-xS-Kh979^Kucl5MTp;=c4G<6~l@UIg*`;w{g|)KcRQk zdi;g9gO#4r+qUcl`r=_#x8(c!W!G}oUu2Z6^zaF&7_Zc(7R}z+osSb37vCfNV_ScP z3|J9nOqHXGBcETmPGb4Wj_1&wYmMfwLVHrFC5cKTr)*#SK1$JqKvyhgYhG7NT0Zxc zHZ|u|i;~bxa)|`g$@cr9p05gdo1&t@{Ng_gIf&`q<#@YYoLi{@q17SEq z-yNMYxMbwImW@Nn?SaP^3fiXLwMkOAHDx8w6|Bef*tWaj^mFKgHltc5+CSA?V6;~& z6V=UZn~FI-r(4Yq@m;*?iecDsYT-6()cf@_z;6w{p~Idex}WE!&@`P_DgYEKIrfQr zFBkuQ3`ndaUNq+JEvX8Fz#q&wkIm7*758m`8)Edxqj8U~RDjrxaxmDfRo2%Z|F^P* z1kc~SMNUq3iI2d-fs&$P42C;he)tdST76nG`8g&gMqxh^0AxT_(ZIy9RGGa?0?^@Z zNo_x2_jt{JsVb^ecxU*a%zJs_9zE@P|Nn?RKvhM4ir@Xl@Jv!evGqswKkAdVzr$h3 zp_yU)abl|1J9Ww5gMRNJFm6c&sWjd3BOFLzg^n@PK;sV}YNm?nXwgQ=4QR{?=1ZmZ zKw;$qNDjA#5G7iQmJgud{Ci?FN+b2cXNR$?(ii`nJ<`-}g_4gKenRD+|=rBB{h zoebyl%NFcciZ9pFnEIsSODE)DP{kf|wJrO^Q~{K(u8#lLH1RkK(5^;5l?{JVM@4g1 zVK19GS)R1VlFXbKjRFkKc87~lZjXW;7w+9s)OQ!+tdb8fj5 zwJ6a)ep-f#=FBTb3X72>`Ybvw_r!I*T`PxquJ$|SYJy-pC?z|;(M+J}sUAV@L(j_s z>}TyIm=|H#^JdEkh6{o$@HB!uqR8dQQGJyA{O0gRnC-k+wO+dr_!_csaV6d=3ll%T zf^>RI_1(54g!7@2GGTX+K}suS8=K?5Npcw@)A=#gNT}J<0Az=fppf6SoQ1*9dC;_Y z2|U>j6@SZAz~v(RC!2uY-L!aPx6{xu$G;V{=Y5)<5T<5e+&s+?p1K5?@kyZr!D}{qeL>FOQmf`(Q9ixnN7;vZZA%57Osxd z*a}W)-S0SeE=Hsbx=U%j>zo8fg$oH{PUihWyKgu(p-)|* z#Kd%)Qh7wz-NPAbmNDER0=^Yx8?!S(zAE?zJjtGDXHq>5OcSUGV)A{SD25tqkfK*y zhdCr0o16B%uM;$q%e-FwE3j-SG9Ek%sgu8w?Ogt`Hc&Bx#nZpd=M!0g6b>0LOs#G5 zwY)k6uC!w-3<8EL!LbZ(*d&PQU@|GmG${d8x2z?>z`QG$E{m_3EcRF0{RM%Z!KmXM zXWL$JD!|1vMAoNaN>Y5ABiEh_WqtkP13uK|R$*3aqpzvTD=dhO(%@j5*t4H$8j@OzyGiZmS#6wb7s8IpwWs^?-g& zs~pOgHNuWQHzGw!{_viVq>BI0_iBo4NrwMTn4OVSYG!7(9rw~~8%>c5o_lEg`S}4$ zO_L}JCE5eU{fuK&vD~7^#w392v9Tia{+^#pVFp#l7Jr5M!Jl*nEReY8XQ1s&^3#uH zb1Ef8XMA(K&ZXt*YB*MNrYo1pe9*8pDUpLu=dtN-R=>5|Dx%zib$?{bji)%Bjc9Pz zO?l(0-v>Gj-;XHBq@TjU(90Xqk;Tr#gMQ*-8+TsH_ud@4eBn~maF5a=taBmg*`_O~ zvl0r3iL2@<8Z*$bZfxs=`%D9w(K7Ef?0IL~YhhlJE7=_*VJRZnlY!UqtwzM!ypxrQ z3-EygK@ZM+Ozh}e-TCF>75!d^Dm5RZFd?DVs7C-Bi#gK(LO~rQ9aS1NgbfpheZ*Ob zJ^}RxyGyaZytseOSiXB((lNoQ6Fjh3+)|+J$rIw+ZVyC_%bdmFCf_vqu^wYYnATn1 zvwmJrqj=Dw0U-BI&)DHr{LNdW&hIz7nS!g3eCzI|geJW^+b(9oHEZaj40ddBEV zU_&u|TRf}}>Jnyx5NjO)@(6cZlGET}2#6Z{c{+uigdD>-*(|O&>GfwKg9?W17Y>|# z4D5NsJ!o*pGr{QlZ+5~xx8`o33?&i*g)KhD9m~zWDUluRjSAfOhDnT>I(Qo(T16)7 zD>bGou30s7tiy<-QDmqGrT@2W$m?$!zJ-~+@N|w!Fy#OHs&3fzENHrx!icXkX+A>w`w}$;#BlQ zu?$X5;e3BjlAAq^d!K zv$U#*)dV^Cqw`2{>SPQ2LVW4m13giT!}W>r<{?~5QVw(HwDVO2rC42U!Kzcn@OH+l zhG(^XY8C?cW`d5eB__P=ucbFh0H{kGH~|YgUV3Q+e8$_AY7uPFRPMru_<-tg;R{z; zcZ{?-BiWIt`wKvB9|W>MwETVs_uoaH01!R~AT6%zZx;^kVQ6&#oVi@2AyOH@bW|E} zD9VTidKn>S5A1=gnxlZNiSY5uGul?Hz&<^Vdp2+0&jA4BlCi!vMBCi-{~&lA;ORWN z=LI;(%6RcJVvGov2Yf3ct9=2}!xs9rAq zKzSdYv|$CEqP#BMdsh4Cz8owOA2E`5Qmw+IE>3Oh!P~)*>VaSji_K2?CuY0E z^&y;FQ3yJj-LOrJ&n?yWX!#>fu`&vNmL0olEnf7ePQe3ciHN)Kvw$Y-vt9+#kg|pV zTUzGw5SZPv$9aTkk9Z9DL!jNQY}zCE51j;v69#}cU~_Zx0())qpXU+5|NmIv|1~1? z{|q`B5EV-t>;SeO=tp+z`u9`+{0vT8)J$HBYJ7QX*D267Hr+rr>7! zaJHnd@c8BLU}gxv!0`pITX)=o_d==tli$us;CNTTScbjV??p`%2jgy2Q8nMh?7bG7 zK(P8xFme8W6G*z)uI=EDIF454sL5@-vbx9Ix(;(0X%28MWfva)73Elu?Pp^Ins$<= z;!GZ-n>X8EGHvL1JE23zJ(izzG3D1w&g!yXbmH#k&~@I5sRS3XiPNjM%a{c3|E56B zUo=&%UmldAGdK-cOxsUG$iG~j{_!7q7Lm^}*OrzH z%-RUY2b(>P6HDy029N&w4tp8+?sJ%BK@QMpefK-=Mqc1srC>0&2mb*-; z*k&mOQ$q>ZZJh>JY~8OkRi#6;KDf7Fm-6rO{nod0QP-ezb=IcOyn8vio8h^jtYhe| zX;60LCYGb;%-NUDdU^g~;zeV?Wgve`N7$&XQ*^m^jk)6)s6yD4kvzOMK*EA03+R#s3H${6O1a;c z;k@Tf>CXoUv3j3EaMBiL{c9cOMXOdJcFD8XL{sw`s~d@G^PjR$f`l+p=dN%0{_ zf_rO1*9|0SXA5P3aVSOfxs7~*;|fD$)09mfh$a^}aT+I$dNETvb5tSqDt2Nm(6M|( zKs8JClluPFCOJ#F%ca8}6m_8y=Q4|luoyj{LFmK}JN#0KkxYtOeYG#T#Mo-|2IPtm zal5E^tpN)k;OaQ!jWpRQupAcWQUb|a;gtn9rtJwg;eqI*ROyLn$+Z$-BSm|tlSx8| z&8c9L1ga!c9<>O9cGTcTGfDP^!hW#Ihx`01cBd6ocn!B485Axu>%Q`mVZLH*Bc4b# ztMYuaS|3k@`kJg&Ig{&i+kMwE+kFc&y-kXe@06KTW*{Scazh_Gn#6RFx`qk8oJF-R&9gM|!7goX6>1dcc`R zS^x`8oKAJt1=3)%dU8gLY|;~3-pnyq;%u$b6E+IK_MXxOTns9g0TDYt!!(YT zNT;3jUph~@g(;tyR~I5fGb!P1#KRwz@g%j$Xr4DRM9^6aIS5Yp@_cx~>8L?W9;jJk zV>~<0YO~j=uhC;{Jo|X;&l~))kyaq1sKRB2sOE6ymSM@<*+MOWDe+sZx|>?0!8cs( zIlU+N6E{z0#Sc$u-xzCJzldt)E;%8uBTJMf&#tn$HViL39Cy=6gX*zkhbVMNDWZl* zgrLjiiL)09{LNH68()uDw4BN3zR(e7JFZno;(4fhiJfNNcA8Ru5*5ZOO7Ej~Jz z>bFf}ZS!NRk15o!4Jw^l>d&bH%Kg{;Q0W{P@LdsQV0#^8>uk=Va77m10GJfU={2{W zb!Nv`QLXAGGPI?;uhj8J&Rtipxq417GGcm%g(+or0pd8f;N)}r+t4}>m}+I&CWC^m zfk}vj&CM=PE_q5$|lPrrd^obhqc ziA2oC30pFea}%4!#XOtR6H;wzIduvYV-qVdJXXEB7!%y6UX(Y6V7s?$ezuQCnwX;@we$j|mr zrpm;1Q@?jl#=$EQFjZ`>ZZoiATAu)>leJ-w&yHBf2*TEUGIP{;xpweCTeY92_s+KH zO7$&i(pJ{ywA$fki>Zpln5kFXE~#X4YaSWsNB%cac&$pJh{R*tPym zB}sgT&1udXlMbm`Y-E6d$ug2+ju?$ZLi=nW%gzt?HH8)VikxZ>S8a_fYW5-39XVra zmkO6V=3T;C{n}b>D%scS$v_pXp24pZjclLwFspi;v|UBm_65RRyt5T1I#CbVGjRrz zVzbDUCm|Z$(_2iB&(U7tD^x6t+9d~x9EigLr=^Czz}_?sAf-f;*M&O$y3wdAJ|LylBx{j^@T~xSU$V`jRrm5 z4o6O59Ez~P-3dQ!U;ogW$()#Q=%F5$zux095i~^hb*)Ot5y)Tl%ZZ?_H$@-9zEwm+ zMQ>+&$)*%M?zhTR-YH1Gfw8dN0>o0HvOPk&=jxD~vIkQ_9C!{@j+nJb`_9y(LYC8X zGOTKImm!`UwGnkKRV=gfy9JRz_KhW!bIO~jH&J-HRgbEEBZmddwr&M9FweJ(N2_M9 z1*3CSCj2frvU8PAA*0tAdB1b)1_|U$rPEQ!^rF*f?_(<`n-Ond`9>#dmXcs-6i@z~ zJE16DWS_WG&FUI1BYZ4>DL{&A2gbn<=W5;g5ftQWUwSwt(#Eg3ya?Uq9$H#7&oFJT z9Zdc4RMRzuRPD60o*6XiRQY*nTWw|@YbRZ0ghZXrSi%2#o7Xf7RaJAG%R85@T-^k@ zI=0!)DeM1)P6$f#P+qEYvRO6M`Ft`=P_L!j5Kt@kuPuFf7@`(Y$^r@X_d5m>5YH+nuD&HtFpClJs|;izU#y_ zAFOpLi)xuSa->`pVkIOzp{eUk?n!!+3$?XRQn!2?B);J?G01T>E)gc=l+xx!Ah*G$ zq#d?H$wN|MS<#y4RMtr*Eft>CNo`d)WR*^{pi#S!H*aRN6t}dLSVX|=Gr(}@-Rj3M z&0~at)x_UM@2HL-AR{`6JEOcW6ry$kL{Bbatf{pq%qO?h&;23KMoRV8h%CF{Y-k6V zREM7no#GZOSO6)P+G9A76aRN?P)c4yOd=@B&k9)Zaxu1O7Rk|(o31O z*+^Y=fIsIGnt!)gv}|rHFe3)KUte%L!)IV;C=OLv`B9Vdd_{VN{sf+*vb0DiK(FMD zF5L3=knHFzF7xzpyhN+qU)-&IYGK%4RNZy#r}nnSM)fBF1qEGgRT*hBkkf<#7RWJhnqAt`c$3xQCn6UcUBgtI73xI6#0qFn4ewN88Fj9)_IW! zzHm6pMc2A|ld~qhm5_>+6&Y!{8_3XUnWol4hcoeWUs|xSl`w#;>~b9X%DlGfxPn

WKdR9fT$lkwWSqe$rnu>tV(dVxZTq@ z9s&!ULws;GLqi;lXAL#3yYBbwVP;Q`-ncWnlJB%4R}>n5ixocl`Z}xT_*XaCPA#S8 zh_AWW;keEp^Fs=vkZI7X?((85niJb)u08jB+nH`K#V_~q$Y@pjr)gR*r^K%)t@-IW z%~@WNZx?)mQnpPcd)tB%qey}du2MyP1g&GtRGz8k&cDsfoS>@}OO|Pl; zN4!zt@@WjJzTm2LDfe3mbvhyGuuCLq!fXZM~@7=t%r1VA!`5KUScW`kLn|Yo^Z<&$wD}w!> zaUHBZZ>stw>(C}UR#(=@SiX%r_DBw}>q#v?kdCsJ4CiEN137tlTAh&)bILDXc`57M zZc?6g?l_@pWH;|lmF)+1cHB+M6%70lul!56mdvsbmfQepGt`;6>VF;8$+`R0sQr6m z?kwQiqny`nK75J!Ee7AAlLVH+I1?}V<7=-%Oi{AOu1#1si2euwTDOF_|Ki;4Ju%rX|BrIbK|0Y13)R`5Z!`T*S0fI}xV}D4JWu8koT8DnNdT># zH9^Kh1E@sPQoh@IXWzZJ{xVuYo$F^Ae`fzvKttPTc?|5!`J?Gl;WZ z2gt{-hu%J?c_35)W?ohL3XQNl&)*s4Tv}O~2b2%JCP@Fe8kxucfvQ^39sCvLtMlX^ z0JdbvybyJwO^f&6RVu9kl}cA00ehYZtC!8kvg*pidk`{UO{jb!)+J#Tj`sJ%OVtJWy}sCLL+=^QWJv{oiZhLF zLu+=^)uFV}mCU2n;D>A6MPk@thI)fxQowS5|4=$9a||)zgrn+1{ZQGpIA$DjE0lK`xwMlwbDr;LU zoZdgy2kHXReALNgx1BY=&YviV*iTMPtZv16&|#Al>j9IUsO>Fzsy^O>%0`gY>8z&i z_#vFO)*?5PQwBzPT5X-Cy`Zs+Qzr9BA&zy@Q&Wv;&Y=drTrcGLlYwgMA-9h#+*e0%Jz&L^ z9gq8d3NE0ST%Vi6s_sJto$EJGNkHP!BU=YlHtI3|3u>sN$gr2Ut?EuksBV#vcot@d^iUCS5Bjytq8HyH z=1rQJA+Ac><3h6-geHGc#>rgK7m_+7Btfq=l8G(w#0;T}gPY1MZlqL*EoA;2((+oo z9Pfy@|I(%3Y`PnVH6vu_b9Nj?vm4DnA}nI_1O3=`JqO1CaT{TVl=}H--d*$?^*sQZ zCVD6~J^af(bEW_L$!qsArb)GG_R!VYxIR0^YQvtPMZeUze5XfK-d_EXGbF91gz|)* z{2g06V-_us9o=CtIoT1N<(foly@EHtKaez2(X&`(g7^G!20PIw&o|O1?6JrE@28Jf zYnhV_^&YW{0ZDKy*s<>Ousu1Rugd#OQTn+pm~XK)2;PwX720bXj}~=gK-zBUK&KrCR#v3#Ks-*k{*~&dadJCp zksTe{0|t~ki`zXjUTD*Kf?ox^R0L@|zQ}PHa{Ex^Udi5PS-P+>A^>XYDmRVMX_P_| z`6s8e@l#i(q0TL|WN`~1|AWX6e6kFy%hD$BS7(i))++6Yv9ZxUT{3MHXUd8*&<8v) z#BB$VASJi!9&WDexagMh8Bv{8(WBK!ir2@D9nma#KcGEOSg2pGtY*Axn-N&&^I|vD z#=@v{AblgHe5_5+>8`Hd4Y;3OKc!J*Rc}hjXzNK0>>ghEQ>hy6-iPve{;zuuLF;4J z#`2xMQU#*lOUaU&h1qpRm}Yex=X;+(HbMb1EF)vsMsF6lqt#e)#k?z)946j?roGdb1a_ieV>pUYD4SED0aDx(?UHf{?G!ZO_*CDnHH( zeeP4|$ROhm1^zQ00I~UUU@@Y9*x?!@1o!Ic-opHzZP3;=Rixp{L4FgPxZS=73&tU=Q6zv$^^&#FLD0c;VpF6M)3Cg@7IE5 z|9juIJxrN3@xyL;EVskHh*B*6-MXT + + + + + + + + + + + + + + + + diff --git a/openmetadata-ui/src/main/resources/ui/src/constants/Lineage.constants.ts b/openmetadata-ui/src/main/resources/ui/src/constants/Lineage.constants.ts index 5d42310c619..6d673ec2003 100644 --- a/openmetadata-ui/src/main/resources/ui/src/constants/Lineage.constants.ts +++ b/openmetadata-ui/src/main/resources/ui/src/constants/Lineage.constants.ts @@ -83,4 +83,5 @@ export const LINEAGE_SOURCE: { [key in Source]: string } = { [Source.QueryLineage]: 'Query Lineage', [Source.SparkLineage]: 'Spark Lineage', [Source.ViewLineage]: 'View Lineage', + [Source.OpenLineage]: 'OpenLineage', }; diff --git a/openmetadata-ui/src/main/resources/ui/src/constants/Services.constant.ts b/openmetadata-ui/src/main/resources/ui/src/constants/Services.constant.ts index 2babc958d2f..a6b500e6364 100644 --- a/openmetadata-ui/src/main/resources/ui/src/constants/Services.constant.ts +++ b/openmetadata-ui/src/main/resources/ui/src/constants/Services.constant.ts @@ -54,6 +54,7 @@ import msAzure from '../assets/img/service-icon-ms-azure.png'; import mssql from '../assets/img/service-icon-mssql.png'; import mstr from '../assets/img/service-icon-mstr.png'; import nifi from '../assets/img/service-icon-nifi.png'; +import openlineage from '../assets/img/service-icon-openlineage.svg'; import oracle from '../assets/img/service-icon-oracle.png'; import pinot from '../assets/img/service-icon-pinot.png'; import postgres from '../assets/img/service-icon-post.png'; @@ -168,6 +169,7 @@ export const FIVETRAN = fivetran; export const AMUNDSEN = amundsen; export const ATLAS = atlas; export const SAS = sas; +export const OPENLINEAGE = openlineage; export const LOGO = logo; export const AIRFLOW = airflow; @@ -386,6 +388,7 @@ export const BETA_SERVICES = [ DatabaseServiceType.Doris, PipelineServiceType.Spline, PipelineServiceType.Spark, + PipelineServiceType.OpenLineage, DashboardServiceType.QlikSense, DatabaseServiceType.Couchbase, DatabaseServiceType.Greenplum, diff --git a/openmetadata-ui/src/main/resources/ui/src/utils/PipelineServiceUtils.ts b/openmetadata-ui/src/main/resources/ui/src/utils/PipelineServiceUtils.ts index a54d0e0f940..7c32d177512 100644 --- a/openmetadata-ui/src/main/resources/ui/src/utils/PipelineServiceUtils.ts +++ b/openmetadata-ui/src/main/resources/ui/src/utils/PipelineServiceUtils.ts @@ -23,6 +23,7 @@ import domoPipelineConnection from '../jsons/connectionSchemas/connections/pipel import fivetranConnection from '../jsons/connectionSchemas/connections/pipeline/fivetranConnection.json'; import gluePipelineConnection from '../jsons/connectionSchemas/connections/pipeline/gluePipelineConnection.json'; import nifiConnection from '../jsons/connectionSchemas/connections/pipeline/nifiConnection.json'; +import openLineageConnection from '../jsons/connectionSchemas/connections/pipeline/openLineageConnection.json'; import splineConnection from '../jsons/connectionSchemas/connections/pipeline/splineConnection.json'; export const getPipelineConfig = (type: PipelineServiceType) => { @@ -80,6 +81,11 @@ export const getPipelineConfig = (type: PipelineServiceType) => { break; } + case PipelineServiceType.OpenLineage: { + schema = openLineageConnection; + + break; + } default: break; diff --git a/openmetadata-ui/src/main/resources/ui/src/utils/ServiceUtilClassBase.ts b/openmetadata-ui/src/main/resources/ui/src/utils/ServiceUtilClassBase.ts index a0c3cd00420..3e4aaccf0b1 100644 --- a/openmetadata-ui/src/main/resources/ui/src/utils/ServiceUtilClassBase.ts +++ b/openmetadata-ui/src/main/resources/ui/src/utils/ServiceUtilClassBase.ts @@ -59,6 +59,7 @@ import { MSSQL, MYSQL, NIFI, + OPENLINEAGE, OPEN_SEARCH, ORACLE, PINOT, @@ -335,6 +336,9 @@ class ServiceUtilClassBase { case PipelineServiceType.DatabricksPipeline: return DATABRICK; + case PipelineServiceType.OpenLineage: + return OPENLINEAGE; + case MlModelServiceType.Mlflow: return MLFLOW;