From a25f45d5fb59cacae75b917de0660fd5e329dd9d Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Tue, 24 Sep 2024 16:29:54 -0700 Subject: [PATCH 01/25] use parquet meta to estimate size --- .../model/compact_partition_params.py | 64 ++++++++- .../model/compaction_session_audit_info.py | 26 ++++ .../compactor/model/delta_annotated.py | 19 ++- deltacat/compute/compactor_v2/constants.py | 3 + .../compactor_v2/private/compaction_utils.py | 18 ++- .../compactor_v2/utils/content_type_params.py | 136 +++++++++++++----- deltacat/compute/compactor_v2/utils/io.py | 18 ++- .../compactor_v2/utils/task_options.py | 135 ++++++----------- .../compute/resource_requirements/__init__.py | 0 .../compute/resource_requirements/utils.py | 127 ++++++++++++++++ deltacat/compute/stats/__init__.py | 0 11 files changed, 410 insertions(+), 136 deletions(-) create mode 100644 deltacat/compute/resource_requirements/__init__.py create mode 100644 deltacat/compute/resource_requirements/utils.py create mode 100644 deltacat/compute/stats/__init__.py diff --git a/deltacat/compute/compactor/model/compact_partition_params.py b/deltacat/compute/compactor/model/compact_partition_params.py index 1ec92b50..9e9f04cf 100644 --- a/deltacat/compute/compactor/model/compact_partition_params.py +++ b/deltacat/compute/compactor/model/compact_partition_params.py @@ -23,6 +23,8 @@ TOTAL_MEMORY_BUFFER_PERCENTAGE, DEFAULT_DISABLE_COPY_BY_REFERENCE, DEFAULT_NUM_ROUNDS, + PARQUET_TO_PYARROW_INFLATION, + MAX_PARQUET_METADATA_SIZE, ) from deltacat.constants import PYARROW_INFLATION_MULTIPLIER from deltacat.compute.compactor.utils.sort_key import validate_sort_keys @@ -104,6 +106,20 @@ def of(params: Optional[Dict]) -> CompactPartitionParams: result.metrics_config = params.get("metrics_config") result.num_rounds = params.get("num_rounds", DEFAULT_NUM_ROUNDS) + result.parquet_to_pyarrow_inflation = params.get( + "parquet_to_pyarrow_inflation", PARQUET_TO_PYARROW_INFLATION + ) + result.force_using_previous_inflation_for_memory_calculation = params.get( + "force_using_previous_inflation_for_memory_calculation", False + ) + + # disable input split during rebase as the rebase files are already uniform + result.enable_input_split = ( + params.get("rebase_source_partition_locator") is None + ) + result.max_parquet_meta_size_bytes = params.get( + "max_parquet_meta_size_bytes", MAX_PARQUET_METADATA_SIZE + ) if not importlib.util.find_spec("memray"): result.enable_profiler = False @@ -414,13 +430,57 @@ def num_rounds(self, num_rounds: int) -> None: self["num_rounds"] = num_rounds @property - def parquet_to_pyarrow_inflation(self) -> int: + def parquet_to_pyarrow_inflation(self) -> float: + """ + The inflation factor for the parquet uncompressed_size_bytes to pyarrow table size. + """ return self["parquet_to_pyarrow_inflation"] @parquet_to_pyarrow_inflation.setter - def parquet_to_pyarrow_inflation(self, value: int) -> None: + def parquet_to_pyarrow_inflation(self, value: float) -> None: self["parquet_to_pyarrow_inflation"] = value + @property + def enable_input_split(self) -> bool: + """ + When this is True, the input split will be always enabled for parquet files. + The input split feature will split the parquet files into individual row groups + so that we could process them in different nodes in parallel. + By default, input split is enabled for incremental compaction and disabled for rebase. + """ + return self["enable_input_split"] + + @enable_input_split.setter + def enable_input_split(self, value: bool) -> None: + self["enable_input_split"] = value + + @property + def force_using_previous_inflation_for_memory_calculation(self) -> bool: + """ + When this is True, the memory estimation will always use previous inflation + and average record size for all data formats even if format specific metadata + is available to make better predictions of memory requirements. + + By default, previous inflation is used for non-parquet files to estimate memory while + parquet metadata will be used for parquet to estimate memory. We only fallback + to previous inflation if parquet metadata isn't available. + """ + return self["force_using_previous_inflation_for_memory_calculation"] + + @force_using_previous_inflation_for_memory_calculation.setter + def force_using_previous_inflation_for_memory_calculation( + self, value: bool + ) -> None: + self["force_using_previous_inflation_for_memory_calculation"] = value + + @property + def max_parquet_meta_size_bytes(self) -> int: + return self["max_parquet_meta_size_bytes"] + + @max_parquet_meta_size_bytes.setter + def max_parquet_meta_size_bytes(self, value: int) -> None: + self["max_parquet_meta_size_bytes"] = value + @staticmethod def json_handler_for_compact_partition_params(obj): """ diff --git a/deltacat/compute/compactor/model/compaction_session_audit_info.py b/deltacat/compute/compactor/model/compaction_session_audit_info.py index 4eba80c7..c6ac3412 100644 --- a/deltacat/compute/compactor/model/compaction_session_audit_info.py +++ b/deltacat/compute/compactor/model/compaction_session_audit_info.py @@ -436,6 +436,22 @@ def compactor_version(self) -> str: """ return self.get("compactorVersion") + @property + def observed_input_inflation(self) -> float: + """ + The average inflation observed for input files only. + This only accounts for files in the source. + """ + return self.get("observedInputInflation") + + @property + def observed_input_average_record_size_bytes(self) -> float: + """ + The average record size observed for input files only. + This only accounts for files in the source. + """ + return self.get("observedInputAverageRecordSizeBytes") + # Setters follow def set_audit_url(self, audit_url: str) -> CompactionSessionAuditInfo: @@ -756,6 +772,16 @@ def set_compactor_version(self, value: str) -> CompactionSessionAuditInfo: self["compactorVersion"] = value return self + def set_observed_input_inflation(self, value: float) -> CompactionSessionAuditInfo: + self["observedInputInflation"] = value + return self + + def set_observed_input_average_record_size_bytes( + self, value: float + ) -> CompactionSessionAuditInfo: + self["observedInputAverageRecordSizeBytes"] = value + return self + # High level methods to save stats def save_step_stats( self, diff --git a/deltacat/compute/compactor/model/delta_annotated.py b/deltacat/compute/compactor/model/delta_annotated.py index bfd3a3e5..051228ae 100644 --- a/deltacat/compute/compactor/model/delta_annotated.py +++ b/deltacat/compute/compactor/model/delta_annotated.py @@ -69,6 +69,7 @@ def rebatch( estimation_function: Optional[ Callable[[ManifestEntry], float] ] = lambda entry: entry.meta.content_length, + enable_input_split: Optional[bool] = False, ) -> List[DeltaAnnotated]: """ Simple greedy algorithm to split/merge 1 or more annotated deltas into @@ -86,13 +87,19 @@ def rebatch( new_da_bytes = 0 da_group_entry_count = 0 - for delta_annotated in annotated_deltas: - split_annotated_deltas.extend(DeltaAnnotated._split_single(delta_annotated)) + if enable_input_split: + for delta_annotated in annotated_deltas: + split_annotated_deltas.extend( + DeltaAnnotated._split_single(delta_annotated) + ) - logger.info( - f"Split the {len(annotated_deltas)} annotated deltas " - f"into {len(split_annotated_deltas)} groups." - ) + logger.info( + f"Split the {len(annotated_deltas)} annotated deltas " + f"into {len(split_annotated_deltas)} groups." + ) + else: + logger.info("Skipping input split as it is disabled...") + split_annotated_deltas = annotated_deltas for src_da in split_annotated_deltas: src_da_annotations = src_da.annotations diff --git a/deltacat/compute/compactor_v2/constants.py b/deltacat/compute/compactor_v2/constants.py index 6a0082d9..4f1d22a6 100644 --- a/deltacat/compute/compactor_v2/constants.py +++ b/deltacat/compute/compactor_v2/constants.py @@ -41,6 +41,9 @@ # size in metadata to pyarrow table size. PARQUET_TO_PYARROW_INFLATION = 4 +# Maximum size of the parquet metadata +MAX_PARQUET_METADATA_SIZE = 100_000_000 # 100 MB + # By default, copy by reference is enabled DEFAULT_DISABLE_COPY_BY_REFERENCE = False diff --git a/deltacat/compute/compactor_v2/private/compaction_utils.py b/deltacat/compute/compactor_v2/private/compaction_utils.py index 786908ba..10ffde6b 100644 --- a/deltacat/compute/compactor_v2/private/compaction_utils.py +++ b/deltacat/compute/compactor_v2/private/compaction_utils.py @@ -152,9 +152,12 @@ def _build_uniform_deltas( previous_inflation=params.previous_inflation, min_delta_bytes=params.min_delta_bytes_in_batch, min_file_counts=params.min_files_in_batch, - # disable input split during rebase as the rebase files are already uniform - enable_input_split=params.rebase_source_partition_locator is None, + enable_input_split=params.enable_input_split, deltacat_storage_kwargs=params.deltacat_storage_kwargs, + parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, + force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, + task_max_parallelism=params.task_max_parallelism, + max_parquet_meta_size_bytes=params.max_parquet_meta_size_bytes, ) delta_discovery_end: float = time.monotonic() @@ -400,6 +403,8 @@ def _merge( deltacat_storage_kwargs=params.deltacat_storage_kwargs, ray_custom_resources=params.ray_custom_resources, memory_logs_enabled=params.memory_logs_enabled, + parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, + force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, ) def merge_input_provider(index, item) -> dict[str, MergeInput]: @@ -463,6 +468,8 @@ def _hash_bucket( primary_keys=params.primary_keys, ray_custom_resources=params.ray_custom_resources, memory_logs_enabled=params.memory_logs_enabled, + parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, + force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, ) def hash_bucket_input_provider(index, item) -> dict[str, HashBucketInput]: @@ -537,6 +544,8 @@ def _run_local_merge( ray_custom_resources=params.ray_custom_resources, primary_keys=params.primary_keys, memory_logs_enabled=params.memory_logs_enabled, + parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, + force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, ) local_merge_result = ray.get( mg.merge.options(**local_merge_options).remote(local_merge_input) @@ -666,6 +675,11 @@ def _write_new_round_completion_file( f" and average record size={input_average_record_size_bytes}" ) + mutable_compaction_audit.set_observed_input_inflation(input_inflation) + mutable_compaction_audit.set_observed_input_average_record_size_bytes( + input_average_record_size_bytes + ) + _update_and_upload_compaction_audit( params, mutable_compaction_audit, diff --git a/deltacat/compute/compactor_v2/utils/content_type_params.py b/deltacat/compute/compactor_v2/utils/content_type_params.py index 2b1d290d..97a892aa 100644 --- a/deltacat/compute/compactor_v2/utils/content_type_params.py +++ b/deltacat/compute/compactor_v2/utils/content_type_params.py @@ -1,66 +1,132 @@ import logging +import ray +import functools +from deltacat.compute.compactor_v2.constants import ( + TASK_MAX_PARALLELISM, + MAX_PARQUET_METADATA_SIZE, +) +from deltacat.utils.ray_utils.concurrency import invoke_parallel from deltacat import logs from deltacat.storage import ( Delta, + ManifestEntry, interface as unimplemented_deltacat_storage, ) from typing import Dict, Optional, Any -from deltacat.types.media import TableType, StorageType +from deltacat.types.media import TableType from deltacat.types.media import ContentType from deltacat.types.partial_download import PartialParquetParameters +from deltacat.compute.compactor_v2.utils.task_options import ( + append_content_type_params_options_provider, +) logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) +def _contains_partial_parquet_parameters(entry: ManifestEntry) -> bool: + return ( + entry.meta + and entry.meta.content_type_parameters + and any( + isinstance(type_params, PartialParquetParameters) + for type_params in entry.meta.content_type_parameters + ) + ) + + +@ray.remote +def _download_parquet_meta_for_manifest_entry( + delta: Delta, + entry_index: int, + deltacat_storage: unimplemented_deltacat_storage, + deltacat_storage_kwargs: Optional[Dict[Any, Any]] = {}, +) -> Dict[str, Any]: + pq_file = deltacat_storage.download_delta_manifest_entry( + delta, + entry_index=entry_index, + table_type=TableType.PYARROW_PARQUET, + **deltacat_storage_kwargs, + ) + + return { + "entry_index": entry_index, + "partial_parquet_params": PartialParquetParameters.of( + pq_metadata=pq_file.metadata + ), + } + + def append_content_type_params( delta: Delta, - entry_index: Optional[int] = None, + task_max_parallelism: int = TASK_MAX_PARALLELISM, + max_parquet_meta_size_bytes: Optional[int] = MAX_PARQUET_METADATA_SIZE, deltacat_storage=unimplemented_deltacat_storage, deltacat_storage_kwargs: Optional[Dict[str, Any]] = {}, ) -> None: - if delta.meta.content_type != ContentType.PARQUET.value: - logger.info( - f"Delta with locator {delta.locator} is not a parquet delta, " - "skipping appending content type parameters." - ) + if not delta.meta: + logger.warning(f"Delta with locator {delta.locator} doesn't contain meta.") return - manifest_entries = delta.manifest.entries - ordered_pq_meta = [] + entry_indices_to_download = [] + for entry_index, entry in enumerate(delta.manifest.entries): + if ( + not _contains_partial_parquet_parameters(entry) + and entry.meta + and entry.meta.content_type == ContentType.PARQUET.value + ): + entry_indices_to_download.append(entry_index) - if entry_index is not None: - manifest_entries = [delta.manifest.entries[entry_index]] + if not entry_indices_to_download: + logger.info(f"No parquet entries found for delta with locator {delta.locator}.") + return - pq_file = deltacat_storage.download_delta_manifest_entry( - delta, - entry_index=entry_index, - table_type=TableType.PYARROW_PARQUET, - **deltacat_storage_kwargs, - ) + options_provider = functools.partial( + append_content_type_params_options_provider, + max_parquet_meta_size_bytes=max_parquet_meta_size_bytes, + ) - partial_file_meta = PartialParquetParameters.of(pq_metadata=pq_file.metadata) - ordered_pq_meta.append(partial_file_meta) + def input_provider(index, item) -> Dict: + return { + "deltacat_storage_kwargs": deltacat_storage_kwargs, + "deltacat_storage": deltacat_storage, + "delta": delta, + "entry_index": item, + } - else: - pq_files = deltacat_storage.download_delta( - delta, - table_type=TableType.PYARROW_PARQUET, - storage_type=StorageType.LOCAL, - **deltacat_storage_kwargs, - ) + logger.info( + f"Downloading parquet meta for {len(entry_indices_to_download)} manifest entries..." + ) + pq_files_promise = invoke_parallel( + entry_indices_to_download, + ray_task=_download_parquet_meta_for_manifest_entry, + max_parallelism=task_max_parallelism, + options_provider=options_provider, + kwargs_provider=input_provider, + ) + + partial_file_meta_list = ray.get(pq_files_promise) - assert len(pq_files) == len( - manifest_entries - ), f"Expected {len(manifest_entries)} pq files, got {len(pq_files)}" + logger.info( + f"Downloaded parquet meta for {len(entry_indices_to_download)} manifest entries" + ) - ordered_pq_meta = [ - PartialParquetParameters.of(pq_metadata=pq_file.metadata) - for pq_file in pq_files - ] + assert len(partial_file_meta_list) == len( + entry_indices_to_download + ), f"Expected {len(entry_indices_to_download)} pq files, got {len(partial_file_meta_list)}" - for entry_index, entry in enumerate(manifest_entries): + for index, entry_index in enumerate(entry_indices_to_download): + assert ( + entry_index == partial_file_meta_list[index]["entry_index"] + ), "entry_index must match with the associated parquet meta" + entry = delta.manifest.entries[entry_index] if not entry.meta.content_type_parameters: entry.meta.content_type_parameters = [] + entry.meta.content_type_parameters.append( + partial_file_meta_list[index]["partial_parquet_params"] + ) - entry.meta.content_type_parameters.append(ordered_pq_meta[entry_index]) + for entry_index, entry in enumerate(delta.manifest.entries): + assert _contains_partial_parquet_parameters( + entry + ), "partial parquet params validation failed." diff --git a/deltacat/compute/compactor_v2/utils/io.py b/deltacat/compute/compactor_v2/utils/io.py index 3fbba95a..8284acaf 100644 --- a/deltacat/compute/compactor_v2/utils/io.py +++ b/deltacat/compute/compactor_v2/utils/io.py @@ -90,6 +90,10 @@ def create_uniform_input_deltas( input_deltas: List[Delta], hash_bucket_count: int, compaction_audit: CompactionSessionAuditInfo, + parquet_to_pyarrow_inflation: Optional[float], + force_use_previous_inflation: Optional[bool], + task_max_parallelism: Optional[int], + max_parquet_meta_size_bytes: Optional[int], min_delta_bytes: Optional[float] = MIN_DELTA_BYTES_IN_BATCH, min_file_counts: Optional[float] = MIN_FILES_IN_BATCH, previous_inflation: Optional[float] = PYARROW_INFLATION_MULTIPLIER, @@ -105,10 +109,13 @@ def create_uniform_input_deltas( for delta in input_deltas: if enable_input_split: + # An idempotent operation to ensure content type params exist append_content_type_params( delta=delta, deltacat_storage=deltacat_storage, deltacat_storage_kwargs=deltacat_storage_kwargs, + task_max_parallelism=task_max_parallelism, + max_parquet_meta_size_bytes=max_parquet_meta_size_bytes, ) manifest_entries = delta.manifest.entries @@ -118,7 +125,10 @@ def create_uniform_input_deltas( entry = manifest_entries[entry_index] delta_bytes += entry.meta.content_length estimated_da_bytes += estimate_manifest_entry_size_bytes( - entry=entry, previous_inflation=previous_inflation + entry=entry, + previous_inflation=previous_inflation, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + force_use_previous_inflation=force_use_previous_inflation, ) delta_annotated = DeltaAnnotated.of(delta) @@ -129,13 +139,17 @@ def create_uniform_input_deltas( logger.info(f"Input delta files to compact: {delta_manifest_entries_count}") size_estimation_function = functools.partial( - estimate_manifest_entry_size_bytes, previous_inflation=previous_inflation + estimate_manifest_entry_size_bytes, + previous_inflation=previous_inflation, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + force_use_previous_inflation=force_use_previous_inflation, ) rebatched_da_list = DeltaAnnotated.rebatch( input_da_list, min_delta_bytes=min_delta_bytes, min_file_counts=min_file_counts, estimation_function=size_estimation_function, + enable_input_split=enable_input_split, ) compaction_audit.set_input_size_bytes(delta_bytes) diff --git a/deltacat/compute/compactor_v2/utils/task_options.py b/deltacat/compute/compactor_v2/utils/task_options.py index d538a72e..32d8cc43 100644 --- a/deltacat/compute/compactor_v2/utils/task_options.py +++ b/deltacat/compute/compactor_v2/utils/task_options.py @@ -4,11 +4,8 @@ from deltacat.compute.compactor_v2.model.merge_file_group import ( LocalMergeFileGroupsProvider, ) -from deltacat.types.media import ContentEncoding, ContentType -from deltacat.types.partial_download import PartialParquetParameters from deltacat.storage import ( Manifest, - ManifestEntry, interface as unimplemented_deltacat_storage, ) from deltacat.compute.compactor.model.delta_annotated import DeltaAnnotated @@ -16,50 +13,21 @@ from deltacat.compute.compactor_v2.utils.primary_key_index import ( hash_group_index_to_hash_bucket_indices, ) -from deltacat.compute.compactor_v2.constants import ( - PARQUET_TO_PYARROW_INFLATION, +from deltacat.compute.resource_requirements.utils import ( + estimate_manifest_entry_num_rows, + estimate_manifest_entry_size_bytes, + estimate_manifest_entry_column_size_bytes, ) from deltacat.exceptions import RetryableError logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) -def _get_parquet_type_params_if_exist( - entry: ManifestEntry, -) -> Optional[PartialParquetParameters]: - if ( - entry.meta - and entry.meta.content_type == ContentType.PARQUET - and entry.meta.content_encoding == ContentEncoding.IDENTITY - and entry.meta.content_type_parameters - ): - for type_params in entry.meta.content_type_parameters: - if isinstance(type_params, PartialParquetParameters): - return type_params - return None - - -def _calculate_parquet_column_size( - type_params: PartialParquetParameters, columns: List[str] -): - column_size = 0.0 - for rg in type_params.row_groups_to_download: - columns_found = 0 - row_group_meta = type_params.pq_metadata.row_group(rg) - for col in range(row_group_meta.num_columns): - column_meta = row_group_meta.column(col) - if column_meta.path_in_schema in columns: - columns_found += 1 - column_size += column_meta.total_uncompressed_size - assert columns_found == len(columns), ( - "Columns not found in the parquet data as " - f"{columns_found} != {len(columns)}" - ) - return column_size * PARQUET_TO_PYARROW_INFLATION - - def get_task_options( - cpu: float, memory: float, ray_custom_resources: Optional[Dict] = None + cpu: float, + memory: float, + ray_custom_resources: Optional[Dict] = None, + scheduling_strategy: str = "SPREAD", ) -> Dict: # NOTE: With DEFAULT scheduling strategy in Ray 2.20.0, autoscaler does @@ -67,7 +35,11 @@ def get_task_options( # 20 tasks get scheduled out of 100 tasks in queue. Hence, we use SPREAD # which is also ideal for merge and hash bucket tasks. # https://docs.ray.io/en/latest/ray-core/scheduling/index.html - task_opts = {"num_cpus": cpu, "memory": memory, "scheduling_strategy": "SPREAD"} + task_opts = { + "num_cpus": cpu, + "memory": memory, + "scheduling_strategy": scheduling_strategy, + } if ray_custom_resources: task_opts["resources"] = ray_custom_resources @@ -81,54 +53,13 @@ def get_task_options( return task_opts -def estimate_manifest_entry_size_bytes( - entry: ManifestEntry, previous_inflation: float, **kwargs -) -> float: - if entry.meta.source_content_length: - return entry.meta.source_content_length - - type_params = _get_parquet_type_params_if_exist(entry=entry) - - if type_params: - return type_params.in_memory_size_bytes * PARQUET_TO_PYARROW_INFLATION - - return entry.meta.content_length * previous_inflation - - -def estimate_manifest_entry_num_rows( - entry: ManifestEntry, - average_record_size_bytes: float, - previous_inflation: float, - **kwargs, -) -> int: - if entry.meta.record_count: - return entry.meta.record_count - - type_params = _get_parquet_type_params_if_exist(entry=entry) - - if type_params: - return type_params.num_rows - - total_size_bytes = estimate_manifest_entry_size_bytes( - entry=entry, previous_inflation=previous_inflation, **kwargs +def append_content_type_params_options_provider( + index: int, item: Any, max_parquet_meta_size_bytes: int, **kwargs +) -> Dict: + return get_task_options( + 0.01, max_parquet_meta_size_bytes, scheduling_strategy="DEFAULT" ) - return int(total_size_bytes / average_record_size_bytes) - - -def estimate_manifest_entry_column_size_bytes( - entry: ManifestEntry, columns: Optional[List[str]] = None -) -> Optional[float]: - if not columns: - return 0 - - type_params = _get_parquet_type_params_if_exist(entry=entry) - - if type_params and type_params.pq_metadata: - return _calculate_parquet_column_size(type_params=type_params, columns=columns) - - return None - def hash_bucket_resource_options_provider( index: int, @@ -136,6 +67,8 @@ def hash_bucket_resource_options_provider( previous_inflation: float, average_record_size_bytes: float, total_memory_buffer_percentage: int, + parquet_to_pyarrow_inflation: float, + force_use_previous_inflation: bool, primary_keys: List[str] = None, ray_custom_resources: Optional[Dict] = None, memory_logs_enabled: Optional[bool] = None, @@ -153,12 +86,19 @@ def hash_bucket_resource_options_provider( for entry in item.manifest.entries: entry_size = estimate_manifest_entry_size_bytes( - entry=entry, previous_inflation=previous_inflation + entry=entry, + previous_inflation=previous_inflation, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + force_use_previous_inflation=force_use_previous_inflation, + **kwargs, ) num_rows += estimate_manifest_entry_num_rows( entry=entry, previous_inflation=previous_inflation, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, average_record_size_bytes=average_record_size_bytes, + force_use_previous_inflation=force_use_previous_inflation, + **kwargs, ) size_bytes += entry_size @@ -166,6 +106,7 @@ def hash_bucket_resource_options_provider( pk_size = estimate_manifest_entry_column_size_bytes( entry=entry, columns=primary_keys, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, ) if pk_size is None: @@ -209,6 +150,8 @@ def merge_resource_options_provider( hash_group_size_bytes: Dict[int, int], hash_group_num_rows: Dict[int, int], total_memory_buffer_percentage: int, + parquet_to_pyarrow_inflation: float, + force_use_previous_inflation: bool, round_completion_info: Optional[RoundCompletionInfo] = None, compacted_delta_manifest: Optional[Manifest] = None, ray_custom_resources: Optional[Dict] = None, @@ -247,6 +190,8 @@ def merge_resource_options_provider( deltacat_storage=deltacat_storage, deltacat_storage_kwargs=deltacat_storage_kwargs, memory_logs_enabled=memory_logs_enabled, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + force_use_previous_inflation=force_use_previous_inflation, ) @@ -254,6 +199,8 @@ def local_merge_resource_options_provider( estimated_da_size: float, estimated_num_rows: int, total_memory_buffer_percentage: int, + parquet_to_pyarrow_inflation: float, + force_use_previous_inflation: bool, round_completion_info: Optional[RoundCompletionInfo] = None, compacted_delta_manifest: Optional[Manifest] = None, ray_custom_resources: Optional[Dict] = None, @@ -287,6 +234,8 @@ def local_merge_resource_options_provider( deltacat_storage=deltacat_storage, deltacat_storage_kwargs=deltacat_storage_kwargs, memory_logs_enabled=memory_logs_enabled, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + force_use_previous_inflation=force_use_previous_inflation, ) @@ -301,6 +250,8 @@ def get_merge_task_options( incremental_index_array_size: int, debug_memory_params: Dict[str, Any], ray_custom_resources: Optional[Dict], + parquet_to_pyarrow_inflation: float, + force_use_previous_inflation: bool, round_completion_info: Optional[RoundCompletionInfo] = None, compacted_delta_manifest: Optional[Manifest] = None, primary_keys: Optional[List[str]] = None, @@ -341,12 +292,17 @@ def get_merge_task_options( entry = compacted_delta_manifest.entries[entry_index] current_entry_size = estimate_manifest_entry_size_bytes( - entry=entry, previous_inflation=previous_inflation + entry=entry, + previous_inflation=previous_inflation, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + force_use_previous_inflation=force_use_previous_inflation, ) current_entry_rows = estimate_manifest_entry_num_rows( entry=entry, average_record_size_bytes=average_record_size, previous_inflation=previous_inflation, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + force_use_previous_inflation=force_use_previous_inflation, ) data_size += current_entry_size @@ -356,6 +312,7 @@ def get_merge_task_options( pk_size = estimate_manifest_entry_column_size_bytes( entry=entry, columns=primary_keys, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, ) if pk_size is None: diff --git a/deltacat/compute/resource_requirements/__init__.py b/deltacat/compute/resource_requirements/__init__.py new file mode 100644 index 00000000..e69de29b diff --git a/deltacat/compute/resource_requirements/utils.py b/deltacat/compute/resource_requirements/utils.py new file mode 100644 index 00000000..78b7db54 --- /dev/null +++ b/deltacat/compute/resource_requirements/utils.py @@ -0,0 +1,127 @@ +import logging +from typing import Optional, List +from deltacat import logs +from deltacat.types.media import ContentEncoding, ContentType +from deltacat.types.partial_download import PartialParquetParameters +from deltacat.storage import ( + ManifestEntry, +) + +logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) + + +def _get_parquet_type_params_if_exist( + entry: ManifestEntry, +) -> Optional[PartialParquetParameters]: + if ( + entry.meta + and entry.meta.content_type == ContentType.PARQUET + and entry.meta.content_encoding == ContentEncoding.IDENTITY + and entry.meta.content_type_parameters + ): + for type_params in entry.meta.content_type_parameters: + if isinstance(type_params, PartialParquetParameters): + return type_params + return None + + +def _calculate_parquet_column_size( + type_params: PartialParquetParameters, + parquet_to_pyarrow_inflation: float, + columns: List[str], +): + column_size = 0.0 + for rg in type_params.row_groups_to_download: + columns_found = 0 + row_group_meta = type_params.pq_metadata.row_group(rg) + for col in range(row_group_meta.num_columns): + column_meta = row_group_meta.column(col) + if column_meta.path_in_schema in columns: + columns_found += 1 + column_size += column_meta.total_uncompressed_size + assert columns_found == len(columns), ( + "Columns not found in the parquet data as " + f"{columns_found} != {len(columns)}" + ) + return column_size * parquet_to_pyarrow_inflation + + +def estimate_manifest_entry_size_bytes( + entry: ManifestEntry, + previous_inflation: float, + parquet_to_pyarrow_inflation: float, + force_use_previous_inflation: bool, + **kwargs, +) -> float: + if entry.meta.source_content_length: + logger.debug(f"Using source content length for entry={entry.uri}") + return entry.meta.source_content_length + + if not force_use_previous_inflation: + type_params = _get_parquet_type_params_if_exist(entry=entry) + + if type_params: + logger.debug(f"Using parquet meta for entry={entry.uri}") + return type_params.in_memory_size_bytes * parquet_to_pyarrow_inflation + + logger.debug(f"Using inflation for entry={entry.uri}") + return entry.meta.content_length * previous_inflation + + +def estimate_manifest_entry_num_rows( + entry: ManifestEntry, + average_record_size_bytes: float, + previous_inflation: float, + parquet_to_pyarrow_inflation: float, + force_use_previous_inflation: bool, + **kwargs, +) -> int: + """ + Estimate number of records in the manifest entry file. It uses content type + specific estimation logic if available, otherwise it falls back to using + previous inflation and average record size. + """ + if entry.meta.record_count: + logger.debug(f"Using record count in meta for entry={entry.uri}") + return entry.meta.record_count + + if not force_use_previous_inflation: + type_params = _get_parquet_type_params_if_exist(entry=entry) + + if type_params: + logger.debug(f"Using parquet meta for entry={entry.uri}") + return type_params.num_rows + + total_size_bytes = estimate_manifest_entry_size_bytes( + entry=entry, + previous_inflation=previous_inflation, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + force_use_previous_inflation=force_use_previous_inflation**kwargs, + ) + logger.debug(f"Using previous inflation for entry={entry.uri}") + + return int(total_size_bytes / average_record_size_bytes) + + +def estimate_manifest_entry_column_size_bytes( + entry: ManifestEntry, + parquet_to_pyarrow_inflation: float, + columns: Optional[List[str]] = None, +) -> Optional[float]: + """ + Estimate the size of specified columns in the manifest entry file. + This method only supports parquet. For other types, it returns None. + """ + if not columns: + return 0 + + type_params = _get_parquet_type_params_if_exist(entry=entry) + + if type_params and type_params.pq_metadata: + return _calculate_parquet_column_size( + type_params=type_params, + columns=columns, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + ) + + return None diff --git a/deltacat/compute/stats/__init__.py b/deltacat/compute/stats/__init__.py new file mode 100644 index 00000000..e69de29b From 7a2d3599979e2008396843451c39a5814a0a2127 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Tue, 24 Sep 2024 17:54:45 -0700 Subject: [PATCH 02/25] enable intelligent size estimation --- .../model/compact_partition_params.py | 15 ++++ .../compactor_v2/private/compaction_utils.py | 4 + deltacat/compute/compactor_v2/utils/io.py | 3 + .../compactor_v2/utils/task_options.py | 20 +++++ .../compute/resource_requirements/parquet.py | 88 +++++++++++++++++++ .../compute/resource_requirements/utils.py | 88 ++++++++++++++----- deltacat/constants.py | 3 + 7 files changed, 198 insertions(+), 23 deletions(-) create mode 100644 deltacat/compute/resource_requirements/parquet.py diff --git a/deltacat/compute/compactor/model/compact_partition_params.py b/deltacat/compute/compactor/model/compact_partition_params.py index 9e9f04cf..a3df9066 100644 --- a/deltacat/compute/compactor/model/compact_partition_params.py +++ b/deltacat/compute/compactor/model/compact_partition_params.py @@ -112,6 +112,9 @@ def of(params: Optional[Dict]) -> CompactPartitionParams: result.force_using_previous_inflation_for_memory_calculation = params.get( "force_using_previous_inflation_for_memory_calculation", False ) + result.enable_intelligent_size_estimation = params.get( + "enable_intelligent_size_estimation", False + ) # disable input split during rebase as the rebase files are already uniform result.enable_input_split = ( @@ -481,6 +484,18 @@ def max_parquet_meta_size_bytes(self) -> int: def max_parquet_meta_size_bytes(self, value: int) -> None: self["max_parquet_meta_size_bytes"] = value + @property + def enable_intelligent_size_estimation(self) -> bool: + """ + The arguments enable intelligent memory estimation that considers + encoding, min/max and other statistics to estimate memory requirements. + """ + return self["enable_intelligent_size_estimation"] + + @enable_intelligent_size_estimation.setter + def enable_intelligent_size_estimation(self, value: bool) -> None: + self["enable_intelligent_size_estimation"] = value + @staticmethod def json_handler_for_compact_partition_params(obj): """ diff --git a/deltacat/compute/compactor_v2/private/compaction_utils.py b/deltacat/compute/compactor_v2/private/compaction_utils.py index 10ffde6b..193d158c 100644 --- a/deltacat/compute/compactor_v2/private/compaction_utils.py +++ b/deltacat/compute/compactor_v2/private/compaction_utils.py @@ -156,6 +156,7 @@ def _build_uniform_deltas( deltacat_storage_kwargs=params.deltacat_storage_kwargs, parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, + enable_intelligent_size_estimation=params.enable_intelligent_size_estimation, task_max_parallelism=params.task_max_parallelism, max_parquet_meta_size_bytes=params.max_parquet_meta_size_bytes, ) @@ -405,6 +406,7 @@ def _merge( memory_logs_enabled=params.memory_logs_enabled, parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, + enable_intelligent_size_estimation=params.enable_intelligent_size_estimation, ) def merge_input_provider(index, item) -> dict[str, MergeInput]: @@ -470,6 +472,7 @@ def _hash_bucket( memory_logs_enabled=params.memory_logs_enabled, parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, + enable_intelligent_size_estimation=params.enable_intelligent_size_estimation, ) def hash_bucket_input_provider(index, item) -> dict[str, HashBucketInput]: @@ -546,6 +549,7 @@ def _run_local_merge( memory_logs_enabled=params.memory_logs_enabled, parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, + enable_intelligent_size_estimation=params.enable_intelligent_size_estimation, ) local_merge_result = ray.get( mg.merge.options(**local_merge_options).remote(local_merge_input) diff --git a/deltacat/compute/compactor_v2/utils/io.py b/deltacat/compute/compactor_v2/utils/io.py index 8284acaf..7088e1bd 100644 --- a/deltacat/compute/compactor_v2/utils/io.py +++ b/deltacat/compute/compactor_v2/utils/io.py @@ -92,6 +92,7 @@ def create_uniform_input_deltas( compaction_audit: CompactionSessionAuditInfo, parquet_to_pyarrow_inflation: Optional[float], force_use_previous_inflation: Optional[bool], + enable_intelligent_size_estimation: Optional[bool], task_max_parallelism: Optional[int], max_parquet_meta_size_bytes: Optional[int], min_delta_bytes: Optional[float] = MIN_DELTA_BYTES_IN_BATCH, @@ -129,6 +130,7 @@ def create_uniform_input_deltas( previous_inflation=previous_inflation, parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, force_use_previous_inflation=force_use_previous_inflation, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, ) delta_annotated = DeltaAnnotated.of(delta) @@ -143,6 +145,7 @@ def create_uniform_input_deltas( previous_inflation=previous_inflation, parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, force_use_previous_inflation=force_use_previous_inflation, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, ) rebatched_da_list = DeltaAnnotated.rebatch( input_da_list, diff --git a/deltacat/compute/compactor_v2/utils/task_options.py b/deltacat/compute/compactor_v2/utils/task_options.py index 32d8cc43..cdd907de 100644 --- a/deltacat/compute/compactor_v2/utils/task_options.py +++ b/deltacat/compute/compactor_v2/utils/task_options.py @@ -69,6 +69,7 @@ def hash_bucket_resource_options_provider( total_memory_buffer_percentage: int, parquet_to_pyarrow_inflation: float, force_use_previous_inflation: bool, + enable_intelligent_size_estimation: bool, primary_keys: List[str] = None, ray_custom_resources: Optional[Dict] = None, memory_logs_enabled: Optional[bool] = None, @@ -90,6 +91,7 @@ def hash_bucket_resource_options_provider( previous_inflation=previous_inflation, parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, force_use_previous_inflation=force_use_previous_inflation, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, **kwargs, ) num_rows += estimate_manifest_entry_num_rows( @@ -107,6 +109,7 @@ def hash_bucket_resource_options_provider( entry=entry, columns=primary_keys, parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, ) if pk_size is None: @@ -128,6 +131,11 @@ def hash_bucket_resource_options_provider( debug_memory_params["num_rows"] = num_rows debug_memory_params["total_pk_size"] = total_pk_size debug_memory_params["total_memory"] = total_memory + debug_memory_params[ + "enable_intelligent_size_estimation" + ] = enable_intelligent_size_estimation + debug_memory_params["parquet_to_pyarrow_inflation"] = parquet_to_pyarrow_inflation + debug_memory_params["force_use_previous_inflation"] = force_use_previous_inflation debug_memory_params["previous_inflation"] = previous_inflation debug_memory_params["average_record_size_bytes"] = average_record_size_bytes @@ -152,6 +160,7 @@ def merge_resource_options_provider( total_memory_buffer_percentage: int, parquet_to_pyarrow_inflation: float, force_use_previous_inflation: bool, + enable_intelligent_size_estimation: bool, round_completion_info: Optional[RoundCompletionInfo] = None, compacted_delta_manifest: Optional[Manifest] = None, ray_custom_resources: Optional[Dict] = None, @@ -192,6 +201,7 @@ def merge_resource_options_provider( memory_logs_enabled=memory_logs_enabled, parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, force_use_previous_inflation=force_use_previous_inflation, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, ) @@ -201,6 +211,7 @@ def local_merge_resource_options_provider( total_memory_buffer_percentage: int, parquet_to_pyarrow_inflation: float, force_use_previous_inflation: bool, + enable_intelligent_size_estimation: bool, round_completion_info: Optional[RoundCompletionInfo] = None, compacted_delta_manifest: Optional[Manifest] = None, ray_custom_resources: Optional[Dict] = None, @@ -236,6 +247,7 @@ def local_merge_resource_options_provider( memory_logs_enabled=memory_logs_enabled, parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, force_use_previous_inflation=force_use_previous_inflation, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, ) @@ -252,6 +264,7 @@ def get_merge_task_options( ray_custom_resources: Optional[Dict], parquet_to_pyarrow_inflation: float, force_use_previous_inflation: bool, + enable_intelligent_size_estimation: bool, round_completion_info: Optional[RoundCompletionInfo] = None, compacted_delta_manifest: Optional[Manifest] = None, primary_keys: Optional[List[str]] = None, @@ -296,6 +309,7 @@ def get_merge_task_options( previous_inflation=previous_inflation, parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, force_use_previous_inflation=force_use_previous_inflation, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, ) current_entry_rows = estimate_manifest_entry_num_rows( entry=entry, @@ -313,6 +327,7 @@ def get_merge_task_options( entry=entry, columns=primary_keys, parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, ) if pk_size is None: @@ -336,6 +351,11 @@ def get_merge_task_options( debug_memory_params["pk_size_bytes"] = pk_size_bytes debug_memory_params["incremental_index_array_size"] = incremental_index_array_size debug_memory_params["total_memory"] = total_memory + debug_memory_params[ + "enable_intelligent_size_estimation" + ] = enable_intelligent_size_estimation + debug_memory_params["force_use_previous_inflation"] = force_use_previous_inflation + debug_memory_params["parquet_to_pyarrow_inflation"] = parquet_to_pyarrow_inflation total_memory = total_memory * (1 + total_memory_buffer_percentage / 100.0) debug_memory_params["total_memory_with_buffer"] = total_memory diff --git a/deltacat/compute/resource_requirements/parquet.py b/deltacat/compute/resource_requirements/parquet.py new file mode 100644 index 00000000..ea9dfd74 --- /dev/null +++ b/deltacat/compute/resource_requirements/parquet.py @@ -0,0 +1,88 @@ +import logging +from typing import Optional +from deltacat import logs +from pyarrow.parquet import ColumnChunkMetaData +from deltacat.constants import NULL_SIZE_BYTES + +logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) + + +def _int96_size_estimator( + column_chunk_metadata: ColumnChunkMetaData, +) -> float: + return column_chunk_metadata.num_values * 24 + + +def _int64_size_estimator( + column_chunk_metadata: ColumnChunkMetaData, +) -> float: + return column_chunk_metadata.num_values * 8 + + +def _int32_size_estimator( + column_chunk_metadata: ColumnChunkMetaData, +) -> float: + return column_chunk_metadata.num_values * 4 + + +def _boolean_size_estimator( + column_chunk_metadata: ColumnChunkMetaData, +) -> float: + return column_chunk_metadata.num_values * 1.0 + + +def _double_size_estimator( + column_chunk_metadata: ColumnChunkMetaData, +) -> float: + return column_chunk_metadata.num_values * 8 + + +def _float_size_estimator( + column_chunk_metadata: ColumnChunkMetaData, +) -> float: + return column_chunk_metadata.num_values * 4 + + +def _byte_array_size_estimator( + column_chunk_metadata: ColumnChunkMetaData, +) -> float: + if column_chunk_metadata.is_stats_set: + statistics = column_chunk_metadata.statistics + return ( + statistics.num_values * (len(statistics.min) + len(statistics.max)) / 2 + + statistics.null_count * NULL_SIZE_BYTES + ) + else: + return column_chunk_metadata.total_uncompressed_size + + +def _fixed_len_byte_array_size_estimator( + column_chunk_metadata: ColumnChunkMetaData, +) -> float: + return _byte_array_size_estimator(column_chunk_metadata) + + +_PHYSICAL_TYPE_TO_SIZE_ESTIMATOR = { + "INT96": _int96_size_estimator, + "INT64": _int64_size_estimator, + "INT32": _int32_size_estimator, + "BOOLEAN": _boolean_size_estimator, + "DOUBLE": _double_size_estimator, + "FLOAT": _float_size_estimator, + "BYTE_ARRAY": _byte_array_size_estimator, + "FIXED_LEN_BYTE_ARRAY": _fixed_len_byte_array_size_estimator, +} + + +def parquet_column_chunk_size_estimator( + column_chunk_metadata: ColumnChunkMetaData, +) -> Optional[float]: + physical_type = column_chunk_metadata.physical_type + if physical_type in _PHYSICAL_TYPE_TO_SIZE_ESTIMATOR: + return _PHYSICAL_TYPE_TO_SIZE_ESTIMATOR[physical_type](column_chunk_metadata) + else: + logger.warning( + f"Unsupported physical type: {physical_type}. " + "Returning total_uncompressed_size." + ) + return column_chunk_metadata.total_uncompressed_size diff --git a/deltacat/compute/resource_requirements/utils.py b/deltacat/compute/resource_requirements/utils.py index 78b7db54..93ac4487 100644 --- a/deltacat/compute/resource_requirements/utils.py +++ b/deltacat/compute/resource_requirements/utils.py @@ -1,6 +1,10 @@ import logging from typing import Optional, List from deltacat import logs +from deltacat.constants import NULL_SIZE_BYTES +from deltacat.compute.resource_requirements.parquet import ( + parquet_column_chunk_size_estimator, +) from deltacat.types.media import ContentEncoding, ContentType from deltacat.types.partial_download import PartialParquetParameters from deltacat.storage import ( @@ -28,22 +32,32 @@ def _get_parquet_type_params_if_exist( def _calculate_parquet_column_size( type_params: PartialParquetParameters, parquet_to_pyarrow_inflation: float, - columns: List[str], -): - column_size = 0.0 + column: str, + enable_intelligent_size_estimation: bool, +) -> float: + + memory_estimator = ( + parquet_column_chunk_size_estimator + if enable_intelligent_size_estimation + else lambda column_meta: column_meta.total_uncompressed_size + ) + + final_size = 0.0 for rg in type_params.row_groups_to_download: columns_found = 0 row_group_meta = type_params.pq_metadata.row_group(rg) for col in range(row_group_meta.num_columns): column_meta = row_group_meta.column(col) - if column_meta.path_in_schema in columns: + if column_meta.path_in_schema == column: columns_found += 1 - column_size += column_meta.total_uncompressed_size - assert columns_found == len(columns), ( - "Columns not found in the parquet data as " - f"{columns_found} != {len(columns)}" - ) - return column_size * parquet_to_pyarrow_inflation + final_size += memory_estimator(column_meta=column_meta) + if columns_found == 0: + # This indicates a null column + final_size += NULL_SIZE_BYTES * row_group_meta.num_rows + elif columns_found > 1: + raise ValueError(f"Duplicate column found: {column}") + + return final_size * parquet_to_pyarrow_inflation def estimate_manifest_entry_size_bytes( @@ -51,18 +65,34 @@ def estimate_manifest_entry_size_bytes( previous_inflation: float, parquet_to_pyarrow_inflation: float, force_use_previous_inflation: bool, + enable_intelligent_size_estimation: bool, **kwargs, ) -> float: if entry.meta.source_content_length: logger.debug(f"Using source content length for entry={entry.uri}") return entry.meta.source_content_length - if not force_use_previous_inflation: - type_params = _get_parquet_type_params_if_exist(entry=entry) + type_params = _get_parquet_type_params_if_exist(entry=entry) - if type_params: + if type_params and type_params.row_groups_to_download: + if not force_use_previous_inflation: logger.debug(f"Using parquet meta for entry={entry.uri}") - return type_params.in_memory_size_bytes * parquet_to_pyarrow_inflation + column_names = [ + type_params.pq_metadata.row_group(0).column(col).path_in_schema + for col in range(type_params.pq_metadata.num_columns) + ] + return estimate_manifest_entry_column_size_bytes( + entry=entry, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + columns=column_names, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, + ) + else: + logger.warning( + f"Force using previous inflation for entry={entry.uri}. " + "This could lead to overestimation of memory when " + "enable_input_split=True" + ) logger.debug(f"Using inflation for entry={entry.uri}") return entry.meta.content_length * previous_inflation @@ -85,18 +115,25 @@ def estimate_manifest_entry_num_rows( logger.debug(f"Using record count in meta for entry={entry.uri}") return entry.meta.record_count - if not force_use_previous_inflation: - type_params = _get_parquet_type_params_if_exist(entry=entry) + type_params = _get_parquet_type_params_if_exist(entry=entry) - if type_params: + if type_params: + if not force_use_previous_inflation: logger.debug(f"Using parquet meta for entry={entry.uri}") return type_params.num_rows + else: + logger.warning( + f"Force using previous inflation for entry={entry.uri}. " + "This could lead to overestimation of records when " + "enable_input_split=True" + ) total_size_bytes = estimate_manifest_entry_size_bytes( entry=entry, previous_inflation=previous_inflation, parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - force_use_previous_inflation=force_use_previous_inflation**kwargs, + force_use_previous_inflation=force_use_previous_inflation, + **kwargs, ) logger.debug(f"Using previous inflation for entry={entry.uri}") @@ -106,6 +143,7 @@ def estimate_manifest_entry_num_rows( def estimate_manifest_entry_column_size_bytes( entry: ManifestEntry, parquet_to_pyarrow_inflation: float, + enable_intelligent_size_estimation: bool, columns: Optional[List[str]] = None, ) -> Optional[float]: """ @@ -118,10 +156,14 @@ def estimate_manifest_entry_column_size_bytes( type_params = _get_parquet_type_params_if_exist(entry=entry) if type_params and type_params.pq_metadata: - return _calculate_parquet_column_size( - type_params=type_params, - columns=columns, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - ) + columns_size = 0.0 + for column_name in columns: + columns_size += _calculate_parquet_column_size( + type_params=type_params, + column=column_name, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, + ) + return columns_size return None diff --git a/deltacat/constants.py b/deltacat/constants.py index 99934c73..73f02657 100644 --- a/deltacat/constants.py +++ b/deltacat/constants.py @@ -53,3 +53,6 @@ PYARROW_INFLATION_MULTIPLIER_ALL_COLUMNS = 6 MEMORY_TO_HASH_BUCKET_COUNT_RATIO = 0.0512 * BYTES_PER_TEBIBYTE + +# The number of bytes allocated to null values in string physical type in parquet +NULL_SIZE_BYTES = 4 From 8b964293dd602ccb7e228cceef595de8d75a8fa4 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Tue, 24 Sep 2024 18:00:48 -0700 Subject: [PATCH 03/25] simplify --- .../compute/resource_requirements/utils.py | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/deltacat/compute/resource_requirements/utils.py b/deltacat/compute/resource_requirements/utils.py index 93ac4487..15488844 100644 --- a/deltacat/compute/resource_requirements/utils.py +++ b/deltacat/compute/resource_requirements/utils.py @@ -77,16 +77,19 @@ def estimate_manifest_entry_size_bytes( if type_params and type_params.row_groups_to_download: if not force_use_previous_inflation: logger.debug(f"Using parquet meta for entry={entry.uri}") - column_names = [ - type_params.pq_metadata.row_group(0).column(col).path_in_schema - for col in range(type_params.pq_metadata.num_columns) - ] - return estimate_manifest_entry_column_size_bytes( - entry=entry, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - columns=column_names, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, - ) + if enable_intelligent_size_estimation: + column_names = [ + type_params.pq_metadata.row_group(0).column(col).path_in_schema + for col in range(type_params.pq_metadata.num_columns) + ] + return estimate_manifest_entry_column_size_bytes( + entry=entry, + parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, + columns=column_names, + enable_intelligent_size_estimation=enable_intelligent_size_estimation, + ) + else: + return type_params.in_memory_size_bytes else: logger.warning( f"Force using previous inflation for entry={entry.uri}. " From edff05da5306b5532c769e3c69cc7155f8bdfbb4 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Tue, 24 Sep 2024 18:41:23 -0700 Subject: [PATCH 04/25] fix by array size estimator --- .../compute/resource_requirements/parquet.py | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/deltacat/compute/resource_requirements/parquet.py b/deltacat/compute/resource_requirements/parquet.py index ea9dfd74..0d0404f0 100644 --- a/deltacat/compute/resource_requirements/parquet.py +++ b/deltacat/compute/resource_requirements/parquet.py @@ -48,10 +48,14 @@ def _byte_array_size_estimator( ) -> float: if column_chunk_metadata.is_stats_set: statistics = column_chunk_metadata.statistics - return ( - statistics.num_values * (len(statistics.min) + len(statistics.max)) / 2 - + statistics.null_count * NULL_SIZE_BYTES - ) + if isinstance(statistics.min, str) and isinstance(statistics.max, str): + return ( + statistics.num_values * (len(statistics.min) + len(statistics.max)) / 2 + + statistics.null_count * NULL_SIZE_BYTES + ) + else: + # A case of decimal + return statistics.num_values * 16 + statistics.null_count * NULL_SIZE_BYTES else: return column_chunk_metadata.total_uncompressed_size @@ -75,14 +79,14 @@ def _fixed_len_byte_array_size_estimator( def parquet_column_chunk_size_estimator( - column_chunk_metadata: ColumnChunkMetaData, + column_meta: ColumnChunkMetaData, ) -> Optional[float]: - physical_type = column_chunk_metadata.physical_type + physical_type = column_meta.physical_type if physical_type in _PHYSICAL_TYPE_TO_SIZE_ESTIMATOR: - return _PHYSICAL_TYPE_TO_SIZE_ESTIMATOR[physical_type](column_chunk_metadata) + return _PHYSICAL_TYPE_TO_SIZE_ESTIMATOR[physical_type](column_meta) else: logger.warning( f"Unsupported physical type: {physical_type}. " "Returning total_uncompressed_size." ) - return column_chunk_metadata.total_uncompressed_size + return column_meta.total_uncompressed_size From b5d0d05d3beb4b6102db393f6d6a4d818127d4b5 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Tue, 24 Sep 2024 19:10:03 -0700 Subject: [PATCH 05/25] append content type params if intelligent estimation is enabled --- deltacat/compute/compactor_v2/utils/io.py | 2 +- deltacat/tests/compute/test_util_common.py | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/deltacat/compute/compactor_v2/utils/io.py b/deltacat/compute/compactor_v2/utils/io.py index 7088e1bd..ed0c740f 100644 --- a/deltacat/compute/compactor_v2/utils/io.py +++ b/deltacat/compute/compactor_v2/utils/io.py @@ -109,7 +109,7 @@ def create_uniform_input_deltas( input_da_list = [] for delta in input_deltas: - if enable_input_split: + if enable_input_split or enable_intelligent_size_estimation: # An idempotent operation to ensure content type params exist append_content_type_params( delta=delta, diff --git a/deltacat/tests/compute/test_util_common.py b/deltacat/tests/compute/test_util_common.py index 78aaf07f..d5b60a4f 100644 --- a/deltacat/tests/compute/test_util_common.py +++ b/deltacat/tests/compute/test_util_common.py @@ -267,6 +267,8 @@ def assert_compaction_audit( compaction_audit.peak_memory_used_bytes_per_task, compaction_audit.pyarrow_version, compaction_audit.telemetry_time_in_seconds, + compaction_audit.observed_input_inflation, + compaction_audit.observed_input_average_record_size_bytes, ] for entry in audit_entries: assert entry is not None From ecc0c535330e61e20e08c95663bb57dd48ea14d6 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Wed, 25 Sep 2024 10:52:37 -0700 Subject: [PATCH 06/25] Few more changes --- .../compute/resource_requirements/parquet.py | 26 ++++++++++++++----- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/deltacat/compute/resource_requirements/parquet.py b/deltacat/compute/resource_requirements/parquet.py index 0d0404f0..4b7feffa 100644 --- a/deltacat/compute/resource_requirements/parquet.py +++ b/deltacat/compute/resource_requirements/parquet.py @@ -7,10 +7,17 @@ logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) +def _observed_string_size(min_value: str, max_value: str) -> float: + """ + Pyarrow uses few additional bytes to store each string. + """ + return (len(min_value) + len(max_value)) / 2 + 4 + + def _int96_size_estimator( column_chunk_metadata: ColumnChunkMetaData, ) -> float: - return column_chunk_metadata.num_values * 24 + return column_chunk_metadata.num_values * 12 def _int64_size_estimator( @@ -28,7 +35,7 @@ def _int32_size_estimator( def _boolean_size_estimator( column_chunk_metadata: ColumnChunkMetaData, ) -> float: - return column_chunk_metadata.num_values * 1.0 + return column_chunk_metadata.num_values def _double_size_estimator( @@ -46,18 +53,23 @@ def _float_size_estimator( def _byte_array_size_estimator( column_chunk_metadata: ColumnChunkMetaData, ) -> float: + uncompressed_size = column_chunk_metadata.total_uncompressed_size if column_chunk_metadata.is_stats_set: statistics = column_chunk_metadata.statistics if isinstance(statistics.min, str) and isinstance(statistics.max, str): - return ( - statistics.num_values * (len(statistics.min) + len(statistics.max)) / 2 - + statistics.null_count * NULL_SIZE_BYTES + return max( + uncompressed_size, + ( + statistics.num_values + * _observed_string_size(statistics.min, statistics.max) + + statistics.null_count * NULL_SIZE_BYTES + ), ) else: # A case of decimal - return statistics.num_values * 16 + statistics.null_count * NULL_SIZE_BYTES + return max(column_chunk_metadata.num_values * 16, uncompressed_size) else: - return column_chunk_metadata.total_uncompressed_size + return uncompressed_size def _fixed_len_byte_array_size_estimator( From 95a7e6716a694a54a3838f157be4c12719d58ed4 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Wed, 25 Sep 2024 12:16:08 -0700 Subject: [PATCH 07/25] Adding UTs --- .../compute/resource_requirements/parquet.py | 6 +- .../compute/resource_requirements/utils.py | 3 +- .../compute/resource_requirements/__init__.py | 0 .../resource_requirements/data/DATA.md | 19 + .../resource_requirements/data/__init__.py | 0 .../data/sample_no_stats.parquet | Bin 0 -> 113629 bytes .../data/sample_with_stats.parquet | Bin 0 -> 85575 bytes .../resource_requirements/test_utils.py | 367 ++++++++++++++++++ 8 files changed, 393 insertions(+), 2 deletions(-) create mode 100644 deltacat/tests/compute/resource_requirements/__init__.py create mode 100644 deltacat/tests/compute/resource_requirements/data/DATA.md create mode 100644 deltacat/tests/compute/resource_requirements/data/__init__.py create mode 100644 deltacat/tests/compute/resource_requirements/data/sample_no_stats.parquet create mode 100644 deltacat/tests/compute/resource_requirements/data/sample_with_stats.parquet create mode 100644 deltacat/tests/compute/resource_requirements/test_utils.py diff --git a/deltacat/compute/resource_requirements/parquet.py b/deltacat/compute/resource_requirements/parquet.py index 4b7feffa..be8e7d36 100644 --- a/deltacat/compute/resource_requirements/parquet.py +++ b/deltacat/compute/resource_requirements/parquet.py @@ -56,7 +56,11 @@ def _byte_array_size_estimator( uncompressed_size = column_chunk_metadata.total_uncompressed_size if column_chunk_metadata.is_stats_set: statistics = column_chunk_metadata.statistics - if isinstance(statistics.min, str) and isinstance(statistics.max, str): + if ( + statistics.has_min_max + and isinstance(statistics.min, str) + and isinstance(statistics.max, str) + ): return max( uncompressed_size, ( diff --git a/deltacat/compute/resource_requirements/utils.py b/deltacat/compute/resource_requirements/utils.py index 15488844..4cc48f7c 100644 --- a/deltacat/compute/resource_requirements/utils.py +++ b/deltacat/compute/resource_requirements/utils.py @@ -89,7 +89,7 @@ def estimate_manifest_entry_size_bytes( enable_intelligent_size_estimation=enable_intelligent_size_estimation, ) else: - return type_params.in_memory_size_bytes + return type_params.in_memory_size_bytes * parquet_to_pyarrow_inflation else: logger.warning( f"Force using previous inflation for entry={entry.uri}. " @@ -136,6 +136,7 @@ def estimate_manifest_entry_num_rows( previous_inflation=previous_inflation, parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, force_use_previous_inflation=force_use_previous_inflation, + enable_intelligent_size_estimation=False, **kwargs, ) logger.debug(f"Using previous inflation for entry={entry.uri}") diff --git a/deltacat/tests/compute/resource_requirements/__init__.py b/deltacat/tests/compute/resource_requirements/__init__.py new file mode 100644 index 00000000..e69de29b diff --git a/deltacat/tests/compute/resource_requirements/data/DATA.md b/deltacat/tests/compute/resource_requirements/data/DATA.md new file mode 100644 index 00000000..88064fc7 --- /dev/null +++ b/deltacat/tests/compute/resource_requirements/data/DATA.md @@ -0,0 +1,19 @@ +### sample_no_stats.parquet and sample_with_stats.parquet + +-> Number of rows in file: 1000 +-> Column details: +column# column_name hive_datatype +===================================================== +1 registration_dttm timestamp +2 id int +3 first_name string +4 last_name string +5 email string +6 gender string +7 ip_address string +8 cc string +9 country string +10 birthdate string +11 salary double +12 title string +13 comments string diff --git a/deltacat/tests/compute/resource_requirements/data/__init__.py b/deltacat/tests/compute/resource_requirements/data/__init__.py new file mode 100644 index 00000000..e69de29b diff --git a/deltacat/tests/compute/resource_requirements/data/sample_no_stats.parquet b/deltacat/tests/compute/resource_requirements/data/sample_no_stats.parquet new file mode 100644 index 0000000000000000000000000000000000000000..2ae23dac0ffe794068f26c163b7245b3509f31c5 GIT binary patch literal 113629 zcmZsj2YgP~|HmIh5X6ipPilSC+YL==^ zYZOIQQQH6Kd+z-P{rz9BzJI;F&%NiK^F7~nzUSnZ9Gep2=jC@|Z$2W)?^p>xFO^DF z|FPv&MU|@Kuv#irzu@zCqm=8QcLAphDA%v{p8Y$Ya^3NO*M^eH_0fH^YgbUN*PPX= zswmehidT;+r(Dn6p1QM`a@}#z&}q$->)fE6+4#Kpu6uQKq-?4*XDkfOz!wYq;mgt^-g1IE7xzkOU3}@`qARaqZ%mJ z^{ei9!QcI{!Jb|G-4hv4(~B$51ob}Bl4Uab-hghEl=};wba(T3gS`*ei&pMi%Jtmo zqg;>lOT5@dxxUtP%z+ll_4C&!OSe|8Zxz0>g=JfAwePF;%KiB*drYjXTyH6`#KCed zcWGFmc;&vObN;98lw{Nbcjmr1J2gLDSh=6r?C)ve%Jr$azNul#b?M|w7g;{})23U>DEE6ExFxa7 z4Gn9XxSb8{+BL4pZT?to8p}CoU{FY$a({mRl)kLPm*a+2=Qg8vZueu^?izmIpZotP z@b^`$Zxb6vj^VL9`EC7G*5||HgI*;l?e{sgB96c12uh6SXX@Urx3Z}6%!az^?3Voe z!_^mpliVgBwv%V)0ZpgdDD>{n+) z<@&mBtq-tuVQYQHYi&H&ZTfGI;(l&Ryi=WJxFG+X1_hPpz3!H;1lJ(mebf7;oo!@z z;gYeeZ@r54Y*bcxrnFe*{mA;T;nMF%W0W>OXrIhu8J1q+O62RY_EDic-owv*{$Trko>OoK+oM-p z{4M^j&-CgMEW>j@7Y*Wep61pGV)?vT{nteHW6_02M0QmA@XB+vJ&$+%@rozeP7kE- z*Rg%IINQHC%QMTb*Rw*(yFCIr{=@B;Eb(iWTDgB`@y@HP4{ry&NaXJ>JAW~f@82>^ zIm-T_*nxR>y_I+4SMAN@Hgmc>h-Mo(*wJm^@s6!Kvj)GrCO)KcY2{saiFrqOj-H-b zx4XJ>|DU+i+u1Ijzy3F~h;qO1xf?&1Qm(H(`eq^9Z~d1yi?i>2(r(-DY`>$2E~~`v z);&}{jP+;w$8IrfYe6-p?%}!1?bm%V`{AQ)w^-O$_uAL{9NU!7&RMrvZqBuqKyLHE z-4RFFU+s;Vp2Rw{uF=x++@GvITTZjSeW*R&$nDfV+wv;g&hYJly;wKaE!!Exy40kJ z{wmwi?)AAZ0~I;kspBrj@Ag=~D}m)x?e9L9*)KV^3_HMj*zD1F7rFh>KMk6|eal~E zP5A(&&08D4zrpf<67lXG&tvz1fBUl>#%h0@$k*4Rr{-`R+;DgFS(d}Ax66NKdCqJ( z;#M`K{lU*(7G@oOFLgY_eHc(<>pQ+5*zeRL_C=Li1@>f}Y`CLwBKPOsii*QouLh^Z zeZ%9`#Xe|SUisGi+6RAcrCcv6QE5LvKj*;PQS6hw9f6LQfsBnl4MTWKPGl#xzUYhM? z!?nfv@+;3Dz8qeh?b`Off`{7)8k6*z^|o-*YKiSSI(<$Ezx!hGKN82p4{aN!u$}g+ zdvq4Z?q-3TIyY6m_2%SH2Y8ITFU>E_KDpe+xf?hxFWRB*$g*9PvFaiB=dy49eXM)$ zSJf!bb3NUZ>ttPZZ|qZr1 zP31mEj{Q)T{q*Yd6L9)H#%d^8J6LV2Ty*w5rm6z@=#c+~*~&em%f*k+Jie!u;J6bJ`d7SDr6@HR(IH1>5bg z4Q#*H2RorYat!rdbGJ7?@89A>y(Y^2Yni@|Jbg}dSy=zC8&4GExtKmCyD9hQ(b5v- z>nQK;$d5V9 z&bIJ(>(2N8lWlp~yN6IoSyx}rTO7vy_h0;O4EwjlHx)Fjx6P_Fk7OU4SYpuJ4$8ah z!lo@QrCeWJ*K-TY{CSIQ>HOUvHxE3@u^?{Ea|`>H=gWtU;b-RD@jT)F1eb`b%X#2~ zQ@^d}T%z#!{)1R<&W1@lSXOPkN(Qmt=sK}PceXptsfNkXO5Z|<7oE)V92!1%49hdC zdHuCK*C~H{Rbahc_tVibJa;A2_8g2=+IiZgmk-N-MZ2rXY*YE?I~Q{7J5jsbq`bb> zPHoIG)XiF!&GIpvT$#gtP?h;9hx`0$fT4LgeBA9ViuBe!2$^Y|b36*lh% zPx$$R^-C<}ejYr1syW-~mLVq>@U?gK=JQzo12Puesib_%acs;zmU+r7^)A-oiQ`AD z2vnZgvFh$Rwgto5yT#aEe%xj#oTNN+!qw{~`$J2&<*hk>#ht!Aiu+(Gy}LKJGrWc~ zgLNaQm{cZCX{T_r`frOV*FOCY&ShCOGkXQ|GjY=U!~WJW@=y-9qbrxRf@62Ji9=%f zTftdd7U#))R?;ojfzVe+7O;&c7izSRWt(xTOnH`P{Z8%vV0)>*__2w{*nQ%WL)_0FuJipK zzx;NNbv3-&&Ij5^ z7d~=3hM0`{>>CSTJ+zv~=-V!RR!61HYUfJFvd-LDbfc%2azFmxHoI9jX8y6IF3)4f z*fJ~lyZwA8+~NLI8vSxG`yW#YXExjKWbM0C{BB}_CVSb(?)BdPPfg`p2{kmmxev#t zUmMJG-SJ@O>ueW3;$T$6NSYN29_D#47i7 z1H#|2FHQUQuV}XS{5d!0s+DIl${zD(`8Z+^yySWB{-In!p6eDpZu_%-9(#T(m*@9X z#hl3eN;_}g{yD0pa;+X0(w=o`TZLacag5MRKcwTnjW6?LD9faRHoP~-({dL=e`1?@ z@@t5Sb$dnYLep3#$!Gf%D5!jE{;gg?-2Z0hMpx%K+8v-N#yL-tvGHr}+w_7B1Nb_z z`dd|k($2yBEq~#8T(F`z%JoNpKJQtz69LYS7k!`n4VOu*~Q@YG;UOy3c zj{Vr>;V(k?`HpYxbJ-uhew*5sZ7}0W@h5&tf1bF@2Jl#(HhFuQ{oeTp*H7`>C7=9u zE$e{q^SY_*w}`#DGgh$-@A)0s$ZPPuZMLr9oG|d@ zlTxfF#Rkkg$-41RmHRz-j_#O_9^o8pSL3f{c`JPgh(27L=Y3B4#3Gyzujqf%&9QPp zxf2uE9v?+D`km!A`u8nX@_bq=UoQ94_xnv_c)Z?gR^DUVS@&;7 zk#K$k%eGw8vRw~&+N%QV^R*vSF8M3%>{|DA zbMAkBXQ9n(3&9<>H{rg$-?eBs+w#3hRVr{F(i)t2#Ce6`_bFFchSmN^9vh*w|0*N; z1N&S5TWjuftnc{#Reg>}^*^+>vP?SexIUNLdGdD7DVG1##i~&}kD<-S*x43}t@&#r z`|!7hV__U0=O37JkbUw@qqw_Med{nFc)v)Z%V z&NsG3aG&4r*N=@;-W|IrxhFrw0WXU$$o76 zPlj3-aXmgWCryWlsh^BJul z_F&xzdO0YO=XzQ0gKxOcZAM<-&N_T~d)*G<%6G#D?>Qf&T-UxDHkR*CT=|=d{cZ5w zQ4jdt>&;VNsFml-l__zDbA}%`udc^)x~AUN>NS*St_`bnfakIHnza7xXA;wA)#155 z<$b;;_w%}Az;d=z%Mjmi_S^Szjske($j{ROaV{~bjRl-+=U$ab}XU1RQvA^1#Xkfp6^x@1&+|RwfvmUY@?$#~8 z!}1ACK7WGykT^1K65GYadDANJeJ@*bZPxi_Q9X40tv;zYJGN5ByJ^BpHRlqWX8rOz z=T{B)Z%$~VJhSc5r8}G_jQ#q+J+@Q#z)L3X&-;?bV(fG79NXYmQ+ap8xY0MbZ%dpN z^7fKWm3$w?G39m6ltb*t#s~e7mn$wx_<3fm(vD$no13iXZ8}As<5+z0rbf&2SY_w2 zCL9NS%FSrXwqW_Q)4wbS=a9dBd5mvnO#X^xm0V|XpN7h}#`oVF&bDw(`(gvBorG>=SZK?^^PGQ~CS3{I0vJ*_UN|e)Hp9Jde}M9IEfHe5-K7%zCVQOL}(R z!udmwwr`8GAGnshO~?Iwayi|=avLysqJjNPwY$j^xSxevz5K}Xxt2IJlgHKZ{l4#6 zC&&LZbQa%Vg}NoTf9_e)1{}NHvwkbUGR(hj&r)1LEA`@eG)=hR!?s^?UWI;~6TYfiVXU{({7=-9a~>*1XZKQ`uf z=X9Ks7f;{hJ6w+C|Ge2LZ}w@|?(PlcxnA}4&ZEVY_PqzJeBMa84h<;iX4~w(^49M> zUmK=3QSFNT4*xarCFd=D3Pk1A3eHc6I>dcDdfiSszL&ZO-49^`31f_uFktx&H^oH+aOd8vNbI{%kw($B#7P`CV3E)#z4AJ1>@d z%Q}a@?t<~j&fkqLK1a=Y zMU#Rh-mv_WvM$}`{GsgJxF@Xh-rJoe*jL|qGrl_8&fB|#HnR_LM7w*i4F4JN>(4Cn z7Ne6RdA<&G`r&q7|Ltks+|T-vTYIp7J{Z1eB-@~BaH%P57dJvq4duCB@aF6;&M}T| z_p8hEb?e(d%X1&nE@mMs@%5)SNeysEhL!@5C*;pRnk#MY-P^ew$NH{4w(Mrx_wPBl7WZe(<9*ruO!Aevdy)(0QgH`loT)f(RagXcQ5-P{K}#yO>qtze%M@aO7p*$?XuG=XW# zeRw{-=r!)!(JM2j7FX^^_kKKq?ZS6w`Uq}QRqaMGmXFW1>h=7U=XG0m{KoIzD&A>) zLFN9n&M!8zU3}R7<~rXGpLQZI2XLg;yU))Yp0eg7%l3|6i8!8X|G0&DXU39ln&$Cb zr>W9*72&b8zw{mF$sN^AYO@SGzWVhB=PfB4%jN9{H5+f4%-@DW*V*FDb>)Z6Q=ZmwA@2Z>cO`y`I?cTKg2`rOMN!xht z;s^ba$^A@NZ#}^FI49?;$Lv2hwXZC(o}BuwY!>UY&y16ayjJpV(eD}S!05>fUa(D7 z(X~kkRlaL^yS`)x<@)KQ)s5RK*9(dcsL%RPfBee7*GNS6R!QZ(^0_hhOTN zcaGwpl!U5m`#lx~*JQup8(FVCw^R1)%`z;LHaj=w<33-DFSws$Tw%%3oZI*RDX19h z-qS_9O0d1`yBAxS6?`A!@b7NZ!%XVF- zQSDi_*X!Sp<9^mIJ7*{Bum@p3vkp%cR%lD{FWxs~o%g*HE6x zy8Lnu>!G*r!!6wZw{3fD;TX5C#7jdf<@u5Ajz=_7u9HXoG>FG&Ygg_v%jZRb{R?<3 ztA=iiU>`DkdRQyA>$k=x54g|OX87G^y$W72c4UCke#cI0HgL?k^~Y)-)&Esu$lpq?ICCubEyZ~;w2acu2KdR=%5||SVU2k#*Jqro%{KnwA(t!;x&7vuJK z>>HVn?XFtOeKzjft0U!Sb4;HXJ9!w-*Sx5C=UEOrZgxvi#_p%`yH*DOH1s=1e5hz@!DlveYrmFyWf`| zFuKOE3c6hBSVhB%rN=5+cHBKy*>SYS@ha}i(($TU50)OUHu&}33V z#`wTxCjusi-#bxrdVIjiz}&8yleOmQmYuATlR_ z?^J^=M*~hb+;LfRy3y_j%T5RFdwuV85lv7n!Y$%^M_`yFKd5j{`SH0 zA6k5PegB7+s)B*%)n0zO^Rb0%tvDZ7EaJiWRwY{pUWhN-O?RPn1^tQ(ZK`-4Txjb* zCh%fH;B4K+c6C>-xR}^*=YxywgTD{FloWnNcd5gxJGrwuHhp<{R;QNv{Bo09mFkw; zxowTvxhd^KuH<&<6zey;YnRU5W_L@KX3y^4d*GFB!Tp95?A}8&rFHk7X$!}6?`7Kf zefQqB{RLC|IM23D?VE9POlrTJXWysxANsLij{zf#x9K5`tva@cW@4jbJ+xDsd-v4M z?9isCeoo)9J=5k}kM&GnG{n1?VcC>6y^O0Cj_qYyxA9mn^QQgYy)E0$w&`u%b#rWQ z+umo#dfN|t^zP$0T)b_cfyb(j>*GAt=y)I3x#orXx-WKU+t>4B-*J62u3L}y&Ac_F zP`|8uQ`+{+{&nHFemPGz9`85k`Tj!v2fsSow*QbJZ*Gq3KlJ^x*PPb;{o9(;V|`)*XSA-A9XPXH{XK!RIy89~ znA1o|;vw(f)mbvar%BzVV=Cp;T{fZq-nz@DGp2ZJ?Rm1d!RGz%-#6HD z$fs$;t>0G~)NtF$`rkI(ezwWq4R>5fY}#n&m0p7y?fS{|ZKH2)=KS4g_uYw2gZBI~ ze^Aig$Lqcg`u5r0zk~L@Jk_-E{y(n|YJA|`lW!Y;_woJTjSuE4+AR1`!ODY!4;N{$ zFZf9DrXPZjmTKQD+8Xxr|PZW7kawUw;w{! zgq&^`b~fVY!C~iOe%lxJL-W5sgq@Eq+C2P1>&ipIFScv2Km1aMrXRyEcW&Q2;!3yP zLn40cY2F`kweO&h5!a+i%_D!(e?27fx^eyf$e*p>evG^^@O1MiU-wn(UpKQJZv5-k z;J=>zb$fWBptpBMm$$vWJHF1QxA!JTJ%4+DdfT9P4{}p&?|zw=zUkes3o@U-d$?p= z(ECR#=GxvrUbA}B`zIT|dH(*lEysiYe!AnT?eAy1A8z{l`M$rN|NZ;HLXAJXI9lHR z;pK@sn?Jld6ZQLtKhC#p{PFeWRQtz2uhpEt|IN>?+x!2zJ*xVFxA2eWTHqhyA62q{ z^m+sjSiuY4SM|9+r2o#ZxJP(t6*3qVBAyBvkqVwyg=|rUEK!AQPlc$fLPn>eB{-s} z3SL@;Xs1GyQz1I2kTue7iD(~LuL{|y3fYAUS+xpLh5i~yg^WRkm5K@xO63n~07L;* zO%Mo>xv3C2RCPdIP!CYL)c_!~Q6VC!fERBZsVJ=!5xA=6eN(yI_@RY{-&K$S(+ z2_%EgAO&;*T|qa{9i)ODpeN`BdV@ZoFX#vQg8@JS8lVL_pa*Fn9Tx|203657z~Dhp3*)v%wrN7t8}+f%)KTumCIsi@;*A1S|#1z;dtxtOTpTYOn^Z1?#|i zumNlYo4{tU1#AV|z;>_$>;${OH()o|1NMS%!9K7b901>egWwQ2432=K;CpZk90w=B zNpK3B24}!oa1Q(c&Vvi!BDe%DgDc=ia1~qwKY{DuXK(}D1h>F#a0lE4_rQJd0Q>@e z1rNa^@EAM+zk#RV8F&tU2QR=&@Cy6^UV}fu8}Ju+3*LeE;BW8&d<6f1e}O6={sH+x z0YJ{7AfQr6Ay61l`Njv()Ebs&60R2e?Z8=l~l|W@s1ylvq zKy}~`YJdPx69fV}?p+(y0d+w=P#-h^4M8Ii1R4X{@}Z+8p&$%|g9t!dgHa$F&{4T2 zpebkungc3pw*+bs3*ta45D!{|HlQs?0PR2`Xb+M=2S8=DP9Pa{1}T7!G;{^sKzEP| zdVrpw7w8T8fWDv~=nn<}321;8=zt!ifplO1MqmPFU;$QO19sp51A!B`fE#!~2FL_i zARFX>L0~W#0)~QNU^o~7MuJgbG#CTMf^lFxm;fe%NnkRV0;Ym#U^%j)F5o`jR!4|L;Yy;cD z4zLsK0^fk$U=P>}z6JZhesBPM2M&Tm;4nA>j)L#OF>oB504KpIa2lKeXTdq}12_*Z zfQ#S~xD2j)#6fx%!17z&1g;a~(92}Xg@ zU0kz!31)#@FdNJPbHP0D6_^jc1`EJKum~&$OTbdF z3@isLz)G+RtOjeqTCfhR2OGdfunBAiTfkPZ4QvNHz)r9Wd;@laJzy{R7VHE2!2$3c zI0z1b!{7)w3cd%&z;SQ_oCK%9X>bOd1?RvI;5@hhE`m$oGPnYM1XsZ|@DsQWeg-$d zO>hg`26w<+a1Y!E55O%pz(?>8_!m%3Bp=8R3IH!q5O{+^fXatO0Bt=M1;s#dK*uFYfRdmTC=JShvcM0N z1LZ*lP!UuDl|dCy6;uP&fj_7L0zgd=2x@`apbn@D>H#_u)&MjFjX)5fEzMvM0zyF; z2nTc&D-uM3Xb=OMfTo}sXbxI{mOu?+K^$lW;z4Ub+aYa10%!*kL3@w{I)IL#6G#S~ zK?f_L9-t@a1$u)%pfBhL`hx*L0vezNI-mz>ARQQh5tx7(Sb!DSfE_r% zK;Q%};07L$0Wv`r$ObuJ5Eu-GfT3U*7!F2&kzf=U4aR`6U>q0^CV+`x5||98fT>^_ zm=0!unP3*k1+&2%Fc-`NUxE4HYp?(;1dG68ummgx%fNE50;~k9z-q7ttOe`9dawa( z1e?HSumx-d+rW0P1MCF5z&Btw*aP;0Z^1sW9~=PRfrH=>I1G+}qu_gR3>*h1z)5fl zoCasWS#S>g0M3I8;3BvLE`uxJM{pHf13!W5;Ae0H+yuA4ZEy$N1^2*x@BsV*egzM~ zBY^Ve^lJDg$!~H2`oHJ!hS|niC7VtXR~C)l>87jJ8G1{$jjrPLHls^VSFI$c#qKJE ztM-!H;FN#vB3V6pOF`UBkep6B?X60*yYzIUwbf|Jri(5fmt>=hSc_3Bno9Jz+(vPW zn#!g&c4s!-?qJj!^cIWwXo^JF)NZWJZLr(Kw-WR=XSNUSB}&?X7>Us)`W|bu8ATWH zsNH6?)A&0YwRVfX2(DTgoJJaeBt9WI)6q9KHQWIWxLl%VaeB8~7KfxY)9a~{#jSUe zfV$dsI=a)vAUW+q$Z>Z5Uc6pocS>|hOBFAn4;CR&y=0?*CmCH5{ksePo1SRxwCP=9 zzLM-7%o0f?MY1@=>uvRRXS$vq$6&MasUQOqulQ4G;;5cAyLXoU`1vW1xOy55Ny5nZAdvTZ`jMvKm=7oAEnLUpNk zP)@s;+awyAtZvDA2%KK%YP6DWiA%JocL{OCqkk41eTbA$e299I;&B_OMa)j7_*{y| z<$?~;y;zzsdI+BrH`+nzgdE#i>`tSYoDPyxd?^L0CpqO_vT5KqGqsZfixcYC$|)Js z=}vomL8wa-+M$ldIq{6F9o>yuw_OY*RSeE5ny{zqoivxp2BXF3pjYS$m(;&x$zv&s zf6JO`hZPCUOYvyLXA?0AE_xZZYVe3Dm#3fNf-1!>2GBurLoAu}SgZ{KmvA8*MuI&P z1Ec$0&_}0`KPEFn=pTk?6c>s1Y%QywFwyQtXS&g5l;{(&R?&}+cBfVLJhpUb1vS}K z%nyw)Rm!%>x&a%fmlExw;Z%H7HZe$C%uu4<~LmWJ8j&ggiQWsdf6u6LkxC{BwB`~U>@`+$xN68I~JvN7u=svNyVeWImpr`wfwdb^iI0jUXM`= zJ;c1i9_cbspO)s-3lYF}aCRa8N^2?H5_@Y=B zq@$?5QA|gBa`n^#MyHn*g`AJ9RYt2dTl@ykp_OkzJ=|g-l1HAtc)KSZLMGF0Z87FZ z8ofN01idhHdbW$!?sf~=X3M^{HJl%{+tpyVN}?NxNOFHjQ_Mnx?6PuomCy;2NEd_A zsdPTkqqGChKi#u+3l)?ECyb88Wh-p`~Qeyk0 z4ke?TcI*_|s917mb+eSRT?)3Y+O=Ev)YKk5d-d*G^;($m=(?IL1 ztEEp%Z(#5<2KNo=XKG}&Bv}1z_Qd9PNBK4bn>t;+y^Fd-YSn61yojesMrJ}nkF4k@ zLw4Jo7Rj9knFseCqUn_|blC6_1u6{iu9P~m->A`zL&oUG#zc+_P8qN2UZ_#q3Hc`0 zpES8s?C5;8yA++oll=hC5txLxGu+QN@K&Ohzibv;UgVoealFWx>LFhTsePc%#gH9pRTbH zn*$ouT{kdcbkRb~b0fCQ+UgiGvgNj_^$PWCnwj-gY;mi3yTL83+A*a|hCQfwQLnb% zg@>72w9D)r(9XQJ+{ncts-26MHQ&6eqMuuzUash(sO8=}E6kWaVtT#E_U^1Mn?t9p zZ@D)2t9d17rIw!4@tbCK8cd#`-Z5jvtO@mJv|J`lu-iLiW)IojvSjP1oz-jg-#FRT zdiuQ3p8XaEOj=NA&-kr7*6;4R%hS%>f5Nbsq`i~fgH2QF^<6(MFs0zgyA zb&bktJNNA`yI9jCpm=h1Pobi%>V&Pxn9|U!nLK>ex`gSwdt1`V_f7~rFe*IBym@mQ z^}rexyX~A_XZ&~ez9U@5ovzts#&#cgP&2Y+(Xr7Ew|UAS)z;>vs`zDEnrGK49N%+p zRJSbyn$PjCTfWCs-_-EUyYiP>nznfR$2#KF_>nd9V2V;}VBE zwtA0TSEjAxPKX>3UedmK)6mp_Zilj#49hK8ZR?1`<5i2-c@JEuD!gt+liH>7PtY|B z&@3>@+RBP2rd>-&fv*T>WWn{FCJOJTF&Iv^#Zj%!8NJ zBXK8J(*cQ|?3hxF@J7PXc$_xj#8M4Lw}>VQc8e185x$XZ(2H&CE+s=k_(nJoIc&1t zX*S9)$C10H$dIDXwjqBLVJpF-Q$nloPa^-5ot?Nunro5dEFU>Jo+QT^hYt#WDUw4u z=1Nw(Xb&+7SuAzBHN{-|94GU1QzYor*#wUbou#X;b|-Q;dQc8J`W$+&14Vqg$V;V= zq={J8)^643P&bkgDTKg0PKQB6kv2%%^zsla83;S#v9?Z0_U=Zj18q{rTiMY!n}{f} zPOHm}G*F~%9kg1zXsC-VTc>wY#EG@&vyh?4nJ0AtuhPfkk+f6yk`ZV`KGa6>XGIfX*nOuZFB6EdUGrAjV?(Uy*t6n!;TEdJ;VNK14|2Rh&j_8hUGYHPRU zNEW^51F~QpRGMB(GP*H1x@u*&JLG<~^|&!AOdAPSgmm#aIZH+-={99XLI7>BnvzvG zRWBi_p{5cfn@i3$utLd@KNKgi6?H+6MT!B9u!(gRObtl^jV}SYl|>m*oIMk!NFR%n z%(DDjIX$9N@y0B>SlGyKD%VE0Se8-`;yg~J71CR|2W>E=@&KVDHkxXomtBljIWfVK zOrBD^6DA>)%P8egYv`y!vXX4cn2_SoWp~MirEQMTAv-c%MKB#8Cech+I16Phf%S>p zAXytVoT4}Cs0}QS&3KR6pbUxaqOH@ImSz+tA}5rhgS1$eR}zx&MoHN=tw<)7rIg&T zUMK#o|*tls@G^}q#d+afhp-UT%51~ zdL=>1mOU0e=oBN2#e@mUr&!Yc~vuBIdg0iv{9xlbMgKVKO)DD&rNUz2P1tU2M`VF>gre@{hhR~K&y+tdQIc+dr`VFQ6 z`&Vq%g{3Psk>E);=!J-Akn~s=t-)&4xk*|uUYMogp3sW?t$55c{i>{sg@n*s)M|t+ z%b7ksASw}h8+%sRrVuMSW)ZfocqI9EEI{mXcudk~$_s=Jm}}A9R*0dp<8CEc9U8n( za!avOSxxDlC`yQj1_voR~S;Tg$6Rxe>_; zUr0Z9m9i1c=n6WB1r&)$`7#MU0a1j0lto29ici>W)I;isCBj0HP>8Zgf)P%L`qfH8`ORXn)3q#6W$s1Ii-sBJgc6ErPR>hru;|K= z{X?vi(iwU#g@Ue~z9MQOLN0n6$HJa=k1d@ZgG%accKQ`@EeA12-1In+DbQ6C!nEwO z5~NIenQpeXk%RjGR(1tNyA=wG&{PhOs%Y3aPWt8QPAu<~)t!%6-W9D}#ZN5o0xH)G zq}5%3vb-xFT$C4hivPpt4N#TAck72mo|+e%G+t0lCf^WWbgkX3WOrKB zdG+qow_pDO;ZkvpwqvqRpH`%2v2>|!gu!SsTdX#_qt`%((^bLk$tatdm7O!lU1RW& zp@q8-D_1uwy~*%mt`Q?gHBGjTR*gxhI5xQFxbYJvcAqqP%E(Gnr`e~s%MA6&oDu2Y zd}j2l+}U&H&THIg&Q~Q{SDDat{?~J6E?Bt8;Hlq?eY@o*OjFkek?S+N1t8!x?kN=$UO2|e*40xxj)R#3@Eh1 zx-z!-M&BXo%Rlvt+BD$ARDXZ}%9Sg32woTAt55gKZh5`4#uk0RFXCt4;9`ZZHatG= z#`YyQuPw_+ShhUrR{k4~6USQ|D;K%x(vDHt{jc|#Sv$6Jv}66M-uKHt_$7R!*Oea&micvIxrb3lLP{n(P4gdBYklHmpSY~uyN4fp zP_a&hk~JT98M^3+f4{5>xpz1HmiE+ADfj8pXT7&I@tO3zkRx;RncsV~xZ>+@9BFl9 z$;?M1&9!PRiK(aYOv>n7z^}-Q9yL!7zQ1Z*LY(iD=gS6r?!5H7`*6^RmS0V&(rD^W zTGx@v)2ICWUHL_uu&S=!IQ|#w{JJ@pHiVr%Vd}Q@c~{lWxevD%I8^_fZCH`CukR0T za4I5pNI!c*k}JHg(XT|iSDx8@QsNVD4CrJINHE3Sj@!JmaG{B#|1h@r$+w*K#_*cI zZfZ2GgWr_-(P906dRB9WVeq1fz2=;qXui7SP2Q5l!dv1H#{0)Df;%WvUa?bZJr9(2*0UXJv8?XJ9;q9r!7uzXya4g}vC1-;nIbn0b*j_t_H>IrNcW!t zP)T?S$t0G6!D_AD8U#wvJ(T7s_J^pAuAp>XdEC-l#lkRDEp{G|GrFZ9+@Tf_;NiQ& z)ah17I^C;)ds+=G8pG9AqtUI=qrw=JZqEqP+DhX$b0)1LBe+xeW?9@tzC?@4NOhV$ z%cjqCQD4g7zQG`TZj@TDcX*r@x?KvlO=)7i8LiINTO4$gzKWbdEInh?$ZC-Jd9X;K z9#K0kgf{Y~lG-65W2RU9a2xeJq+J?yrqOJ4Ajc4+qW+k~4_Y<$o7{Gtlua*?Ak3&* ziuJ2bjoPHviM0`Zf?m-X#Bx@z)>!Q6ND%oOnoPtOqbm*fJ$eKB%|Bojhgx{1D;>f| z+HA1|>2)6R;jT=1#nm>x1w!y^xNgF0yAUPmj}65qGiAf)Uu% zZX@Q%ZlkR&v=!tr(<_uUkZA2{i^t-jZ&b!z2|Y!i)9M}nqXqK3(r(#6d`2hfbV9sh zYIt6dJe+n6Q>NA}>B zJAQmxf#B+V!ByW~1WNR%4RgLbm$`ATAVa@iszS`AHb zH-dYRpf&iFzNC}5k`WT4#(aK0P@RP$O!+VoI3`l$-uaUqqL>meaJwN6UMHlf~SOt*0#^)vhZEhj_l>NK51ewG?Dxs*~)hy%XOB|buWWY&nXPRKyD z3*teVuBFdVUv=clLY!YJu#k~S)O|u+a2Ez)-gMXH5H+9>H|#MTQNo@XjESCq#j7JXhWmb4uf5b_j80Aq|w$9 zJa1@>8k%C(yCDoYkO@bYnO7JKMT|2zG#dIa3EM1*Dp{xomYbnBAuOU0K|fM=E$MRM zEL5vDNOmOt{DZ?Pm(fCXpSdD2YGr_WmKrrfouL>I3j4A<$U~sp^t4S5exZnQve}8b zB0~)|s436L)N6!+leVC4Dc9>ljqv9N(kfC29kwc|bQfw;TO<*J@`h#+mAp`dw``b{ z)X8lKvXdJo-~HIHIA zJQZ}n4)e?~c!Yduh%SlBf}sxRmrmRiik~J*grOASMDnGT==T4UlhJJp&NiTU80vzt zD$$8rHo@*uNep#!c1Aa$QuIuy2SH0{63HJf7X5?#GvHF-($Z+K5x#{dB{7heS&#;zIue?bH!5U}l81Ug zH84@d3=2l=&{%c!4k@l#BTAZKAs7UDX-{v2onf~KIW!!LhU%PQp=yuTr6XTR-?54O zF)U1tBL!4&O0$UP=_jie<bc}b40RiY3lkHrqY7ylEYAZ^h{2+3pY<9)A>P@o zTCDiVTB}oQCA6LA2VvSTf;^2(gNP&-Z^Lv5 z*;;n<2!B>rnEumjz!_4cGzqH{>Kc}&774e(?s4hqJ<_)fSv$khKTQ?g#vVIT`!K_2 zu7FfR=Mcr=FeBCjZoAeDx8%0d^pb+tVpM*f<+CB z&Oqnga&V_fLb50))?rpP%76w;1Wh6RyTWXr#%T@Gh^)(|NYnnA+aVun5J^Xv zL+!vpGqEh8fjH#o7dB7{q{x=+B6}bOrkG1*?l9+n{DmV!E_#Q#)F?tp7LoK}^fc2B zr&0wEb0ho00V5^vr|ERT9xSdf1VhCG2wNdJYu%zu9+vTCNYWvBBwNT->#}V+Sp&&L zoRk!YAsq5Dj@ocJQBDubR%tQ*0ftdzp5>^%-tVXDnbm{U^fSz~Y z_?jrehlg-z6aG$VCOi~fM52e-r8Nh6T%tSaa-}{zOg(TQsR0Q*T`vxSxWdELMyZgOev)vZ1S34|N)<7*`X5`vj@&MA*lr~ej9BPXeNN-6{Rymmu zPls=_=U|~Gl31)nX)7Y!Kv^D6$-yqMz7#e>T4ROUxDDY(^+22gaSLCCZqQUAN2N?Z z+{B8I<`k+fFNMR+d5NC*ws7O&7IhZ98Vy7!N2b{52)C*!xCw7S<8z9vE!_6SqEbj* zSW&ngYZt3dNS#`NXdpnC)+b?*ak*qG4M%E;#npB??j=a;R5%GbBV{?nW zobXI}J|GnGE?T56G`m?}I%IOYoNtCB8rswCBKf7(U9$TO&rwTukx7xsm}KjX2v)l+ zMl4K7g(%GU<2SQY>=;Fa$WaxGG@LQR$^ifSY4svkVj@COSU_c4gA&Nz@svk(!XhSCKY?Q}0wL%UQFk!qtsn2hiO@o&s-R(nAxiA ze?@3e;X$fMpAb@$S1)#3gqB>H1qrW^C^bcX-{y|ceOfuvd?6{qc2R`>KSdr=CktlX znH_;ji!mK3rdyxa6lFYmTSU4#-D1}WO%{eND?|jDqE4^13L_*LDOH6CqdGH;tXT}2 z4!K2`)GiO@WXKTAav4FyXDjW|MSQW&Lo?!Uk4;wEwu^wxVI`*#PK9*Hq9nBuHjEo8 zDv4QETvvo$oecqDgOz%cHyykZjBw;>Fb&(R6ks9-e!k8W1LXFcPz&tbiUCrGO=fuu zF~X&`(FQhsf_jDZv&$6$lS9-L57SK(l`0}UN}hu#B$hDum%+BR7G1AZDQlXDEVW0r7{qT9v;*b7h-`HR>PmTD-30~37G^|_=mj-PF~MZPc4lO- zI!%128txzyls7gbLs0jksugs}X4lC2ORZx;oE;geHt4fF!pumiGSlI{Y>{D~)<|M< zKdtT~!$0dIjg;cLRv#IGfcmKxNIob%uLH;{zg&Ni*t8bfT#bZH>0qouT7xhtCHIbu zR^xo6T=pREX_6Odkum?xv4qNtDpjQPpG=N^LUl#nSdP>nKSeH#3=e59jTDs~S|~(n z^J-%x6O9u)y^*@STAlFdC>xqxk$QESMar-{T|rho>6I~Wd}%bt*q&!iG|6)unU18# zs7HAWNoE&oN|HCA9t#%*4;*RweAPuG zu*i8*q&ZK7G#?fv|A@4Z`%xBRLWauYRSVjI6g+$VzCqGN|ZKVb$*BW?L5*4bpYi+V_P`9Y=l^GTGDY+xX zaB%@SDx4D@QS1>WDd#*<5uZD5p&L#J8IaA+EN^vzx`TQOBL~E?86sG!+X+;oC?ZaE?-JK!Ql;v@>0jZ`O#Oe8T=m#A)iZ()VuoM(hp|2p5qXS{l zCb$Kxgq@OuyaFn2WIWi;GU}quYO5Ptr1C<|sR-U8f{9qai7Z84cSKu-cd_APZjnfn zqG;qYRWvpUluC@yHMw*XZC7WyOe_ectQhT3+Z;|xZ^e+HXd(Vnaz$tTe_o2(q}T^96`lR%x{}1CSJKw#9C&UcrocsaQD&qg zVKKqTtFgf6{!%s*LpG)q+DT6MYY=%~GX^C~myi?bJ*vns$QUfmU_*ARD1ggu-zHC5 zOgOS!n}j)|k&{7~T=J>2n27)8EA$;orKqAC6N#)yBhn?hE$=8pmMA z0L64=LdAYV43=n4JJxYo^i}GpiSn8llt#X&tkO)>hU_0>%wO&wQIFH`cb>?k zV=SLM@`exnNOq)!deh0n7#9+w40sXY z4e<=pD9o+OpB^B{f)0n3Rmk9O$;y}GlI06`i6jxy#XnL_m!w|+B&5J zni6UuA~h0Whi6cV!jxx1P1Djc?M}1Ebx2V%6|og+T#21q%X}Byy+Jv+U6=Vq!PMmf@rI5ZQG*FJfl1`njw`+|! zy6#eHk{V^P;ga-ZV=zkx&PRwBMe>v9ZY6BhAXoZE&wn`@Umed&6+9flbqWq_vH!kXT#;2T8vSEYNfe4>N_NA5WT0&NdA_K{f9?vU#OAeHosX_=d zA~V&pPN0^;c>+Z<{B0_Z%t%i8K!BJ}I%h+tZY0+i^*EYO+_a(eOj$cGkf<*@`4orb zfh^=rb1}t0zB}y!n`Zo2_>YzV6c1fA)bYqUyE|*~G2d$ESNLgwVHY7Y&QgiA) z<^jbf3H8c>9;41Kbdx%pSAUbRb@!RZh?%9%YJ&fpn&2%J{hXG!4ACIs<@qieCUpaU z5~2zFR4<~p=qK#iQV~ZJfq8U!MEIiq;V;@nnxTQoaruK^q3lVxc}+AX5Jxwh;#@?g zqFox~jqD<+DUCQop^?P5KRa%vgVmx^r@<1eQQn~o8VKsu^hZP*En*)w@u^D9=46(4 z@-;9g-m6jc0edny6{JDdh*uo)UL!ST5@jR};us3uI+S`mdD{ip2oZm`q)EqYa?wpp zvo!(j zEdO8DM$@A#^fcB_WiYyp^UE%m1{(pNG8OT(TuH(iS{9Gk%%(6XQbrBVcv(d3B3;9- ztehcf2I71NS|Taq@E^HclNm14sdnK^oV?v5;yZ1Wq2i6RUCLsX9z~f4g*uHpFIyBM z6#A|4h-eIdFXgQgQE8A}s3t>RmC%*}o4X`${An^jpGNwKS=KoX>XPzF9FnaYJLq&m zMFTVXB1@tl5s}~sG$>B;u7tAfAmk@DP_?LxNW7m#F3=%L?po}C%S!?pv`GXcZ78Ch zvg0rABEmvTv|)LoAP1Ylg&r-AntXAzkrEWK@ur2L$>mn189Gm({ggY>#F%)qTZ@_` zM^uCj@+fAvIP@D zQ7Q!>E$zB}c65|plQ>dpv3x@mP)^a&hcYQa(|)nRO}DjJDrvF1{n>FRF(*o%r?n$* zb4c>pLHa_vyw#?q?QJ7!emI`MYZj+SC$$6f)}=IuCN4l~v5foAi7V1iI$s9Qt97Zv zB7-qP@*$G4!KVFxi;^VZ3}rq%>NIU=o+lN439`&kTeDp_cO#@mO6*W7q*@r^|DN4Q zQ?^62dE4A+PLhEMY66W%7}RFx9o-{6u~5lLn?ohw&vu|lI_!US2x($#Q{H2v>Blw? z#RiuygzcTg?|@0upQ-3V^VZH}SE4Se3*)*ArFc53qv*uplm8Yt>C0xT{Qq;|(nWke zdZE_xIc8nt7xh$;nTo7g7ll;cP1SoEbs8V0(P5hagBSZ))YWw1HFPnbZBbC`m|$AG z=%g=3Ouy(9sp~XYR5?**6lZMY@6zs?PRkWcA#FV4)_oQQ>6``Wd!}NQIvC%7RSuO6 z%2Rb|9JNU~aIglcs}3O${)E;#LZRinUuXE-4O8=4N+)&3|4~laYvFA~8f<%e(0xoc zbWMrQCMuoi%v7C0bw@s3N~3luskqMa`LSD(-peVA4keWTEm8}yz!r6|#n1N%CE01| z?4NJ3kcv2HQ%Z;ECzozy4uVhhE-xWO=g6g+ z6!)#m{*;f3c0>W+r9<%hbeb$pt{Lltai|7<=F^c}F%fy@s}G^jC09R4P8KRSBJ$}& zaX#El?u(+UjYE_^?8_y(=$1TpdhBuM)geeO22lvapA?|zkayOKe#n%Ul={feiy$O< zR9oq<9$aWF}q+k+KoRrQu5GbTf`569S`bT3Ygu;}j&Ok$&cEC`e zp>&$iftDe})8FqnulsqPca@Dce&6T0@9R3R^W!*=^SrM6Uc>th!C{l$n3MX>R13#j<@LFvw>&2#Pa@H5~|?s59G}H?Qs;A0O6fRo8!K`3=V| z*8`33@>c6xIiR??)wSBy3mXLLG4IG}qXiYTU0pvyL3cGTYt zcUmlS^Nl*?v37QnFZ2F!`}Q1;-Go@K6z?+cHWR*85U6Uj?DA0D7M;n)jU#jIe{J#j zz%l!QWR0A@7QMGfVw+c9Pc+^2$om#O$+sZGy4+2Om18T9i)nr7EzXCmFGgJ*()QwX z$`%KftEcGoTekJKrapCZaLnm;Hc%WcKF!wR);k^!P?a|~eP(lOe&$r8i(ZKOTkLAD zNRm;w6D4tYaqGraIK3Nb5b{(J3bvo4U|Tnr=d&H^R>yF**h8yEleuFxl+6%ULz)z7 zwAS8ot8feIxLy$^5SM+Dt?jE@&=`9ju2m5z4m{(q#X-s1@bW{;SMt=>?t07nsA-Yz z$6+{Hhhv`6+gjU2Q&Aj-5S6x8ElB2znNoyq9X@buGQPg-!EQa!`L8l~i>RtJBu>@& z{4F2J3FO8+)_hx>+5*oG?KikezhKju{go}MHO|@t+{>+e&dKpaG4k_BKnI^{d(k>$ zTX(OX$Xa|Zr*n^X%4%!vG$a*47AG!j(fK%=J^YHY&7-$lZ=CJ-Xz)XEVe8F8m>VIP zeq2@6md^*UVZ~aihtSTXZ25dn_gIZfY>#Yh!xUHLy-VR?1p*bn-6n0VolZ;|z$JaA zYMY02ZqkT42(CK&c7^x$a8=aFIrf%#v^@fkkc%xo3ZdnhX1a`=Jlq~rcjKkmit3C& zfl5PXdvf*g-of47liHh-dK_rB?djEBj%yJAp&nKj+cOV1wDUO8Z)JgPPL@3QOw}!S zLbvTl@+<}8V0g#5(e_BTZ*blo6K=&Fwkr*6(~VrMTE_<}dBSP?*0NRX7=%K4)Z3e# zuX*WVYWmG>sN$XEmj;_;$nEWS!c<&VL*Ln^UR${=Rk`l9y}Qn7HO*;Q=QfYwtUt3A z7TGAx_5m}?9yQ){6Ne(uwmFhaYHub|XSLJ6eFv<6;0&oIKlP0KHc@zX zpEY8@)p1=;O?twM=fyflqh(cHo9)-TF3(n||4qov&HZhvwzJ)0^$l!t+IaiTt8eZe z@$x@tcsJawWdPmphC4a1t~}`sL8^Oa2O#j6ZNeQMBUp8y{oKxrW!jEA1pAz*@6fTU zx^@P%0Pbz`pI^lv>gYT1xPHDgnY9J+&ZujJ)Uh_|&3Yfe&e$V9^%#Rgp~`gkcg{RR zV7}t8hL( zU(`JgXlU(qovxi5sMb#_YgX5BhxX=g@Z`o}?LzMG_`rI%Fmkkw_8mfXRk90QoCd6S zyzG!6+n`vW4hSSTJxsR4`DQlvMIkkNhjM}kN_RK{w%h~P^A?!~c@ zJi|b?Nl$v`O@v!JV&e21UNqPsn!_|`zd>U>Fme->>yW{n+ zR?(FG1O$Qjj)2p2ZcCru-=(%YyZxmkZChe@=5EyT!ERk!JG*a*7ukWPyWt}N%UzA`S)aY$1GM&TQ_3Dswsw8~ zg4!;z?0G_Lxi_h6?RU?Y?(vN4O8Khtq>7lmWvWd7O&8%2%QyEpzH&2t<2t+4&Ia#I z;DhaCdB(4_u*U(+^@h3LLn!!;t-TqCyKkCC^t5Uq?#&-y@;v*V$CdULtK$j|zterr z20H2Z=$LMK>nX8E`Q{#vlP^)<-E^JYxhZJzp1oVk1N9pAR_(6uvFmr{`N+iWlFaY9 z`L}u^+M^14+w1*C`6937#ntzC!r*~}9KoEFvU||j?u5Eomq%h%+#ZL;-r1t$`~0hA zC)I}7<0TFcJmcg7?!azHHh1?9&oVQ=+oLISw7vI^hslElok#Djorcy07A;5Y5t*xA zk_tSj&AUAwbUnIfqt`G#Y!5l@`7B-AIT0T>!#?-!v_$LuKInsj?j{7ZipJlm!!mpK zRuAT@%;i4D-s@)>owDZ;@g9$qufOU**$=8J+k5j&e^B>5sqC}I0|-GqMI?4WpgkU+ zJ90G|ga?msDH;;{Jlpdi4XUZ@nXdgl8}%gZY=t@)I}7*seYCX>l{qI*;8X4F6J+&( z86PFxwZ77RXy*#rUBF&a(>~A1;T9j2xOaNuOi6Le{b^SpWxuEPv-as^z9U1%hqDpA zKR>gptV7*ucLjQE$AM-i`{YxZh!reapyZs?X$oB zoNB$3A-ggATeNu&?_a@@9D^s8(-_|8tqkqNMlu5derKQTw$_Z$ouS6ksyqAaov)vg z@w?YMxZ5X|*0zGOui154_u0Z@3!ifG)UA9~>wN6;Qh4w4@c63o3NAsU$Ocl&?g+51 z$!q-UEe$*u#CuWzPz=+VNzn3u%sv|kZ(d=Ww2o}u>PoKtlQa24(dqR!%Cyg3-X;cR zCyhj)I_u!;{@KQiLhzOn-%GiFC&;ndElK^}9a`JJi|F041gFY#GgK)191mMRyeO=m zq_*GZIXd4i&x36sCsE;7TdVVT`_N5a&Ea}0V3t_B&(j8Hdw4k;n!0iS{^~SMdwOo4 z_ctu=A5)`{iyhAO?ptLX5PRzhJAsks6A$`l4^;WhQ65D&7^KWy9#H0sJm|UaHr@e` z6)ZPC_Dr{rySk|ov+t}oU*M6?Ld3$wA>|p0ib4nclGmq@TiR<~4)&Mv5z*>Q!T~vM^h&csVT&S{1-Ns@3~4&OYSVI&wfybQxqFq4E2Ae)!-wB%KOFzOJ(BfK7<5dmC-LmwG^lbbZ??j?nei4|_|) zxd9vOShmRz>B_A7`1&FHA3^3rUZrwNTIUVR7tL@g53VVvWm!{rk{ zn|HivzdeC}$b;mtn&rVxb;wh0?#CQX)^sbmHk5t2*5UN(zD+8hQ7uIFu0=a;a5!6S z1}VZeEZBE49-dQfIYyfcO<}E+WZpzIFKKnN#Q)Y*`x~-fs`*>9Onrg!mlt%w67H zw|p^M>{cxHh%{JRv$0!RkG!aF?Z93lsh&kYBJy}ryb*9S^P*hkXvi~d=znqeQA%(i zGE~LvbSpiqqfvK;JHgV({}FFisdDnxWawVQ8q4Wn=;z&+QUb@{b>D#09x*(Hh7NSbhm8z zII!Y$=k$;6uf6amcy34d=&h@pAEgDpb9XSha%+?8^KYu$&7Q?EjnDP!Q`7N2gT`)- z2kU(>C2a0E<8!4!d*)d$b z2+bI~YXNmkomiyM`FXJ4jfrFU>DkB9w0^y};Mi?L4q*7Qv?KkI#IOF^>llxDUT^Kx zqVv;!^jWIo8*3GkoS)1JvG1PaQqz=J90g?+OHr|^MsDZik2hEQsH*ay(Dr!ifs^O8hu1Y8Q#vl6o>tD2wxM&(OCH$NNZ}U*%(?PocI@j!u+e;C z9y;cnS@-5D6||PBj}P9lt;g$7@;>|H<*PJV9H;37!S;^0V+zGqW+z{Dad6B=#MuMb z?d<@^WWw%Tnz+__2FL6i{szyoW~SmA$EVAuyH(^~w}ElY4*uGE0M*uMWzFNeE3Efp z&(!3dh{wcUonrQ5%C-g}$3~wIBk3JCmbCDgCy3UScG_Mmcl;)G)R}iQs^?q9_>Y&5 z#_G6KYX_Uh6o#vF3}6h0uurl9A-=>T>-@#ZTF;aH8ZEl;7|e+$t?%6IxaMJN>7MY8 zvE{o>vZ5rBlO+RX^!REXXFeI9Db#hN?)`!%K5yROF8RMTlTUcL-&%wy4-YT0L2)u$ zKdG`vMSQ}u=I=ZM#lGVKWOSYOIWw>6tv3Rm+*qEGPgJicDSFe^;!#)WN3ZLBS z-rg6CDmrvRye-+Miqgw@6g96QKiPV)@Kr50x(3tgt3|z$8q04u^Z>YbS@l zQB9VJ&g&vi*bHp%A5^2fqR@ZM<>XkV(B4O-ekdt--#R&YM-SJ1SMWV)`{ef3W8OZp z&xwgUM|8N`$rJv~M|X2hPS+mPas79yNI&73ht?d_hv^;pl8TeNXI{mmynA(ur5`#6D^ zE;#XO#OY|HI!`>^?mg(V%C?^^*xO{pvu{RGHePwr&a2*Df3{DDYTISE>ye^%@FJ#9 z-}E-mu&<7z?RgI5wlR68j*6exsnpw?w=TLYl5pM>47fdAZ$G=qb=y~Mwl`*aGqs|7 zX}39MT@OFTV&;I`Y&$ILLd}0Cd$+f5@CdK}dWruqO;?R8^{s3#+`id`dSa$MgK&Iw zd$sML`gz=*R`_jS!_sYLNBwpxZgXU2?F5ix->uV^x0CX&Z0E^T9@nv^p`fu^JquIMc5U8% zeSP2ATiUBPZ*#zN?d>+&)Pp+NBDl?yL?Bzv`spA0@Jyf8y$zdu03NdHI)r`utyZ@h zRyXRcTc^DTs{{>0FEvHtX`en6FTicb@?7lAs-RDKWgiC}Av^AAv_p1{K%Da0qxF*n zihfY%x=uNoMw?(Q;%)?NPYJKP%UwBJ&j)u-c_GB|Ew8%hNgc>KQ}C3nymtP@JmlE9^d$$CtkUF{nb~Vf9RDTxpnR{ue^Ho)#on!$c1z7eSG8GOV2%d{o&^x ze&yN6Z#}&D!sCnU-*N7h_ia3W;lkq&Uwm})`g7m$>a}who_qD$#&Zw9@bIG}t+zWUO$Kk(9pt5>f*ckRgw&piGoUw!K8M=m^kdE=EQu0FAG;iYRYz3bdd zPd)sZ3s=v5%kB%$z4FAxC$9eXS3h|5+$*oX`ogoXp4)i-rB6Tpg)cmH?dr8hAAjlk zLr-2i_uS)OdTHa@g;!sF_We)2^z7oPC!c-v^0}urUi!fGFY=Gmzwb}|$a}ADy!6CN zPhEKEi6^c-`^tMST>sAR+_?Pg3lA?|c;doS*B-zAC`$U$AAiWd>;G=Ue>9(8>5ut4 znOzwShnF7fUFnam3kZq^va;uzcLUO2mRTVexDDg<153_V|+5@ zE`A?QuFOZC5$}(BSBB#&{oee_Z0Ifh2~+ilOf|kT>gAI;-w*p&W{xw@u)_i0kMhZU z>acSrs^_Ruk@}=O48AUH8IiPN`E%b z5aa0;q~-I`P)V77FdiWSR}B5vd1hD8%JfRlY&IAo^$e-{{%;96M) zK4o<^X}`w=sHX3K1IKanfxq^cX)rC(m3hd*P~FIX(g=~bY;@=UG%oF+ zOYU));W*k+geYL1EBY93fLR8k!IgQ0?V~{6fil1@{eH%EnRA97=3d1gsSR$T(J;@j zgC4gd1_qz{ZzPScpaIt#OCu#l=(FJ2)Z3>xO&o`XMQ8mXE1KdU15`g6)$&oF##J$s zk_;v5PrA)0pG;Vn!uNE<$QqwPG)F7t_i;R|JHP=DD9WFdW^;Fapw@d=<}6~Omh^Gn zH;!LTOS6~@<@V={F&cY|`XUPYXrk5>S^Lu~V^$D9oXxL{Fa)BE;#Pf(pd9LM9&c9f zQ%1!#SY;MH;sR9JpQ6ZqCSsVGv+5q@1n61C<7Vv=BL=T<$)j2ME5aoTh80Ky*3-kL zxa2r0&=VsXnygW$2s$2L(FkRAJdB&BS1=w5H%|0f^f0sPj03&#$V-l?o2vmk;l0lqE{Rui;)sL5PJ^C6&!189Y&eG&wg2N9N%V{Ij8wezeW3%d-h+8gy$FvM%#RITSd0raYKKW$h&Cf<0i8j5-Ji53 zmW;DH{wDyWmz7CH$s{LDR_=v^;!wuxDDjz~$Hmd_tU-sm$Mao6^Lv(;xq!n=?Rs?# z@UST#*J9)(j9DdJ7mfI4qH!4+k3iMt%@*j1EwOAoae=xhzD|G?K%sUMJ$WBpm5FP~ z16J689jzlP&J5=2Nr16&?{)cI5+VpNblg?zX=d)r^38PMfVOI=z%nE*h>xdbR-=GV zOsl;(Bi{wm(C#Fr@d}qpu;mU$Co!508$k7?F}fWe#apEtY8}0i-;F=hJT_qzKPs0t zNvdl+a$H))+I|>1R-q2S#w)Y%8VErXD9l+ljUQ<+{BqC;4nSpgV)cnYz93roeUf3t zSfO0IPNJ!3Qf&d&!DSN{-?H#wu{qYm^7#L(GHO6<13`w;OW;FEkO9OLg}g*}4{9R) zywW+Jkb}foTqnRJ0Wgl$Rj=V2b10b>i0_ypSvKi4W+&3jwdS`na3-38Pd8BBna1?CP{E1`i6>#sh(fSRGh}?Uby>is^;9$DvO$gya&{)l1-NPBnqU z4@1L%dgfsbz-ka41?qDGcFX88>0&l~8YQq0*)!`NC(9WiW_7cxaF_T|esD2Xmf@%q z;=MTn7-T##Ef|$G$Cd**I-nV6QTvXG^!;*4=BTinM^6a`*QWosdyqBh2t+F%4CirB zoFT(X*)a70OU{dVnsdS3zG4n4&k5&Zr3h7#2Y=9CSTWczEEgiElm@L7!6I9P0*fAvmued0id8kZVFquD zQM4GjS20;@VwGt%N@A{!Xv5)6AY=K!Tm zqa+W3Su)79!Ju*ra|hR}kTO>vTc{xeHJ;8d;2Xc8r79*=JC;#4 zJUpmovX+5_0Y%g1G8W*c;|h3!1?kL+$wk^6(86e7P~1%obA<)}q?}7!#}$OReziBi z5JYB>Dy&UK8kpdnMJbA|QeARQ>lLUB%p2IAl5e8}B+Hy;6OD(545MV@hAB)OP`EpcreB^+QWC`un0Gb35=n{51oc2ITC>w zrsFn(odQ>%k($C2Q=kyAEp0d;Q!8=OhHe)Va&L2Tq)}Zkg>fw^ zn2$O@#c^yzc=M}^ZP}D8q&56o+ADU2S7V|oDOqgXk1$VEi1e-}nXdGL{_%0`JZZSh zEZV^|RN4ee(*V}%8$BVA5Re;*$#ei*8ct2@3KRw~{~1YzHj--W7$z8)b95_Qb9b*0 zRXwvhlVKJdgl3W4l(-L!7Q$2{3F)qevCCTGIPPQY56iz;FVgE$1+L~FtpO~ITMlG# zQa|gQOe=-c{k$-h7I>@aSU?Imi3k57f0czHZVapJs8~k`=8{VFT9Y7}C3h7diO->p1ZzPo0VhIm3-wZhT?izR5;GlX zWvTcZCYsiYS)ISA7N?D(^-nZAE>z+Q>6I*{>j4n}>HzF3uoPQ?1ktP)FIR9H6-4w&wLSl+a}3VZ5E5}jpc6fcv!%dg@RdYS!!_=HY?NGPDNP`d%q;r;!# z5;Vd(W1WI-HFtYSo!s-{%|x9MO}Olm{07?-0o&!m}z0DL`?6PVCD4yy!mRz^U!#E8DYq(ob7A!-b` z06ty?113znNs>3Mw2ICpggR_78_TEAVDfatgaBLWg&)AMV5B8Y~ZF*HQ-sk znVCQtSA24lPRD1hb(~yRZZ(;$;ygBP-wtWVVdGu&~ano zp2l}@XPieqIcC(V!oszoNvfG;)shV6L=g^LwY>GKq#qZrPtsLkfO2XfnHv)4EWBT> zdWSK48DZ6ZQ7lVO{+qY3$3v7%!xh9BWlqEY^dp`09hGW{6gFNjekAW#mNf%1boP>> zJCm6OwJw?TCH$vRV1DWc+_ftpm_#{_>&yHIT0=P|5ZI(OsF)U9@{V;1-ixkg<) ztPBbCk3EpMZOohDSucqtcC`XS&h%1d)qpV&^#u@dR?dOug1Y?gI0gwz#WN#XaX}U+ zcmu1GTp}@gWSWvb?qkxxS-5MYvv8q!zF$ z$0z(WUT@pc;P&Hj01)*vGbw1iQaLCT{q|b_nK{Hom{lA+g$|?&WG;kkY)x!0!=)1< zj5<%1kZ}ie2fHX_!f(^jb+UQYQ3fq_C8}y$u^3IZ$Xe$WICwhJVr{}Ckz~x4%QOgV zo~MFIMH)6}s0ZwjR(+fb2_3&}r9_o*BfEdjtv!G%!J+MI z$q-zQyXMZU;)QV0pwKL&Hrc?Bnyym^_i~>$;g8?^6wiVN1I*wgo z9w6pJG~PPaT*^%b<|NT3v>>c-RwpA;E}IBAw=JtC_S)=*T#(#(R?&1Hab2X7to^az!E*zC-;JC6_Hw8v@Or|KDC{@st%mB+bByvw+ zUU@@K)QRld|A3^*&?a9f9y5)KVd(mVsi8Nl!jG&3N+!{SjcauB8PHUvFoV0AZXp36 zghbq4w49WH(xoQz>d|s_mJimV8U@2H&J1&$6jk=1RU*H{B96tKD4`lACON1G!VD{n zu~h7)o~vC8=|BzdN#o zQs{%QXhIKJf_8_K5bDEX8NnZ-i2vT2SVOYdmh!8)6 z!H&Q$0yx(tk`03dE-EM=l|G@?KteG~BNzqf?*)@wf^Rv|EXh!p_ZJ2(pH-nTGeebY zpe#P502)^Cv^q{?X&e`;2KroU@jWcS;*_;D@3|NukgSEiF>Tsrk!@qxuoW`VO09-p zx{A5c3akvfmFEwVbB#ykOprN-#!X?h$woVn za%;u581+0tkua+99~GSu=8(iI=5-Gam-siqH=shtTvGK`EY5Iob%;h}WjKr_ z)$0{xYJIJPvc{Fw$kGanYt`})ixrvyW|etBrGz!NR_7#sY4+mQx<>A0{+O$ClKbL~|GT%dFe z+_p&c0zgF9t`@Dp&=?Tn#;9akmz=I`fv`1?j?2yE{N`R9hEGfJ#ETi%FlyLk!PS!c zv@K7=(w#BF%b#=zM$)9C7H5vZiv6Y^D{D^{XvY;`h-V6E7+$A?PUyX&gm|LVac2vq zjwd9ttOhkD>K`V-%ckvh5~ z1R$+^u{)Kg)k)LNo3!B&=*dO3Z8H6VIaxsKQbWIignU>o31V_N7z|XV!eJWsob|b3 zgE(Q<^MNgCf{_8vuD8otbwzVz`7FPsP$CU90H|MyEo@OXH9i@a4~R)M!ViEo`KlQm z*UOOTwRD7>xS$_5LZm0*vQd03M&)q1P8~I7TJty2fG?Ml3?#(GWVN?QgolsAX~HYS z0c^$SqiU!GmYoa4i2WtJ%xW|ZtK*JF1T?&|XoJNLcbK564IB?c67qH}7xOCtv0t$U z!?aFAc%hqBx;h}g$e^fBxWfI4F=RMV21}RMG82R43|1W{)HPDTqf*$PDwdNtKmR$2 zUswvVj0#OLMjx<~{>!A2Orn(rn>vSrJ59mLaO z0UC;5v_q-Xw9F*>Q9IIM_GTEPs(7Yck5QC*-k{MTkVE%Q9O5FfaH(Ib2PHYC@E~J3 zfaC;6GK)2WotjX4tUku*fEI?_@%lbQ%n^b3p(ivi%dH@c-EpbNlrqFbT`n_tF%Z#KDXFKyb#g~Oc zTGz}|dBAn@*TmaShY$sQYp2;i3vO|qZBlICehwgVBir>7ZIKELpB&Ux*NaLKpTcyOSO?wG>WH~%wg^v{{qDE~VzGMwVU(Fw390@t*_!NJ%l z%BuMlJ&@d!U2-qsOQ5u4eo}OA0hu9U-h#LYhid9bg3A#*Sn)LNo^+9!GKFhFxdxNq zAYRGg0R^F5w=Y~4ybL;NM(c+76SHH@0*~|t^=c!bIEQ4ynb1savGYNsMVI9Z+DoV6 zf65n-SDZ$n_?=2o{W^zMs&`Du8WgzuLy@0=Pg*J4BCPYGL9)f3| zOq+og9g2aS25D>S0riM3_i0BvsF3s|*H;M^6qFlDe8)8jBIvN_YM$swjey2j^{607 z%Z5}sb%KFqv3QYuQrV5=x-tvY_Q($rCr8muY7}tG#)W1bRh8%mO2*$R_>BQ$1%gz3 z5h;ZXbUm!2aDu6fLGIB!1Hsiu+_ZYPS!2){@QRC?NWpruWOW@=S-LLGJ}N3;zSe^J z$0SztCtXa-3@ZBo0Q$AQ)+^p?co<3M*Va)p;{vWC^{nDZpfXeFtaS`0Gh-g6XdTTg zg0WrWplUSzP|QHz#6tn3FvRrJXjOuS#)21h05ZA3_4;u?4IX~c3V>jgfuMSXepUwd zx2iS>!5A1_X(u!d1f}{=r@}`#Olyg8g));VE1iiYNkuNQDp8%h;G6#q8gPSQsaY%I zJl2tNjT@o<~o#gX0R4fGjRdh9# z7;4^Oc&L^FVFw_9iWJ2Y)1$1GzZuoo5#EdOl4wXtsA`_yQ8`lRWtE5u_+rgu}%{*kW_xfCegYIGkxNQ;!y>aR>RSv^n9W-2rSkSwa;z0 zPL3$TOp2L^oqQh_TWqwEAZbNO*Hp?(&E6=@&Y&LMo_N*1+Ck49SN#PJEL?sV&9Z&asQumY6SDohuMkR@Xo)4OVsO+|}O? zF;3^SVAh9vdU0owzd>8 z`GX~uMI8hEpu_?t^8yyu{UOjvtEyaG=!J$pm@eb1zHq}6lm$nyT#vb|Z7oxX2tYGA zFzKW<)O00hkn&45^Sg_8Eu>0sa84ZwRo8K)h=gePV5PgXz)&haIICHej1{L9Q`;eo z!YvL|hvRa>4a?~|HA_Lg9+M9cNy{cnrX7_ffJa7eoi)e_M6T-&PfFE7O^O7TyEy~W z)cXg-38S4c+6fWdrD%o{SM@3f!hEi|?ILMj92E`B>hKKabJ2Kcdxk{?269+q;{Zkk z#^IYu>qa>R5?ZbF>+xYE;X^{Sr3~B@YoysXE1HcUmdf$4VTu<+&8{ukrsVY}*2QP7 zt}-CizTr12Fc4Y!NR+EgnPgoJKciy4HJuY0LT_ujapm3-9jC?UB#TY;uWK{f027Cu0@@RX|GUA_Ry zZAtP*lg&C~r9Cok_99b;0!^xVA{68wTw4j7ViaA%aLDi>7~YO_q$RHCa;u#(Q$;;g zAG{(=lfvaS%!&$(<}h1HmG(fo#Hbltw za2^gOz0rKw?>-w$MtnM%Pe+q^&w2dQ#vM^!0fxgK?%Qvl4QD8)H=ps%aO7vBIX#SF zZ#D+TMx*}Jm%~hkL++c;fkf7dDyF$~JRVNDcr+Ox>U=t>A(>@1nT==U3hJtE8fHcQ z`GCd62Zp@?_s)9^F&X5}@n|@lkA@@eWf6l}8*S7Z4`u-KtT&>#SF}yWtdt&j6zF zcruQnhU4LEG=xl`tR9+ZV{p}QJf8IsU^4dQRxC*6O@^~Mt??m9=%-`un6Z}ml%j(5 zPkSzYgpku=uSdPdV#h-a>9AuyVY=y@!Le1d-FQ47j%FkB)_B-&3K{miVuCX=!kAl^ ze@eVr?AjZnThgqbO<36kL!+J!^h-f=b8p)wNgx2m#zD zsJXZLe9Ujt`MfvjO_APl$HVzxj@`$kAoQ0^I_wcB!wIo~zhqsx5+jhyp$BtEX&q(~ z)?}hP;F@zUL$M=TZbZXG_Zs*aCYa-b1E!eI$F=yO4oDEqsU9bz(K6gdy}@LHStq#r zlmf5rM?O~B_4*|W=rd{d=xRl&AEJ7)jA_$$k~XCv0XRl zP`vGpMkBmTO}8}?WK0co;DbG{Rl{Q}18y+YbWX_o>uQ)GdIpU^Xxx4&^BAoYQd31I zFyaQoF+$4fcjYq5>Q0gLupC{m@bAb74%Op_@(H8pc&B{xN z3j7%ZVk=a>x)wCWZwwh$UXDjpj!7fuNT zKAUtFgtRg*I`od3fvYeA+BI2Z&2j@WQ*S~OWY`+bTm#j>(Y$U_%abbrvZJYZK+DN1AScfmQKq1|`H-c#l$n8c?O9GZSbfseCdYqPrFVI3-anxgcdVyq$rKOrXo* ziMoh?nw#{701eJetx=7<;Z!Y>ks%Ay_V5xGiV7ugv$+~8Ho&sSbcGNCKUfV)_8)_6 z1N;Cu@biW|WJ|n+uP{>0ubVLq=|uLm8iMfjvl$Bn$?IA(4WJBEj!}1gHbBD&1sNos z){{0WL=$p~;b`i(-5<+5%nGyXfw}We=Ku>P1>A>Zi=G#^Ny+gjLN*4-mvp5(Q(F~P>uHrE@FYXUk6XziGBqmqX zTrdAK4M06W!GdB`fNtp%;y~y&OQLom_gG&9YKi_D8V_Sg6A&w_+<+T_BuqaA`1?!u zEHAUB!%Y~R)C-@eQOJz6Qy|K)u%g2)?kSQ+6bR(PGqI=m{>fGG5D5&7DpC~^kSjrc z1|GNo6pcpVczW#^^r*vS<_>5Dbe>v%)L4;R1>vZ9QB4l3BNV<#s)K4!;WVvM0&tg? z0awu`_rM@vzT|pXZDmiCfmiVntac57a5X{E0tOeV^+Iw%SMF`;n9M{8Ni-q=wc3&_ ziW^38ymeZXiuw%)fw`HX0BH5|1b+=KWP&83UKZN<*Z`gd6EKf=V2v}t=-I$!oE4*X zApg{=t$2nK8JNflP%8vYIfN_8G}=h9iUrA{#9!yfNTAzN58-jmDL@F)iP;Tf7Qe?} z7=cQP#0iahM+sX-roMot^+4>F3c*RT+azha#BMPOk`aEUE*uSZGwYlcC<6s3647@2d1Dh%cbQ%O1ppq($b%>0y~& z?#0ab3D@ymqnA1$D8%w<H588BAro;#7X5ck=M2kuU01ycxh~v1?6q5~% zg!TtZ?63qVfqtlc!nj*qnF|7iI!G@K7}4~Uv?umJwQ6nwjaY+UQg^j^to12Zh-xsU z^%P(zmgKZ0`%OX;4~%g>)8)&o&ffirjR16LK9qzu0RR;plcHd6a8lw4os<-y6i$tY z%*gQt?O+3Dg)M*(IFsWxi3t-DKD1k^m3b@>;WefwP%8w^>kT*T464}sBTGLC4=|nhB*n^!iX&|;UlTIp{tZ*#qP(P8Mje$blG_yJvPaoFQLE*9tIqyOF zbUzvm#g144)BzXZp*4YOpk76Fh}2(pPpSf-mLw@qh*0*n41^}|dkGp0txQ55gWa(% zHQ)Hv#JHid)%eiJW%{i`9QffUUyCz$&yL9mTaS1qeZRi^dTV z3P>%}nRbC3pkU#v8jdyN1UI1Cv*QBwYd8n6!J*Q6YRX~usMRAC)Lw4)NE(`}Dv#b*F{%m$l+z|rMN6gStR#ZE_y z&iES3#Y&fVKFSEV}I0SI0)!u zi=jzMY9e=23K6<%(PfNMYzB;0vI09`G3h!S!Z3($b0?0R0RJ-2kGS0&I81u-}xQ^4u$7>+91)n z+XIXXZv&bzngb{^4w2f;iUYrZXfzX%&&jPgwAr=s@-*A*4oI(Ch?8jK<|u*XuGlsm zQdW)6G^~Th;t>c#nOPk%WV2?qGwilcfjq2{JRsOmp>-ZaV3E>D^*xoXg6qG)x?c&%W8#R!0GHpwBN$w*IyQK)%=w4kwFM8Q-ia3BL?g0Y8 zk#l6xa3=>t&1ssiQoIBC$Al;tlM@BfDQhHQb*N3di`>Cylg`GZdK60nWH&G2n)ty+ zm>lC*I|<>a;jHelCd@9%w=jp9*py~9;aS%2HY2IAvLBf1U~%{|761gn5rs^~03Y!W zo5hoCg;kma+$fXiOAr~@ztoQ{-I2njsjj-MILZxR#RiaI)V2qKXl5p-QmNIw{A@O* zSOtsQcsPs=eP%Wo>N>pL5G9;Nj!up4x{!8ctjJqe$_m^=0diqmenv2WF4#ltV1#y^ z^)ulH#V6oaIHbd}yWUG^qizh8s@B#^qhDrsH-V^f#%J7(ASPA8pE0!*-MZA z%=>jh&ISN0fspDzC6JrDv@LK-TQwS@J|{@sYeZ*^)JY{%lzTGYB==Wm3k<~p-RJOr z!N!dO%3xb@Kr@aSjaY&sB4#IYmoyIzu~ZyHdpdyErWzq!;N~*mrag>LNn$(NzF7%M z9)c4=%M7ihjVK^cJ{3IHEO56Yf+v`LF-$eM;u#PsFaQ!!k=IFpkzw@!tP;>W^7GiM z`q!{?T(3$Nl?^e&p;XU$$j~pc2q3FR0jZq<81l3lcgXHfzK3twl7v>Vpm}pb%Qkuu zfFkbSnq#t^KidbL{CmES1=QSgL)ykQhovplHisGa#oDI)lx!GBL#|9sn&lVFnKqgsNW_geXaSL!sh>`kpmS-7&BOS%YXk`XFN zAQnV|{sFM0U4_s%4@DSRfmPsB%La%E4+o&`Ln8jxr~x8taL+hads)+|3Td zBpc(pgl(@|C81~Ul!gYirt-{yy$K+!;NLAI zN&;MkC4u$L)O?(Vka6qOH4WG)c93Q!O-RuP>B6LB|BZOjlMPzZbTag<>DrG)P^048^Mbv|SW?!2$OFc*&Lp+qwKe=A62%O61!L%G- zv7|ss90P|Z4NMyBkQup-EWjAk;nJ9RB#|tDvx;t9jA=g#1f{t~liInE6%$dGTgQ`I zqf#!wy#k10#jM7K6lx6+j`iEZa@?erl9#!N51IyR-eU~lxsU=tyXOtRnZLHc@LRhs z*c1BNl@RzZqX2gN!Ph5C3fx3{peDaqwm=&vJWBVX1H3i;%MSl*~VcjPQTbGaFT;i$?ou zAo8RIERmM9d8-1E-OF`%eAz`hA(Ag|meAHxQ z1zTw@ClIhNmgYE7k%H&~g%L^k1o#EAmaMoL%Oy(46)Wmm-$5?X3DnB{#$}o_-J&O) z6cP+Js}Qd*mP#*(;UKS3U_OiGu`x*0=46vd+Mv@{9|@UJA?rkB_`tF!PdNfM$`V-( z4TH?12O6rn0RH6s{%JBmMkxBJ7bTPRP5u8iH;o?7v9P}cq{exT0{eHg5bZ;duOUgv z$ID3VC_8qOt^}TAFR>;vdr$1m8gW}P% zE&*$Bmke4`EP&1EvBV~9C}wECkS1r@)wZGl2;HxR5|V;?%i^fg5(|M8ll<%8O|*u5 z)VILPuz<$osfGa*=nWmSnk^wI*EOv-qfuF424YN%MoGLc0_bw{<|>y+mIme}|m zSZMW$#oDF-84kzeydassU4_CR=b$~`EkUQgLyJ~e)XW5BBXEq0r`lfz(&JIB2b1k@ z>gQ!QU0HEHi+0RcBZ83b5aGgjSom=zU?`8<{fensC{jmXN`17Md4_5ykKIK5uB?Mc=G^YdLt=G{3&7u`N4eGU&FF*ems zsmB0J+s5WhBM?cZQZ(zpE(MTzJ(^*Lh0U!dhXJx|HhC-vCKYXNEF!Hz<(_qUCjN`f z7?RYYb__QJA($aoz|?9;AdA^ez*Ig#(zW&EG+kGmFbxyfse=1C!1|x2H_4hizxY*I z8g&5{G&|rhI7R2r_^)TY#6r9AVPy^r|>|RE1x4{N_Q$3kXLDW zauZ1#yTl^3Vm!&3Ns30oRoMERa+fp%QL^>7%rk;1lvc1n*kM~Vd%S9rz&B--!trQY z#Q;5}6<+M1l&;LrI`LHM;tpm|D<2qmxJ6A)!jtdYpat4;FEt8~WiK6;Q#BgDfFCwt zQ5)s&x~6jv%9C_*pbY(V)J|a5IuX@hxiCWM>v9A9sl;sxKtT>VV|znh4w6p5CKzlNgmVyjO@}eY`2EGUZkRMo1p^UueSb{J7=vufELlRKDo#Mp+>Sw~Tr>q?#!0BB%F&RK zUIQPXfn1dmg+5urqc~7o7Jdt9f-JF3-MXkIFYH6uVXwLjAf~Fdj|1-rbeZKz0+e&W z5i6wFCUU%5Y7gfjWpE&Y%%Wb}C;}jD8TcURTcJWmHg4>{p#d6#bt%int@sbIYA2yS z$DVK~oB+;S@AFD$QZ-^4?Cxb`L!6Hc$Dl;fGBiqo&5(j{e z09N9bB1n=xo56NhH5H8|xAr~Nr)xBC6N-p>@DE7wGm2TtCre=>pV{*nWS9mKX$lka zW#tGo$7k?rl3yo{XBW7L?i_?&X$o=(ANomfCP;^$*5uaVSPMD%TUCs^*(@8jtE~$) z1OeuP*Wm))ziowTm=~`?A$3j#NAR2j`lp>x08|^-(%B{YN^?*GgV8`~Fy+22+9oy4 z!ihBv5T6ZE(pT5|AUEkEv;x~dCjr_LBmf5CRy~eXY{(9!wxgV6ic{$#fC{X>dOda& zL0{t*O%+*7C*PboOlB_!Lv6Sa z8(}tp*i){Lxe9gxqr;crSLiI(OtUxSmvNhOfXPCkQA&4fWpqjgq8a*I^LR=FM?);w zxS72~ha|9b0jBfCY`IxWD=ytHK&vjJ=)_Y!EhS$1nWm@bWX6#=uN4Bo3fTy4W2(9v zu4E_I2cAftE7yY1p$wC!kl^P6W`#cJ0G7cbxXo{tHn*3AUEp)|&Ruk35vA8!HhZnJ zW>40WKwNHR8pYToheWYqB57_*lMq-+kv=PEtX)AnSfx3&vIKmHa?l>UMv6qv)cvWU zz)|=CXaIngm&OZl_nd7tp?gD{m$Yk6r|(Gmt~G;McJY>V&}!D)^&% zSr3&BCeC$+gWX_~J;AW7p#(*8QijC^dMel!lwB@2Qdu=>i`sz?NC$q%Q~=3(Bj3=c z$3V47{%#vUq*`oa(j^9$A~3ZX8(>*QDT$IrLNocipc8zew?<$={%r9VtYirQvP6mc zGGo?KJ>`p+)TQ7A6w=)SNxH*iN~0b1ASK~f{T07!zz6FB zyYaknBZ-~|!GVMXs-7B|wg-t4NG4x(WRg&ZEs&Pu-t9V%B2oOZLZs~^z@z{jbtovJ zChA(0M&vODE?)Z-eqNAe#becTCj~%Ha)sT{DdaW!#rZw1r1oL3TH|Zmzl|Koh+j@6%gnEv;MA zAukzx?U)3hhg{SLYinW-+Sn$2BBcD;eMuj7P`Rr-L>hP7ZPa_}!c#x}jr{lGQ#@>Z zVPnI8==2juw~miC@9Zuv^2^5-M~hqe>C^Y_@Y%zDwzGS*@Dc8pkMG^xxqNNwbob`s zQGUOEe0ulZ_Tu1u{QS)Eoul2w<;Bsh%P(x6zO}i1{5p5}aCP1F?Z>B^i%0nNQ@h)n zr#0DkEe>zg6u)nAvN&@7XHFMy?H+jDtNW*m-J?zCd1Z0*)`HRf_?3G{yU3-fUp?Br zyLs#K%Xb%dH}B-SoyG0}5;)MuPd7JjZu-;nr;DSTo2utyZ*AV(x%}zPlY2J~c5k+& z9c-$EYp2_rM|Y9fIbJ?KTpZMNAKyGZTpZoMFPV~(2!S3tbM8~Jw_2Z`&hr3NLpSZg?xUcG; zxp%scbeErB+&OkpFTb&QYxBr4p1X6`IiBLj>x+ZUJ1k;%adi2W&C}D}yT_;ZHNo|r z-Gkke6V!RfTRwJrytrE?xPGiuO8fD9sF8KYvd`Q**j`{IuYPg!#^UJs$klz4ukWLM zmHDaT)2-u!{mY-Y;|PBH^1<=zn@sM{A3Huc*0@>a=E36h-W^}x`t6Ii7W+GQ?kW9-aQtud+&~R`p!*0&$x@j&C}!K_{yu>Oq|(1d3>~u zD&x1v^~U19!+!$9d-HSK_fPIR&x?z@XxpDYap!cgc_56ua*DogEpEkgUpc<9*glS{ zFYeviy?pI-!5Y-ywXN+PTu{lC^E-Ao?;b4<9sT8dw`w&XyLTg(erogX&gQ8~iRkF; z?!xGP_V|#gZep9x16|My$49sBoi5_P*B7UY?R$%u?74%>FE0*WU)(xAbuBM0j@u>s zOCg@wJlNj7cX-*cHxG|*o?_IS%fT-nZ(T-^%O6uWD)i;k-ODd7j`kzuCtk-j_-=9M zu6B9x-kY0;H;(U}Zi{u-7bo`?mp`@n#^sOS!)Kzgmp4zFPriIYTzb#jVXd zJO1{uW0Z6G(=5>6uD!+ei`~6a*C&rpk8j?LsJ56$WB_`&mw)8ozV14sC--NPG;8*cv)u6X5L-F*wR?9A<#NwwH;)b%r~6*=$@>d`b^)I}eoJ(^eJ}9vDV7!6c-5Y2SI!LQ zr!GGA=x@FBXs4k|k9PVI!8gwH_p86@+#}!f_S>&r-FWh`N51#9FZ4eAzK6d5waYuR zZ+hrwU;EcruYU6r4_zI6@%l&J_p?_|-}Tzy8$Wh!_=}ItKL3^1e*Rzo!FO+bb))y_ zA9?=##kWU~Kf3Yfzw$F5{N+FMTi?F+@Y@gl#5ce9>YqD*=?5?VFFySI?1#ViAD(~h zp%0(?*x+mb?(*+=d!zs8e|Y=axsU(T_doiLKlD#tyZX1z{n5{#`{keC*tqn$uYB(E zqn94}`1jws@z96AZR0042A9sg`~1T{dE=RjA9?$;C%ui!&u{$L<-hwEU%dQR{^ZsF zJ%L*u@_@_wvT2$8J6Rbm={(zvF-Y>?fZ1!4EyT@wIQe z^w?kh)Bo4iKl_>g?c&dUCJ^KZZY*B<`j#m_(U@L&CB?|$<9##ZnCxraV~ z?)P2(osYfp_PKZe>ZQBi^2a{=FQ5A}8yEi5^9TRn-&}fZ<1c^ZXMgvlFaCG$|MJGe zmmd4#=O6jn!+-oe&wu#mE^NH}H+|b5J@?x6>yKW1>|gx3-lZqs_xX)4zw*T&+<5Bh z$FG0p7cYLpldpa4;m6+Bd-&yp!H-_N`JM0TU;Wf0pa0s{)&9n<5B>F9|K^)EUi_t> zyYcw7cYVW;{6D{I{NWd#JpZTv?C9~oed*~hzqaw_7eD^lhkpOy(xcz~eZ#MP_&Xo_ z>c9NJ*FN$)|D%mR_J?2l((~s&@|o{{cJW*O*^Qt0rK=CU|6LdV;`e>q)nEL(Z@#_z zvtRj4@5Q&jaqs&3H?D8|se`|9>Cu1q_PL+F@P%`uZ}{>vuRZjwm(KsGkKQ;J%0zx2c}eEFl-p1Sqe&`>3`Fp?n%P&3j*N?udckPkSU)%V~FI~MjxccKC{v+o<{o%iTV|eN0 zpZ(Can^*ehW_Mrz!{74ZAH1;f<&De#`;TvY{)OLm{Re*Z?)fLr_fCHPvG4!s!^`h~ za(4a~Zhf`?OOIUmZ7=UX^U?1<|H?0pE?s{9q3_$hd~)T3AG-FHXaD|>UHsnP{g-<$ z^e>+O^1teDJofHyzVPij+b|+_qozBX44ukw+*~(`1D}P7xVnme|q3mt3mjU zJDJT(`Na{ZV}F^{ebb(>#6^Q1U)8Y}zaBl^A39RcUzja7F1I-qV)AsvOUYCh4ZLI8 zUwv6GXlWDlonSim()rT^rJ6?&e#vSwxZjRTmho~nKb+Q8vq?td2Ird0oon!PDU=`c z$IE9m#$@CLv(ju+e>60X`kP(y&(=my3Ka>tUqw!)P2Bu4JR^uBm(8lauHO}9-sv=B zPDkFxqV$ew4ADn^uN&}1_&n2j<80!kSjJyUk@GBW9J7y~*3z?RjXTspJ&C<8SNcSL zfQB{Dr2MCOM*EHLqQ5u|qB(BWG~S8rwbf)?OLCNXtU8V`v)qM2@4)|8&)7KJqr)vYQjX}=1&jfsm(G`@jSmUo|layc6sM= zUcZ=@FZaqD%0|6r=lsK(N)e?-K-3Zk?`$%{rO0QcS;fg0tlVWxvRb?(PM)i#%;Oyj zjrRSbq6y9Am$(zxR46!b30;o?cYf}rZ`SEy?kK ze37EKZtH8Ea%(IQ$(Pn~`IAX$Jx(wzML9+}#H<{mMh|FqT1U5W(PqwCYZuG7II9BF)JVj6Smki6#4qJ~6H+7a* zEcJr)xjCjlYg}F@5>#4qn-Tzpk_DE|>6&XNf(y_5@y_P3g-Q%t3idL;Ww}l?u7L7e z#_fPp>&_}txX9lMMgc5-UcHU08;rYS*?25J8T|A7o3~6-_!=zZ?r5g(js8&*E_T#v z$!VfdP@#D)Qde+xuDF66hIXQHxoy9&WSLx;C_WL(skO}E_yCJPuL4<(W_aYt0u$Ed zDdR4Wc~9w*OA<_7hAQjD^sIfQt-{I@4*?>0Mq}g{W41o&EdI`Q%rj54BWnJ1UgOB~ z7Tm=VmY^_={wG7PZ#s2`c>{LuoOH=AN}F`TCnPJrux=)g6#)~KJ55B+0}0jKY_63B z#AExR2bDJll^$9^Re(g9^LCK}YJ*uhOydQWqoz$A1^bZ#BSxfT#RiK}G@t58WgULe z{VGOKb1mJe{8Iw=qyAbo*x}j=*NNkd6vYBbjx_BoQ#z>Mh!ABdKdqr_EsWr3F)af) z^a?AH-w}&!u!0QV`HF&$qk|Sfbs2t}C3I@kkIRjKK!q)w!K!5iSLnEY(>ZdkYwJLA z?^->Fx-R`7wq1Hh#GiIK*YS&q7Wb-&kw@&F# z>}RFq;ViDRymOScyP+yGchWa^shb*aN=|FiCrLHnEVC3#Xy%_L;EKyP#?W$7TGJ}E zOHvW9`Nrx|Oy%&Z7B`g+n*&S={aC6?nl>U;%FSpEyejuJg-)_^mY_e?L0QM|(rrR< z5@y!qlw^WWp3_YyeVHPn8 zjV1+W)0m*Zd$J6f%o0vHN+49i!3(tR878Xa#d14y>nP2kn+sP(g9j;ZVOaF?ScdRV9(HJk5cCG6* zKoG9T*t&sBD6WrH8%Wf(v3(_nsvY<_s%_O}%+%EjE`6NaS-Nsf2BoFwJa?4aSJl&8 zw)|J)d&fFNO*-n%aGh(k95xH;F-LUKc~hJ)|K6x~1Q46k7dj?Sbyki}uV2r;kiKT#8S&->{KJsPnFgm{w^AkNC>jrfs$;uwaWtc;DoV z-!8|j3rlOPHdjWy=ppXfQUA*Rfk^C{ubL;<5XDM+#R}tZT;pZ4#?zwML zMaGKU{-mgljObZahqR)ps1tU|A9c{~C>ya-t=&dGGDp>zTQxu8ka3CLV|Lj^iwKN+HGrIsOPQAE413Pyt( zgO&DJlE^Vlc~C)9K$`o|m#Z(LS&&}2Xk2a{2Ppq3ZjT<-acd3m6aSThl&@MG9rGx-B6r<4Vi^8J89lMj(OX z&vFk}B`D%2O)>KmE@Nu=|&uKvi{Y9;uro>s)8hORm1fq|}55M`BK87$H9U_)(^ zZK{-7sZJiZCI@^s_0xaW>^oa*mEAOp(aKtM=D4u z27#=(EsJPN4Inkgu0`PT-o9dsB-KPmqgxJ9;G@l4DdCe?O~inms;&A|(SxlX9nZ@f zpf_gbCvAYZL|nf`Oa}vvJ9!x-C6aahoN9JqJ`g-YbMVW8sX9C)RFCQd&)%YsLTZgYmhPN~4ncH!tgg^(=ap)o|LL|!$p&LfFvrUi2oSU;X zFo^C7N?WBC+(2f5?e8MiCk)w0K$qeMA0LC$~0x>e;U@K)k;&u!um>4=j^emV7>O zx2&46jU&2l<4}@^3x+vP#oUqK?Dc);mt*IVqvVeR`u=q(OPrONsw{0%4 zKn+z}ny>VZS=)(m#A?IwlNT*BMVqtOT6~m2B5a2#&bOlc5j}x{3kE_OT2`O1YN{$7 zS|hJzn?QQ=lgefDaNkkka{yJvHIszD`FpEA+txso8nd(V_{y?h*)n9?Q7AoCaCMr~ zXDT4vZV?#s4X_ejEn7AyFGF{dtATl-dtCWCO^$Jmu`Grhz3f6DM`>NvppmNof$gK) zWnHT}8`Es&YichXT>2LRy7U@_!4oV^4p4R+m3vA&Sps%dc z8I^0*&}3 z0#ttxFwD)JxxBsGr7qBmdsGXj_I6O0SDoQ%%3KP-La}fVnvY|y{Lxo zg=Eo)sz`+m-0je-H$tu8xfa8>Ua)GId+)sSzL>i9PrLp^A&62{uM1TQFBMnoeGAv< zSj)tm_>4BV$6MOZ&Ocr2!SGEyhB-%9%cdustzVUxvuI0b93y_DDE0|bGJ*W@`)e4tlSC_9j)w-xc;(alv&Ga+Yzse z0Xy#I%+;_0cr!{MuY=CS1K+6CRj;khJ}>Px2dNp6#qldlq7gt(71o<2)G1?Syu5+xI0Bcmhby^cASo&<{%fir_imQ6Js)l7jRM4ZRO-j2_ zQuFeTwnmJq)fFORb(Yt_9k@i9%L=CuX<33ZW+v;HNqxNp?ly4o9u^mmD5tAbx%AYP zOJNg#sUq5L(=tujD&5&LtLi9lR=lix5-G(%;ERe~We&nb z<;w+&!F*c49S_I?PI(qBRR-P@S0odYx zOHn%JzT`j%tFmUvXRTOUWR@&yS^`MsUl!H~6wgg+;JWBG5sEnIdR0n9ZD)89IO1T{EaO_GE$2b-)mjX|N_<5_ z%P}iN5|Q9|tdMC{7XK|S|848H-ADj{_F%XdH69?6JDuu5%w> zDk$KN$Q(q1@;Ps&DN3|FBND;uA6Rf@D9ovX?`_`FL8}3)BaAZR_!)3moqzy9i=Cp4 zEIa|RUN@9)qM}R_;n0A>#cQ}l4YdYr{Ze}-g?nnN+XyWYG6eT1b@*R1<8o_O^;htq z5^wJ+g80!a9Yp#ueiN_3HPv?6fmOzzmh85wv05MZ#G{w7%Y6|rP|hzdr8+Wv?4ZjX z;b?C?r&MQj)Y7JlWpwVSo*C0P@Urh)32%9X3K>;&RmNRRHeW2)StmrbrOym$+1Ao) z7qS_w;U`9p*TgVwX-hQmn^_j%PyAPW8$tQGZD61|^;1N%j(t7QOizd)y3()Z?`rf+ zyYy2^jd2~_3F?Tq#XRi0>1PUP(6=gvj0sdRB^WT{Rz<=OR{Pe#)|A(HqcInaY`&Tdh&EI?F z?_7P~51d2vKg8ejzw!sqo&Wmi+`Ino@AD~nNB+^T z{mQ5L*1zvNcmCi1UcUbS9zXZ+zxydaxy0Z1o;&{wPo8_^>p%5VAIUY3eEsWhe1NOI zXK?QPf4Gth9{KwB{K{`V_YMF4-~Qu2^nc9ws7mFx@O{>xW? z=HGt!4_y1qL)TvTCvQD@>#v`?_CNl;zxVe}-u3@p{heI!*Z#$?=KKHY@Ba^1-~A8& z+{y3ynTyw4$3Gf9cI|I|^Y{I$Kl#O9y82^3`lCPkC;sfa^80V!`Gu=~|KEJ!pa1{0 zeRo__*ZY4G4toU@#8JH}A~+b*I#Ge5f{LOdwps^-08t<`30ha1fFf8yR760QqNs?9 zvu-W8s5h>*id8EO?(qc|;{HA7Ibqb+e*XILBDuNup7We%yq|GS;9T!U$${+lsl5$+ z;6GG7Z@XFgM8J>PqPO0(zdY0m`?V=B^cfEq+WsuSW6``9-jBxZ@0e#K!24&182NlV z*zC&5mtr4k_)LwV&-M|2&2z0FK6HHd$u@XjlKt8n#B%Q7XN7cT>E3$N@juKHV82tt zMJS(Do}~}HUUA~JH>LZ`|Bg46d&8J|Z_3A%#}BdJ{AG3C_0HccFaB15=PIWm5bB<;?76gLF&x{DSDL>ky~kKdAJk zbX5vNq&sJ!C62FJ&(6;W+(UVsvTq@u_q*1kJ^Q9UL%JO$3apa0;IBuAs z4dU#_PUld6l`F((&t-=1(GE%Z4EG0)St2g2_-b$JC$HTtk#6_jnV;?Ue1*8~>nlVY zA3ZI`V^!zhaGcA(*CGCe7Yz{)Q`_A|{l;fm`A~mYHkq~0%Be#?jbFoX@cfPe;)WbI z$MN!}iO>!`u3MlT!|IKYfB%(Y)K_f573A+xTO(|*%@*KzfrALj-DMZUQ_wpR;xKn2 z)4$(*G4r=$?=U`@Pke^<{Ilp5>c3(98`S58puZ3&^AA2l+>W)YL%k2&dJpmCS7nL* zmiLn>@~Mh?i*g>Ze~x%b4t$Avewg$WkCR57NBa$Q{eZ{WTb`kwYR+1t9p3h3aqqRp z9OF~N1}6830~RPx!V@!;?|wSN`F9BoINpjgcahJUcjl;AiTvNWX^i@>c49bmKJXOfK6~mp>iJxM7GI|RW`+K0?05s| ze%1a3;$c_2`zW^~{{+u-7BT!R+w&L3`2k;4qkiucnc_Hm<89DietT(*aqL|()4Ow# z5nlK0ZH0Omzm3U#)1RRHE+bf6Nw{EvIDhiR0~|Li_%hNDa(jdNjU8|g@l{+WM*Cab zXh1*w_LdpqrPB{psGn*_8`PVH!~o;SjlN>U|00MRG@d^@a|`3&wP-Vx`_1iVD0hAd zi+}A$St4J9%Y+y|I}3DZ7cV8l`JNkQD97~*GxXzCo>ws*W|kPC9{joY7}rw1eT4iO z-(Y#J3iuSjPf*bxIR3fe4EMgq?@+G92vgLX$M&aqKX^6s=X-W$D37i{i1FjaQiAc% z%gP4N2Y+FT{?vB{izm|_UPgX$XI@2rm^9Xa;p8B*)1^^toc5tci1${6pq_4j_6YUv z@T4C3USh=Z=#$7&l|o z!_rN+P~T5{m|ov}aSQPocA43$!Rk8V_q(xX=oiTcSX@bMsK#+lO?`v@ll$^lv_s%J zmX9B}viNE~unP5Z8U8ht=J|F}EbdwS%JhEF;yKD+=VgUBbQ@)XxH^<+h5Q};U4*#n zK8w{8tHxR)eluN!*e>*qCF=3`n0FXg*6w5ZW>Us=pK<~LQWYWe=}pAp?+U0Se&yx z$l`YJ_ID^($*H@j?-jpWq5R4B?%?$h89_Yzr3qQx6ZZn?JpY{4VJU5Ipx*CPzeM|3 zUlSnTXM2bc&qpP9@p|e2mNx>cpQHaqiQi)UdpNZU?J;08lQX|P!^6TEV)V<6!(XG{ zMs=t`oVQvoM*CK6Ge!E}*Bc;TvA=3D?)CfG9OH|o;{|Nr){@1a>^K%bl!I7)-St9@ z`VRQ>EymNwQ(2uEwD=|Bd1n%YaM+D<$SWl1obOe@)YTXJ~Kf7IQ`lZ`PsJMA@ben+C{`gNs%GS zs|aTGdj9nW)c2*S_wo4j+u!k6{L2fJBepBkljeyr#<#pq%+FLO%@K#c*1f^Ju!l3j zcyi8?)#*b$pW*e~pN$c>QE9KyPGW;dPwOF2=>{=2d8igW%r~js@W{7D#{Dws&~n%8AW8 zMglAKZ}5J*aSVq~Up+-S>5#{1-G1ceBjh)7oiXy^C15yQ)QaK4wS?Jq zN0u?#bzetT&tAS#jr=Z;F+}{BU3i0W!0>?;+I_8=81;JaeKp2WS4T1Gr#{dI?eFHn z@HxZk1={7_i8{pl%}iF0e%q^mB|4&V$)};d4tHj6;`Jv3PW_ zPJrVEZ)Sdb!1g)D$)5?!ryTMzeazAjti}?OFzyj%K zyl=qs15h8+`OCFG3{hVT&OXKSL>a?bZtWA~d+1>%-|1hO9vnNc`NfD|Ezr-u3Nb=l zI=+5^bwb~-YS1sA=2xR0ekfo#t}v)U++~a~L%kY|u|m5o4ywiLU*&y3`yL);jdafp zw?_HK{$Pwa@_QvheQe&x;$|Jk^m4O{CGtNvgw3b8fh^wn9JEGx6TyD8?x>NOpWIu@+;fI z>Y>`Y2JF8M=Am@nCh&cQ{7G)II8N6w=)C)9p4q4Rf<9geYOr1B5pNM^{Bc%CtS$Tt z>76I{QGVy2_3<^;6mi{A&Eh1dcz}6Ex180*FIurUVV_`t@yiP4Q*@pb4)Z!XuT8VG zM1OX$H$i!dwz0SrQh5n+VU}os`7g1drA{55xHkzd7sP;kd_MvHUUd3|kLyhxHFS-~PG}oA1BO_#Nq-xh_VzdVc$Y z;Un1^<*T9(0CCk$vK43iGGm_0yyzF6z6!_VG zgw=K0FB-7l-hIsfeZ7RJx7)*wv0Vor=Jyw8u{zx@`8D!$QOoMv8#yd~@;jMbFNsZ% z@3Yqo(GO*}ncnX`V)N6R29MDni)zGpUpoFZ^0xuv9bK|6J1y4){jy;$<2UtiJ^IasqE( ze1rNfa3P4h?dyj#5PS?wNKefhucJk5_7EdO>$GAOeJELQ=_dU`%xBnW-v$+Qw$6=EN>ZdS> z;k@e;A=?Tv7dJy%ZH;J_p$JQ^M zT?{b(Zh<{4@@*(rJkTX!;ZypHlN zbg0I-@yBI0KcDB!>e}6f`h4W~8sqouJ+JVX1oUaVK7R57>f!Ag7N^Hsu{z~lpZ93T z*pAOouKv!Ck)I1US>E*N{|NaX(3RCCNw?TM{r(wNM@2k4kNljgw?hBCcls9cxw_LY z=>H?m8DqQ}YxflKk=jRy_^KVp>ZmMON2L6@?`7?3N4-aXSF~b&7NIc4xDhP7hkWik z!{(E3hqHCnvcV0=U(FA7C|_>$O^jcqKDSXGi@pT$Gp7~Hn{^Qk$Kw)jVY~^P!19o> z{5JZTcqLnZxIKi`_4{1w(augk-bMbW!a4+9A54A2;+#e24>AN~U(Y|! z;`J$5Kc@B8;>|qzf7oxVF1Z4EjLsXUo3eaqJCWr#K8WEV+LhHg53gGy|I$BAQ4inw zvAUwb{SoS!7c)H$vt{cBYkSmUJ@;2Bn@<~$e~af`hg+f@=A38i2)$1jU|n=_Dy!2E zFJW<|>$5))hq`{O-(J=Kp_1SjO^^(38b!@rZiV|B4@2 zJ{K0V{LwJO0ORJ&U)j8F*;&T#ZYhhGf>Z3e!%6l%!<8VGZ(K`RULO6Jt>^s$>q#{4 zM%Yv$o|mq)LcTi zzKnI&gPG=-4^P1QJB`<;F1|s3UL4HwOof=Ohj{uj{Ve;|9P66&^Y;MFd zC*sF1?H20SHk{S-y{g#*6*g7C4S^xCs4y{p1%oe)!L9o!GaCeP6a|fDrw* zW@|m-eS0K}vxhy+F&+$>WQO`ZE@kUeUmkykdKzuW=5yJdEl{8P6WILpkHI%k|D9{u z_b^E{Y`t^&P*w+=Eob@G%$&`m>W8xLBW8=)_X`cchX8(hhOoF~6wKD2wqJdNIFp_{ zhjv(glGV9mzk7vpeb-Hg_Ou(q)^X1ttwX(TiDBz9=AO*I=DlHcZShk9<^l0Htp2$C zHRJ2gNvwYRD(M~S@jk2*(sh`8hzm6SH;j3X_?RuYgmsl^PjesGV>8_wen;Cq645<2 z#lU|6?9`@+rJF~>wt}!LBkp7eJL1L1!X9lZE+C{8G@77N#7k7L8DKOwEH)B0-jgcq z>1Ei;O|F2w`jjfpr>PIz!2Ou@SGZx{NhR?r*=(spa~Ifi61KjS(P3fixk?pfWs)?E z^OMe&M$6-*3K$vN%Og~CLKRZjH4+AC9GnZFJBa#AVFSY`x>cYg3Yt?AXl&eP3V7o+xR0+Fp!arBchOPcuC!O4ue* z!bZg{_P~6os1dLiAGKW|Z0iSGoKm6bhK|uv*df&hPFh$ae6~y}SI}KF|E^uiZ6NGL zD1}WtBdJ(yfHrVhkAja?{~6q(`j$s>U&<7cc$o@}tCU5?a=MyKk4Z7k^?qX7XF4X}SA zRg?eRNEvQANcSrYmI7L&3i{76U&>?R;sO0lgEM$g(6^;jKLNcEYKZYti2_8T%Ft_- z;^Ob-X=ak5)h|>P3RsmZ!T}F*1(k|k3JV3i`GnIAIO)!ha0i$VphNh|=)RRIxZEfc zz1qlQkU}1*2#tX~M=9|Ms?dlCZh{2lq#ZiKrJ&HcOpD<&;lF?IBO5)I4nmPSRw9Xy zg8^V8O285*B{UBDQj_|`t6-y9dZ&e^xd6&@P!w#lN&(9{%x%U3)%Zx*0~aJ{a(ViP zLP1fXuv4uBNTbYBJm-<4Ix7w z0YH{3Xwwm($|%_JlmT9f*Ys}E#1BPaARrcY*QDxfbbv;0M<;EhNn=p6!8ruC9_>_H zut%p=SF zbQagBKFZ06fC-JJ1rB`_k|^L&qN1by1J_jD%&s&%(9lnHI9?hFkdZ22A7u>6)R>LR z(Z{vXG=Mb&kcI;4)6!Ta!>-;}sHOvABjkWW25Eh)(Q5~NU4Q8#?J!3KKc>4g2ZgGl zVDDhc6^(5$QVgIS;?CH373?4lE&lHCV6d4Wa-&YEt^my+MKY=oOe`&P0QH(?QPCk; zcKuuI{U2c}fU31QE7Pb#t@vd$ZuQ$Tgkk{sd%B_q2$VvkYUCaUqo4o*^dx~5J9eSb zFPJ)Mp!t{H0TjYb*av2)Q9g!iQ(EBySXiei^zS*AiqI5YxCs!2rK(WHT+~rxQsNlr zL9ib$pb4S-&jdlW0~r(2EEo;}MjMWhqJ>);gCXuG11)XTUbx$Th76S_D2? z$ugRX7Pan2p@*5~KlDh$T}$M%GMM^UGov61%b66xEgM6KMb%Swo15$7HB~}O`vm-t z5;%m$w~vC^e^d?F{3B3NVOWXPa$tSo)v`jPmp&>{Xy1RD=zl~NOv^3GayCh#+WRM| z0IH>r$(AYTSz5Jt1vcIkaupzQbo7 zi9;tNj&$zQ)v+7tMx1QBI}?{4J$v+d>Xpv|BGA}=y{ z2uFr`cQ*4OzF+#0VZ?O!uo1*RV5En5)JU_@LrLpEb3(?96|{EsdeGIm>HEo9GDAip-ec zXiIt%V*Z)A{g@bY(tiTn9uO;U?a?ESeA~x^j8G5*WoxT3E|bmuRBo0eo^&4Y4RIv1 zWkXz|K~cfB;%EcQ!U0u21nv!f{4e(*@w!3JaJ zX-EngK8(08jS&C1jC^U8nl_Nkwd}P#rPF+}+?@Jkl92&kYpKlHE2U-4kEVx$qBBx z{)uK@^BoO`PH07j#dwU%i|Rl+k04~2kVFzgVsD=NSz>;H(ZZbeg+0QC4_~*QNDAhq zlA@vHYmX^wiDD!Xk!fV<+70RREH}0*o;rd{^tE>F;Wn)Q4-2oNv~R5VL8mw-|OK&crs=oY=>Zjonj;SCpOq;Ev`cVb|+Tgo>*$ab<=V;Q`I^sq@bBbI><2|k@kAn~;&9difyJ9O&|GH*1`HzYkLr;$yC zWVnEgcWci9{#@6R@ua7%Y!yivKIOAyvF{E}>_l7zo}LnsmuuOVI}J&)!B+$A=PJh7 zI*{=b$?);6f^p-0`rDFLyQa)1WcPw8%OrIE7f}th_?nPDeddjG>oA;zcLbrj zky#!J`5>{uTF+r*>L9P7$$N-JTZ_?-#8TtmX75Zd#WXhsfmK6wU-%tuw;7+zLn|qd zVLjPXooF8OQc4uEIMooCLMT+u-J@jPUA)}%*XaF|t8~dL%J-CS*QpE3HO!Tm=GrH^x#J=lHt@#^eHYty+$nS1D`HDwR>mOe^amLe1g_%-}NZ6<$! z-=SR<%J(HiOUPGsCq7aHoLZ z#+UMWGQJqN*aB2D_zj_OeGOl#P2sa+wYl0H4*KWwW!fC=;wXNvHbuto=JRLs>$OXy zKx3OWt@ouAAhU%(tj&ml!ME{SW!lxiX&zr3%dgR9hx0{z0jJI9cksKktL4B+0emcj zlWZWJ7r|+>_!2lTgJCnFdzJ*)N#%F(`S8Al&yE7ob3#Fq?bY)aHOprP{@EAiKtu zFQ8L~-e`SK_%xZ{4udZSJ!fk(eE5yPSiUwHm|Zg(sAmHkJHTf7+EpX?Lt}yCHT?D; zz?#AQYB)K_ACznJIsPXZj9UtNDA8ue0-I~Jt7n743cyTRe6gJ02UacP3uj5SYT#x& z^eX|PfB}xrRDrhgw5vhWMKEef6zFypP%DNDSrivb;CdOHmTJ>Demw|u2s&m(X%~Z) z(m>{1xW5am3Vmq94YT<}+EghFu?;AIu6xJu>t))-v7obqpom@69^w36AXEUo!IUt} zZVv8j1iHKVZ3lj(rGr0hO4@~4Ym81b9DtP zGw9ng`nH4h zVp2^)5&fXqe{bEg8qJS2nvxpLBKSbt0z<)Ya0do-iA9`;{@6!<^wb|e*B`&oAN%T$ z{q)EF`r`oov92JafRe9Kt7#?fG01QH*dQ+0$9MEFE?y~BC=t<)9q^N0qx)PM56a;0R|fv_h;{ zT#;IlRE1#`RmVqU+0YnD?Kl!oXn1|qg5q~feFq2I0zzgwRrh>3k97Vw- z1+Z`6D0HAvW(2H@`hno67N< zu_Vt%xrLcLOS!i2Uu6ODQneR_L<|SkA(U>QYYY{K!D*~Ouy9=BR>@ko|hr3)^ zaryA&vMZY}@4b?KWy=-Kk>4&goIvj z)!Buc&$$oc77pT6@_1P9aZ|;_xrIf?DqWRP(&%Vqlw2~)!!-h)(Kd*q?Q4&wRPWk& zckNPmuRWXsr?rQUak`8{x(p4xBeVf(xG zn)vDpbJO6pv#v1jKwaU^Rq)zhSGX&)u5kBqx=hr>EhVT+Ur(Xf1%C#@C**oMaLo;w zKE=`5(b>giD#-C^3HsE+=kEvq%PB^*B)K# z+yg$ky1M=MuT&**i-KuoWpkmZN(vkyR)D=GbH=A&m zavB$^RL%+S=I%~q>r%D#>SFF{(p62>Zmw$g)r6{_s!F)4$yJ-HHeFp*RZ_LBYRA>3 zRXeJd(Ftr_!HO+)1aAta(@KAj?w5^qW`cWUH>6_S>u)EjaTqP zckG{Y6gnlXq;E8cC()RmM1yz|4d+SO@V0!@acFuryD~jC?5{m3(Us|3WqSOn#LD!_ z9o*@>$^)=48pr*zLM1Gw=jclp{s)@SL)RH<$HAMva?GnG+3`@>5`Z0)Ok8}Hl0M5??1b?Vu^0e(i1z+ z=2j$>?>L)PevDdb%*4^7XTTBmH~9bhL)Rd0uUvMo=*XSJG&!;OQAE*mb=}qVSBt0^ zg)IPhV{9Y)QA%87_WRfj|rUx(sK@Ni9 zw#I4;!XbH;0AHE3GA@HYTc9swNq$$bc0HscTJe=cK_#>b(&8a0$dd<2wSWV$4)A%C zv~UKEX_=UzT@p&$LE<_<^HMa_P#Lu914%NXA;WHU)vl&lr1prq_Q+;ufB+<-BWhjp z78-z{WODCAtxS(JP`JCF%|4as*qeTpfbh{2G8Ap4NvHN`LG6)^y4+(miAQy-cj@Z| zo%)!*TF|ZD2{)k$G^AABA)?Zt;jCM|qr6O)2|*N651iEQ-`kveTsRI^YnEBSZ_Cvh ztG|yAYc&)i6a|}2;J@UD!;2Te+dz1Q!)qSA@`XZUH?=n>HX$NBn20pqoV$sb9)jG7 zl}6CT2JXR`IH;Fs862Ye)3Z8yQQ#;}CL&<~c3*Y8i`XPcq!~%idTB-pS~-wD8gK7b zRnTNiVJq4}-CbRS zLPEqs(_C*gd}^)nRyQ(cLO$0BTxdgkq473v?;0~Qtqcg5lVG63nUjdtLW4aZk2w*E zg>AY{p+m$10o(ri;fXH}xB$3_ZVhyV6FG6UL->Bm1rz>^q3B@KS-L3+qaCUYMGiva z!|eJIZ|@6ctg#8{C>B~DRd-7_EkpSl3`N2jdRjriy=b4nsgZIP@=#`WLS!p6-OFhG zsCL)i=JX^TjHUs}QnfF;UfqUr83rU<8&FOxsB%5@ec~+0WDPS4?541pkp$xI;hfl? z1PlFMt}rMn5@<3r2C?(7vP4@Lba{WgrR+ljdER2)AI|I^-H9z2MDVu&~e^VvX&Eh8LT< ziK`7ot%Zh&&RzzW)Q)uc)LzlqQQ%k1^x>rP*y-UaRSdL+>mpgWz|I0bm_^7G zO4W3DoT{`N(4m)1qeDNsVh9vv(T(?w;9dcCLsdAO84PEn4237u&Shg8s${C@CQeKw@C7t{*-8o9U$Ac5q3v}i zlMZ%9ZND%fMs5E^I6K;kedbVl&&z?_gUsVVAeXSkVW(fxL>=rb^enVcAM07=YG36_ z+juz`O_D`N%ivGK*huS-wCLBYW7h3(Kl{V|oS=!9gW=ckhiQ=xhCbn;F-q3;&*!Ww zGRcg=x>aio?BV4E4ZIv$&6y(&oh?=LQFuqn<&n`+*YKvnkJcMDGp#ouBav=spKj;` z_d!&6T5eO1b8lD=9J7Uu^}Uvh?9-QvoS>DLgVlfmuJFijSC78^UE$f=4i@gN9-q7R z>Ei)c`sgKC3U&jfF*QphjLNwuLVI{Q{karOWz!cPDBr&?Je9i-Jl5S4o;Tj8xze|$ z&CE9fic#3$WWT}52^x5P^1-T37uK6KkKK$vMj)w7Xb&&i2uL(OF`{wg%a_bM*qQ0s z)&rIG%b#PPKgS8~LofQj#{afkjMMWHCvaL!@zvgmN` zbNKVD&%;Bd{ldcB!#|gXhe;y(4d~+;D)sR691zi`e?+)vM8610go{85DLlw$yvP3o DbfIxZ literal 0 HcmV?d00001 diff --git a/deltacat/tests/compute/resource_requirements/data/sample_with_stats.parquet b/deltacat/tests/compute/resource_requirements/data/sample_with_stats.parquet new file mode 100644 index 0000000000000000000000000000000000000000..80c471b01bf6d9fd1d66fe0072bb853c79d4e056 GIT binary patch literal 85575 zcmeF&XIK+k`#<^_O>lq#0|KI=5+n$UNEfUVH8epHP_Q#30|}&=0t8!7QL%Ti_uhNg zSg`kP@4f5RZEt&@?{Yu?>-_%bdd`dU>YNwnx}ML&CQN2lxz}3vT8ZiMj0kU$_ol(# z@6@T@$7*|1o)lHpKAbzizQ5i~OgTo~pL3sP)WK1TKkzHpeCD}Jiob7SIAZS2AzFC`!bJjn&>#z9-d~VCE?uP%W6NwAE{Se zCqAZwnPUwdG{oh^AJ4{LMuq?Ce&`;}Fafax z#SF91_o&#P**;l;8s1*Dm;TC99&Q#~W$$F^DotX!$k+x_igg4}Y)@Zcxj^?J^kmk~ zUX36H-8yRz?aMmWciJwNFud?xoWsoN+Mgc7a-TolLn*j7Exyr7tbK}8C)|+=YC|rk zl8+|R4%WU@7>|~)Z+@914Q3r4>Mp1=cjH1ie%L?WbHbJO&J*WJ(Gb@Y_S5O14wunk zPti(L%FOjLUnVqZnjDvOue^n(cND~P=vi)rGmj>Sw=YDUPSra@-)9{$@15vUmXds1 zLFcmE&zHsM(1ou<@J##Jk{VK9miu1th?H))^J{t&OZjJRMFkue_+PYywclHGhumI& zXbReA#6%z5(SF4~o!n|+p&fzXsJ&(YuHe=uFGO@W91;2S3D#b?X&0%@spBVb69+4^ zkp?||ydCQ8*j_c9R)5!mJpzu}9r>q%VZw)gLB;K-%Zg-1rcm+-mpiC{-RLT+$IrXux>i?@qA|Dwian7B%Wm-?mPnh#m$>`2a)yC|2c`6 zHp>>0+XqYwqN`YL*Zwc$z7s=96HikRk9*?Y_u^ElYS?@YwbT~dw zc!S90Vow~Dg){Fn4&lL$NhxQ@=xz19J;KG_sY!@<6Z7iCb#yD&ZE+MSO!fK(7e1f^x$w`*W{8X#tNhrXtZ=|jxs`C)w+!|LG`Z~*vuGg3JnBj4S6ssybOhU^+L#xOGbOx&AFyE6>6m-8zZrhNI z<(Bg87(7%R`keSVGxMLFBqDq}^`h;J!+GHoyo)=18=YrA>bV!K+i%PyDH`M8^MRPd z9Wxs}DMhDk`aK6-&U`C-$1Faajkcv8p8ZOCaO7qeQpwc4<4_t$rvU@-VCw2+7Ukyh z@|{TI%`+aCIxrhHT!g`O@WE;lFQ-b%$&bI1^ALC~ydv;sF4XTECc&}g>q$q_BXe*k zuCmK11REvLX7n`|TwWIirT$*=2G>&yCuhhOF@qI1(4d>L+K`t*zATq+V!8OWeMles zuN{gGwJeQCNN|y-+M}TMA#3u;vo0+S;ZLw!*YW4c!>)}!pKC_a;U5GUg2C}F zvl(f$udC{jj;cyrCr%QD&926{IOsBl#&ZWdg<;O7f?iHXlWG3B5iQ05&3K?+Bn^^pf@}97O zzRqxCqdh5xvEPyVaL&xH`cp|~Ogl1*%v^lTE0n;V{csgZz};{1lVmF%*UNYb>rg*D zPWoodIv->k)ZGRZ=%Jgv|3Q_gj60dqU5sO44bNa#mZCF0W3sj1bP}ZZS^J_@&d7u8 zjXHFDjOoL1W<*1j$nJGnVha=3wpp5)F`YwbddI96>465E+G$$c>ce6##5 zlQ5&3>qw8T$nHk^_vx`9w4bBV%(i63I`DKf-o&kGwG0VQ)Rl9jIHnM?q&pNXtI5nh zG7OsI3jSW1cavl#*W5_#aWlp*K-q^kyoDL< zSMep%&$Vo;$n^01X?+p{ul6Ki6!pAy8J*;K&ir_iY=qtm9 zV01(jG$RwoC)YTX{j#)>U&T5WoLC@5ub;}Y$N?Pb_Vc@~NG}IZ^Cv?j;Y&|C zm*GC{)=*5)v&Oicy`^Fyp2%e!^^oSU-1SK#NEqE2xg6_R`0KTZTw~*-r>Cz5f)?2Hq4EuzcTh^5!Jh}I!wtXb8dm5F1kr-`z{{Cq~zBkoiKSwwJc5) z>@7aQQvE}_k*c=-^LH@4%Nc@)b9|dCS;v7*}wdW>htftc^k{M3k_%(U&&Akp;Ix}ls7j*EJ%kQwL z#1_op(PG)9Yoz`x)jPR@G<5&Q2gyJ^(|!%sfW6adqBY{#QF!N#3H9;clA%ix-Q2oW z`_b5r@usQbDp3=OhGSm8k)w;bqrRR-H>}!Ka}g07SE0dZE*d64*J?LAmtp| z;#`w|a36cOVcSSMMn1Ajcd=X!QvOnwTi!XBG)H@%YOzBUJ9l*)lU8=0v{}kUF-8nm zeVfdroLl?ZxQu#DWs*_9aAYeo-V8sFOVMh6Gc(8=+B3CC6Ft3i5z9qRvHSrZnf`bM zd3S|hHELiVr+7^>5^na^U-!!=Gi}*g;#JI-t6Rx%db8LQgE}U>fr4S~rk%t6ID6+f z61!5V8=mZFbGZo#-uK&Hp`#oJcna_0)+)79ym)zXAZZ}K2tTorS+GKmsmRgy#Vs;f z_=JBVzdhy};R>!!$G5ouKg&kaNl5UqFTiUf4KJl?hO52J^PLMr?d;(h&a%|Uf;r@p z?cdiSh%Coj$kA@iL%h&ShwnAQu9|%_;6(q{^nRbu|LL%Lw55k>gx#pTPF)=>ZQy?46r|s29YGmyVVyDp08LoWZ@+y}4HLNx^2l0Uecz^6+&ooxNV{Sa0 ziV?+)edUE8jyv}TlU@wF-3xK+INyB)3DFl(r_qoP4t_v)JLX6HP8H&&w!yQg=KEWb zW>`I;J(7>r6Stx>s5=LLk>Pt^?u6=duk7#eqJ8+AXwp&F1KZPThEnd5p;w2FIfP1b zTfRM(OBimJ#_VS1XvDL5F3(}ICH;tD2vX0WS=V)`sF42AI7}jpynIf&if{^oj zgEy(+Tue>INyAAWI_|GV98hPTBueq_n#aGPGSsq%4@oT!ua%MZ$W6UThcndj>XRp! zFel_F7?3S;$oMTVY``{RXK^Vy=w$j*43x3i@3BBN=$=G&qkD}j#Nnb(dLM@yW4g1F zUxZ!A0e_54s_x?1(qPt}Djg~qnW~>p&=i|C4kBF<)Hj|qXZkpt9kJBmjS;x@oQ-2h z8NF+E!@Wf>dSZxEiOtecV5-%#m86kodPI^D)iz&42GP*rlhK@%b>$v3KUH#k1FGvc zus{C4;w?^8L?b*rOPyIqs_2Dc7qiLt5m^LcI)&mrT%$r)DVlWruv=tNUBB%yY108u z)3L^@qP-gNN{t`b5JSyTPd7H?ohbSv?&P5E?IA1fwI44@2tUf5LgISB zS$E8Y>}w!IMcLzeH(B zt zo%%}%A%{oLi?|ilR?-{ApRnZT;Jms?BoGn=T^>{K_|dF+D`bmyy=*Flhz) zmO7(cKnmD=SQFV~CcnZ!7Q9nKk`Z{0ExY5zUzvwZo??D2I5LUE>D`HMQLeZ;MtKfH zaekh`by>T-uql0lvG1llB|faZhHwOt`Zjr@42!B6%HM)t_{OoLpt}wxUtu}R7=4yL#n!Co~-R}Q>dA+&o&g%{trJ2Ke!d~ zO~|x9s7olu2-S052!6A_-xo~si<}erI3ltSjvRm|ZEKT5df-MII|7f2y!wvR^J343 zc&@|L*DM{wP<;oJSi010lJo>?PZ&Foba(Z&Yce~tJ6c7`H%pL{mi?d@Nrrp#rm@%_ zt~o+>k&gb8&fth-+MxitigA3~R{fmiG+X~6>%-H36lgtro2dw!j_%d+>nvsdkwhMw z)z2A^pzJYGw1eSJbzaWlr0EHcc^Jp%5YO~F7z~F($TE5*;|!S(#?;D2!yLOCiyCll zA3mZGRA|59$m*%A63^0lto``86=YcTX>byS&UJSa=LFxr-r)w>nEBmaO49#=BF|h8 zoTw?Dkn!EJQ!_HpZTxF4nGd6*zhUx>Xnu*b&{u9U$*3nV%`sZ2*3FvAt8f;4`!E9D zVch--6DRfM4LK07Z#kMJ4rdx3bjM(E%(g$oFS`=&lG_I~b|W)DVZbp&wxjW5&(b=K z{rjJu*O9b_JSGXY{fB?8iu+9S>lIk9?KcdSoT$;v3V+5?($;g8D;8*b;|fajZic7Y zgLM?%wBRk&%2VFsQxRXHk4d<5Avd2kOe?+;t(I;Z*F^s*_A<`}OVKau2*^(Voj>xrpUl60Q zn$+k?O7nI2M43O+KBOUe*XimL$m*$<7c1!NZK*#KCSFE6+N&;3B|B_y%KLg9?}v3l zy)V@EmUugPJNfxC4s40ZhXTF~S!v0KM?O^We|eRnsMpRPUx{ksKQ7d!U4~I2e+MP< z%5~68;37v2Hu}D!rlhm~A(2;B?jf6yqg z(T~nVUFh88Z1m+U<=L3)s-38pVqa_`^(e1b)dbc zr+4z`F}-`DbjI|IE)iFzXLgf&&&bN`-eX3OY<|X!o_z*h=@H(4h`47?-ULO@UitGY zdiE|{U){5he!n=ouj#BJyI=90itPSnZ>qBg4E-g}$sO*Nki(B^I4UP^Y>Q(#%8Biq zda0(SCG-+z_8ZkJe~#{0uYv_boO-L5Oi1XhSuuZ9@4_|fkM%Cvu-~bVcFWm>KDr%u zM)lF}d2_6f;lMAazQ)6DiG2qiYdE^E=~Rp3ea+|E)#_)tl$O}fdbQu^e#JL+$NQDs z9a5`*>B9+${cTU@kM3Xga{clCgWm41HDK^xXA=hu8S?qg=mA5&zd1f&*x$cu<=Shw zC*_tqH5!vU{BZCt{}FDjzxt2#NQskHc=s-qj;dFDah{O4*#!fUzNK zb_I-UvFB^R_=r<+O(w+LENwC|{^hPFliGd%+GMiaB|dP9qP{I~YI3vPfz#4je+!)6 zJtbZ?qepL>Y-X>*-LhHz%D%~}__6UpvxPafpgEc~yMyNH_IwMPH}F(^)A`n$wx$bg zFLyUxIOO}cri;p5S_dz#s9zSmq_Wwb;H48)oU&$XRkkUuaJl(F9yXBhh%bn6^>%ZI`{N>G;d-hskU+<5s ztN;37Ow$ctAC7PJ_Uog`iDBO!PtVqWds0=f;oH-BC2zkyTQoZC`}1Y9^xt2sTDjr- z%XK^7et)&;c-W8E+pg((linbM}9}Wxlun%e@he4}1;!ImFLfR4+zCIo0#^FbM@3v&F<)GzR?u zmBpg-uHkc`w$GX=w9m&Oh=U;J|H*V(FBG5aQ{FYmtSrF~io)-X_TH!3r;_QJe9ph9 z@yCU=Y5ax!mCU;$z(FbH1hs$xXK(>mV8IRCp*Bds18^^@4tRk#_&{Cog?dmQ8h{@( zghtR9{6Pu<&;$ZO20_pif*}N&L30R&77zw4Asiwg5~3g)Vjvb;K^(+GYiI*)p&hh` z4#0sNIzlJt3<^kqL`Z^UNP#Yp3Te<4xkae;5F{ zz(XD=K?MTjLjkBk1BFloTF`+W3}A$TU;;B(zzW4s0;OPsG8hDdVF(O`VPJ=H7!D&~ zBvimC7!6~f62`(f7!MO*B20qGFa@TCS+4QpU6tb_Hi0XD)W*bG}>D{O=9umg6&F4zrwU@z>0{cr#df&&i0VK@Ru zp&E|CaX0}d;S`*PGjJBp!Fjj<7vU280hi$lT!m|J9d5u)xCOW24%~%%a33DPLwE#_ z;R!s2XYd?ez)N@qui*{6g+JjPyobNw1AK%}@EN|qSNI0s;RpPLU+_2l0~C$_K@F%0 zA`pWU)B*;a!3A7_1vhX9GI2=213aM)c!4+gKwa>KdQcx4fFCr3M$j1iK?(uT1Oh<@ zLC_R}Aq1L1a|neN5C$zF93mhRq97V#AQoCd9K=IwXajAb9khoIz=0e(LMP}93P^xN zNP=WYfi92=Y0wqAK{|AY49J8m=m9+;8*-o*^oBmr7y3be7y!AzLmntW1p?$l0jNO( zg-`@q(19KdV1$8S0y9{^3dK+YrC@_H7zBf12n>Z`V25%T4kKVBRKO@04P&4Z#=BD0X-oba-bLVhCa|2`aypf0J*?J9wbcET>$4SQfO?1TMq01ko!4#8nK0!N`5j=^y_0Vm-UoQ5-S7S6$WxBwU768r&| z;R;-ZYj7QIz)iRXx8V-lg?n%x9>7C*1drhfJcVcQ9A3ancm=QF4ZMXv;T^n(zu*IW zgir7pzQ9-b2H)Wa{DfcdH~a%6>8GIv)C6+)A_ga@1q?WY3%CLcZr~2JK>{A&33b2= zyukwQrVtDv&iVG@OC6a1PGH1-J;8;19SASKumKgX?euZo)0N z4R_!!+=Kh@03O04cnnYADLjMc@ZSzqMdS~@BC>+xF!g`@h8O<-L)8E89iqlPS@uWQ zwx1sV(Tx+!E~j_)7A|*Bk}bcSks9;la%Q?hb|ovThj67wj}WLI-k zGlZ+X)XSG&?XBJZB+SLc5%@4+>ze-*ZDEB71#5| z$2`5RoU91Cp_<+!|AtT{thkXs&-(O6!J>+wo9bmV@^5NZEnji7aNYK&H;Xn^2i?+c zyOMuPx9iD@Tl#&Up58Jz#7%D-k9rr}9(Y2w^0w(r%(L6(3yP+9ESGx}+_7F4R^BPT zZGCp9xr4H@$F$;x{}e|~y)Zy4ni zeBUneQQt3Tf>zxh&c;5!KSGia{9vS4PxXTe-~3e%M)?&#e=ypARPe(wfiu+)D}z_8 zdN?+8$Mc8d!jA<%8XtXC{b)km(^Zcqw*CD4(In0(C@HE77YIU;@LvGQ?uubM*0*!Upyvg&GRMWV_!aBIys@)i)GV$7QR?smA~f2ih0E^ zU#wg-s@cm`%Vrk7T)k?=nwM+V?Rfce?WSYRUai}9weZ#YT~F7%+OY5Q%U2s6PR(C$ zI_gvOdh>~(wXe6FiGB5Y>xG2oZ?;|TS@dT6_58JOcHAz0^=9Y&QO)1(dOWk}?e6C* z*1p~IddI7`d*2;v{^!1rSBw7K|MltGKM(x;{OZqxRISi=4t&=2?hw;--MhnVtJm+2 zND@QeAN9)CzOVKzSoi*zU&-tD$Nfi#{&ga7miDic!7JDObt-h{>%UHi9}oR-Cipq-o`{nhA^IWYKA1`#StNVB{sp+#-ch64?Hvam2yX4PbU+#}?`S;hy zvkZTK3mWwY?dM(NKc}cQG^&4(P3L6#Y{(2cdVmhPKFrVef6S5pml^ZFo|8^))DCpQ%qrLMdKE8rk!bS-lZAHiQWOhXn;!qsSZlg)+R;m0cn zqM#bRcpod*>owvLzPQGq*BGRfZ&!`dpcS0y35?pLp$}2KSn11~3h;Of-ODQtZ!??4 zbJ&i8#S&6GoL3f+=E~-^7QrNyvRMX|iuRObs(F(^CWty3JdD1bg*=0aSJRDsJM(&t zpcT>c7y-?|u&Ej|k58I21zy}rqA=+NvuqQaYOrcTirEZaYZQ0zN)!yH0)cMHprEpQ zzMa&9$)k@x8XZCcy(#tUgzE*9MlY+xiBd-q!efC~nntTK3E~;3n_eYeDQ-+R z#8J|%WP>DKKoikkOqNC`?<;nd;H{`{ZFIYy*Q#iZyh5y@$Fdn#iyGe$MhljR4PF^m zvsut89#eAZW?IG|=ENRsGD1O?BuUg7Od8pICJiqZdS#&7d6V;sfel!^P(=4d3+h#h zVUkWJUQuSUp{+irK(o>_f7{n)KyF|Q>j*w&+3eb;dywcSgt;R@4 z(Jl=6xk!@ETeYq|)-VR6I=PI=u;z(NBq?YhGu;j?XI5KzaXZ;Rz9e&{G7REFzG=J# zty4lb4V3HAB0OHFXm?4Hp@cWd&Pp=zc$3U7`+`raQC^MMnN2a+l(O!mqlKk@JvF8R zjUK(J6v=hsF_NwZla8k|C31ZMhM0>eOOP(2Q`l_YrWecE6fzWIL@5SwFDkoyBQ0e+ znl!v@iC2nXHfxl!p^RM1SL84$yg8&Dl_7g7^GDz8!f}9&h655RpP9t!6O8RJNrfsTBSKlaHA4mUIy` z7PZ{htD{hi*i?AD=LI(^Pr$0V2pMMI{eNzlz_NtcQ@;vjP) z-IGe=8;S%i9WRla)mkC#2c?lsqPi@8MK@qkLLoV%e2DCqBt@gs*~ES<#<9}54t`)# zn1LC{8gkILv%y-R<<0ypUxijv#^(vnXGBRtMFt%z$y6FF7Fl1$_IuPTFihzjNtW86 z}z}!4mxFCBwDmK#Z0!N zAumrH-?kINUW>j%qjg0u+K@y)uI>G{_W$^1LM#>RluafK!Iy;LU$s8IuSo(G-#uW4 z;Z&@qNbFRLadvTK-P~(COFTU5czOHO^{wYrzky%FMveWY?qVjONgyQ)Y8o8Uta)gQ zu$JKwkx|hxv903bTeoRjw_W=VoLtTZ#vCJxZ~_)nT#9uYic^3+y^4T=VO)|nPCK0T>bk-7i$m>8Gx88ZW%W_7Gt zl^r;s?rfXTWlp+Z_K?=Y=gyOK&TM93%O{PVzhGf*k0eR1>$Lnu!>5>?;>vQAiyK)y zQ)RSPIVrPO;q>^(0TGd+^x9F$4JLJ1B3;xkf}aw+bbd|IvKUQjdeL%GQO)K>nf+JP zS=rltalA*DRotl6p@yiGF>BT~iuX$mT-+*mQIh^G;OnS$nXwZ z8iv&B-=?H=w%kouv{h~4S8SV*S!@V%a}^~zF~f@5CztdING@9KGkjqLwSD1|b{lup z^R@^DKCTN|Ep^)NJ7seDY#W;VTV>v=ZBpT^%HfR@)oP2CN@%R!pvn?A3pMz_@v}c8}S-ZSAhC9oFQc0hPnz zQ}>Lw3@)4y(r@kPzzoUW-V3^}b}t*EkT%HQzHh(R!o1c2Zt0D!wOl(jjapVbA+#uO zynV%*q{+MbX!Gm#NeVnLB09BbP9kqfip`Cd7&dMgnjO&NQ0by!)5VQ8mmeNO zEnMR?a6ZMXnbJDQqei8wZ9v}AI--Uy6E~Nx*rx8jL%z_Kb1-V++7z9xX;6GbO3RE$ z#r)xImq!h*zkHH+r8Q#RkRzT)L;8$1wy%A#NRhpw`e4e~Zj+)DnE1@<1{;FxKao7L z|4ANQ@--&2Wq=-Q*uP2S%`l(qO@q$8qJvglrv3i6DdWH9jjw~e7d1@ve*16UxT8)w zNrX}iCKazINg-xh0ZAn79hZg-hS!t%DMO>hoa`*=Xf^50_?fNNSOiRINd_%eQfC*o z1Zy;M%mjm(I9qKpjA0{kXTh_gP!i-^?QOd z)6vNHu-=x4UA>T%X?dmC9E}ta*N}X)BY8lu1otf#oJoqHGlqZ;= za5WMPWT1y6zcWT!@0DcLlLEO&vJ56QV+qnCDJo`>qL}FV zl0<_JfAzrHpcl`Uq*+ZywYY~Q0V#_h-X)Q1i!q}&WfM)jO}s&((HU_MeQGBI=4idy z83mP_bohRRL0>@kVbhdKgE-kMQ*ToVCM0HBp%mFeNDe)M!5ioyqRv9dGD$j?6@h-5 znjn}8tf;OF?viHETMAGT=U2$)(8eY@kQCjlM$6Ew@4q>dO1y!}6z9oAUYS-rzc|^& ztE1JT#l?Cy!(cW`Ly_qy4S8Z0HeDl_LNZ0&v=N6GzCiE1ipl0J5uYU8&>Cg*Kj=_F z&^lknx<_Ahi)9iud2TMEu7XF3SJwioP0-V}--#W)!IxwlMOuycu3t89R%`SHmgH6v zIoS`$da{YUN#$1Vns&o zu$kBaC@Q?rC3=gKY!zcda#`Y)#AEAXalVArwnP_lhv~={h1i)+CaYp5b|9sQysK22 zxV0PWrLmBGhcJ>sbh{3KFo(5UEcNCK5bZU-3T6;@wKqDhmVA99vas7mAucpopafhww59!K#N6D6rOy;4ml zNPcJ6n%cO5pc3;UgIT6zunm~75TDd8ro%*;#&WT6M3SjPc11tX{Dq;JZPFB|Ewn*` zK0?ZmcR4Muu8 zIwLPn8s?X#RQ20v^7O7bx;(K4!YNCL*3Ac%)C zsRm=R2NHRWLSQ>07Axv97=DMmX@hT~=J&3*d@8bji$&lM&^()j=BIn$l2W=h)6rlP z-y0ZBhx=t1RA>;Zpczi@q%7hjDyjPf`T>(6*g`gWp^3wskbP}z(+_^kxAS16*t)gLy30vH@_5vT5rZ+lCEX2S){{7X_|5$`n%-! z#z3==3_4?o)GLd(VK$^g@a=)e3Ne-8Tqf;@%2XOdn$rnlr3gc73HFr<4+hY_5=_(@ zQ)SElwX1TIQzqG6whKawZt9h3;q_YKiH6NK7*u?(i%UW(NjoT4`>={)%)cKnVadPK z%M@pYZ!AVWQ5MOjpuU)jv|59-Fr>S) zS@L@uE=$4A!M1)Yl^*aX!y@l4uJreko5(sukD)RiSn)-Le%NE_LLOpUqkqpzdKs+0 zONfpX^alEi1k+xbCd5P*1QRb@l%!%ZHpom|5?@k;nV&47UG!vx{r4`?BS-CJVZ)$TM&&S6M1rv|m}ddO?1T+SArd ztG#v&zpi;^$lUcqO&gSrHcl~a8g0l-A0T9ob+XM8rRHs3*>20$x_;@~f*Nk@xUhUs z`+4IBZEvYqvtwuNU89G@?~d5BaLjf)KRnkuU~k`jySB}DiP#@8i;QG1+IlRHOQy zEu5Iq|J<-O>(6f**30|CAo;SH7iaA*7?O0!JnRoT`SL1>`O4yTS8LgG`tG26UfWcw zX!!LShWV{#ou5$>P-~fPx!i5N`w;H(jsC4Rn zxC7oXx7@>7=340S(YLoQx^sO=anh2dsdsDKHl8@%{+LhfhCjBAunoA`cdA=-9rwH* zlVdNgS=~-(y!+n$2la~9KU^t)*kaq-6@4Dnef%VPz39qSah<30eV(;C5`mnfaL)5a ziW4XMb}Zes%YN)}y{5jBCNDCFE_mtRztng7gAK3pUu)}6f4%rkpDnFj#=Wg&EZKPG z&z$yG+>OQ~oo+9h`h0kiOtvUKB+ojoxO+`+=XW_xP7i*xVsuhR_m^*%47T2X@BQG} zpz;p0C-}9PctdGE(qQt0TLvKc(3Ee?M$t)BR^=` z-*ETQ-L^M*x}n9SH17#>;-UuJc++Hxdhmj=y=R^sTXb#F$(1e(w~Cq#XmE3d^5bNu zE!Ot|AJ#UU@7l4mss31ncg8LzY(Z+`xvgF`LjRm#X?Mo=iD#$IU21)Lc5v{hcg;t( zmCk8WpZdI}Y1Z3?Q)={G{`$V2bM$%L@p1goarp(uM4#hKLqivQ-ZUfj;_!P}OLn{U zx^Yh-yxU)|ok%`G_BN2;9-=?ieo649ob2_Jf~K#n@qcFW&Ny}WpNaf|Rz5#l1l^od zBdF@v??nE8pEdq#GXM3T8s3lJ(W%~B{{6b_&x3Tj1QQHj_&cEw=adFr7}Tb{7U5up zWcCr9#vE3ls*9@x-R~4XlKXuOT8wY@@?m&|?ih;I5h;HZSD-T%&|dEblJf^;9!dYB zIh{sh$rEsj8&+T_4pZtq@wcK9vL%SYKTJ0iXa({_FT9F;fou?BxqL&ZUMMk>*X!VQ zwc0>-46QgpFj`Go^0EhBF3kVEe~9C3g4RgyqKi`n>?g=ZBAzoC1-;o|H6c}=N+-pc zjbvY;H}jm4NB%}f;X8e=+TwuIWXt19G({RC7Hpml$JhOMalJq#hXZU278x5IA$3kF z!Vwdh{mm-QB&d`oB+DLAZB^>uTM~iG(;5nJoQP9I6_-~+&R@*=c+o1TQ9h4)SvUkH zI~8*QS{EA-tu{c^i=gIkzErHq6L8mbrS|tGM#B{sA*tt!EJB#cO5cw& zkyW{n%fr2tSflB1sV-cF>K3D(MVu1*8XH~hr!`@xW7cvZP5hAx6$oME3xUmubT<=bEke7PTJ-TUa9XR@N|#X$q`m!7p5NOf zV@IbN^rmPv(h8GpAbL*4t5mW9;_q$MzjsY04()5<3rxIX1|2~aCQHN3XhsXpM0H`7 zVh_3?)IxSuW(!ABMS3v4cIhqK?`!1@C1F~dbVOUd?mxS&V$Q%9VH;qi>dIHQ)nVhM zFX6B&;&C9R7m~~9ag;8^C9ss^$tfu|xbz2nhtLwJD?%IDZ1^|T(w9tFzDXVtm#0Kx zREAzhhNTs>vMsc4oQiCf$|AzKLJdx~HDX_BtZ$JPhkOwc(uX3gLA-|!i!0QTy__u~ z67RJcdBr<=0#}$%_I2inD9(nHPqW4%o~7k_SY!-h)89Y zc?O#!3|zj7Co>M67FTHgz2QaCjTr1^u^@fWxhU`V-Zx?(XSSh%ErOCRpx05V^3$RS zQ|zG{G#_p%B->$ggqbrN)WSX5C)#Yp<`ea`Ag~L}bO~>x3M+z{2rDPQ$nR*u`xbLb zj7O_DxICS~WF{RDT*4{U*xwrjeL-c0qJlG)U_&jKB1$=olw#h(E4IrESaS()SzPTO zVdG4Ek<}tde=uki@n6x{J!;A#$~c{&3=_-tlW~|ivG*2Ck>Q+1S6~&V;@=S$sRen8 zKLRN-g5%9aQii_4Dho7uv^N{c33;o(@E;h=5cHP7$S90?y)8_sq`kvc`rn)N$Y{<` ztX?daCQ-V|DS?qOnBFxAQJ(Jgpl6^KEnX5C%Nfl*hEe>@wf2BfTXEL|_qsQdSnAW}7docOcb14@X9rNih6 z(m7s{2uOpmJb*zt=rBru@)sM4xmHsm9ZVJStC>hO=dNa$5+QH7mpm*ECndj68OkCx zn2FT#p7QPCs=VLl4YtTaPRm!1pqGnpwbhc72O$!Zs|^c}2m3!G_hhV);cB+`m`Q;{OIQ_A%V{UeJpmM~uP z-Dfc6g`tIH_a;P^aCztvFKH}QSbqTH6xIBfP zBG32NA`viY%#mdt2PIgH!+7y^c>0Y9h3M+GgT+g+(qvYNT#4U6HFm$jsBo(jozm|%70C?S3#i?QwJe!!*IMkXqs^Bjl|NexzffF36|QpL#&L{SBFMaWvZ zuAg3oLnL#QdcLc?PP)p7$p8=4AR)6DltmbN7K8MjoDWtZcPWc1s|S5f^+ zS5Z_E{=t2WFQrOz_0@8S?_$CII3xCxUZ+YL^>SsSbR5s0qKbvhbOV1AdKvu}rLXs# z(S_xC7CTe=7F{XB{F23;DYd7Skwh6eqshI$vj`^^-zekBzeQ04-z2kG^9+YNRUCO> z@zo&OqfA_ExQH_r|9h?#W#*JvytH|A24?FDUoWGH9COM0_*$@D;v?nyUtOi={mcj) ztvSjXhJWP3d#-o&S6VO`psK~O)0{*qWuIo3`%{HNzJGK$huN1`;qwd@|NPsCuL*d5a27>J zT)b8B&MmQRbR@{&wqmPc_1`d=uO9!$Af~v%QIU|XV=J1sU zANQ%wl+N?1YhZK?huIkC-j<*X&h(5p9{HQ8C_0ui=&xn3hbr+r!TLtWW2owQ#nAGB(Y)#R=i%CDtm*=P$bZTsag47P z`M4@NZ+|e08&Z7NICUY0uQ^)9MWf^2O&lu}6WjW!@D)2U>|IJn_u za>^LyMwbUsX3Yvw^gwYx1d-=SH;;W(vCn0^(PW!KzwzU-UX=0C=7iU-ly*vOuV@RG zXEjT+Mdp(KgS|HoZ=!1BhR@&$PPWr@noiSc37OE8w$PA$sZcUNfU-#i+yy4dH2cyd zZD|!TN>xx0P!wg+pjA;(QCx7(gH>@y)JNQ_q9SgnxGUcspXYtQ_q(q5y{_-S@1O54 zN-~)_=RWuHyMOn6&Ty;MrVBeWsHb);mL%pmIdyWP2+ms?a6qI^1%?gOYlc)FI87<* z=xg3kR?>bI1gt#&AzD#X?c0d7q+7*h6gmfZz;rmx{H36lqTH^j8pnW7OfxPOlxt(r zr*uw>FPDghL>taZr7$v#;QDO2$sp&n9zAm+Aj2rYcIcuAii4Mk6OqKOt!#4IAfEug z9!QBBcTjqj80<-rPl*LA(P{SqJWIktnU+zza5l_&<=E+XXiafD$`{(2q^#5FgB-^k zO;g&Yz5r0JDW}U9jYxL>5X!K-R63^{B2*^945_KJP-fTtTBirJDK+0p9W!JD|B$mz zFHy*~kecPiEHh2cOS%}i?toL=zF!FX;>rIqw$1=#WchfMzZhD0Vx_mqCENQk0gz~Xtyw4ItuexMCf z${F^B6X}5eMe2G4BrS4^SE)i$Fewf44cwU;jWg=&vzMk&ccEuFKgPnp8CAfnMl{4iF3cz%mK3VrjgEQ^x z2j34qh`O9_q5@y$vJyMH%^@QDpmL!u3l<9WyzB#N?!;L99~Qdb zne(N{It3MkF=xpr_-X#P2dQclBX?-9vxobP#^#XhJS1|9)%DKaa=sXfn*O59 zV~8by=yq9rU9u?BlSk!%egDIFyR1Gq%0UaMVJaQyc!FBm1lYIVW%EU(6+HlKm)7@% z!Fi-y_C|x+bhz=|3hZ)>xsCUayr~IFE~n2-YYUg4aCHH>+AmV^a$hh37-zr*8U)cZ zWe!GoIm!r^8$`I=QJbAiPjZ>EH28h z3W&@@kD(B2QejunsH@_&xkn@=V1+Aw6o{yZOOSvLonyn`D|a={S~`X)Jw*_)l?UR#CoO9mG+5&bdP^2#aa^3NR)78ATvq_ zME^F(ZY6dLh`GDeOsCSJq4Cu7C`Vo(8h}EsEHj(MNr{`yif=!36L#g=_TXWiaZsak ztI+QDR8eBbKyi!gSOK^fS67~HGq3puND8Ri1 z&+FU$D^;xx2h%HDE4mMWKl3xXSRmaWa`i7^_yZ%A%C%C0+2>$dn(fuMec z0#d?_%w#!sTgeu6F{CPU7ThIOrYR_)-=xyIZN79c)lpB~Kz?HAKDWK`u-tTpehN#w z9X`FU0;RbnRVc@PPpjNcUufS${4Em`+5Z+z2HY;2a}*>OOR(E*{abk zWzOT}vmkpoNPy^TX>&9$Jcva0pbC3LW)dr`Mxosw(Vl>B=G&(hYU?3>L1oPoY(E{D zWJFyFY>y9kxW2;208YxKAdf`!8kA$EGr|%OnwO?MfQ;_1sLyw0qmW`IWMc|5QE=U- z*b{EqS}W`X;Z;?rJP}`jOK_BBau{+S;7)R$C_q?1QcRKthmd4VN|h((%agweofR{r zx4xV}iB6Ok2eqEKFPZP9?xvOlzt)cu!+vJ3n)-q30J$ay&(x&gmU(81QZS(ayOqJ7 zl<|a;`U%G}%zc$wPuka)i&0Bu)#|01j4uT^{E&jm3D1aFoAPw~qzp4f$6t=ZY`hM8 zvJ0*qP5q>W=FXhtX?s|0dZQkKfhX@v=9n9`)C$sGWk}^bJwB_;fAEZBjm#igG#p!1%7*F3_HM-q3Rg7$#MWGY~u zIq-mxvO(E0=d}QsOfWZB$?hl5(a0+<=Nk2V6AB-^277Hpt{^k3N>fpS*(z`F0>}vh zHk`Se_WX4v2p#F`YHNDpoeca{noRdY$ce(o}>@mp7D&F|W%Pfgn0wPSt?5 z#+fe4e1n#v2;4+J@^E5pyh5Q_C}4)T8iUl zC~0rZN3N%FZdrXT$LP+Dg6YiNM?Y*L^sH^FPHD>>gr2pwK_Kvq4#03}IhJbRxJS@Vu}5QaC^-b5W+T7K_pLGY~&epG!bZ zKPSfan5p^t}|3^(g9X zoTo5|DWLB%z!5Tk*PvkZ9SmvC9-}y&nq)l2s>F;hK>8|2)`@q}!JOEkMcs`}bm`Ul z&_AUIG0VyfU}r&c{x}jFhsB(2b(N-E#7U5rN-XBdy~@I6RIm6ogoMXLNU9Q#zoItJQdWIjNwg6bWu2KWi3b-1-(@>L$OBoi;6dF>n?ZHPr9t%HMTu;qMF$P8$=pqwMTB!RZ(ml*Jm8`(M zR*3`Q7O{%z5`R`n1A&O^6czQdA=Cw(8IVp!Tb@xu-*o=KZ8#7cnuYqfs2Y9%K0A(@+mHu36L|Jqpudfi!xn_3^cM~c?0bh>rsJED#59l5 z1OeiT`bW9KFDS#DSEC975y_saqE4XL@E{J_8s8l&V=;JZ_jnw%JC(&Hs8|11B@Q~; zo*+P!GGZMLI`!vOt7j390g#+U!QY5t`nRBx;r?wV*IsFe=l|((33`k_(g3*+nAnCY z9Q4i^Q1jDJu;W%uPy~}=&Zsrr!F{aO1_Qo+F*0}|Hph-EB5Rt-bqoqFxM>u18d+RH zJW$vOneL&X&`GW3FF-ldK2`n#+9y@+V8{TE?|d zZX_Pp)4_zV#XO#h0-}4tgo3sMda6bn1OyRG+5X)iJWQ@&$oq0#Fb%v{Mr0cxiN@hD z^BYJb!`w|Ep9Bh+%ra0vgN;mA;9$4$YBF*ZxMiBA2ZGsY=bD6D$TZzVu%=+nXLng( zAcYx_Fn83U;28xi6-T|T8#Fz>uAq${;-5zG!7WYcRIt~V?9Tz?ek3dmqYOu?aj=i5 zwC%U?hOYmz_d(-pdep^4Ye-c?J&+^d-GcqbTVXc3L>Zk22grBTDB2j~c>SJBR3VEm z&+M;f=hg8iQAqt34q3au=T|jkz>`IRBMRA!mJ0D~23O;dT_33=|CXZ=nOq6bUNUOR z)P>aYik2Fn<4KXcq*wLjLQeDFh`JAFBh391amZzxT+Oc}Q>*F+#E{z;O6SQm!vBuC z^v-db5aiJ*?tYHC8-)j<+~sY#MhDbVj6FSL@_RUeA^!aY!aX@2#a9gbL;S`Ed^)#+SN zWM{U(n&!AwaD#+-*#zd0l6|-dhmyiz`?Hj(Mguw$O1U)kSnMN`S_M6wzLwK#g>jIm z6gXgU_4ic@LAb%PIvncOzcHSNkk5WA$Du5VHi#4XS5TzYi*r6>pMq}$1b#jaS7HnmMrZTPJr!9!P4v6vX)5J(&Ow88otSq0B#bR z%=#r8^eyC}xU{j+Towybf4^zpB7VKCW$CY=Uw2v9k!4 zT;|_2`~c*_89A2RjnnnwM_@}{1(rMuelSt5FhAAPlGjJtCeisomP@t@P80xY7E7Y> z?izkMbD$MVf%eVKR0k~Ah_*o*kb=HgI-N|A$wt$Bl&c2Ca79m{x}erc2AiY=+J-=C2(Y?GYVa$h%KDQdn?!+(GR%^$$nU(_>K zR06{$u?{Vk;R5ob=&N$GOSN1%nVQNe~R6rDp7BrW~MUE?SR%xHBjmb%+_OrkzQ zqOl2Xl0AXp?;!J|0p%JWH2MeG$Lks-=n{YwCS=a~-6;3XBswScH0D&mZN}I|G%fY| zY_Cl!eF$?7fMomF7kTi^hGyz|a!ZW{s^(I^xrd@YMfujJ(ZHHN_?i6M#uJUW%VLx( zWcP!U_^U$OW%Z@fB6C$Gv$VAdY%Jep19xfem1e^94w-dbke&=gQ{-w2*JY8kza+jM?66rpMucht~^x8NuFVE7ky!i+cZud&xj29#{*?wO0(J#_P~j%cbvP9zpG{;3CIM0Mg$Z{B zjTf~7^%`ougC6J#={uQ{rFJekq0zwO)7jyl_}g(Tt5??Jt}fSTBEb306Lno-Ul-7I z$~-ewF<(!T*&^rPC_F(f8IQVJ8tWi> zX4b4lU5#-qx2gd!RzJaA$+oIySVPg6s776FpVY8_z=!F|YYHy*qDdftw0#JgftauL zbQfTOSTiJNXKSclFpb>M-1;uS0#Mw}k~8(3z8*!mcc5mF+h^xf;QU8bsk-vOYurVX zyq%gaj3rimncnH@F}evEtr8ZaBv3(Jz3sNqfaHJxsKQ-+u6HW=kzH@MQ=f9zIC0q0K1idKe#ZnHw%S}!bcOI_aT>zV z>=6VooM2WEa4up0V{|TTZ#wzRVhFo@$(}?$nqgOSz!1%4$jud+u-g~SGKO3N7j(3mGa;%?9klR#;VGsFV}B6?MthqVTShI2<(pjD+6+ zs{UY84TM8JeHBa1fLZ|=F9-T8EFC`tNGr*_Pyt?{3-bTL5phj@7RsG*DIA`8EA~Ig zAB7o4DrfQ4+$a|gN6jxN_^lAQV+tIOxpt2gw&|azQMhefB|ldS4f#?yVfheNs<{Grso2TId^tdw08Br%+CCCohjN zgtO4Plk1bRrdxRj`{Gy}&doHm@IL~jjwa!7z8xynoLm9*1{`%Eg}mHA?93 zs>I>m?pwitzf;x+1LbDka9{hg)qKVS+Y5s6TzG|VMKlHbG$B%!AI76lkAV6sPsf=A znMVLQEQGlYEcm%%Xe@Jh5{_8RF_zMiR3+J}MiHU_a+3YKp86Ig^)a;}VgtC<6HfLp zi)iW=kj&%baK!HVz77IIUvd~n9QuqHHqNC0rzSeS>iFj=%yikZr z=%=#~YvxqqY+BsUoK}SxvTu_9Lxzq7eC(GKsmsBbVp<#tPV1#jCN>4u9&#DAvMagH zu{a``cag$Al+`B~9O2$oAbFGlVda2Do+fjl@bHy7fPImKuZ#Su9e$i2 z2fRaF1&$;eA5>9C%MjD4i2(g6k`vYBR40k6Xef)4kX$Q4rsV5A@~QX|%=5ci9LaR@ z!w(3PQLMTiI8eR)H0oX~g~^v3j$~&VIO<}OGiU&!L~^#Bgy&F13gAfIcGGD*$881q z_P97KTWwt6#gX3OCjsULOP3M#IBm8+(&rOxuRD;`sYfeZvy9X#s1Pk{pd7g?i|Sin`lB z<*3WK%is1d1`kki?8(0@`q3{~H0 zI2t!D^*|jZ%Bp!BO*D?G8+j8@&m)s?G^xLe<`bxkd#j#~rhF~UR*J*%2-(_-qv_(( z+UHE>iLmPz`XQiT(ihEhl?-p@7Be{7ZM=pB+Ku_T0wuX`IULQl{A{2?z`?Glq@y`s z%NI^+9;2BElAigE0<7!H8We3QDwsFrszeO%$X87$N>cG2 z=x3*4u z6NMOS)#8}9?Kve4jfTt>4p>~D5#F=43`Ht+IN&1^uS%t(x#D4RpBz|@AryS_2Vxrw zyTUI5Fg2gYv5>KK5`P%r^R%%zCXs5bKB2+p9%g-2W2Xkj{!LxuSCh##1{@2Umusn& zDDOXpW07gQ`NC56c|ymcKI6sakt~42*EKj6qb^?H83mK;ga0I^J?fyI1T??I4Md@9 zlbourk3qF96^^C2cU;C_+Xd&6S5d6x^NEmLiLcZHmu;TR@KfkaGD>dqXk*aclrV1~=DOTdW5+*{h&qIp0x{@{uR)Dk&Gs`%m(l_}_phRF9)$Jw9V{D&Jw; ztHZHg*NbxM2?Cpf`eJ>qr+w5D?CBHfTx^AHxmS1``c!&xtY5u>x)tRIFV@5cfEuM< z<<}Vxo`&O=nI~yE9D?uE#;w3H<+&KpX3(M{_Q!2TXCsw|IR3dx6^EWV*69-VaWfeB zB(B(}rk-SfQPszx%L7Wft)$OnZ9qXi1eUm4uT_&@p&raW3M%iJ!<`CEg(CB<1IN9# zcD=9$viQC#92bp?YN`9ct8cfUxUtVjwbcL9p_+;Z8=qI0_A;L|;&`a-O3dz_faB8f z?>Bn@7>t%T#k&9_Ci~6MGDFP(*g~;59yYG$h0~dPo59Fz1�$i1hJVIv(|juNy~= zgXFGPKUs<6F&kA2rS0HdH*&fH z3VS!0g&R;r-zz|GRXjE1Mfny*D|fe{KA!faSD3{hcZ~^!^kFrQXKbF))Z-{*94~K( zcl!bow5b6Eoi68}X5VVT@vKcnQG*aj4#Q}2gSJMtw)q4WQ%w6zRXpzt89%l|`Y&-S zcpUF>-Pj~RV*vAdg&_{z8s+01a?Gx&+`Hoei5f?ysp8s#FW@WzayIu)11J zkaDM(Wfq*UG4B!M%8Ae`6(@Js;e?~*11LHAXQM#tXq0ICYX*Nd64!i;6R!3fnZlFs z6nMKsf+!mS^8P(ufH0{yt8v1k-=q}hK=|9KQb`G~k2|{skf4z!o8@#u^l_7$K7EZJq)6t3I}eM+xylCOK8#-T~f17Y3VpCU9HvtlSw*x1l(|&Y86(Gp$ zg>Pb;F{AV&(eDv2#))`Oy=*JH!hoSgeP|{>iTPkMP9)93B<_wjoJh4H4rRU>1s&2Y zuc)Z2A^zX2#EHxpR`?pscfvS2m*{TWI|o{3zP6xrneI(w%^`+t1KMQw2@ub*Dbv>5 zsTTm{eMK}-EKKBEepOP7w9uNLO!T-;nIx`c&vDjAaZ?47kqYsP+0Eb@V{e;@n3@fk&dHa1W=Oj+0jN zqcy^n&;xnBCJ7y}A+m}qx2WrBraoy0H3dLS&gp;UGob}h}|$2CxhnOT{7p$O%5rQ z3_G%eIW`%zD|jgVOP2iGVo98 z0k#ZL;=lRpS-ArzqsGrCQ+Gon8cxQ{$1BAOoakbv&q2vKFB*+QE=@Aw%SDX`Ow8w z?Dhp^6viZi_YH_lk zOS%z9dKm5mjgte!+k~^hMrl5;WLH9SKnhy2*vrP49%bTM8uqcY*e$|@Z|2~Xjmfrf z{mPN+k=QYzDP{LTV-}pFl5B-LBx_4M0CDYQaLU2dD7ZW3=}#ze%BlW>E;pU*K{-8~ zZ=Z6tS!AZo%%n=3ayRNV6wCg?;*`gF2uQf%L4X;mDLCa_*UX#9pJrV?pAw;HGh*el z1tm8Q?iXl_jV;q@%nfp!3fj)W#an?Vn*v!W*{CCq3XPb6iV<_Zy>S_27VI};ptr^R zZxx)-6DiCF9zB;LBWl}b*n>)jl^Vd_sjzJ>jVj5hzSdW8D#Gn?iM5~&%PO_0C?u@| z+>#FFUlr85P|-S1-H?hwM>%BI*&eoG3RRDxcINZY&|ulN(LybOGAibQqc1+L>J@|i zVmq|(Fy9lL&9R$QIF%%qji*+TNE`R`c%b8!D{v~+?rW$1B()3STmqa*x1EQeSe9qT z&cLZmOT_|y5OD1$joMTTzl>)eFLTU5`^e;AVu2Nrh5)M4JFymFg9iurW6eQ`J|%0`$2)n94tf zGhv4Is8TC@*_LZNscXpE31p;_P65vU^>i@_$@TF`p45O(YK2S)atB9wW#__pZ_(kj z)s-duBQURbX`Hs1e`Tm0^H0NR`^aY~DSl0s%W2?4j|}UN@i^_6_O1Y%Q6y5-;Dym$?EDanOs?E;CmpLT!Y6;lgRR{*N*6maJGpEPRF2p7z&U`QO>qK5&!c>S-Xz z$ccF>8`e~OvN~WHMS0?!vf+Ai)nKt4w*%21k?TLNQ+`hxm}cd zvX)K*+D}eoj;n=Pp!ps%PRI14nl$AtR+-XjIvw{JFP+2lf8Bx8iD~=m^wVlF^n7nZ zIFKz;pks%sVH2-$Sp}*g&(?NhoKBlxmQnk;+Z{NaXp)VIoG90} zr44=-0r_T)cTE8#w;SZ1v%P*g1pn+&8*Gi=_a#+BzF?U57vgk}{`QGbe~N9MiqgQ9 z_0HF$fU!z}(|s-{1?oI@0Zy-I&og`kpzgOylr|n~6ZS#i7L7PP(7BJl4F|I%UMEMr|L^(Vhg0nd>hX8ll!$e6E_1@1Guf7vb{;nNu|*I^&RD4xo;3i7Ebn~ImV@0XL1~8fqaoI147a!WPmCf1+&ZLsbTdzQ#X! z2+Vg)rgj4!NEOZH~qoD3r4yf<_jBPRC~vxZt*(X3^i{d@%l15)9unvP!{HtA)*U+ySY!R zsV4A3us5Nb*cXAV{eA{et9f>E9q#saMobTsFzyyxv;;Ds9CPOwT{o0+px=OTdZ-4hs?BT#Y#EX>p&<&tYF5r^`a05mYqEYdYrF zM#v))M=pUrM%QySltfNz(`G<_0GsURd1z7^d<|zq`mVA3cwGuMUi4?B_6J9cU0{P< zlX14I@kW~Og@8Gx8o1n3Q480+OH1 zE>z=eyzx#`!AyO!WG2ogTuKOw#x)Uqu62| zLtO{3dubKUW{&?2*NNQ4CYWz--q3%} zAp8H1?)~r4y_ro{+g`F9y4wDluWFs+(85LQoc~^a?K;+BuU;KHE7j6jA zeCOeO)dKI)y;Y}A`R%8wg@W9v?ey37XwT?qen`7$zW0Q7ac76KddZS=daBPnC;CwJ zSr_!5s6Ja<<7Cf~ZtP*tjqZAgJukWM1bcq>0cXtxJ%@X0mi8ZgsAk#9-%ivlFUVaw zaj>>m7g*E1^PRvp%iFr~|DV1DZRJ02LHl1{g7&=p5;)n^P>^ve442AV{^v{3vc~@& z;=676mg41g*>{RpjJ;I1^U86X&fd9Z{LU>q*G_ogot;-r`j_t3tL<-`ed{{+CtGe^ zf7%c4+in9iJY+s0YO8*ke**LL`}Yfqn6J9OQmGtL>>d{*Gbq3h4f9v-@3 z*`>9&ZwYQX=k^=JJ8!)GruYMgZ{M2!SM43!a&Me-$M(KYZoFg1g+Cm=^`XpcT_+t;eckkoh{j2W2Cw_e6y!)Q~ z<&&-VJ@v;A@7?$G$x-$9KXXde`S+Jj9k=cNe<|(n-@m_Ndi|bfs}`NVXSgP~ZO?Oc zx%c-xKlZZv2VNL=?fDNJ7=P=w2VR`;;QJ4}G--eRgD=|;o&Vq~?oYQp`08mtzW?BB zQ%@PQ_x0&j7wkPaYuxs|Z_KrSu=mjE)5kpY=C*nBd)_*8*}9&$&rQG5^Y5jr)V=Qn zu9@F^xa;P1z3;~Ed!_fi)YIy|_p`6f@B5(lqjh~B4t)1Y-$xgpRIm8>(uxHuMlK(_ ze#IwiC%wAj)AdtH|7V*{U(kQ#`ep0;KffvcYX28IR*``(cV4q#;OOl)uOIm8uKQjc zICk&TyHMtrEuNu)Go&_X#D!;4i&><>SR)%vOAGyHE)gi~Um_sCY_?4BFQI%h{FR*x z8+=PTXI$BdeLlf2Xz?`7#wFFh7A!NM#g)EFggb>!In`M$I~)5bQ*^d}apmYrAHSr* zM=f5mWcG|D6q?z2hGxTtYQZPan$Crzv;yVV+|r2zykyCY+Um|3VzqV#Z2;ueXsO+;>J54#N|c$AM*R(uF9tMqpEMe3l7xy@B8Q0TtR&=DPb+FW~Mli`nipTPw%Y3(-n)*)lC*G-*;y;hl31Do#Mc1knR3g5w{PP^G(kJ;&k3oI4XQG*@E=z&pKDd_d4aypwAMr1X^OTgiJYFdNUX)`bS=jERUeb6iFgg>3yfUNKYJ=fgsk3OC*(=*^v3fyVFmDjE)haui z`l5WX%`Te^6)^`-bZ{}G{~*iRHVga$9tMl%E}yCAG<*1VY%!a@ng{~{XwaY8{UY*74g;BZq#wHLsY=sBi zF}r1F!X|x?In3H?=g+0=4K|0(?1C{k-0C-F_vox{vlB++zf$Y*njH?a@B?!A&w>Xz z;e=raEUnl7D!+nuTg<{i7?E3cf_6B~{9ITwH~$b7Rl(0ZRIf0$Lx#M38|^ULg{iV} znzC*>ng#6eP)`d>kxLql?9mP_thv?V@n4Iarc+?kh{?;hAcy=+og38D3HKdr{u$;A z!?+$hTzt3KB<$6~@EvBS|1|1atpx_|bqSA|Rw8$Y2fJLtyVP3b6|O>d>NtjhyegbZ zy@s5^M2bf?e^O($g4+5gP}4w8)Jga#X0XB!Q^c)IY*I} z@e}PZOVBWAcZ5}VN$f+m-5zZ7!m!{q+lvS1$zQ66HLyD1df^AzIMX7P*KD_lZIna0 zyR6vACVvk*Z5@xRz#hG(li2DJD&g&LuOqjt($tNd)XT7xF4@(ve}yBc_qX@}Nk2X(L%7Yg^v?uThR3S*G1K#+^NAG@sn zr}#HuyN}Edf-qv2ZO;SP?%3_4zJSB~Y&KUhTYRy#3E9L$I&fm3G9J@wpc~?A;&bri z=j;-KZdO~*<%H{3Bk@p z+4KC{*lLfKz!(O<6i>%CtA9NW4h**Nc=&Hjv#F!SD>NP$xKmoGv4LA~6!#UMkvWT> zsK7tCO3op>@G3Z$3sm57?ZSF+3U0Hvu*)odgFN$p1q(0!A^r~Xb$CmARp72X z)|I6Pk^O)bc}y*)50RChLp#mR<|E)D;v*i}546?e63;KZjGZoO1ae8I#G4ufY2>I1 z9JzUSCk<9f{iw0Qs(Ac0d@r`U3hm$#WfF20wt_lSO)`zRw;&V#7Gl)n{1>oMZ_Ul+ zc{oZZH?qiDa={w}pT;)Znl^DX_L#+aumrMR*fKn{vpfl_%e0j;Qpb>a*B8`M>~fb@ zVzX^c5n16-0HF`Ik_q>cbUE*|2%X3-UV?1G`>>cto4|LOj+pp_%Iks|ok@dtA8A9b zVqf$a4PMcIz3ex6$H)iB&b~n-PxBsOc0sEGt+H_=G!52K+5u}jbUCsX=h0@jlZ0v6 zEnv8%KGQqMG!S>{6b5rr?i(T7R&0`iS~Km+vJp6kheckHk1QQr zB>F~vrXd8IJ|~|*?*|R;dcGXu=g8BgH{l0ETm8Rci;di-EZfJ{M9ZiMD-8eG4B2;1Ew<6(vC4?M56g8Q~P#S&rUrlfzeP=}mD zO|;izE_n~}c@o*8YpD^+Q0M@E@9*jCE{i*QVd+5OD`v3@cI7V|0ScEK-3=~d zmGC(%d-Pmvww7LmQ&61pv-wkzo6XAC9Tt4l1LP#^GnPul1Euxlargt^cnTjQGxn4ku0_1T8SnxNWz)Bs_`Y)6@saJ$|N(Ycf_A9xOhVUVbN4DlcYJfx()P~lD#Ya>S zH{C>|fLLW0*7Bp_MuQK_jpPJPZ5&)UI9&La28U((pzti!L|z13DKWshddLC-;guJr z6_=2&<-=Pco{uyZh9LeuDSJU>ahqLa0yz-&_aj$f12%i*YeDQ7ie5xn;RmD%Zk=pU zVyo5kSK({&l3c9ln2sF9_d#{Y6pd*vsPj$YcSV=XKXMc4q{(CQb!p@!i`56#$$q5E zFlynE=HZ{n5e@%m;Sq4PvUOySviKflI=UBm*bYLrkPbCjpb(y-#hTR7XC3bfu-$>4qlXEJN%d~SOPO$|Mxd(IP zbvb!j4amdR{3o&s^`t0=n8emHAa-OO8U>kvwFvl*UqkM#0Qob66~k*uNdxM4BX-zG zSWfmb$ZaxVu!a5bICtTXk&T3v!|KVv%bOxZttgyUI>8Llg^Q?ri{CfjN7gBMqqs}v zC-W)*1BFK3<}W0>tN5GvXJ8l?4t-ae4zOmaDMo@7WEx#o7-#f2I4T_vj%oO6{wEdS zYcCU$O9nnr`o45t#|6cA0FH|L!9tiyd0_@P7G|GP_G9U)(#n;ug&#Ns7zfRD*f zIu%1ScH79!&O9mCOhc@O+>ZeA@>qyv-bD`SgA#z z+@?xoZf<9n%8{k?CbADc1dCIO@yobh=;Bw@>mvu)*%~rK1@Vo2uG(K1Sx#0~@vA|v z7YK8Zo4m)Wz~wvqpHn9Cy&8nySIz&DQrIYXa;^C8=SkSIKIvG?`!p>}#iP0u0-+ zyl^(P8JXq1!tsWYRy#HLH}pW z0R>#v{07-vG5o}ujm29Sw}O1GHZ?-_MH-pH0l+Z+i>a?iI%t>8tba&BFacbBE7)6< zz%dv$2U#OmFCkBBg7oaa zQ}(6w6AXc@Rlsz%Dquf=t8D#FGkc+$svIJ)1>S#*AB}9K?I2g#3$t*P=lHLIb!9%OB29G1 zxq#QmiAvI2A=~1QFryfD7hPxp&~!y54ar#Z2L26Lpf9S)rV7~;WCaHz-ChOT?o3Yp z9&#r*T977n6M2s&E)Hh6L0)*9gcTY9j&?I!t)PJ=VNO*PSNP|VfRfAvtfxV{GW?XjD=-zv#W9W2+6HM?5HE-GA5GYS=X8p4cIn8zyMRI`GwAVDwdF7?Qx z5apXsExkf2b!3*pBo?u_sfGzbz0|>+a$Iz&9I{VUKci}KB(nRG*owR^| z2N(bg^dERS?u5N=`)SfVrg@?;Oaq5j`Wbn+L9Mu{a6Oq$4?knNjUNrUvv-&QZ$e(E zAdS`Qf@F!Bic;&=m2-ggIxUFu z-iVyc#7bo49v>sF0j=?p{@Q{-ZdDKUlGqs2#!>^Q=;2DDl>2+hj+)|=<-E=%8waSK z2Y5=>RmgvYP-HHB#6RDB>EKp!f+JfD(O*bSmG}$6x(>%k6`7!BRK&E95W2*b{)Ie4 z^8i4S`LG#TzLS54sVC%5<$>koY&C^JN%91k9$*dZYantI?;#6n$lr3=)q@8?Qy!}( z2NlsFN>K!4U}Kze!qhX5RzVVHnn)cHQSl{q(pX@PrjYY!i1-J^X0Se~0$S?)GmxDu ztzZ{bKteTJn9MIMd;&V%2_C}9-&Hd1`it~w#Sfs{#W@5o5z`>o$gjxh&6TYDa%4G5 zNGmDo$W>!aqsdVfsM}ZSHIpP#8YU55Hj~^#56z+?>~!L<5FR5d8KHqF6yyU!B_UP% zN(l(qW+7XYkPB=QWrB-v8rcc)Ve+$*$*cIKq!MhGIm97bhe60vaj_0y)2Xs&glXlR zmYY*S&aWnuYsntQv>E*6W#lmAugrs>viu6q}6 z>^0&VOMS}s7xx#)npgSp;6!h$N7m?9Am>^IAY+Q9-lE9-r;yFxS6EF%4d8zl(?kzF zO(w`8xnSqe;!5acc~dEa?B7B-g>)lw$tp{yLn{v~$TxB*vTvh{7nMPc%d~fRgggWX zVD6p?xuC^Mo-pda9Lpah@+u%w)?7xJ;jAI^$QbN#kVOjYc9b@h9d??Ceayp3GM8{; z8Ca$p#tp4OwxO$vck6^=7@y~kx{hh0iwjB5c`E|Nu*-UXnv%4#qe>N+tf(@ z0PFIC8&Qj(f}kMCqLf7z0a;{G)a!na*LC0D=YGAe*YkS*faeD* zO(!#-`FzfP-shYwTUP^>+ZeU}eX&!1eHJ|rTZw->0ZXkz?5${iQb1*|5|=Jc%#iwu z9yI21m$pNVq?g_@Rw~pZC$V3zI`azA%*5*);#Q!CC5D#nG$l3GIkHWg9sd}wvYVu( zDEsixWs?DsRqje?D`fr^aPPuY-0EYv@YIxrU+-ua|Cutf7Mnzsj=Zg=SWlXrPj`u&R~k%;f$heq!u1q- zBo+8dJ>Gcs9nydy&FNFW~zS;B)j+!S2 zSN{wxOkw|~GX}AB3PjyiBijZsvB)yvO1qZ520d=nMUu=24J%#3k+6%60WRN?)qJP+ zE%o3Xi$e-5o04$?lfCGD>0BBY0LGb9qF_T zoqhaW9Am=|qvK`hT2e454F;EuFh~~}kK>lE0}nt(7xLU;(UGMY872HhJXxAa4@0(u zpQ1^E_8#MIx=CM9=ylp@4Nqp%rD63MLJA-G6iOgI+U63l593pGtbX8>iT-6lTM96z zZEr{VwZL(jX15ESVMps8JBvt2^mg>yq-{KL#3+$>KuQg_XpW^mf^aI$vNeuIm<>T{ zmD(8DRh752ePp(S6vq`fYPEOhM^TSc^1I2K*_cbW+1L)!&v^O^4zL&}YD?J7$X$83 zSz0GTbwrYnjux_;5|z>d8kf4Bl%kA6wJg}d^YHeZE9h~v^t2JFRd2)mmaoa1jbxv%K0+2dJ(^3V_OU_GOt|006i(Kb(^q+n z8fwpd6T1)EKuCfFWZNdmZgdER_8wzmFN>{=u}6eQVPoWWtZ9 zzYK|WO}B)PGoyVLeMnPGbZ=(-1Jh5GVP|BZoIR%e5Vkc_m?K3epd`oLSP1iPgBW@@ zu^|DwIoyeTUiv*$3->0R6Dx=fNa)F$-(f4|78)YhdXm1Ps~vUwJ9_(hx3xj%J@SLYVTmbqAu20$2jjOrJm{ZQf_D(Vn9hW07e?w3tgM3 zwk@jJJ+>ntM{m5fhUiJ@R+?j`v!HNK0z?$fVf=fFk{L@VHX&^`-&mdeh#o7^02>?XYz>)*0PSd>CP-;%G)xFrAVZ9|aHnkAO{@Ui)b3!{o8 z@r#xp`&0bYf^uUyhJ?1wB})p-tZVIU2f9ezC7*P~u@S8s>3$If`i^L#4SSwe!r2ke zAm%7tzzShbI3_UO$kzb-dz}L0N=Xro;dwLS50Jt68R9KL*u;Z?I#}3o!d@fo7mWaE zD3#wxzcPV1++4>xD2GrrT&NM!RL3tc921{7jC9-z13%`40CjZ`S}Zg#hc;yLhhi%T zBDcoV_+dyx<4rm#u)`8x2_IXSE@vxX-w19K`CW{{{mBj&x4`rk_ov2Mq1`rRcHth% z8*Zd2Am{MW#3V!(rDso-3d2n_+9r)+)C=cuO#Td8N-?)N)<4!!`aPP2Pj1M1;*mym z3ALs<+4LLYMpR-tHHi}KLLXB196mkq0^a++_Ddf~^2vhDELw0mXFknBgpIqxenYnmM&$MUjVpxSBKoD!11U{kNn7G?Q)8MVlj8RH z;n0nym9dYsTI2`M4xpdXwYf>MV!Ja-$gzOxvE*|;B$UbwEvNIww%5ZqD<#@&f^0IP zPxR2(PEMg#34<^o-ocEQtXQsV#GI~}pCPqhJz+u7IQ7nKz8tVNg(`&vLV47ZPu z#NKR4gER<6a)nTu5pW8X9JPEDrUiX)?>e78 z#8?U&>8HHhTLEfkp-E}~EE;DGeFu6rqYIGG$^v~}%Et@g!7@sGHv4yY8l7oR*=}K! zNm@t0rf9c`0x!H^qhWgHnhW>Wd+RPz_;DoWpvXbCjdDAt(GO`Lkx?DEBlMS%Qq%J) zMX81fwv4pxvm?(fm{Z_mDcoO)dZLt;5n`-JdY@crD7<83ZfHw)@i~(y@!I83)KS~m zMth6~rO{g%jJ&gn*aG3}4u}p*;~TrdAlajvB5L?Uio=U(Uk@gA4yL*Jj$M+E@k@`z zmpN7Gn6x1DzBnxl@7g0HwmdX~dzgKpeiYeqDnBMy`_KjY+$?pXJ9G4qXXg~g&rd=G zYfBpKOXCq77BXCdPZEa+abwJe*Lumt?$c7}FoiItra6!CN)JXaLZK#^g+3|NI-5Fl zWWDhrQWbT9%3Z`rZ_t`tx+euD#ht0bPNK&gq3$#`j~>YpmZXKcOK*s4(n;th6lBn1 z^lnyYm#JwyT@fh2OWEQ*X~Khf)Fy`j*?$tQijSU3CDnLpDczBR;EnC5E%sWxH{(sz zNWsJh5b}55uBRaW6#g=Ve$|TkkLVp2>_#pJOZ*@WC@go7s%~>Ugd1H#a0@t_xqZJ*?rW?`W@UV5-2 zAXN?QU-Wo!QWx*p;c+dIj~F*%=q3cp+winWjGT){5aq}y;ipb*5<_BB zM=c3yF330R)-bxu96OTOA>3o&-Xo(x1B~G{w2$C2F}_&)$OF^}y zF3lYY6+E8$WkRj_Y+G@JN%law=vX%VR6PH^VuJwKa_gW%;08U4Vl^V=4%iIPlWx@m z_~ZXyxbdIDjr>=>_I&L<@wKw2SBG8ReM83X@*OOHWtVzn+KFBMWAi%f4xC&*c6ad1 znpbvrJ-_wD?%OVX-r<{WzwI0Q&Fwc&zVgi-fBtsjo9+ykzo#TAvwTm;;8?S#htYR( z4?;=#dwW@imhTN)BWw2d&Z#}Qw~xI(|J%O#E6TquEo@x#?VUw$o&5GLxjFy4e%}4% z-<1VUt@*Bhw`(WAySs#Q>>JQ4GqP`BUq|D=z7QZ!^>Ahz8^86vGMzn6>pvT{@%&Wjsv5n?T;L|Z|14S1NYCqcIvtUFTkweR$i zse8g5k51b+bo|lUgOPPdryr?3eRRgL`i?)&Jh@{0kF(A+uKV%9^KYI0@u5r29e#?fT=77rlMv_!IK>!V?R<-%mKPDDd<86Hj)#e&)pDl4R$} zCB4Mx$)$Y@noch3r=C6e)PUa3Q_F|k6Fs$J__(H1Pmh{@_S7?@A9Mb^vi#}j&(BU+ z+w}9Qinq`H{M_X2&eP9N`#yU5g_%D$oqlol^|Pm6s!Q&4X7xOA;+gn@f(>V0UZ|cs z^U9LmozAXVe$T|SjVs4(IJ@@w>F3U_Tm4w4bL-bUJ@H)A`n4O*ZP@tsxpS{>-rnh# zja$E;_{*kuf8OxRYad=e_si?sk~^Q@{JA*k{FWUB|DP*0F1~;M?O!f_aA|wzUqAfq z`$@lkbo1v|fBpE+>%aW^3Dc(N(zc|mic1MY$BmahHTutA`pn#?=ZUd z-kvk#{N*p~^NX(R$bY8d%Fe=d8?Ss>^v?M!U&&t-UH#g7pyKMT!0C-wcXzvS{^~a+ zZCt6KOa+upt%H~sc)KmUc_z8lcTb#32}VU^eR4mp{&|d8X=*v+LKr_Q$!6?_B)j zm(5>vxpjW)fvQ^<-aY-=t&1PtxOnT=ZEcGGy!3fi^`Dn_bbS5KD_{G6{qyRcKE;3i zwr^PVU)K(ffBmoPM`rx`*NtQIi~qiP@|o(te=oRrllz~-jg;u*2QXj{jhUzZyDZ?( zms^ReK4A-1pX3Vmz4uRH!2i1l{jcQ$-~PyEzSxLzfrY96RB|*`kJ86)s(xS4hY|tR z8$`u`Tg68VZ}qBvw>v2N@gF3$pS5&Te126{@Jk@5`Y=?1wU~XX?D4QBs}B(5fKO&E z0mTjD?**P8AZkT+;|Y8>uU`oOATI|4_!wih zEG6C`Anl4A@BwS@^}78T?7Jn3g0BRFZco7P2I}48=Q?-u`4m5{^r{}LC+Jt%?sz)x z9#8|m0D$#wfBHky6g&uP4SKM4w_e7e$R2zvDC5hjhwIwS=T(%TSMlOYSg9vq>R;rQ zeVza+A_8(y<)-7NieFzPVCFub>T_i4p9H)LN*S;Xvg$P*WeW-wpD!TeL8?#ScS^28 zRS5*$vaA4`$cnvu*n|Mq8T7loZoHUZX0P?~VikTxhQfr(4EPkEhZ_cfx9nl(T7r0B zFyP)O`+Pyg8}NF7pI6-6H%2^LX7jA7J0Rn0rgezNOCRRQK5UE!)9OxiQ?XhVr-65t zRb0d_#f~a!z#H)69fL~H9Z-N* z{s6uu2i>IxpWn@X1<*FiNEFkP2G#8gO}45kHp3TWpY&q$Wt=w#40KpAnrfVP#X9aQ$231u$ zZ2)O-wT0LR)MEHSbt=jlJbXiu5(EKwaKm8G$6sKS(1?7{Xb_oX)$4VP4|rvds$#oU zkeA=%b!1{>gCCO#@mO!!MjWKi9}K9Vfq>r&AVfF31TMt7aZ@aepJ`A*SbyYzVnMcY z&@=$+MCE?m zv{Xe7`rOeMx?wB4AO#SGU-jae(mFd>Cy0l+)zBl1QhCIJhoWkOb)x(M`{el28Oq05 z`rWDzk9SW9y=n&gxD|d!5xyA2=K1|NVs^Z90^S3RND`OP3YAGMEbjME3f1xV9R*+YEj{-2QJNx#d<^&^WjX!omj`0&b8K z<)-nBN9tqsdV>&)p5}g5P`RoA<^3VR*?wQ>7x2l)IpB2#6;mwJcFSPy%fLGgQ9RG{0EiC z2%8(kQGyTr@_S3{cyX`@*ekcYp-4q37lgJNR049-1nk4Z1~s6z>^DFLhezUI17K7| z_J$rnjJ^?s1YQ7#fEG~0=H;6mcVLc;8`9q|a~X+b$88oxg@6o(xU-L0{6VkJK2w8ByBcwd-gu({L|6J$5r_&h25bBZ=1-k@Rz-nVHrXTe81~~&D5OjkDRon&y;Eyda{Qxlv zd96aq`Ecs>=fV%-kg!F?$Pbi0iC0#kS)gSg=UI*~XOsj7yoWn<1<(aYW|0hCiT!}U z^~rp?UajIwG$3IVGee5{RCN1~JQZ1J1{niQ(LcqkLV7B~V~Ec9wV4J;g7}?gFYFpw z`o#$Gq_jK*lA-Sm2uSwthEs+{m$6o;T2)nq*LhaNxJIxZG_ewU6+kj)#3JLg-GC9M ziU~dHllT{scZn~6op(lbiwHVb_2Q`k87hSvAG*)z2gkI$V}>9_X~EjiZTJ-m78(jh zktgUb8y0&S05{)GzAPniJs&~@m6$%S-_H(pO+GEzLAj{~7`=RGM6UwN z1*T0u$V(cZ+AH(o3(|VYCKdbSiSJ zuVb$jtOTnAxJx78)xd;rp|?{Dp^>1rA)OU<*9J38>VU##Ab#y$3JIgc;^>rRfc%Og zI2}I*gs#@3d5;m2$%n+VMS;yG`}jlJ5$%z14aCPwpwZCnMpz+gA{q66Zo0~A7{vwn zDR0IrE5Y3BrW|k`-qsUnDck}ULG>VHI@F36V9)J{u+ecK?2&gf!V2}4%EDQSe}f8C zu%g519B;>dq4-vH%PtW$mwq`^ZHAfTRhlZX9#FJ`-d-{oRN)5E0ixk#fe?iRh&uQ| zc4$Co6&%9lJB=`ad~}0nTcodJBf&Mzsa6~elwrW(7B3q>)DVLn_LImAJ4{Czwxl~4 zlaR@AO#(;7N1zr`ZE*2j?YJR!9A;MT?Iu{8!NhJON`qxCV1~J-04bDafI+#}4Z4_Y z0=c-|?Orp0c_^C7_*@YBcCMygEKO|Z#2Vqm$2+a=BqQWnLbsThy7b$Lnz6l1jCjpT6xehRH_Ri2< zDBCu@m;*B(YCjl2?@qr1#m59p5-b6#F~IOvxkJE_!0gklLJti~Gj+3iQC<#d55O5cV+G6F)+Sf~|zi)cRdU zSVF$~c3gjCAb-cyJ0EhrXd&ur=w$?{$+Q${vY+{G9TX|% zl`w&UDhiY-DnboGoO*qmvgunhEO6`vY}SUw5ICVTAcBQJ-mZAZ0v!!ENVuBnTrS-S zNE-F97(m7JE5EZhii6x9trLhoKrg0;Yn^S)ka&JSt;i5djKW~Dz{G=veqnc!?#94; zfeGXd(p7!-0~BK$>XV6TMNg%m^%Fp1-rW1sjc_o$bVOjkMv~}EYIvodyJ+C)S{mJ! z!4EXTh<2lNA2KJvZ-s({HiS+XZh-ujz2WEnu|A=1+;lJt36rL8L#Q!PXb&ZM6=E&y z0De63UeiSbq;u(g)Y^>TLXb9D;5h`H!>OaFbFxD7%)kZ)xHS|>p@je#;i!Z3cZTB( znX;@NNIzJ|i8sP`g#fNn=@?*sG%1C%fP8KlQ1haz&$)cTgDE-iO|fF1XK5X(VpMOU zdy+p$Gc(wC=$v4L5Diiob%kEc+XO-b8>z&$p-F)}5OwGZeP%R5(#q0(Y_|~V0DZ!X z0BZ*n)$~LmoLjiGh={0a-X=5kw8CJy1rS7cBd)z4<4OZ`o0&2xx;54VTN7}Ldj-d^ zSTf^n`NizVJNw1IEvP|3nXLQNo1l1)qcdinTHqVQ9HYljbH4~RU0Ad9ArLkXQ#_4- zDEnRKnX*V>;))s0 zC=ycNLDLE!17<~Z6mr$;7Y=5#Us4XVlba@_?pUkB0qf4i@grDw zH-NeTL*m8(7XB>3_avV3hGnB!1Y{sUdN3SePuUJzq^T{ZIz0~MASd27!=?5Fr@(&$ z30=-a3kx-Llzln;sWu+sIjCKC_oS*U-KwSUVV=qZgVJ74it3 z?dJ#OBZT9JC6Cwy{4M8t6yw1QX@KD+H?0y~WQK@_$ZK!rkqK?-j}VUvr5f;Y(_zB- z-HD~JBpuH<);bK1BQVqen8@R3**77x_#kK zq@flSni*i-)GleQ4RAH6UFwBF;+H188>lJt9XA#r9BAVcf(6kggp6{V+QMGK@i%0I zzFRZ|L|O+p0%ot+v)PxRGu4(WD9*BbOiO-w5Bbv65GKaG%*)y0G(I-G(HnPuO11Dk878@uF83s1V`9Zw{72q@>*K|d%g1Ifn zpkg@di1ac$1Qx|bYJx>2i%(cE1KJLs!;eUn+aruN!}X>{JOU-hr618@q@C%H95Xmh zQFFHf%q=X)1sV5C1HuY8o(?HASx2wib@A+xb5~@6hKDoWdZ%q13Woypso9LUghw^K z5VIK&nHHW;$qnbwA24iCCj{N(2AE!@RVfD8(PR;F4`a$M;%>Cb3So^fjf1WH9bG;Y zg?m|YHZ@~8D{@-;H@=bhP2ZazNH@xCYj~hI6qvhI(jLTQvlw9(Dl|5i=3)YIP`uuj z=JRAtM~P`A7)SBFhpSUW{Agvou#z^JP-wCmfFJJ)rYHz$3qKDQ0p=ioF4D6oK&jsZ z7dpTX(LIk?o*7~c{h9`t1Y4SEy%A1_BJ@lv%M0znwq3W=gNEEQMnn}`<~z3HY2MsP zsb*vfz%mgu&;W=T)qa6fSAi!BDse#dbY16`)6W}Vl?XN1`<6#dl@_>Th@d3~!grR# zi(^xbeq@9|5|F|k;rncK3<$DaUm`=?4*jBKpD>F-Q)5>&F`L;ZNw)kk}D` zHbc}80UZwX&>d7a3yZK|J<#sJBM0cb z1l-(k1l{P*wonavk&z;vu{!U^CkM!Y;EveFK$V#}@C^~?MhQ=)ncW%cVGP1Mr<;b@ z71koW3qYpumBoNaRQgsNSX9t?=JXVk>QxW{uiO4C(((GIs$ zp@I&KJ9od4uIJI$0GI@&6{VljUIAK7+h>L+5C6Q7kHUXMtO1yMafQ8cyd6O|*m($} z^6a6iJ63XK(0%h!->2+tH7J^_nc=vl30wjljsO=xl^w zVO+s5b?{5*FgKehpB9^_9naqZHxdyT#8aRp4`*>sIwXNU;~l`@eR9Pdi#CDEMcST& z%4hy|>ec}U2=z9>^a$G8iF)W70eM6kGxALZn5NA5S|k+mryn&S~DY zA#t2oodx^g%e9(?dx5~Blq^?kb~(Z!h-T8eX|B%LvCR;I`EgaSrf?S#J9E=$f$_2r zg#*+h1#THk39i^p`;Y0xlCGR{YSU0WTQajH5hZ6nmIav0I2>0CF5i zq{smWr~PI|yc(YCT3{;0G*K7;pIV9J5$dzXvJdMXj-!lXBqM zc0{e&M{R|kW+)e5@WK=gT`UUm^c}{ApJ)Lp05R7x@iOCek6{%4F(w#W$*2X((2xet z(BKa$-m-UdAZOsxqquyqjo*yEOn#Vy7=Z#OvyivwLD8|fi3BV{U#B&h8VlMki;xEE zRiVYY-f;`E-hqeSGGTkB>)QT!A!b9$G%m#g8xN^EVNM=G#wft@QJoVZJXv}d4L9;p zMiZo58=|Z9p@kRdQZB3u4}ELo#_YTX0tmoqZkma0hG9uF9lH3-F}WTS9!7;LaTa6oOK?t3I~KSBAC zrl3k5N-`^rk_ATI+XHj&Jv11z?vcI|U(t(C*U1h1eA}k zPl8USz@1k-r6(e(QVTVwL&n|AZ~z->(<|&;ZEeY^bS#Y?cOtfAdyZ%iVr>Yot7^-v z%9sTT9Ns^s$wNUXi=IN6vBC}}xn1tQyc`645Pu8G2@MgM-q@?onY0q>QCOIb?88Sb zqZnT(8ypK|EY2}5LemmIJz)alZJKXH5LqQm!~p-a67t^a3=I+=B^Y#_bdQLh64<_q zYy-BEdgQg76D|sp0+SYDy6`We-W-L#E&_o(nkj{xkQubN1Gv4CXDw8cZH7OHo8Jrj zMhPF_3hAM?v^EbqSRHy7UE|>>DEh2G%z;XybLlA0F6&EMJ6Q3WAb;6b?3B90kB50# zjM{BEH>$f`gWI8p8Q6f`%`en3!eW8=RI1IFX+d@X1|UY(A`68A@jZ@B|6*k6Cw!$_ zr@Rpm8=Yv09v)Oo-Q%5M1bI~r?YnX}3yFCaIA%zZ)AMclhfwi?U?MUkbR#=FSvS4i zWVPU)0hyk#@P#?ZG9mcv^~I&7_h4`O3D{? zpaa((-}^H?b^x!`T^k;I9nJ)O4{W%LdRgmhTjmHybYgodC7KIu0|FS*G4=vZa5#di z^k!-f>?NcLeX`gjXjbakj$X`%XO*`zCDw-|NMh9|9*|I3?x)_xSi8$6c5qSD1mczh zE#nNxrHGFTbRPDfO!G4U8qIxMkBixJasKUf>QJM*_2hs;LkkU{D5qQ3q9rg%ee}M7 z2vSqJFfm6rhxuY@U*mRmOCe|jAsPfGffA`rycBu{F*gK~0^-f~MF_bglYk5{P&vC$ zah14J@3Zk)X4yGV7zTF)2~(Qng1ZZ7a$8zfh!CncASD!o2crrtHXO1ycbyIu&o&{M zgP4~BC@+Xb*iJfYEl;C4$h8FMNe6ECw59p1G!H-n_%#TsTr`KC(v6%mJbLK4NdcH& zH1R?!k*+%66|9DyE7Rf7c}29lJw8AylTaFcnq@}72a4mVBB`cn8NzYMqRC_++dKIV zKmDAipVuZ-YWv*bZxArYb1Td_NU9_7;t%paiD9^=s1^Vh=JrF-DCmnFpjSIV-w3OX z&I>s(TMJAbIXOmXS1-NNjtS(>)+NbKn6EPJ0KK?PJCF=R-2vX~ z#r7y&ooSLnWMyjs1P=4GJdz=Cdc@mKWV8A!wJ z6SFg%gYoC2d;?_ll;L_Cv7YVqN{zmJL$BC=N_Ik!CUulT^mZOiM-^~eGty{&B|JZt zO0D(fKhTASIn#F;AdF)>;#-&i%*Cy^H`0BG)TTSuJ077%84QsJ5jAVN5mHZuPmKO0 z4gUhg(Pb1p(@g{=;SY#^rE(>Bt$uo#FR6X_P7{C~um?#@b+xCvGO$P`e(=IL7CASdKY-h;ash2FZKgm`Jp?`-5b32%y6&9v z6;***V#q51QHD#pUEFTc?lQn6w5{qwsVRDNW&y$%3Q-qnjbk7Rj};a(Gg zPH&~ef!ENDEWA#^??qPlHDDt*twD7f-h`aRP|05HQ3JBgr=Ay=!Gc6Na~K{PmDt(w zgt#x7d>jzgWN^nRBZVGS^#Eay{N~jKCIXPdhg8*xRO*n214snHoKu~>xt|z+f-?fQ z;pYdyOb^nLJPX1vK(`?N9PU8xWKh&VUv{8HZ5t|J;6mISN}*j@um!H_U1=WA)ARyK z*9iJoJKZI3yc+?6?3Z*a;et;{OK~*eqIU}*=C7dWKVXAA@zM`^D}j%lwh5`UF$ZBZ zuXGx+E%#sTAvy(zODg4_)3J<@=!ljgl;oBFJJdZLh%B5Q!hm}4NJoqO(+&H-Biu_E zHm9znRZ7velGfCX2{CLhdm_m)WORbFmYea{37bPq&a`7FFRU~7VDggWR=gu=^rw8` zn$J_R=2wxOu53Q<9WkXYQF)(kq>{ zrf*AU?MvqgcgcHuG5x!-VGG;W9cG3MNw3RflO}WGhzVP`toxFVHD`^UCvCTuFW0tb zPgoP*ZmZbZvc2u(&(lkony*8B+OY@Ix3!y@v}AtnLrL5EFl^GE`R&<1*}e>V7Z@bg zGG>RwCYhPMhNRUy@)mlJ_r)cb@S{X1$zRs5N2%lKQFBTQR#ga0M|RrrQsyOGTo_;A zNpUu=thtk6S1)_K)2o}&TYJ;H!nV%16<%t>dgqu>(&hd00#p1^(y_0K zxBc_iq>MWGGxOkuZd|<0y#t?lzAXBzpTY*N8og|<4rR)-6poZp&J!dy(cgwM8Rlytk2B;U>amNej>;Rts#f1Lm9 z?(Tp8*?IuO@?JCgkPjHZB)79LJ=lCp-M_~q;8o4N{#kQ+sW&~GIl{6HxO+=o@3vF+ zE$xH94rJ_{d0Ag}VXr}@?8<3_n13VixjloJE*~X9-0t)aX4pMLo+@LH%p2Ul>xIt; z-+jkl7Utjc6Ie@T3DcXKY#GGn1CamEgZ>AV&kh*6V(2h!#fp3Gow9$$um@t9PY-94 zWO2m9k!Lfv0h^y$H*(UJ%#wSnKH`Rrs^QGrMzMQ_K6Bp;4torg{xIgD%b6t)Ji=Z2 zXS~92W*#?SIKx`S5@x>i^l;{J*Y@GeqTqFF`O;E+#`eRH|Bc)q!8|wR`)9|qvy(?o zczIFpkxb*W(`^%*n%=fe0)?!a^m;-psd!^o?|YfIxY74A@3iPQI0+y>dr2&*{`fEa zx{WhGKl!utVWaTK8KamTj;s>qOK$Th<{w-?TatAj!}i5zY`-j~c0V^72!Cz~KAZ4P zyBX}X{|)nJmkhg~`S}I?y4BCT_#m6KIrkxUs}A_*%rDKpx_8(Ej6S1*`TguW?H~P% zJEnvF&7&FCVBJ>7CY6k4xPPWKB(K)t|JJ&>Y&V0ZQ%K1?p?_bkUcBGRJjRw=>;BK& zhB5f7{eP+f@ax7MW1e8&y_vVr{iSsqN&>8Pi~PxTW1sAryt%{T+mnxtW!OIr@WY_)Qvm7GKpk&#xV(7^1ZKx2)uKPa&9ZFWR3xWmDMpn%#SQXvyUcqyf@Bd!# z{FE<)9bc$DG^^u_v(CNO@ui28kDY#rWo#v@Swq(e%)e^|l64-+-!OrB!gFZ-D~ppE z%y&qZosBC-KImNgZ1Uo}gj-152{o~`4nD@@TKZ*HIC4qV4X4ZID#g>9Mwzl~ngnO&8WB-k8 z-*A-`>)vMzuD{f#;Eg{xz2*IXm6-n((Ee#w<{KIPGB2V1zQ8aosYCR}?WI-a)m518 zt0m)?o?4Yw&ZSrf)lS7H9BVl)9UrOTD~%&D=$#*Hz7sQrs@OV9 z4}IQZ1zTtxIte2~t0L?K>ktjIo2#N+_fbP?tD;zij@27l6`d9-w-26%Q9)C=LhA$! zYQW>zQcItjNMvk;O|kZ_`Dc77yHkD;bK>lSB9m*!R7A&`4r2(X#N~u)FgIXYw2CV? z53a7%D){Q0J{a_nE!0>i# zguS;mwVK^;89XCW9;sq`<%FkV24Ganl$H#@w1=r!RaC3857FluPOGk&mC9xpPKZ`S zCr`$XOl`w{iIL9QH0NB)fa-r<`n8-swb%!|gyWF4M{UJ;ttRRqYrn`Ct*W|;{V}IZ ztI%d)=cKay{?#?(sw*bhOQ)hKL9TUh1%}{ab3?|S)fLrfRoW)hc0i;;tEt6waE7@{ zdr+G+Vd^xkDv5o@5UHtUcbR+6np{)MtulmaD%l>gPBCR#EfyNBohq$2j)+u7xH;CI z7!X}kT^%km55=&ya<0l!R$Vo|x`O}Kf)&irX0Z<#OL0K#?dI_KS(B%+6O8?|Y14T} z-_ofyTBITaw;fW0eJIz;?V#_HBEz&8s zZ*^69ZH+dSHChK~HQM-E&GBYoxWYbI!z9!4>KgQB@26FpIxv$m>vMWUD#k}^EA8y4 z{77Z>*cxqGbgX^gPj-9tI6EF>?r!Z1_FP5KvcS7P}X!P-bgeYv@>hJ%rk za_)*^UOm<--D~a_9WzUlstu(z5ov%~XQmL#m(?;;*>rPnt)@DXp3e5Lj*M20(Zqq<7N)OnaTJ-{Hf<>66=Q0-LhtI}iEUKn<&)l6bL7|Ld8tdUI^%BmlXy=a(H z8(}vZ`bWn)pJxjULu+cKwQQ;(R52#@Z0J5qNT&o%+Gx1DQghgiLmwOyalXb5wD;5| zN9^}RYRb81tv#xzO`V}tIf_cpv3=RW*6`Tcf6ijC^AVIE;2k2He2^BkRgow=$PVJd z$gs%x8pjBx5BqxKx*~luDnZ#g?ZU7;XUhD|^nXuQG5qHhBUZ!zensrXwdCKXwf@hP zl{Syo)r~G;EXFqTMlX>I#a54vwoeGyS~ZWpTvC#o-m1j2azH1sxuk|4?aAsHQVtpe za}%TYUY?!8>|ta>L+{qfbzZZ9X-s^Uf9Cm|x)4{_YE73lIMiV@3F3Aw)QN`+v4lDHg;B(L-_@4l`I>Q zTeF=y78;X<*qGHB#-usy>eUCi-hrp)9cVqeRUzBcbL^(QLmlHb{8<-b`y9_UWG!nM zUGh2m_+0kPUWT#GNzAbutZn){Rz9XxVOM4i2n)@SZ=Tf4PsRcgt5j$zn3 z-eK$4Z|&Y5s#O-5-4mD{i;@RDH9Br(*3S8Kj-A<-pUTwPWYdO%eI+r0$#4DLXz5_z zjh?4U3aqA@4*O%J=?e=i%(1RkH)>Un0-+*mh#3%4?PZVztr$rMX# zW}G~WZ8ev@)9y45!Az)2)@hC>VrRq8Gf5A(uGsgR)yQm2G~Z@j`Em9(hO-)1&TVsy zd(zaqaCZ`ulGgQccJzP&hGgT}+H&b5lM)_|xrr)96W6U8wb;Xs=zQVqi*3Z1`HE*{TZyU|bT-hhq z>TZwKaDTmWtZ;;J&)K|VozTkbiTR^f_hvf{f2^0*_T-o~he}#y^U3S$yOtc>IkPU> zoER?mtIL`=Ae$M$EU(yVH5{m8KTTS~dQ-Rc7~QJK+IqQbOl!L`r|H*t_L}4kv8Dkt zUA>FciS7lNhtj{_<_cw&&rOVRn-|#E`!{$7nA;T2(DuX($p!ni&AWfwz*cWmjg&)e z=7yNWp^{`zNxZP3^`OG-V-)M;lgmP5ySUlFwCVF5g$t9IZH)ayoSECV-GBwpPHSyx zEl=KSeC*Rod$uLe`up-d?n7;o+6|u6!*ze_AqTxyd+%2DQTyaB`30dwukThR&%14- z+}EAldfP8<#+Z_uRAYZq`R>uW$XAL3#Q+Prb?xOYcB}YB4Y24$RqSQ~lE#0Sn}8ZJ{Fz~$`JQ5r50?a# z6juG`nqGQ~;LgaZpo)s^n4=VX{P=s2LrkdH!xrcheUJt$#?`p5$;6_4gpWmP3L~BU zvguw#kJ%{jM{SMXVua1_#uHJWU(EZsjY(=Tkk$b%^q)JSWb8>EFb1!wjB&GNjGrqr z28&TNSjKq`_y>1^Tg>?QY3ODPgdUHls$37o&Cfx9C3aMjr`YL2tUrJUt6nx6BObcA z26NYO^*R?4vRps*5jMu@7jy)uza5YqK#8zr&WHFKYyp~CxY1S)T^2F@)`vIKpTCG* z$BuN&Fa+>Bs^?S|-{eis97MYMGd{=LNxovK$%%JxhIo&g;l^PRj%keQyy4nwLbE?t zTdo#x^=7Q$2paj!K+=W(M%sWL_%Gn27~b8(_)7=j_mF<8Pl2nNhFt3*M}Jc!^E7*i zHzj$C+3QG|`COw){apVfrC4gqAJMnP8~$Do9~UcnA5>Q@`&rpb7@Sj#CHlBGOMCI- zO@N|sB|zaj&pMYdvg>V>JlH0}vUEkomy!;uw>UPE*{S)o1OxNpEqHSC=IUm-?-B;O%kxPqk^$lE8 z?QCJ&hCYZCm#)Ak8?c+lqUeNK?3P|~euqlF7lMzmzo6YYhOBrT+oOFMwPB^B7Z&|H zE+1L{wY0?5E4-R}3op65bi3<*?D8tK6p1_Mp}W~&$2>-HzQY8aX|9d6me2wxB1X<| z^c6;5uG^IsdW){@VcRZ7?!l4x&{pPXL#nBXuSEi*SmBO3)7T@7Km2Q1 zC9i-g#xpAV#465-*nUiHU~T{49hPqziOw3wHyIlaF=}{d^bNeOV+Ip&+|PJ-qNpYq zwl7s0Yw@^f6YwL^N0~tQBgPwf1N$&0NiMFh9dVn~yR1vYH>mGI`G8tBnfEzIORW-N z4qY4^AK>NoMk3NJ4t+TJxjU~ok2`NM!LpaRy}K?jz9}Q?2b6xfYqe8o_?&-^Q5Jpf z0Ai#RN&V6eCiD_+JkKbO-Xp$@^(cLU@#4RLV+Z4Rj4qwx`T&~=n#F&rGXO{wz)R^( z5haQm;6IY>d8= z6f8dV9wKiwg;w+%7xW=7K5O8kz%uP{DNcZU-OnX&!=V#c)qQ^|5 zR)*H(;lHpYpIjKZX&+azrNuGW^?h_(*^UMiHdJkx$jA-%F@Em6sV#o?SmZ1H#MR!p z0DbWrmhOEf0??5-(a_vMdsBLigD@yPv_D1q6)2H-CAyo4Kz1vopYU~zuM};`B5!CO zsYhZq6O5jOEPJ!Li*qk_`q6X@X~kdNP0mGk;c*Maqv#VB{=oSe?w1t3YfBWH+&6@2lBcnFWtN+e@aaSAh5ZYdwZiM5EZ)Y_VJbSI| z_0qp9?{Enbn0I9lgOe^dwJ=IUFGe+;rH(1KJ5nwpLI8G24(xX z8~iVP326FDaMZ^G?~FcbqOx(7pGL3YvpZ0kxLOa^qa9r8&L8++=x{1~w7&JJrY%=1 z3pYtF&K654Q$Ro+ww!hhbD<9+Kht&$9h`**G<}Y^_>Q(!JmcqgV1pY>WU3I|04X(% zZ@7)|?))NpoKc!qId|{Mt+bRa&K%JASsBn=W&f+aHxFwfdlrVf;WV5MNk{^;5D^n3 zNK`gg+!G{7R1{DV*C9ZFD4Q(q&Lk>IR1_5zR1{PkKv7%=cU(}=WJJdWXBSlcA-&NE&mda47(FKmk@OP^5Z%o2I@vq`|deDNGEeH2;` zQ5O#F!3%)`ts{IBJILSirZFq{%mSYJ6wxo6EMCuZk|m0V+Cuf|C->4F5|;|(Aa*pY z3t&3hhz@czEoa!BF;2>hV1?YTgm1YWuQ!A-2d<2vza@GaEsO!;L$;;EK|wrWvLLDY zyK3M*&6V9(OLRL!?W*?*f9LhoM8l@feS-sw3Fa6qIn8_}b{EP4Q|2k1I2U9S8hCBd znQr85kvkTJ@nAkd4U6^|PY$z`o;j=Z8sTf?(2vCtHo-4EE#yl;xl}vUZWIl*g|N?= zTX-`{@39Tb4@AGxByk~^v4DB6T5H`g&!s@NZ18#s=P^%hi9R}ICRFsEEmwH=b(X3; zPufL}MTkgN^bN@*RYq z@QlcpqplSP@k;w;FJK~}NHQb+y?6~CpPUJ8lPCtl<58Q&L(oL7@F(fmV3>?kx6-|s zTO!*1S+qJu><6uQ?Im#`WEz3lqg`%fELs$ zjJDzq93782PBR^G_AvPj@m(8TSLI1%xVCHX_?Bj~v#iafqiylP&?K5ic#Fz;Ob?)` zYT5cU(Ha3WoED$4(YgZ_xy*RZv;9s#YIZ}V$q;_;`%Jb6gsm4)yLczjH}MMpnoUa; z(W@2=r-uiKo|KbuXz(`r42TCb0#vLx)y8PSa52$0^qq3I`le)-+ zESQa@W9Uu64Kw$v%IU6&8$dETikEPCEzc6*^))7fqpmTAs*{3i)W1TVn#|a7lu2iT zsPjbc(!Ma?cZ>I%OPIr%D^S6IbD=m7M~d@o)qaGZZiD<6+}pb+KUBbk0KviN^fkje zt86zaT`cD6`}Xb#CM^tY=#tVUsto8Iv*R}HBXwgnQ5uQsnJz7B<ccPYehHv>Vg=9X_i(083 z@3O>J1de!qykAOhWgjtK5tnndFqi9{sRZxU6XMmhdL(3I%%PywlxyfW;%9Uzs4ewn zpkWKt*-yG`&m{Mew6wLf5CU+Wvyq1eyVYV|nHo$2daI5vmSgph@_EcNJ6I?!Zo)HL zwE7lZ%TF#N_}bB8*e{^J*1#?Zx5>BFjuCb(x=-_hM-qMH>%=oU82d5V5caHgJmI6Z z-e1c&@PvItiH^)Fh*E5RC95U+$Z8ogIOAmY70={}qiLN7kk>(rSfj`=vP@O1)H z-%q_x-4D7J@}KF|<)T`R_`N-JE&Ish)z4il?5JPNQFI&cE;AI1W_RF^(o^N^`T!Rl zk)ILIfG~2!=Mt~cJYKr|@UvfypItsNssIvT@3d2>x3ydyr9KbuE^$2{%U?f(2Cmvk`~eJ@ z{nvmgoo3K#I#Az{v!s_Pp1FqE#$!6z z)ZTZgb;@jJ28wDg1g~g`7B9l|9bW82H|1;j=W;O7)!&E*+k{Sfc-bsm<1hS;{0_?D z^o*u?*=6+9L`(Teag#Isi+DJ#c67PIFqZ+}+C=qtjES#1%4={vVoP68w<$OA8@Q#g z3!NHjS7tfUTisPnh`Zy`L3~apL!s(B9&`dSQ?eImYl7=qRuO%T4em>E@$Zy}ODC+0 zHt&+Z^1KZx^kS8Ck8&k-f_9s(u1-8JBa_1-19+pT)H{ zEd}C?PFh=Wl0a=E-f1uYEbe2we#Dtx>Y<_wHdjj$kI`=vryBEJtUAbkh22aK@zXiA z{g?p!Y=*A8_z=fovs56T1Kk}mLsj$@(*f4|%BC`hIJ{W3y*SSqf`axc?M3u4?f_Tq zbFdmpHpcTQJ=tZdtty#WPcvWhnf@TdzN=xA2?lm^@)af0@_1f4{rcG!+MVc?Il~t= z{g93ZUEJU6YpQ3ch4`sB^aI&TT`Bew8dbp^dzM;L^)8hiDUdX3sIe?q#|+{#5iYLR z9aMLj39w^?Y@(X&?hcj5ni=C0a3rHIhpyDlN$e&+Ew*%IUfM!um$}bZI~je{=JM}a zz)?ojYD=$V(%eeVmX{KJGOjAQo?P(~2{lQ)olnP!4|S050>6*`qB=}JX7<~9uFT$7 zK3Sg5yP|G3&m;V1E>e|b_FzJ>_#Ix;D$>!6C#>tFkGKqX&@Q<;7W$vQ4D^%xsY}Ik zFz>E$9IQ19;XQ*`)1~n)b9mIS^2!V=&0M1dtj;*&LYE*;=1Gy2gz$Z~*(K3QxJR|u zNb_bJeZ&B^zvzsuT5Y^6KFKj=LA*U^vJY^@-mv@yLZyoy z_G~B}C@ye(`L8n2(-T4YM8ckq8j@bCm(YKxadMXhe@Q0kHYX3uKQ1q}BH0#wqm<6C#xg z2DL9N5d3co3sB*jrD7zguomc9e(*%m)oo2C9l}M0c?SdW@U36LS8kR<& zwinhCgFvB|Ag+Iqbg45eh%kB_z+JNaUShX3K|N{N`UrnWH)Rn`<_ijI-M6GS)^;rv z2$p;7$bpS9cWqfTqFdRXvm?4AK?A4jK6W_)%;;A2I3x&R*A%Q>;dQ+A+6wQ}?U@sH zE59r!K+eO!2h(i_7dC%ZKx4};C%76ktg82QHrUQevK=K zjqd+w%i_@kp6oftC!Yx#AW)w`U`xI<4I4A~tsupg{1UBCP-jcBD+v@9ynqm;Ri`B( z{yrXu1jBg!UkgYG*(W6JLdOY7X6lmAKyl*v(BUo%k%*KUdy7bvGj6TAYf@f zF?u2D%|`SS3YeQ`4^n4~K;S z!pKhKEuntutRE9EPMv*sq4l&mjoHJe&3#m`Y}&jhm! zOP!`yN|$vKHT)|CAS`Tg7p1r^mzPz#uJAn8KI!k*CA*I(@?UWPA<2->I3~xAhWUP# zc&^)8b+@B#>w-lJ{W?|(|A+_}6?D+*@aHO6h7_iJt3Hq@rB`Vt@z2_}^kEfeYEw=eg)6u$%cjUfD1i`Ys3 z(Ia%l8$@*-$&FE)N_U}zOS*5-ocD6FaOsHl`TlMs%A41%4DB4DFWY_K$8O}_Q=A!l z4tzgiMmfy)cPH8Tev7{%2O`1bJ^N!^RwNv_R?@9_h*N$htGI+7)5R_ zKk^eSniy(~ln;++U zpL+M?xcBK#??+bs5lzr8q~eTqmn}Xd$J>;04m@mUo)?Wt@Fv|C9l?LCGoYh%sp;`C zFuVN6K>Q(+X`bRS!M>*8++Xvta2jopzefCYx$Y#a4 zHy%@z5f;jz`}6E$4vP2P`+P5JrxtGCmXfGdr^sURiXy*UpRWMqQn*Fq%#+&k6NQptwlc<9Yj#=?)Y z;`{O06hl#d(uIp8iTQNzWv_eLyyqPohgGk zUHom8(s+DhUYpDZZ|_RUJn-TyM3vw}hb*W{ZbTX6?!&#e7%QcYczv!FrQmkWCzWIl zH|TMz0-ohOq^J^ZcQICybMUOjk5oOzyMCvd@dlZYiokmgU7`x{n|upE?T07<6wkMi ztCTgP=Oi`9V;Ag3*%_GBFzKx1rCw^SrH-J@h7+zBf_DWnZ} zN)(b|d@R0I*38X*4MYmUk>o>4fTcI?0$y>RcNAm_p6^Id^<0M&q*7LeerlrXFd8IK z`FLcl;whov_qT7brP}bVjcrsFP78iaLB7TB6w(N!`z-gtqqT{>X0l&F zO5}W1?uS={vDJ!BwnDMyI8jtQwjCl+RpH4ytYz`|yIY??0QhYWsX543M@#Uts^3)+ z+=Sh>N=**VkKc9508pKU2o{(@xfm2RQ0&8KdGw0r!ER8jO(S>0g;)CXKs`?jd4ER(iPPgh z0!vOZuT?%cEBH5{;x9{ILZK#%ydtZ@1H68R>sh6*K$x2@+X8Jq_BFQQ_XeBHLe#dC zZo}t~a;ZA*q8A*cvK6)DsGISeMQwoZnwrM|z3G#U3><$`1W=CY%U@D8m}~tzRe@LP zIg}$V8`2_c#m(L?K#Icrc~JUC_pD`l)Znhp!X8Io!sE*qUd!Uq_kPAmJoP$XTY;_J zeg=7o==cW6d_dd+N!&~?V6U)EOVkRy^3eU_W{5}D0J8RI2Mt^~oFUZu;OoCxXV#!k zS@JocZLAZq#|*M_Xhx#)Sso= zu2zAHz}%nPv{`uO0}jxu^EdUN8XDYKWxw4W*7)tPlGApR0 zL6Xmz_1HM`DTETV{<*dp*FWG;5*)JQN2oF12ZE&fTYmyou{f3nG<^5Iq$FtEdR-lE zcHlrof(h761%9&}0WO31La5?{Ve^F$)Hl)?f~Wm*6Drb{>9@54JYkF_bXAYZOVG8E zbUf~;u~M~SJA{Dq4?8^t*>QaFma0U{tXhKb3r{Pm8ATV?*5i&nt^v)ew^;zHJBzMm zdZ2DDrE?JZQ%emd=f4K&q`z*Xe2{IQS`U={s~qF6C9v~59(fi4y*3rHG8p#pO>sP4 zwG7EB@z*!sLq%~Jql>^Ngs*_SiQd)#dil2xWGU!{QXYXn^m(8vL{T^86}XLmM^%OA zjNxbma@=3ww{`sv_|54ScA4r*2rN=Af=Km=KKDjp0E)8{{PAfi)j^vLIYF z`cq~E+I*xo1y9Jm1Ia@Sw5$;Q6inA)g^Xo?*L##U0-vw7&P>5o*1xMtaOjOUAo>dq zwrLdzf6HJRr{A%W)u0)3jE*?`f)zlSp7;vn?S1NN$`6T-l~&?$ds*I=U3e^mDs_Vo zm}WIU0s#)!Aypj?SVhW8uqfpX6@*^*hM{e2bov=s^{C-DSs}LliJ&U+KL= z$`NJvul2*h4X?FTI9~7(B;uE;^|Bn)zl3f^TRW-+cwwYKn}>@7-)k#zymJ$9)^4>x z)`08FX{ZffzqJI>P5Qx{*@pX_vB`|Y2A4YEtrO3v$N4E7StUvv*b<2kyb;J!xazwf z75pj=dN!4b=irdc-&A#I|5Bz9saE`c(N&Q6EU_ZSKL80vv9#K#~mtmk0vFs%kf;F!!wGw>sJrBgVe8DeJ+|EBi zNN`-XgQ8kd>Nwa*9B;kT2E@BIwS~g?<;)xKf7hiqR1He%BKN?C7uJ~#sGu___Vj8S z5P=_BKU6#7LlT@Bg1+Z5B{q0yuUC-wmNV8WpvkKDKyZO&1GEO)(~MwekQ`MAesZ3x ztwTRPHDY{b8lfu0&fPu%E>`K^1CDxYA?#7SI$bBgdybG$aPNh-lmbUBzND=I?^0Uk zfohzYRt}aaZbDIOpCDNs>b!ty#<)uvSEj&uhC5n}rGsr$1m5+8ZMgW09v3^}J2v$o z;K#gJ0>&hAfk7+FKSAy1yEAjRcw`9M#4kGYLY9MtJ|Ez@I26?C%XcrKPUgJeKwEM0 z-cu0BxFTyR3k@RFS!ju!8a7TTlh@;YdDc+FkGL>(9DMUu10-9WNU0Q~lkL5cYBtKwW(sEL0K5+gDwPBl&M+dARv}tEv^<+rZ>-@I=`&Z55ha z&JZ@}$XR(IJ~HZcrUZEmhHziFH&2+Eg|4kL=HO$k1hn7dN_3U@sOWXE2RegVLbxu$ zNXeJ*@Gonw0Rf_xSgSy*9iEp;4Yw$VCCe<9A3%kNbV9U{bStUrwa}bP(g5RNnoET|KMc|r1Td+OHd}FN& z!U69rWCB!jq|^_!O5~3Co+~tPXl^(Y!b6e)##XF6W(zX9n8OruaFdQBbHqG{_p%y% zW$k;Ay&FT+1h&!$tLNaMF;7$}D3oAoICy!06-dzUvL~uYY)r5QF4Vtz4(7y~=eD3> z_KKf@V88TW^fc-=kWpY96Z`?%KU;@7RR#FhFcr33-bAQaDSYhNoz#NHYvyP}u;#EWCp))3VA!?}0u0TW+C`0M+kE(bN zQ`HxfeRJV`koPO71sYV3Gx7!;l1;1Xk&C|614~k$RL7&n)!@gE-q;yEFtf-qvkLKt zl(wQ6gE|kd;_HM5dmLH&zH_bFhun z3QS+O>tGN0boeZfBM=D z1y$j<+#67bYVHUCX7`zIpb{FhY(batGadsO_wHpHXx#A>Nmbzy`dccD8#uNgg`>2; zReRt^zyDrbhYpQkO1O9sL#QK&lMKZ5GWTf%m4En%k-+|gNGc0I478>^kh4*r0S3S`<3_4JwmgQB_o6Q#{Ux`*;PsQD>V$jA3ApJ{XOjwS(E z3fi#uz{H)t0<8e8F0akQZ)sLr+1_rcLQ!Rmgo}DqGJY5*Our=y!IodK>SWp$tj$Ah zbLk3h_KCL&zN0Owt77zE>sCGj6N1~}d1eqUBbFR5OC95a5dG zw$(W}An6UX_IEOBeek<}pQsd+aM370KcA$V@q!ocs(ny&s@e|?ZDU%6c$b!zwc_g+ zQD!0bl3B@uaG!#ow0T@=)@Ox*z@1J!E3QYojzc+d&d;E1amkRY&>pz2dQuvMmqQa( zC&aHwA;1`|=E|zD=S$n_I$VA21&Ffeq|e2bXyjfx2;;+_f#=^>{siJ#(hUqG$8WhZ z2`UT$+pWtB66$Sd(h^kT=<62RAng0-E#-$51$S{3EmtUM41Z==A5z6l7?qCAiND?<@F~LbTkSsTbfGK~@j-c*N)zAQ4mw zRBKdrl1Z_}kJb{P41WHEpgq=;2)H~|(M+{r%V-38(V-qfD4~i*Cej`c`ppK^Q`v;8 zP^tHHcr0q*;Q8T}P*twNE?GI)LvC5DN2O<=dZDe;K=G1ktn&Iim%$c^EByz?9gcmKr7qtch1wAkFu0`PxF-&6=#2I9n3#Wme%m%7;`P=JrLb%jgB@ z;Si?Y1|RWy3j}jt@tCT@Q-9{bH`yPI&6q3w9TegC5<-=Nrtn)T(EKZ<75Ks-3#|fY zmY6aL6m=bni*MZ{!EXJEZt+2)@wE-e6rvWO1TWT{_~j1U$I{-VfrxVg7zl5-&F87Y z5QIzuc@R>G7(apA=5Ej12V?HyoTs4K7_6(!L1eE|FujJq zg%0AHZ(8aQxeJtYTcl;NADaFHQ_n*yFVhwHpjKGygNO^XBXX=};vLbgklGaFwWlQy zje5q^+2KpQZiC#%3Tst$c==jBRKCtblv|trrl<|$C(~^n3h;S#v#btZxcUL~dFHs&a@H108-AC7O>)HjD*XL!8XC8o%-iY)wA+ShbwX=Lf#KHZ`vElQ5=#rP@Mf%gKzZO5 z2O2>;zDf8cvkvL|Gc`7-G=~v%Ksmb@+yPxY$~bb+qM(alltjwaOw;X(E#+L^i#2k!aor&|}TXuxi5wqj5V3H}AEYjm29>c)tMo#Ofui3B=7URPai)01WBwo-w;SrMIie~ zcFL}>%7WXwp~_P5H?QErGgj>m!ICACv)C>7i!bn5PW$wn&w0r?Ztv@n=Xib(3eWKe zy>2*Xt3cB8f&hEXd0~)S(RtCR-uKVjg?yQr(^NY|#1vICQ_}DMXWqWji_YoSH5WV0 z`7#T4|6*r7a;=dpVcmPP+_txS_SSUC32?1-%N-Y0E6a=7{^zvY64y%}J9DEh$xC)_ zztnBdp~g$yE6%%K?s4dP)Md}g2iq@uop{~I`tEiy{mDimZWu*01!}vw_IwHCPAVEXG-ueZFd5UZuhv+z`8(nIO~2nHa#^XIvpG% z6C|pnIPg!@Lij>>3{G{)so>CoEylyXtUqm}6*$CP9}z*FjD=lW9b{!$~?3>Czk|HLUv%xB{kvzpExm1UI8IQq#1O zkW@`HxMk|VjX9NdRSrx}j*$$D*Cv8DAi2R)>cKh#!{%V%6WoI1$Xm3A=-~8X$b6FMq_i9d z)J=|)4AZ4XXXw&_ylJ|)BuOH@Y?K>d@1KzdY)ea%j0Fy86VpNo^1kO@Zn_wjz|?yT z1E0y%WN>p$(JF^_7@#o}@Fr;|iP8Ymlrf1qvWQ3}#RAAyEs{E8U2>XoB9W$CPF*J%;ZC=Z z)a1BSO(J*}lT*iyP1nT6O2$S5TIqni3*;A~nJW(z#7u`}t&|P@HIYPWm1QAgqN6iX zV3{j;u>xZNaZQRYP4j)&74w(z%Fvyeg`~6+B zFG+>Ourm{-U4{2FeT7`$oODd^cRd$SaFzNkl1vXl1f9BX-*V7cefL=@ zTLr;d5INx~sP5rP2bN2`v0SK84)nE|j`A@E-H0Ke7?tP7e}oij>G~o2ERZtIJb}%@ zMCbO6u~r&gL$RA=EJ(FB9W=Jg&0z?A%svEs@RPLavYpAU$VHj!IC$Y+gOCmUkMj+%-L#v{$RGKz>FvSUL?OKWf5=j4*fW(uLCoT1vWRmM|l{rb*D zM7XkcaBFpc6duB~(wZ(_&MYkm!LWFpHnzx>{}KC6QSE1X*$lX0scoG!o+Kl^u$8u} z{g~KTU9`66Gz-d+2s)f^?v6oi7?M2#z+pTZ>NmK>(?|GRT=0$yErWy}C%TR&?IN^i zrU#LPNF<{ZBj0pYI`n^aqd|FgbXQAqpOd#yQ(eGYFDq2-n%)djTLde6dH%qL>3uQf z-S-oRGBeS%*Ia^(0M>4&Njvp<*Na>!EgvJbX8PGcB$z|fhma5%GSGda4I|@8hXjr! zWbOkR2xI7geBPOsX%O2WT)KD!c;$tMrxPM%hPk~u`3XH zZQ&-*{e)!Rc!JtG`W=)(lRMgpFmCHqsoE!(8Z`VNbn*PauK=bYd-P6%2#3Db16zk* zZU5HS0w-6&zc-ofD-vH?zkczjhdTa-AT2BrWo3Vh;NG!A3+~nWM_{=2nie* zggV$cI*B_vcXC0FovkED>e|IkhGfWH)YSvYyLIp3=_U7uuz;w84Fsv~%j?mtpPyI% z0m6a(Ee4^%LnLUZLdsDh|6u`0h3G)la1^8-;VT$9f-`C;vJbXFX!IDOqhkp2N1jdZ>$K73kw`Ch0EaazTpT3B3Hq{2@zs0X_h112cfTcoJdVnbbo{fcO5T|(L$=Jv1kxKmWhk+F4aYMhX-i-j6p*>B*Y>+ z6gk#Sggg14x z&@9~$c>)k>oG5Je8~`+TUhnxm7YtoEdwdVSn6HK1113cyyNN>~l+jQ$=$pg(DZ3$LHCk`y6CY@ZMcl6!b&&V#KQCzUlH_DgbY{TPWq`mp-CZoQn872w23wAq zXFx_2s6sx=V+AW#qG9~(RRhpmUXRtr&hyb~A2c*|lK-k4v}P^x>W0>Fa|6(_wUgJ! z2{vq`qowkVlB|(=s2c&{{gbVJ@HME!-qIcNbwvrzH?lt*>3?0}{ZM53UqMP6Hi z7IOy8ccX@mwL_{z-_U}1XCxhtkcvcch(e6bmQIVeZnIjrsbgWcsKCIY?I?QNylk{% zD4OUyc{55Kfh^E8w0!fL%!7essqsM=Z*0^V-QR7Bu zA+!~Ve5}03FAiVO$-2WL!{$LKG$?Cja@fF02)WsDz~4&ntnE>cQSF!C26mn{Q>9A#1{bFs}<3U!~f3VmCM0tpo6-BAMa z=d~Gyq3$BxI%Eu-+$l@ozfU4?N3f*4SG2_-uZm&&DU@XyF@TwyI$Gq2!p5V(FfSrB zOz9^=b_XWUN9f>!$tx#_Q|$*#>w(gTawj6#{9|6Iw{swhae+d~&cBxmvkoDy6L*vw;u(TE9G*TXb((i7azAh#It|=EIt^61E{=Fq^#HpPr-L z^NK-({j=Wq$KHXyufuz9@Qwcx|3}|3kAjB9SJmC8C!7gr1BgLs(W$zW^dVrIrlxyz zjn{RR5AsfZ^}5EaHD?c<-P@=yJX?IWBXum##FD1Yz1}I%eKrI%?WyUN_FP-?Y=5Z7Mb| zGiSw{4WOnIgPcy-L2 zpJLi+I%YZwfcBX;Y0X4w$yW|58(aJQRJC21>Hmyb3Kr+jixOca}j$3S?oywP`gbBSzvLHY#Y#Wzj+gsX^%NG8NhDzGC_1I zg79qTd@s0~WZDavX9At`&ACcbF(9_poCU~k90kzx0gZjYv#sWJ!%fG>0LB|ld%pp0 zjx()?n`5S9$>yyR(|0<^cMs5^%$%PDXl^vGp9KWl25ibR?MgNs0j^e<3TJ4|dcbBc zd{+iQK?WqI4e3Cw0`q#H^bW|iEFNgP4xsIV2YIYqEQ9A2aJ$F6N@Cg$1v(BN=Ea*c zflI5P%v<370pKcp!zSE0%XHkFt%WSg0Rqt4GtsnNXU%3gw!zoH zO33D*1l|+_+=Hg_R3Q0AQ=S&Y%M7w~0^S^F%gsu_F_4!XX{J2$GFeRPv;wmcgbp}R z3@`z*d6Q{Rl3CAU+RnWVfcwsLbDlYyB|pd{E8yiiKx%)IxiHRj$ZV8=*c;86(IA=I z;>|@G5RcuE-M(ZvkxHCx$|vLU(f}VQo)<{dQDDsu^o4;vIdE5YbOM?v7A zf}PJRS4&#At<5DVVkI2w;n)htIyg4Mu>p=eI11p{2FDgSHp8)rq^9Is0=)a1^inuA zEnd^K_(ap==khH>3*<7@;$*`iyP}XEv0vBYDLB6V<;~=FE7KREjHFt>GE?luSp)uh*p3`YY7284|Xk&ILNk5Wl8(zK~*R#Lve z&DRUI@a@{F@$vRy`+1Ldo&slp$KS4bSYFwZzrDF^D_lG-%za#X>KUvhc zaXrM3(@WqCcXH_T66Z3{WuIGhZuRAYbL-CKKHqWr+}d-S&Mj@sTii}I{qA&W64vjw zGTdk#Uj*79)3jA>2FYawHVM?yGIRbckksADGeBf^oAS+BI?yNiAPGl72(~6c?K%$c zcbJaF0q3*WH*mBIX+=8dmR<46;3_Q$>`n>Ge@L?-)^rF&XT50$TMag7;oD+UX@VKF zlzEd5^xHb~CURlx_zjSLBgo1@)8069w!~DD4E1}HsVo*y+hN)RRR*AJXqVo-(btTS z*KUBZ4lv%(uC_ogff6z2YCx}UgxZwJDxwo)QbL+}J7|L}km3y>PshDL&^LhG7DDkV zl&b_u83_sYDtX0mkLRa%05##iylL_A0o?|40D8LKT66Qn&5BzkHxJ*+xm9}0aP!cu z{99$@K!GR3L9vjYBYdUzT8A|nDjE%k8V!dV7wH=p85$QY>cR3zE}1X!87NseP?DaU z0o_6G^u!eJsDz|6ue5k=LPA=6a`X&euULreHc&!H@1&nHHXJB^ym>kNe}2OF{FL$e z@zavV+~bY82KcX(bMF_fa+0=ADL%KYF>BFsNFHs}AA7#X@O;mrMt#xQ zinHG}>Nhp&bI*O2X8N`|E|7FayzTOr2gC6 zOX|z&%j@^uUS7YiekD6T(YkF-Y3sJtCrNU3BwIVA3g|RUbMNLMXGaSg^=0svuDwW< zkq7;rZmRrctNxdY+-FM*eXXKGG%ek8>*?RF*dx);BF$*JRk;WLVT>SleXS z^`LSE>3|GN$PgdbK|_WPaCM!(aKTsIT~!t`k{k}}AVw*h1g+chvU`*BJhrR=G)}V$ ztc*vaH97Dve3V4y86|)R7Z#H~jqFS+|!+8*!t)l=KLJ|DE8&>a~sf zt!EE4>MJfUJ7*A8aI*3mvsO3i3mWzL4)FeL<>k%imNjl%et!SuE$0@U-FG?f>}hG4 zWc2t^qaxvb|IZ{bk0b`MgQpMIu58|M^3e%a%_=>ia_mqJvA0!prc zZIVsfW}DW>gT?{-FBw|=d{C#Wpvm0}CJm^he9%PO%vlpm-?5FP2A-6evvsBtP_1BT zKXvmk&+xL9OX9m~Nnmmw{T$*Fn;w?Yb1S zDP*@h$-FHF)NmQ#wSl$MD&U(^_!iW?>A*Pic2KOW{ZSSV4f8tC@5i-Jo&sncwn?B^ zM@dw#OG zam(qZ#ittAAMkiC^1&bir^Q~lN8|ea@B&^#A~GENc==@Z(R(zm-*>j6aRX=-(4BDe zyz=m0x>GKZzzD=juEQSA-$x+09?t)j!fYRnSZdW8#)=#)NU;1I2#3i6{tbX5299}f zY$Zu+Z@oeyz{sNgge?pTi4PXArw|{+HxRC{*D!l02D}cq_a;e`;2fCWFZSu zO%0W-t8x|K5DUWyk{{H=Fp{uyL_G}(g=21z183brawP7G$6#61K$V zdn~aq=;Q5i&WK%O$)U7^z7R6%#^T~iTIMR$4j6?CNP3Gx4_EdEg`W11u#rDAt~|JOtR=APWK6q03}4+CB-s75VYL!N}i01}PRm zwh8u-4LM#SXmE_FBpv8%ZVhE|%z@9%p-~j*fO^Wpk+eQRI|@!J6m=Zhvj;{l0@C)B zUY0{wP?rU5lm!_{&hh}1LjXOCBZNCyL8_p)Pg*-yaL!wZU~GyWg;(f(c%2Ct(3Ch@ zO#=XX^!~JCkA@Cx_EA81drJu3$OUrheQ9TVNeUNDGRW6+VQRgE9_s@@d}1VmAT*8) zl8|zumWBwZ66|3qAZ;$|y+}u7sG@@edJlmBQJn`xT1Aqf#0Dt3grt{Mxl-;#0Hu~plTFe@lN*N>!5lN=v6!M2eS1tfGKsqaZ4tP!u^NWZbSS*0jkY(oi&R_X{uJ{#j) zr;yZfxrXDkCzm0qt2Bp`6)wF)4dkUemjgJ_38vrml({(3>z+Nd;35#Tcr3 z3G%={1$?@D-N8giqQ*f?I3SsYqcF97P@Ihknw<1iOnQ2v%zo5Lkuz)U5QKvT zB(c!N5Dr|p;Kb@u)6&0!c%@nyF^cVKw?C#6AWrCicuGMgx`e-{vx4BYG1}C>KjP?8 zzS6|Rz`R!4-ycJy;J@Gg8CUP`&;D}q`|~LWF#Ka)@(W&H#vA^8%@*KaME&C-Q)lbs z>~zQd<3eW#^v5X0eHgy; z3-`QF?VzZM5b-e1ky6HJ64U+(+{gXsFK}N(;tQzjxt7eOObhoD1u#mq$AZPnZuV@g zW|lU!XR0DDIXNyt>lO2#AX!uSH>iJz)fcd9DW;Hee|-?SmmEYaPy_*V{}U*`wo!lK z_BToV0_o#+3ue=4(!KBj<$nEWJ1lV_-`Lr{J*VynQO0jpl#eS^I6ZahnnZ?$W?}gjhQgui#m4P zW9`lU`o#@XDwUydu)mQ?Dj$zWrek z`|w|=wdb!?4j(-$YJzVc z(MPGA&T=XA-|}M%@%Q{#9x5mH1mX^x$ENrBxAgyxAMjbJ)I|8lPMi=nBRVmBUNq2s zLI`|0Hvsv+gy;1l&GG4VkZTK`9y^(K?K|3!MXhhS-1ZN6ONJ;O)5kTPxp*Y oN%7RcAj)hBG+k+6L`i&mdO-)q=?)l3|4;sid{`0`q73u>Uw0Ff?f?J) literal 0 HcmV?d00001 diff --git a/deltacat/tests/compute/resource_requirements/test_utils.py b/deltacat/tests/compute/resource_requirements/test_utils.py new file mode 100644 index 00000000..729cfcad --- /dev/null +++ b/deltacat/tests/compute/resource_requirements/test_utils.py @@ -0,0 +1,367 @@ +import pytest +from deltacat.storage import ManifestEntry, ManifestMeta +import pyarrow.parquet as papq +from deltacat.types.partial_download import PartialParquetParameters +from deltacat.compute.resource_requirements.utils import ( + estimate_manifest_entry_column_size_bytes, + estimate_manifest_entry_num_rows, + estimate_manifest_entry_size_bytes, +) + +PARQUET_FILE_PATH_NO_STATS = ( + "deltacat/tests/compute/resource_requirements/data/sample_no_stats.parquet" +) +PARQUET_FILE_PATH_WITH_STATS = ( + "deltacat/tests/compute/resource_requirements/data/sample_with_stats.parquet" +) + + +@pytest.fixture(scope="module") +def sample_no_stats_entry(): + manifest_meta = ManifestMeta.of( + content_length=113629, + record_count=0, + content_type="application/parquet", + content_encoding="identity", + content_type_parameters=[ + PartialParquetParameters.of( + pq_metadata=papq.ParquetFile(PARQUET_FILE_PATH_NO_STATS).metadata + ) + ], + ) + return ManifestEntry.of( + url=PARQUET_FILE_PATH_NO_STATS, + uri=PARQUET_FILE_PATH_NO_STATS, + mandatory=True, + uuid="test", + meta=manifest_meta, + ) + + +@pytest.fixture(scope="module") +def sample_with_no_type_params(): + manifest_meta = ManifestMeta.of( + content_length=113629, + record_count=0, + content_type="application/parquet", + content_encoding="identity", + content_type_parameters=[], + ) + return ManifestEntry.of( + url=PARQUET_FILE_PATH_NO_STATS, + uri=PARQUET_FILE_PATH_NO_STATS, + mandatory=True, + uuid="test", + meta=manifest_meta, + ) + + +@pytest.fixture(scope="module") +def sample_with_stats_entry(): + manifest_meta = ManifestMeta.of( + content_length=113629, + record_count=0, + content_type="application/parquet", + content_encoding="identity", + content_type_parameters=[ + PartialParquetParameters.of( + pq_metadata=papq.ParquetFile(PARQUET_FILE_PATH_WITH_STATS).metadata + ) + ], + ) + return ManifestEntry.of( + url=PARQUET_FILE_PATH_WITH_STATS, + uri=PARQUET_FILE_PATH_WITH_STATS, + mandatory=True, + uuid="test", + meta=manifest_meta, + ) + + +class TestEstimateManifestEntryColumnSizeBytes: + def test_when_no_columns_passed_sanity( + self, sample_no_stats_entry, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 1, False + ) + ) + == 0 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, 1, False + ) + ) + == 0 + ) + + def test_when_no_columns_passed_with_intelligent_estimation( + self, sample_no_stats_entry, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 1, True + ) + ) + == 0 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, 1, True + ) + ) + == 0 + ) + + def test_when_one_string_column_passed( + self, sample_no_stats_entry, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 1, False, ["first_name"] + ) + ) + == 2988 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, 1, False, ["first_name"] + ) + ) + == 2989 + ) + + def test_when_multiple_columns_passed( + self, sample_no_stats_entry, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 1, False, ["first_name", "id"] + ) + ) + == 7031 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, 1, False, ["first_name", "id"] + ) + ) + == 8314 + ) + + def test_when_timestamp_column_passed( + self, sample_no_stats_entry, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 2, False, ["registration_dttm"] + ) + ) + == 26540 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, 2, False, ["registration_dttm"] + ) + ) + == 18602 + ) + + def test_when_intelligent_estimation_enabled_single_column( + self, sample_no_stats_entry, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 1, True, ["first_name"] + ) + ) + == 2988 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, 1, True, ["first_name"] + ) + ) + == 7000 + ) + + def test_when_intelligent_estimation_enabled_timestamp_column( + self, sample_no_stats_entry, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 1, True, ["registration_dttm"] + ) + ) + == 12000 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, 1, True, ["registration_dttm"] + ) + ) + == 8000 + ) + + def test_when_intelligent_estimation_enabled_int_column( + self, sample_no_stats_entry, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 1, True, ["id"] + ) + ) + == 4000 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, 1, True, ["id"] + ) + ) + == 4000 + ) + + def test_when_intelligent_estimation_enabled_double_column( + self, sample_no_stats_entry, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 1, True, ["salary"] + ) + ) + == 8000 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, 1, True, ["salary"] + ) + ) + == 8000 + ) + + def test_when_intelligent_estimation_enabled_multiple_columns( + self, sample_no_stats_entry, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 1, True, ["first_name", "id"] + ) + ) + == 6988 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, 1, True, ["first_name", "id"] + ) + ) + == 11000 + ) + + def test_when_intelligent_estimation_enabled_with_no_type_params( + self, sample_with_no_type_params + ): + assert ( + estimate_manifest_entry_column_size_bytes( + sample_with_no_type_params, 1, True, ["first_name"] + ) + is None + ) + + +class TestEstimateManifestEntryNumRows: + def test_sanity(self, sample_no_stats_entry): + assert ( + estimate_manifest_entry_num_rows(sample_no_stats_entry, 1000, 7, 1, False) + == 1000 + ) + + def test_when_previous_inflation_forced(self, sample_no_stats_entry): + assert ( + estimate_manifest_entry_num_rows(sample_no_stats_entry, 1000, 7, 1, True) + == 795 + ) + + def test_when_type_params_absent(self, sample_with_no_type_params): + assert ( + estimate_manifest_entry_num_rows( + sample_with_no_type_params, 1000, 7, 1, True + ) + == 795 + ) + + +class TestEstimateManifestEntrySizeBytes: + def test_sanity(self, sample_no_stats_entry): + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_no_stats_entry, 7, 2, False, False + ) + ) + == 224984 + ) + + def test_when_previous_inflation_forced(self, sample_no_stats_entry): + # using previous inflation has turned out to be inaccurate + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_no_stats_entry, 7, 2, True, False + ) + ) + == 795403 + ) + + def test_when_type_params_absent(self, sample_with_no_type_params): + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_with_no_type_params, 7, 2, False, True + ) + ) + == 795403 + ) + + def test_when_intelligent_estimation_is_enabled(self, sample_no_stats_entry): + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_no_stats_entry, 7, 2, False, True + ) + ) + == 223096 + ) + + def test_when_intelligent_estimation_is_enabled_with_stats( + self, sample_with_stats_entry + ): + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_with_stats_entry, 7, 2, False, True + ) + ) + == 290222 + ) From 0e527c3afcd82b3893f59ce49099d9c7447511e1 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Wed, 25 Sep 2024 12:18:43 -0700 Subject: [PATCH 08/25] Add invalid column UT --- .../tests/compute/resource_requirements/test_utils.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/deltacat/tests/compute/resource_requirements/test_utils.py b/deltacat/tests/compute/resource_requirements/test_utils.py index 729cfcad..9bbcb55c 100644 --- a/deltacat/tests/compute/resource_requirements/test_utils.py +++ b/deltacat/tests/compute/resource_requirements/test_utils.py @@ -139,6 +139,16 @@ def test_when_one_string_column_passed( == 2989 ) + def test_when_invalid_column_passed_assumes_null(self, sample_no_stats_entry): + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, 1, False, ["invalid_column"] + ) + ) + == 4000 + ) + def test_when_multiple_columns_passed( self, sample_no_stats_entry, sample_with_stats_entry ): From 028f1710690bdf9ead0a0c24724428c83f8618c4 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Wed, 25 Sep 2024 12:57:26 -0700 Subject: [PATCH 09/25] Fix log messages --- deltacat/compute/compactor_v2/utils/content_type_params.py | 4 +++- deltacat/compute/resource_requirements/utils.py | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/deltacat/compute/compactor_v2/utils/content_type_params.py b/deltacat/compute/compactor_v2/utils/content_type_params.py index 97a892aa..6cc2b193 100644 --- a/deltacat/compute/compactor_v2/utils/content_type_params.py +++ b/deltacat/compute/compactor_v2/utils/content_type_params.py @@ -78,7 +78,9 @@ def append_content_type_params( entry_indices_to_download.append(entry_index) if not entry_indices_to_download: - logger.info(f"No parquet entries found for delta with locator {delta.locator}.") + logger.info( + "No parquet type params to down for " f"delta with locator {delta.locator}." + ) return options_provider = functools.partial( diff --git a/deltacat/compute/resource_requirements/utils.py b/deltacat/compute/resource_requirements/utils.py index 4cc48f7c..3313eeaf 100644 --- a/deltacat/compute/resource_requirements/utils.py +++ b/deltacat/compute/resource_requirements/utils.py @@ -55,7 +55,7 @@ def _calculate_parquet_column_size( # This indicates a null column final_size += NULL_SIZE_BYTES * row_group_meta.num_rows elif columns_found > 1: - raise ValueError(f"Duplicate column found: {column}") + raise ValueError(f"Duplicate column found in parquet file: {column}") return final_size * parquet_to_pyarrow_inflation From bd36e06df5bdf67a08263338c24a7063565bfa9c Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Thu, 26 Sep 2024 14:26:49 -0700 Subject: [PATCH 10/25] Adding enums to perform each type of estimation --- .../model/compact_partition_params.py | 57 +- .../compactor_v2/private/compaction_utils.py | 22 +- deltacat/compute/compactor_v2/utils/io.py | 53 +- .../compactor_v2/utils/task_options.py | 280 ++++--- .../compute/resource_estimation/__init__.py | 11 + deltacat/compute/resource_estimation/model.py | 156 ++++ .../parquet.py | 0 deltacat/compute/resource_estimation/utils.py | 415 ++++++++++ .../compute/resource_requirements/utils.py | 173 ----- .../compactor_v2/test_compaction_session.py | 157 ++++ .../compactor_v2/utils/test_task_options.py | 6 +- .../compute/resource_estimation}/__init__.py | 0 .../data/DATA.md | 0 .../data}/__init__.py | 0 .../data/sample_no_stats.parquet | Bin .../data/sample_with_stats.parquet | Bin .../compute/resource_estimation/test_utils.py | 717 ++++++++++++++++++ .../resource_requirements/data/__init__.py | 0 .../resource_requirements/test_utils.py | 377 --------- 19 files changed, 1638 insertions(+), 786 deletions(-) create mode 100644 deltacat/compute/resource_estimation/__init__.py create mode 100644 deltacat/compute/resource_estimation/model.py rename deltacat/compute/{resource_requirements => resource_estimation}/parquet.py (100%) create mode 100644 deltacat/compute/resource_estimation/utils.py delete mode 100644 deltacat/compute/resource_requirements/utils.py rename deltacat/{compute/resource_requirements => tests/compute/resource_estimation}/__init__.py (100%) rename deltacat/tests/compute/{resource_requirements => resource_estimation}/data/DATA.md (100%) rename deltacat/tests/compute/{resource_requirements => resource_estimation/data}/__init__.py (100%) rename deltacat/tests/compute/{resource_requirements => resource_estimation}/data/sample_no_stats.parquet (100%) rename deltacat/tests/compute/{resource_requirements => resource_estimation}/data/sample_with_stats.parquet (100%) create mode 100644 deltacat/tests/compute/resource_estimation/test_utils.py delete mode 100644 deltacat/tests/compute/resource_requirements/data/__init__.py delete mode 100644 deltacat/tests/compute/resource_requirements/test_utils.py diff --git a/deltacat/compute/compactor/model/compact_partition_params.py b/deltacat/compute/compactor/model/compact_partition_params.py index a3df9066..96d6623d 100644 --- a/deltacat/compute/compactor/model/compact_partition_params.py +++ b/deltacat/compute/compactor/model/compact_partition_params.py @@ -13,6 +13,10 @@ PartitionLocator, SortKey, ) +from deltacat.compute.resource_estimation import ( + ResourceEstimationMethod, + EstimateResourcesParams, +) from deltacat.compute.compactor_v2.constants import ( MAX_RECORDS_PER_COMPACTED_FILE, MIN_DELTA_BYTES_IN_BATCH, @@ -109,12 +113,11 @@ def of(params: Optional[Dict]) -> CompactPartitionParams: result.parquet_to_pyarrow_inflation = params.get( "parquet_to_pyarrow_inflation", PARQUET_TO_PYARROW_INFLATION ) - result.force_using_previous_inflation_for_memory_calculation = params.get( - "force_using_previous_inflation_for_memory_calculation", False - ) - result.enable_intelligent_size_estimation = params.get( - "enable_intelligent_size_estimation", False - ) + result.resource_estimation_method = ResourceEstimationMethod[ + params.get( + "resource_estimation_method", ResourceEstimationMethod.DEFAULT.value + ) + ] # disable input split during rebase as the rebase files are already uniform result.enable_input_split = ( @@ -457,25 +460,6 @@ def enable_input_split(self) -> bool: def enable_input_split(self, value: bool) -> None: self["enable_input_split"] = value - @property - def force_using_previous_inflation_for_memory_calculation(self) -> bool: - """ - When this is True, the memory estimation will always use previous inflation - and average record size for all data formats even if format specific metadata - is available to make better predictions of memory requirements. - - By default, previous inflation is used for non-parquet files to estimate memory while - parquet metadata will be used for parquet to estimate memory. We only fallback - to previous inflation if parquet metadata isn't available. - """ - return self["force_using_previous_inflation_for_memory_calculation"] - - @force_using_previous_inflation_for_memory_calculation.setter - def force_using_previous_inflation_for_memory_calculation( - self, value: bool - ) -> None: - self["force_using_previous_inflation_for_memory_calculation"] = value - @property def max_parquet_meta_size_bytes(self) -> int: return self["max_parquet_meta_size_bytes"] @@ -485,16 +469,21 @@ def max_parquet_meta_size_bytes(self, value: int) -> None: self["max_parquet_meta_size_bytes"] = value @property - def enable_intelligent_size_estimation(self) -> bool: - """ - The arguments enable intelligent memory estimation that considers - encoding, min/max and other statistics to estimate memory requirements. - """ - return self["enable_intelligent_size_estimation"] + def resource_estimation_method(self) -> ResourceEstimationMethod: + return self["resource_estimation_method"] - @enable_intelligent_size_estimation.setter - def enable_intelligent_size_estimation(self, value: bool) -> None: - self["enable_intelligent_size_estimation"] = value + @resource_estimation_method.setter + def resource_estimation_method(self, value: ResourceEstimationMethod) -> None: + self["resource_estimation_method"] = value + + @property + def estimate_resources_params(self) -> EstimateResourcesParams: + return EstimateResourcesParams.of( + resource_estimation_method=self.resource_estimation_method, + previous_inflation=self.previous_inflation, + parquet_to_pyarrow_inflation=self.parquet_to_pyarrow_inflation, + average_record_size_bytes=self.average_record_size_bytes, + ) @staticmethod def json_handler_for_compact_partition_params(obj): diff --git a/deltacat/compute/compactor_v2/private/compaction_utils.py b/deltacat/compute/compactor_v2/private/compaction_utils.py index 193d158c..818a0885 100644 --- a/deltacat/compute/compactor_v2/private/compaction_utils.py +++ b/deltacat/compute/compactor_v2/private/compaction_utils.py @@ -148,17 +148,9 @@ def _build_uniform_deltas( input_deltas=input_deltas, hash_bucket_count=params.hash_bucket_count, compaction_audit=mutable_compaction_audit, + compact_partition_params=params, deltacat_storage=params.deltacat_storage, - previous_inflation=params.previous_inflation, - min_delta_bytes=params.min_delta_bytes_in_batch, - min_file_counts=params.min_files_in_batch, - enable_input_split=params.enable_input_split, deltacat_storage_kwargs=params.deltacat_storage_kwargs, - parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, - force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, - enable_intelligent_size_estimation=params.enable_intelligent_size_estimation, - task_max_parallelism=params.task_max_parallelism, - max_parquet_meta_size_bytes=params.max_parquet_meta_size_bytes, ) delta_discovery_end: float = time.monotonic() @@ -404,9 +396,7 @@ def _merge( deltacat_storage_kwargs=params.deltacat_storage_kwargs, ray_custom_resources=params.ray_custom_resources, memory_logs_enabled=params.memory_logs_enabled, - parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, - force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, - enable_intelligent_size_estimation=params.enable_intelligent_size_estimation, + estimate_resources_params=params.estimate_resources_params, ) def merge_input_provider(index, item) -> dict[str, MergeInput]: @@ -470,9 +460,7 @@ def _hash_bucket( primary_keys=params.primary_keys, ray_custom_resources=params.ray_custom_resources, memory_logs_enabled=params.memory_logs_enabled, - parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, - force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, - enable_intelligent_size_estimation=params.enable_intelligent_size_estimation, + estimate_resources_params=params.estimate_resources_params, ) def hash_bucket_input_provider(index, item) -> dict[str, HashBucketInput]: @@ -547,9 +535,7 @@ def _run_local_merge( ray_custom_resources=params.ray_custom_resources, primary_keys=params.primary_keys, memory_logs_enabled=params.memory_logs_enabled, - parquet_to_pyarrow_inflation=params.parquet_to_pyarrow_inflation, - force_use_previous_inflation=params.force_using_previous_inflation_for_memory_calculation, - enable_intelligent_size_estimation=params.enable_intelligent_size_estimation, + estimate_resources_params=params.estimate_resources_params, ) local_merge_result = ray.get( mg.merge.options(**local_merge_options).remote(local_merge_input) diff --git a/deltacat/compute/compactor_v2/utils/io.py b/deltacat/compute/compactor_v2/utils/io.py index ed0c740f..2af79741 100644 --- a/deltacat/compute/compactor_v2/utils/io.py +++ b/deltacat/compute/compactor_v2/utils/io.py @@ -1,6 +1,5 @@ import logging import functools -from deltacat.constants import PYARROW_INFLATION_MULTIPLIER from deltacat.storage import ( PartitionLocator, Delta, @@ -9,11 +8,10 @@ from deltacat import logs from deltacat.compute.compactor.utils import io as io_v1 from deltacat.compute.compactor import DeltaAnnotated -from typing import Dict, List, Optional, Any -from deltacat.compute.compactor_v2.constants import ( - MIN_FILES_IN_BATCH, - MIN_DELTA_BYTES_IN_BATCH, +from deltacat.compute.compactor.model.compact_partition_params import ( + CompactPartitionParams, ) +from typing import Dict, List, Optional, Any from deltacat.compute.compactor.model.compaction_session_audit_info import ( CompactionSessionAuditInfo, ) @@ -25,6 +23,8 @@ ) from deltacat.utils.metrics import metrics from deltacat.compute.compactor_v2.constants import DISCOVER_DELTAS_METRIC_PREFIX +from deltacat.compute.resource_estimation.utils import does_require_content_type_params +from deltacat.compute.resource_estimation.model import OperationType logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) @@ -90,15 +90,7 @@ def create_uniform_input_deltas( input_deltas: List[Delta], hash_bucket_count: int, compaction_audit: CompactionSessionAuditInfo, - parquet_to_pyarrow_inflation: Optional[float], - force_use_previous_inflation: Optional[bool], - enable_intelligent_size_estimation: Optional[bool], - task_max_parallelism: Optional[int], - max_parquet_meta_size_bytes: Optional[int], - min_delta_bytes: Optional[float] = MIN_DELTA_BYTES_IN_BATCH, - min_file_counts: Optional[float] = MIN_FILES_IN_BATCH, - previous_inflation: Optional[float] = PYARROW_INFLATION_MULTIPLIER, - enable_input_split: Optional[bool] = False, + compact_partition_params: CompactPartitionParams, deltacat_storage=unimplemented_deltacat_storage, deltacat_storage_kwargs: Optional[Dict[str, Any]] = {}, ) -> List[DeltaAnnotated]: @@ -109,14 +101,21 @@ def create_uniform_input_deltas( input_da_list = [] for delta in input_deltas: - if enable_input_split or enable_intelligent_size_estimation: - # An idempotent operation to ensure content type params exist + if ( + compact_partition_params.enable_input_split + or does_require_content_type_params( + compact_partition_params.resource_estimation_method + ) + ): + logger.debug( + f"Delta with locator: {delta.locator} requires content type params..." + ) append_content_type_params( delta=delta, deltacat_storage=deltacat_storage, deltacat_storage_kwargs=deltacat_storage_kwargs, - task_max_parallelism=task_max_parallelism, - max_parquet_meta_size_bytes=max_parquet_meta_size_bytes, + task_max_parallelism=compact_partition_params.task_max_parallelism, + max_parquet_meta_size_bytes=compact_partition_params.max_parquet_meta_size_bytes, ) manifest_entries = delta.manifest.entries @@ -127,10 +126,8 @@ def create_uniform_input_deltas( delta_bytes += entry.meta.content_length estimated_da_bytes += estimate_manifest_entry_size_bytes( entry=entry, - previous_inflation=previous_inflation, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - force_use_previous_inflation=force_use_previous_inflation, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=compact_partition_params.estimate_resources_params, ) delta_annotated = DeltaAnnotated.of(delta) @@ -142,17 +139,15 @@ def create_uniform_input_deltas( size_estimation_function = functools.partial( estimate_manifest_entry_size_bytes, - previous_inflation=previous_inflation, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - force_use_previous_inflation=force_use_previous_inflation, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=compact_partition_params.estimate_resources_params, ) rebatched_da_list = DeltaAnnotated.rebatch( input_da_list, - min_delta_bytes=min_delta_bytes, - min_file_counts=min_file_counts, + min_delta_bytes=compact_partition_params.min_delta_bytes_in_batch, + min_file_counts=compact_partition_params.min_files_in_batch, estimation_function=size_estimation_function, - enable_input_split=enable_input_split, + enable_input_split=compact_partition_params.enable_input_split, ) compaction_audit.set_input_size_bytes(delta_bytes) diff --git a/deltacat/compute/compactor_v2/utils/task_options.py b/deltacat/compute/compactor_v2/utils/task_options.py index cdd907de..4b96a5ff 100644 --- a/deltacat/compute/compactor_v2/utils/task_options.py +++ b/deltacat/compute/compactor_v2/utils/task_options.py @@ -13,17 +13,21 @@ from deltacat.compute.compactor_v2.utils.primary_key_index import ( hash_group_index_to_hash_bucket_indices, ) -from deltacat.compute.resource_requirements.utils import ( +from deltacat.compute.resource_estimation.utils import ( estimate_manifest_entry_num_rows, estimate_manifest_entry_size_bytes, estimate_manifest_entry_column_size_bytes, ) +from deltacat.compute.resource_estimation.model import ( + EstimateResourcesParams, + OperationType, +) from deltacat.exceptions import RetryableError logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) -def get_task_options( +def _get_task_options( cpu: float, memory: float, ray_custom_resources: Optional[Dict] = None, @@ -53,10 +57,116 @@ def get_task_options( return task_opts +def _get_merge_task_options( + index: int, + hb_group_idx: int, + data_size: float, + pk_size_bytes: float, + num_rows: int, + num_hash_groups: int, + total_memory_buffer_percentage: int, + incremental_index_array_size: int, + debug_memory_params: Dict[str, Any], + ray_custom_resources: Optional[Dict], + estimate_resources_params: EstimateResourcesParams, + round_completion_info: Optional[RoundCompletionInfo] = None, + compacted_delta_manifest: Optional[Manifest] = None, + primary_keys: Optional[List[str]] = None, + deltacat_storage=unimplemented_deltacat_storage, + deltacat_storage_kwargs: Optional[Dict] = {}, + memory_logs_enabled: Optional[bool] = None, +) -> Dict[str, Any]: + if ( + round_completion_info + and compacted_delta_manifest + and round_completion_info.hb_index_to_entry_range + ): + + previous_inflation = ( + round_completion_info.compacted_pyarrow_write_result.pyarrow_bytes + / round_completion_info.compacted_pyarrow_write_result.file_bytes + ) + debug_memory_params["previous_inflation"] = previous_inflation + + average_record_size = ( + round_completion_info.compacted_pyarrow_write_result.pyarrow_bytes + / round_completion_info.compacted_pyarrow_write_result.records + ) + debug_memory_params["average_record_size"] = average_record_size + + iterable = hash_group_index_to_hash_bucket_indices( + hb_group_idx, round_completion_info.hash_bucket_count, num_hash_groups + ) + + for hb_idx in iterable: + if round_completion_info.hb_index_to_entry_range.get(str(hb_idx)) is None: + continue + + entry_start, entry_end = round_completion_info.hb_index_to_entry_range[ + str(hb_idx) + ] + for entry_index in range(entry_start, entry_end): + entry = compacted_delta_manifest.entries[entry_index] + + current_entry_size = estimate_manifest_entry_size_bytes( + entry=entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + current_entry_rows = estimate_manifest_entry_num_rows( + entry=entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + + data_size += current_entry_size + num_rows += current_entry_rows + + if primary_keys: + pk_size = estimate_manifest_entry_column_size_bytes( + entry=entry, + columns=primary_keys, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + + if pk_size is None: + pk_size_bytes += current_entry_size + else: + pk_size_bytes += pk_size + + # total data downloaded + primary key hash column + pyarrow-to-numpy conversion + # + primary key column + hashlib inefficiency + dict size for merge + incremental index array size + total_memory = ( + data_size + + pk_size_bytes + + pk_size_bytes + + num_rows * 20 + + num_rows * 20 + + num_rows * 20 + + incremental_index_array_size + ) + debug_memory_params["data_size"] = data_size + debug_memory_params["num_rows"] = num_rows + debug_memory_params["pk_size_bytes"] = pk_size_bytes + debug_memory_params["incremental_index_array_size"] = incremental_index_array_size + debug_memory_params["total_memory"] = total_memory + debug_memory_params["estimate_resources_params"] = estimate_resources_params + + total_memory = total_memory * (1 + total_memory_buffer_percentage / 100.0) + debug_memory_params["total_memory_with_buffer"] = total_memory + logger.debug_conditional( + f"[Merge task {index}]: Params used for calculating merge memory: {debug_memory_params}", + memory_logs_enabled, + ) + + return _get_task_options(0.01, total_memory, ray_custom_resources) + + def append_content_type_params_options_provider( index: int, item: Any, max_parquet_meta_size_bytes: int, **kwargs ) -> Dict: - return get_task_options( + return _get_task_options( 0.01, max_parquet_meta_size_bytes, scheduling_strategy="DEFAULT" ) @@ -67,9 +177,7 @@ def hash_bucket_resource_options_provider( previous_inflation: float, average_record_size_bytes: float, total_memory_buffer_percentage: int, - parquet_to_pyarrow_inflation: float, - force_use_previous_inflation: bool, - enable_intelligent_size_estimation: bool, + estimate_resources_params: EstimateResourcesParams, primary_keys: List[str] = None, ray_custom_resources: Optional[Dict] = None, memory_logs_enabled: Optional[bool] = None, @@ -88,18 +196,14 @@ def hash_bucket_resource_options_provider( for entry in item.manifest.entries: entry_size = estimate_manifest_entry_size_bytes( entry=entry, - previous_inflation=previous_inflation, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - force_use_previous_inflation=force_use_previous_inflation, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, **kwargs, ) num_rows += estimate_manifest_entry_num_rows( entry=entry, - previous_inflation=previous_inflation, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - average_record_size_bytes=average_record_size_bytes, - force_use_previous_inflation=force_use_previous_inflation, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, **kwargs, ) size_bytes += entry_size @@ -107,9 +211,9 @@ def hash_bucket_resource_options_provider( if primary_keys: pk_size = estimate_manifest_entry_column_size_bytes( entry=entry, + operation_type=OperationType.PYARROW_DOWNLOAD, columns=primary_keys, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, + estimate_resources_params=estimate_resources_params, ) if pk_size is None: @@ -131,11 +235,7 @@ def hash_bucket_resource_options_provider( debug_memory_params["num_rows"] = num_rows debug_memory_params["total_pk_size"] = total_pk_size debug_memory_params["total_memory"] = total_memory - debug_memory_params[ - "enable_intelligent_size_estimation" - ] = enable_intelligent_size_estimation - debug_memory_params["parquet_to_pyarrow_inflation"] = parquet_to_pyarrow_inflation - debug_memory_params["force_use_previous_inflation"] = force_use_previous_inflation + debug_memory_params["estimate_resources_params"] = estimate_resources_params debug_memory_params["previous_inflation"] = previous_inflation debug_memory_params["average_record_size_bytes"] = average_record_size_bytes @@ -148,7 +248,7 @@ def hash_bucket_resource_options_provider( memory_logs_enabled, ) - return get_task_options(0.01, total_memory, ray_custom_resources) + return _get_task_options(0.01, total_memory, ray_custom_resources) def merge_resource_options_provider( @@ -158,9 +258,7 @@ def merge_resource_options_provider( hash_group_size_bytes: Dict[int, int], hash_group_num_rows: Dict[int, int], total_memory_buffer_percentage: int, - parquet_to_pyarrow_inflation: float, - force_use_previous_inflation: bool, - enable_intelligent_size_estimation: bool, + estimate_resources_params: EstimateResourcesParams, round_completion_info: Optional[RoundCompletionInfo] = None, compacted_delta_manifest: Optional[Manifest] = None, ray_custom_resources: Optional[Dict] = None, @@ -182,7 +280,7 @@ def merge_resource_options_provider( pk_size_bytes = data_size incremental_index_array_size = num_rows * 4 - return get_merge_task_options( + return _get_merge_task_options( index, hb_group_idx, data_size, @@ -199,9 +297,7 @@ def merge_resource_options_provider( deltacat_storage=deltacat_storage, deltacat_storage_kwargs=deltacat_storage_kwargs, memory_logs_enabled=memory_logs_enabled, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - force_use_previous_inflation=force_use_previous_inflation, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, + estimate_resources_params=estimate_resources_params, ) @@ -209,9 +305,7 @@ def local_merge_resource_options_provider( estimated_da_size: float, estimated_num_rows: int, total_memory_buffer_percentage: int, - parquet_to_pyarrow_inflation: float, - force_use_previous_inflation: bool, - enable_intelligent_size_estimation: bool, + estimate_resources_params: EstimateResourcesParams, round_completion_info: Optional[RoundCompletionInfo] = None, compacted_delta_manifest: Optional[Manifest] = None, ray_custom_resources: Optional[Dict] = None, @@ -228,7 +322,7 @@ def local_merge_resource_options_provider( pk_size_bytes = estimated_da_size incremental_index_array_size = estimated_num_rows * 4 - return get_merge_task_options( + return _get_merge_task_options( index=index, hb_group_idx=hb_group_idx, data_size=estimated_da_size, @@ -245,123 +339,5 @@ def local_merge_resource_options_provider( deltacat_storage=deltacat_storage, deltacat_storage_kwargs=deltacat_storage_kwargs, memory_logs_enabled=memory_logs_enabled, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - force_use_previous_inflation=force_use_previous_inflation, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, + estimate_resources_params=estimate_resources_params, ) - - -def get_merge_task_options( - index: int, - hb_group_idx: int, - data_size: float, - pk_size_bytes: float, - num_rows: int, - num_hash_groups: int, - total_memory_buffer_percentage: int, - incremental_index_array_size: int, - debug_memory_params: Dict[str, Any], - ray_custom_resources: Optional[Dict], - parquet_to_pyarrow_inflation: float, - force_use_previous_inflation: bool, - enable_intelligent_size_estimation: bool, - round_completion_info: Optional[RoundCompletionInfo] = None, - compacted_delta_manifest: Optional[Manifest] = None, - primary_keys: Optional[List[str]] = None, - deltacat_storage=unimplemented_deltacat_storage, - deltacat_storage_kwargs: Optional[Dict] = {}, - memory_logs_enabled: Optional[bool] = None, -) -> Dict[str, Any]: - if ( - round_completion_info - and compacted_delta_manifest - and round_completion_info.hb_index_to_entry_range - ): - - previous_inflation = ( - round_completion_info.compacted_pyarrow_write_result.pyarrow_bytes - / round_completion_info.compacted_pyarrow_write_result.file_bytes - ) - debug_memory_params["previous_inflation"] = previous_inflation - - average_record_size = ( - round_completion_info.compacted_pyarrow_write_result.pyarrow_bytes - / round_completion_info.compacted_pyarrow_write_result.records - ) - debug_memory_params["average_record_size"] = average_record_size - - iterable = hash_group_index_to_hash_bucket_indices( - hb_group_idx, round_completion_info.hash_bucket_count, num_hash_groups - ) - - for hb_idx in iterable: - if round_completion_info.hb_index_to_entry_range.get(str(hb_idx)) is None: - continue - - entry_start, entry_end = round_completion_info.hb_index_to_entry_range[ - str(hb_idx) - ] - for entry_index in range(entry_start, entry_end): - entry = compacted_delta_manifest.entries[entry_index] - - current_entry_size = estimate_manifest_entry_size_bytes( - entry=entry, - previous_inflation=previous_inflation, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - force_use_previous_inflation=force_use_previous_inflation, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, - ) - current_entry_rows = estimate_manifest_entry_num_rows( - entry=entry, - average_record_size_bytes=average_record_size, - previous_inflation=previous_inflation, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - force_use_previous_inflation=force_use_previous_inflation, - ) - - data_size += current_entry_size - num_rows += current_entry_rows - - if primary_keys: - pk_size = estimate_manifest_entry_column_size_bytes( - entry=entry, - columns=primary_keys, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, - ) - - if pk_size is None: - pk_size_bytes += current_entry_size - else: - pk_size_bytes += pk_size - - # total data downloaded + primary key hash column + pyarrow-to-numpy conversion - # + primary key column + hashlib inefficiency + dict size for merge + incremental index array size - total_memory = ( - data_size - + pk_size_bytes - + pk_size_bytes - + num_rows * 20 - + num_rows * 20 - + num_rows * 20 - + incremental_index_array_size - ) - debug_memory_params["data_size"] = data_size - debug_memory_params["num_rows"] = num_rows - debug_memory_params["pk_size_bytes"] = pk_size_bytes - debug_memory_params["incremental_index_array_size"] = incremental_index_array_size - debug_memory_params["total_memory"] = total_memory - debug_memory_params[ - "enable_intelligent_size_estimation" - ] = enable_intelligent_size_estimation - debug_memory_params["force_use_previous_inflation"] = force_use_previous_inflation - debug_memory_params["parquet_to_pyarrow_inflation"] = parquet_to_pyarrow_inflation - - total_memory = total_memory * (1 + total_memory_buffer_percentage / 100.0) - debug_memory_params["total_memory_with_buffer"] = total_memory - logger.debug_conditional( - f"[Merge task {index}]: Params used for calculating merge memory: {debug_memory_params}", - memory_logs_enabled, - ) - - return get_task_options(0.01, total_memory, ray_custom_resources) diff --git a/deltacat/compute/resource_estimation/__init__.py b/deltacat/compute/resource_estimation/__init__.py new file mode 100644 index 00000000..f7c2929c --- /dev/null +++ b/deltacat/compute/resource_estimation/__init__.py @@ -0,0 +1,11 @@ +from deltacat.compute.resource_estimation.model import ( + ResourceEstimationMethod, + EstimatedResources, + EstimateResourcesParams, +) + +__all__ = [ + "ResourceEstimationMethod", + "EstimatedResources", + "EstimateResourcesParams", +] diff --git a/deltacat/compute/resource_estimation/model.py b/deltacat/compute/resource_estimation/model.py new file mode 100644 index 00000000..877c09ff --- /dev/null +++ b/deltacat/compute/resource_estimation/model.py @@ -0,0 +1,156 @@ +from __future__ import annotations +from enum import Enum +from typing import Optional + + +class ResourceEstimationMethod(str, Enum): + """ + The default approach executes certain methods in a specific order until the size + is estimated by any. The order is as follows: + 1. CONTENT_TYPE_META + 2. PREVIOUS_INFLATION + This method expects previous inflation and average record bytes to be passed. + """ + + DEFAULT = "DEFAULT" + + """ + This approach strictly uses previous inflation and average record size to arrive + at a resource estimate. It requires users to pass in previous inflation and average + record sizes. + """ + PREVIOUS_INFLATION = "PREVIOUS_INFLATION" + + """ + This approach is similar to PREVIOUS_INFLATION, but it determines average record size + and previous inflation by sampling few files in the given set of files. + """ + FILE_SAMPLING = "FILE_SAMPLING" + + """ + This approach leverages metadata present in content type params. + """ + CONTENT_TYPE_META = "CONTENT_TYPE_META" + + """ + This approach leverages parquet metadata and granularly estimate resources for each column and + then aggregate to arrive at most accurate estimation. + """ + INTELLIGENT_ESTIMATION = "INTELLIGENT_ESTIMATION" + + +class EstimateResourcesParams(dict): + """ + This class represents the parameters required for estimating resources. + """ + + @staticmethod + def of( + resource_estimation_method: ResourceEstimationMethod = ResourceEstimationMethod.DEFAULT, + previous_inflation: Optional[float] = None, + parquet_to_pyarrow_inflation: Optional[float] = None, + average_record_size_bytes: Optional[float] = None, + max_files_to_sample: Optional[int] = None, + ) -> EstimateResourcesParams: + result = EstimateResourcesParams() + result["previous_inflation"] = previous_inflation + result["parquet_to_pyarrow_inflation"] = parquet_to_pyarrow_inflation + result["resource_estimation_method"] = resource_estimation_method + result["max_files_to_sample"] = max_files_to_sample + result["average_record_size_bytes"] = average_record_size_bytes + return result + + @property + def resource_estimation_method(self) -> ResourceEstimationMethod: + return self["resource_estimation_method"] + + @property + def max_files_to_sample(self) -> Optional[int]: + """ + Applicable only for FILE_SAMPLING method. This parameter controls the + number of files to sample to arrive at average record sizes and previous inflation. + """ + return self.get("max_files_to_sample") + + @property + def previous_inflation(self) -> Optional[float]: + """ + This parameter is required for PREVIOUS_INFLATION method. The inflation factor determines + a ratio of in-memory size to the on-disk size. + """ + return self.get("previous_inflation") + + @property + def parquet_to_pyarrow_inflation(self) -> Optional[float]: + """ + This parameter is required for INTELLIGENT_ESTIMATION or CONTENT_TYPE_META method. + This determines inflation factor for parquet estimated size to pyarrow in-memory table size. + """ + return self.get("parquet_to_pyarrow_inflation") + + @property + def average_record_size_bytes(self) -> Optional[float]: + """ + This parameter is required for PREVIOUS_INFLATION method. This determines average size of + records in bytes in a given file or entity. + """ + return self.get("average_record_size_bytes") + + +class OperationType(str, Enum): + """ + This operation type is used when user would download the given entities using pyarrow library. + """ + + PYARROW_DOWNLOAD = "DOWNLOAD" + + +class EstimatedResources(dict): + """ + This class represents the resource requirements for a certain type of operation. + For example, downloading a delta requires certain amount of memory. + """ + + @staticmethod + def of(memory_bytes: float, statistics: Statistics = None) -> EstimatedResources: + result = EstimatedResources() + result["memory_bytes"] = memory_bytes + result["statistics"] = statistics + return result + + @property + def memory_bytes(self) -> float: + return self["memory_bytes"] + + @property + def statistics(self) -> Optional[Statistics]: + return self.get("statistics") + + +class Statistics(dict): + """ + This class represents the statistics of underlying objects that was used + to estimate the resource required. + """ + + @staticmethod + def of( + in_memory_size_bytes: float, record_count: int, on_disk_size_bytes: float + ) -> Statistics: + result = Statistics() + result["in_memory_size_bytes"] = in_memory_size_bytes + result["record_count"] = record_count + result["on_disk_size_bytes"] = on_disk_size_bytes + return result + + @property + def in_memory_size_bytes(self) -> float: + return self["in_memory_size_bytes"] + + @property + def record_count(self) -> int: + return self["record_count"] + + @property + def on_disk_size_bytes(self) -> float: + return self["on_disk_size_bytes"] diff --git a/deltacat/compute/resource_requirements/parquet.py b/deltacat/compute/resource_estimation/parquet.py similarity index 100% rename from deltacat/compute/resource_requirements/parquet.py rename to deltacat/compute/resource_estimation/parquet.py diff --git a/deltacat/compute/resource_estimation/utils.py b/deltacat/compute/resource_estimation/utils.py new file mode 100644 index 00000000..2882c039 --- /dev/null +++ b/deltacat/compute/resource_estimation/utils.py @@ -0,0 +1,415 @@ +import logging +from typing import Optional, List +from deltacat import logs +from deltacat.constants import NULL_SIZE_BYTES +from deltacat.compute.resource_estimation.parquet import ( + parquet_column_chunk_size_estimator, +) +from deltacat.types.media import ContentEncoding, ContentType +from deltacat.types.partial_download import PartialParquetParameters +from deltacat.storage import ( + ManifestEntry, +) +from deltacat.compute.resource_estimation.model import ( + OperationType, + EstimateResourcesParams, + ResourceEstimationMethod, +) + +logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) + + +def _get_parquet_type_params_if_exist( + entry: ManifestEntry, +) -> Optional[PartialParquetParameters]: + if ( + entry.meta + and entry.meta.content_type == ContentType.PARQUET + and entry.meta.content_encoding == ContentEncoding.IDENTITY + and entry.meta.content_type_parameters + ): + for type_params in entry.meta.content_type_parameters: + if isinstance(type_params, PartialParquetParameters): + return type_params + return None + + +def _calculate_parquet_column_size( + type_params: PartialParquetParameters, + parquet_to_pyarrow_inflation: float, + column: str, + enable_intelligent_size_estimation: bool, +) -> float: + + memory_estimator = ( + parquet_column_chunk_size_estimator + if enable_intelligent_size_estimation + else lambda column_meta: column_meta.total_uncompressed_size + ) + + final_size = 0.0 + for rg in type_params.row_groups_to_download: + columns_found = 0 + row_group_meta = type_params.pq_metadata.row_group(rg) + for col in range(row_group_meta.num_columns): + column_meta = row_group_meta.column(col) + if column_meta.path_in_schema == column: + columns_found += 1 + final_size += memory_estimator(column_meta=column_meta) + if columns_found == 0: + # This indicates a null column + final_size += NULL_SIZE_BYTES * row_group_meta.num_rows + elif columns_found > 1: + raise ValueError(f"Duplicate column found in parquet file: {column}") + + return final_size * parquet_to_pyarrow_inflation + + +def _estimate_manifest_entry_size_bytes_using_previous_inflation( + entry: ManifestEntry, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams = None, + **kwargs, +) -> Optional[float]: + + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Size can only be estimated for PYARROW_DOWNLOAD operation" + assert ( + estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.PREVIOUS_INFLATION + or estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.DEFAULT + ), f"Expected PREVIOUS_INFLATION but found {estimate_resources_params.resource_estimation_method}" + assert ( + estimate_resources_params.previous_inflation is not None + ), "Expected previous_inflation when resource estimation method is PREVIOUS_INFLATION" + + return entry.meta.content_length * estimate_resources_params.previous_inflation + + +def _estimate_manifest_entry_size_bytes_using_content_type_meta( + entry: ManifestEntry, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams = None, + **kwargs, +) -> Optional[float]: + + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Size can only be estimated for PYARROW_DOWNLOAD operation" + assert ( + estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.CONTENT_TYPE_META + or estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.DEFAULT + ), f"Expected CONTENT_TYPE_META but found {estimate_resources_params.resource_estimation_method}" + + type_params = _get_parquet_type_params_if_exist(entry=entry) + + if not type_params: + return None + + if not type_params.row_groups_to_download: + return 0 + + assert ( + estimate_resources_params.parquet_to_pyarrow_inflation is not None + ), "Expected parquet_to_pyarrow_inflation when resource estimation method is CONTENT_TYPE_META" + + return ( + type_params.in_memory_size_bytes + * estimate_resources_params.parquet_to_pyarrow_inflation + ) + + +def _estimate_manifest_entry_size_bytes_using_intelligent_estimation( + entry: ManifestEntry, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams = None, + **kwargs, +) -> Optional[float]: + + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Size can only be estimated for PYARROW_DOWNLOAD operation" + assert ( + estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.INTELLIGENT_ESTIMATION + ), f"Expected INTELLIGENT_ESTIMATION but found {estimate_resources_params.resource_estimation_method}" + + type_params = _get_parquet_type_params_if_exist(entry=entry) + + if not type_params: + return None + + if not type_params.row_groups_to_download: + return 0 + + assert ( + estimate_resources_params.parquet_to_pyarrow_inflation is not None + ), "Expected parquet_to_pyarrow_inflation when resource estimation method is INTELLIGENT_ESTIMATION" + + column_names = [ + type_params.pq_metadata.row_group(0).column(col).path_in_schema + for col in range(type_params.pq_metadata.num_columns) + ] + return estimate_manifest_entry_column_size_bytes( + entry=entry, + operation_type=operation_type, + columns=column_names, + estimate_resources_params=estimate_resources_params, + ) + + +RESOURCE_ESTIMATION_METHOD_TO_SIZE_ESTIMATION_FUNCTIONS = { + ResourceEstimationMethod.PREVIOUS_INFLATION: [ + _estimate_manifest_entry_size_bytes_using_previous_inflation + ], + ResourceEstimationMethod.CONTENT_TYPE_META: [ + _estimate_manifest_entry_size_bytes_using_content_type_meta + ], + ResourceEstimationMethod.INTELLIGENT_ESTIMATION: [ + _estimate_manifest_entry_size_bytes_using_intelligent_estimation + ], + ResourceEstimationMethod.DEFAULT: [ + _estimate_manifest_entry_size_bytes_using_content_type_meta, + _estimate_manifest_entry_size_bytes_using_previous_inflation, + ], +} + + +def _estimate_manifest_entry_num_rows_using_previous_inflation( + entry: ManifestEntry, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams = None, + **kwargs, +) -> Optional[int]: + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" + assert ( + estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.PREVIOUS_INFLATION + or estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.DEFAULT + ), f"Expected PREVIOUS_INFLATION but found {estimate_resources_params.resource_estimation_method}" + assert ( + estimate_resources_params.previous_inflation is not None + ), "Expected previous_inflation when resource estimation method is PREVIOUS_INFLATION" + assert ( + estimate_resources_params.average_record_size_bytes is not None + ), "Expected average_record_size_bytes when resource estimation method is PREVIOUS_INFLATION" + + total_size_bytes = estimate_manifest_entry_size_bytes( + entry=entry, + operation_type=operation_type, + estimate_resources_params=estimate_resources_params, + **kwargs, + ) + + return int(total_size_bytes / estimate_resources_params.average_record_size_bytes) + + +def _estimate_manifest_entry_num_rows_using_content_type_meta( + entry: ManifestEntry, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams = None, + **kwargs, +) -> Optional[int]: + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" + assert ( + estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.CONTENT_TYPE_META + or estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.DEFAULT + ), f"Expected CONTENT_TYPE_META but found {estimate_resources_params.resource_estimation_method}" + + type_params = _get_parquet_type_params_if_exist(entry=entry) + + if not type_params: + return None + + return type_params.num_rows + + +def _estimate_manifest_entry_num_rows_using_intelligent_estimation( + entry: ManifestEntry, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams = None, + **kwargs, +) -> Optional[int]: + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" + assert ( + estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.INTELLIGENT_ESTIMATION + ), f"Expected INTELLIGENT_ESTIMATION but found {estimate_resources_params.resource_estimation_method}" + + type_params = _get_parquet_type_params_if_exist(entry=entry) + + if not type_params: + return None + + return type_params.num_rows + + +RESOURCE_ESTIMATION_METHOD_TO_NUM_ROWS_ESTIMATION_FUNCTIONS = { + ResourceEstimationMethod.PREVIOUS_INFLATION: [ + _estimate_manifest_entry_num_rows_using_previous_inflation + ], + ResourceEstimationMethod.CONTENT_TYPE_META: [ + _estimate_manifest_entry_num_rows_using_content_type_meta + ], + ResourceEstimationMethod.INTELLIGENT_ESTIMATION: [ + _estimate_manifest_entry_num_rows_using_intelligent_estimation + ], + ResourceEstimationMethod.DEFAULT: [ + _estimate_manifest_entry_num_rows_using_content_type_meta, + _estimate_manifest_entry_num_rows_using_previous_inflation, + ], +} + + +def estimate_manifest_entry_size_bytes( + entry: ManifestEntry, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams = None, + **kwargs, +) -> Optional[float]: + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Size can only be estimated for PYARROW_DOWNLOAD operation" + + if entry.meta.source_content_length: + # No need to estimate size as source_content_length is already present + return entry.meta.source_content_length + + if estimate_resources_params is None: + estimate_resources_params = EstimateResourcesParams.of() + + functions = RESOURCE_ESTIMATION_METHOD_TO_SIZE_ESTIMATION_FUNCTIONS.get( + estimate_resources_params.resource_estimation_method + ) + + if functions is None: + raise ValueError( + "Unsupported size estimation method" + f": {estimate_resources_params.resource_estimation_method} for entry: {entry}" + ) + + for func in functions: + size_bytes = func( + entry=entry, + operation_type=operation_type, + estimate_resources_params=estimate_resources_params, + **kwargs, + ) + if size_bytes is not None: + logger.debug( + f"Estimated size for entry={entry.uri} is {size_bytes} using {func}" + ) + return size_bytes + + return None + + +def estimate_manifest_entry_num_rows( + entry: ManifestEntry, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams = None, + **kwargs, +) -> Optional[int]: + """ + Estimate number of records in the manifest entry file. + """ + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" + + if entry.meta.record_count: + # No need to estimate as record_count is already present + return entry.meta.record_count + + if estimate_resources_params is None: + estimate_resources_params = EstimateResourcesParams.of() + + functions = RESOURCE_ESTIMATION_METHOD_TO_NUM_ROWS_ESTIMATION_FUNCTIONS.get( + estimate_resources_params.resource_estimation_method + ) + + if functions is None: + raise ValueError( + "Unsupported num rows estimation method" + f": {estimate_resources_params.resource_estimation_method} for entry: {entry}" + ) + + for func in functions: + print(func) + num_rows = func( + entry=entry, + operation_type=operation_type, + estimate_resources_params=estimate_resources_params, + **kwargs, + ) + if num_rows is not None: + logger.debug( + f"Estimated number of rows for entry={entry.uri} is {num_rows} using {func}" + ) + return num_rows + + return None + + +def estimate_manifest_entry_column_size_bytes( + entry: ManifestEntry, + operation_type: OperationType, + columns: Optional[List[str]] = None, + estimate_resources_params: EstimateResourcesParams = None, +) -> Optional[float]: + """ + Estimate the size of specified columns in the manifest entry file. + This method only supports parquet. For other types, it returns None. + """ + + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Resources can only be estimated for PYARROW_DOWNLOAD operation" + + type_params = _get_parquet_type_params_if_exist(entry=entry) + + if not type_params or not type_params.pq_metadata: + return None + + if not columns or not type_params.row_groups_to_download: + return 0 + + if estimate_resources_params is None: + estimate_resources_params = EstimateResourcesParams.of() + + is_intelligent_estimation = ( + estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.INTELLIGENT_ESTIMATION + ) + + columns_size = 0.0 + for column_name in columns: + columns_size += _calculate_parquet_column_size( + type_params=type_params, + column=column_name, + parquet_to_pyarrow_inflation=estimate_resources_params.parquet_to_pyarrow_inflation, + enable_intelligent_size_estimation=is_intelligent_estimation, + ) + return columns_size + + +def does_require_content_type_params( + resource_estimation_method: ResourceEstimationMethod, +) -> bool: + return ( + resource_estimation_method == ResourceEstimationMethod.CONTENT_TYPE_META + or resource_estimation_method == ResourceEstimationMethod.INTELLIGENT_ESTIMATION + ) diff --git a/deltacat/compute/resource_requirements/utils.py b/deltacat/compute/resource_requirements/utils.py deleted file mode 100644 index 3313eeaf..00000000 --- a/deltacat/compute/resource_requirements/utils.py +++ /dev/null @@ -1,173 +0,0 @@ -import logging -from typing import Optional, List -from deltacat import logs -from deltacat.constants import NULL_SIZE_BYTES -from deltacat.compute.resource_requirements.parquet import ( - parquet_column_chunk_size_estimator, -) -from deltacat.types.media import ContentEncoding, ContentType -from deltacat.types.partial_download import PartialParquetParameters -from deltacat.storage import ( - ManifestEntry, -) - -logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) - - -def _get_parquet_type_params_if_exist( - entry: ManifestEntry, -) -> Optional[PartialParquetParameters]: - if ( - entry.meta - and entry.meta.content_type == ContentType.PARQUET - and entry.meta.content_encoding == ContentEncoding.IDENTITY - and entry.meta.content_type_parameters - ): - for type_params in entry.meta.content_type_parameters: - if isinstance(type_params, PartialParquetParameters): - return type_params - return None - - -def _calculate_parquet_column_size( - type_params: PartialParquetParameters, - parquet_to_pyarrow_inflation: float, - column: str, - enable_intelligent_size_estimation: bool, -) -> float: - - memory_estimator = ( - parquet_column_chunk_size_estimator - if enable_intelligent_size_estimation - else lambda column_meta: column_meta.total_uncompressed_size - ) - - final_size = 0.0 - for rg in type_params.row_groups_to_download: - columns_found = 0 - row_group_meta = type_params.pq_metadata.row_group(rg) - for col in range(row_group_meta.num_columns): - column_meta = row_group_meta.column(col) - if column_meta.path_in_schema == column: - columns_found += 1 - final_size += memory_estimator(column_meta=column_meta) - if columns_found == 0: - # This indicates a null column - final_size += NULL_SIZE_BYTES * row_group_meta.num_rows - elif columns_found > 1: - raise ValueError(f"Duplicate column found in parquet file: {column}") - - return final_size * parquet_to_pyarrow_inflation - - -def estimate_manifest_entry_size_bytes( - entry: ManifestEntry, - previous_inflation: float, - parquet_to_pyarrow_inflation: float, - force_use_previous_inflation: bool, - enable_intelligent_size_estimation: bool, - **kwargs, -) -> float: - if entry.meta.source_content_length: - logger.debug(f"Using source content length for entry={entry.uri}") - return entry.meta.source_content_length - - type_params = _get_parquet_type_params_if_exist(entry=entry) - - if type_params and type_params.row_groups_to_download: - if not force_use_previous_inflation: - logger.debug(f"Using parquet meta for entry={entry.uri}") - if enable_intelligent_size_estimation: - column_names = [ - type_params.pq_metadata.row_group(0).column(col).path_in_schema - for col in range(type_params.pq_metadata.num_columns) - ] - return estimate_manifest_entry_column_size_bytes( - entry=entry, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - columns=column_names, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, - ) - else: - return type_params.in_memory_size_bytes * parquet_to_pyarrow_inflation - else: - logger.warning( - f"Force using previous inflation for entry={entry.uri}. " - "This could lead to overestimation of memory when " - "enable_input_split=True" - ) - - logger.debug(f"Using inflation for entry={entry.uri}") - return entry.meta.content_length * previous_inflation - - -def estimate_manifest_entry_num_rows( - entry: ManifestEntry, - average_record_size_bytes: float, - previous_inflation: float, - parquet_to_pyarrow_inflation: float, - force_use_previous_inflation: bool, - **kwargs, -) -> int: - """ - Estimate number of records in the manifest entry file. It uses content type - specific estimation logic if available, otherwise it falls back to using - previous inflation and average record size. - """ - if entry.meta.record_count: - logger.debug(f"Using record count in meta for entry={entry.uri}") - return entry.meta.record_count - - type_params = _get_parquet_type_params_if_exist(entry=entry) - - if type_params: - if not force_use_previous_inflation: - logger.debug(f"Using parquet meta for entry={entry.uri}") - return type_params.num_rows - else: - logger.warning( - f"Force using previous inflation for entry={entry.uri}. " - "This could lead to overestimation of records when " - "enable_input_split=True" - ) - - total_size_bytes = estimate_manifest_entry_size_bytes( - entry=entry, - previous_inflation=previous_inflation, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - force_use_previous_inflation=force_use_previous_inflation, - enable_intelligent_size_estimation=False, - **kwargs, - ) - logger.debug(f"Using previous inflation for entry={entry.uri}") - - return int(total_size_bytes / average_record_size_bytes) - - -def estimate_manifest_entry_column_size_bytes( - entry: ManifestEntry, - parquet_to_pyarrow_inflation: float, - enable_intelligent_size_estimation: bool, - columns: Optional[List[str]] = None, -) -> Optional[float]: - """ - Estimate the size of specified columns in the manifest entry file. - This method only supports parquet. For other types, it returns None. - """ - if not columns: - return 0 - - type_params = _get_parquet_type_params_if_exist(entry=entry) - - if type_params and type_params.pq_metadata: - columns_size = 0.0 - for column_name in columns: - columns_size += _calculate_parquet_column_size( - type_params=type_params, - column=column_name, - parquet_to_pyarrow_inflation=parquet_to_pyarrow_inflation, - enable_intelligent_size_estimation=enable_intelligent_size_estimation, - ) - return columns_size - - return None diff --git a/deltacat/tests/compute/compactor_v2/test_compaction_session.py b/deltacat/tests/compute/compactor_v2/test_compaction_session.py index f1c54129..640f2728 100644 --- a/deltacat/tests/compute/compactor_v2/test_compaction_session.py +++ b/deltacat/tests/compute/compactor_v2/test_compaction_session.py @@ -19,6 +19,7 @@ from deltacat.tests.compute.test_util_constant import ( TEST_S3_RCF_BUCKET_NAME, ) +from deltacat.compute.resource_estimation import ResourceEstimationMethod from deltacat.tests.compute.test_util_common import get_rcf from deltacat.tests.test_utils.pyarrow import ( stage_partition_from_file_paths, @@ -399,3 +400,159 @@ def test_compact_partition_when_incremental_then_rcf_stats_accurate( assert compaction_audit.output_file_count == 2 assert abs(compaction_audit.output_size_bytes - 1843) / 1843 <= self.ERROR_RATE assert abs(compaction_audit.input_size_bytes - 2748) / 2748 <= self.ERROR_RATE + + def test_compact_partition_when_incremental_then_intelligent_estimation_sanity( + self, s3_resource, local_deltacat_storage_kwargs + ): + """ + A test case which asserts the RCF stats are correctly generated for + a rebase and incremental use-case. + """ + + # setup + staged_source = stage_partition_from_file_paths( + self.NAMESPACE, ["source"], **local_deltacat_storage_kwargs + ) + + source_delta = commit_delta_to_staged_partition( + staged_source, [self.BACKFILL_FILE_PATH], **local_deltacat_storage_kwargs + ) + + staged_dest = stage_partition_from_file_paths( + self.NAMESPACE, ["destination"], **local_deltacat_storage_kwargs + ) + dest_partition = ds.commit_partition( + staged_dest, **local_deltacat_storage_kwargs + ) + + # action + compact_partition( + CompactPartitionParams.of( + { + "compaction_artifact_s3_bucket": TEST_S3_RCF_BUCKET_NAME, + "compacted_file_content_type": ContentType.PARQUET, + "dd_max_parallelism_ratio": 1.0, + "deltacat_storage": ds, + "deltacat_storage_kwargs": local_deltacat_storage_kwargs, + "destination_partition_locator": dest_partition.locator, + "drop_duplicates": True, + "hash_bucket_count": 2, + "last_stream_position_to_compact": source_delta.stream_position, + "list_deltas_kwargs": { + **local_deltacat_storage_kwargs, + **{"equivalent_table_types": []}, + }, + "primary_keys": ["pk"], + "rebase_source_partition_locator": source_delta.partition_locator, + "rebase_source_partition_high_watermark": source_delta.stream_position, + "records_per_compacted_file": 4000, + "s3_client_kwargs": {}, + "source_partition_locator": source_delta.partition_locator, + "resource_estimation_method": ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + } + ) + ) + + def test_compact_partition_when_incremental_then_content_type_meta_estimation_sanity( + self, s3_resource, local_deltacat_storage_kwargs + ): + """ + A test case which asserts the RCF stats are correctly generated for + a rebase and incremental use-case. + """ + + # setup + staged_source = stage_partition_from_file_paths( + self.NAMESPACE, ["source"], **local_deltacat_storage_kwargs + ) + + source_delta = commit_delta_to_staged_partition( + staged_source, [self.BACKFILL_FILE_PATH], **local_deltacat_storage_kwargs + ) + + staged_dest = stage_partition_from_file_paths( + self.NAMESPACE, ["destination"], **local_deltacat_storage_kwargs + ) + dest_partition = ds.commit_partition( + staged_dest, **local_deltacat_storage_kwargs + ) + + # action + compact_partition( + CompactPartitionParams.of( + { + "compaction_artifact_s3_bucket": TEST_S3_RCF_BUCKET_NAME, + "compacted_file_content_type": ContentType.PARQUET, + "dd_max_parallelism_ratio": 1.0, + "deltacat_storage": ds, + "deltacat_storage_kwargs": local_deltacat_storage_kwargs, + "destination_partition_locator": dest_partition.locator, + "drop_duplicates": True, + "hash_bucket_count": 2, + "last_stream_position_to_compact": source_delta.stream_position, + "list_deltas_kwargs": { + **local_deltacat_storage_kwargs, + **{"equivalent_table_types": []}, + }, + "primary_keys": ["pk"], + "rebase_source_partition_locator": source_delta.partition_locator, + "rebase_source_partition_high_watermark": source_delta.stream_position, + "records_per_compacted_file": 4000, + "s3_client_kwargs": {}, + "source_partition_locator": source_delta.partition_locator, + "resource_estimation_method": ResourceEstimationMethod.CONTENT_TYPE_META, + } + ) + ) + + def test_compact_partition_when_incremental_then_previous_inflation_estimation_sanity( + self, s3_resource, local_deltacat_storage_kwargs + ): + """ + A test case which asserts the RCF stats are correctly generated for + a rebase and incremental use-case. + """ + + # setup + staged_source = stage_partition_from_file_paths( + self.NAMESPACE, ["source"], **local_deltacat_storage_kwargs + ) + + source_delta = commit_delta_to_staged_partition( + staged_source, [self.BACKFILL_FILE_PATH], **local_deltacat_storage_kwargs + ) + + staged_dest = stage_partition_from_file_paths( + self.NAMESPACE, ["destination"], **local_deltacat_storage_kwargs + ) + dest_partition = ds.commit_partition( + staged_dest, **local_deltacat_storage_kwargs + ) + + # action + compact_partition( + CompactPartitionParams.of( + { + "compaction_artifact_s3_bucket": TEST_S3_RCF_BUCKET_NAME, + "compacted_file_content_type": ContentType.PARQUET, + "dd_max_parallelism_ratio": 1.0, + "deltacat_storage": ds, + "deltacat_storage_kwargs": local_deltacat_storage_kwargs, + "destination_partition_locator": dest_partition.locator, + "drop_duplicates": True, + "hash_bucket_count": 2, + "last_stream_position_to_compact": source_delta.stream_position, + "list_deltas_kwargs": { + **local_deltacat_storage_kwargs, + **{"equivalent_table_types": []}, + }, + "primary_keys": ["pk"], + "rebase_source_partition_locator": source_delta.partition_locator, + "rebase_source_partition_high_watermark": source_delta.stream_position, + "records_per_compacted_file": 4000, + "s3_client_kwargs": {}, + "source_partition_locator": source_delta.partition_locator, + "resource_estimation_method": ResourceEstimationMethod.PREVIOUS_INFLATION, + } + ) + ) diff --git a/deltacat/tests/compute/compactor_v2/utils/test_task_options.py b/deltacat/tests/compute/compactor_v2/utils/test_task_options.py index f0ba1977..dfabd171 100644 --- a/deltacat/tests/compute/compactor_v2/utils/test_task_options.py +++ b/deltacat/tests/compute/compactor_v2/utils/test_task_options.py @@ -1,6 +1,6 @@ import unittest import ray -from deltacat.compute.compactor_v2.utils.task_options import get_task_options +from deltacat.compute.compactor_v2.utils.task_options import _get_task_options @ray.remote @@ -20,14 +20,14 @@ def setUpClass(cls): super().setUpClass() def test_get_task_options_sanity(self): - opts = get_task_options(0.01, 0.01) + opts = _get_task_options(0.01, 0.01) result_ref = valid_func.options(**opts).remote() result = ray.get(result_ref) self.assertEqual(result, 2) def test_get_task_options_when_exception_is_thrown(self): - opts = get_task_options(0.01, 0.01) + opts = _get_task_options(0.01, 0.01) result_ref = throwing_func.options(**opts).remote() self.assertRaises(ConnectionAbortedError, lambda: ray.get(result_ref)) diff --git a/deltacat/compute/resource_requirements/__init__.py b/deltacat/tests/compute/resource_estimation/__init__.py similarity index 100% rename from deltacat/compute/resource_requirements/__init__.py rename to deltacat/tests/compute/resource_estimation/__init__.py diff --git a/deltacat/tests/compute/resource_requirements/data/DATA.md b/deltacat/tests/compute/resource_estimation/data/DATA.md similarity index 100% rename from deltacat/tests/compute/resource_requirements/data/DATA.md rename to deltacat/tests/compute/resource_estimation/data/DATA.md diff --git a/deltacat/tests/compute/resource_requirements/__init__.py b/deltacat/tests/compute/resource_estimation/data/__init__.py similarity index 100% rename from deltacat/tests/compute/resource_requirements/__init__.py rename to deltacat/tests/compute/resource_estimation/data/__init__.py diff --git a/deltacat/tests/compute/resource_requirements/data/sample_no_stats.parquet b/deltacat/tests/compute/resource_estimation/data/sample_no_stats.parquet similarity index 100% rename from deltacat/tests/compute/resource_requirements/data/sample_no_stats.parquet rename to deltacat/tests/compute/resource_estimation/data/sample_no_stats.parquet diff --git a/deltacat/tests/compute/resource_requirements/data/sample_with_stats.parquet b/deltacat/tests/compute/resource_estimation/data/sample_with_stats.parquet similarity index 100% rename from deltacat/tests/compute/resource_requirements/data/sample_with_stats.parquet rename to deltacat/tests/compute/resource_estimation/data/sample_with_stats.parquet diff --git a/deltacat/tests/compute/resource_estimation/test_utils.py b/deltacat/tests/compute/resource_estimation/test_utils.py new file mode 100644 index 00000000..956622cd --- /dev/null +++ b/deltacat/tests/compute/resource_estimation/test_utils.py @@ -0,0 +1,717 @@ +import pytest +from deltacat.storage import ManifestEntry, ManifestMeta +import pyarrow.parquet as papq +from deltacat.types.partial_download import PartialParquetParameters +from deltacat.compute.resource_estimation.utils import ( + estimate_manifest_entry_column_size_bytes, + estimate_manifest_entry_num_rows, + estimate_manifest_entry_size_bytes, +) +from deltacat.compute.resource_estimation.model import ( + OperationType, + EstimateResourcesParams, + ResourceEstimationMethod, +) + +PARQUET_FILE_PATH_NO_STATS = ( + "deltacat/tests/compute/resource_estimation/data/sample_no_stats.parquet" +) +PARQUET_FILE_PATH_WITH_STATS = ( + "deltacat/tests/compute/resource_estimation/data/sample_with_stats.parquet" +) + + +@pytest.fixture(scope="module") +def sample_no_stats_entry(): + manifest_meta = ManifestMeta.of( + content_length=113629, + record_count=0, + content_type="application/parquet", + content_encoding="identity", + content_type_parameters=[ + PartialParquetParameters.of( + pq_metadata=papq.ParquetFile(PARQUET_FILE_PATH_NO_STATS).metadata + ) + ], + ) + return ManifestEntry.of( + url=PARQUET_FILE_PATH_NO_STATS, + uri=PARQUET_FILE_PATH_NO_STATS, + mandatory=True, + uuid="test", + meta=manifest_meta, + ) + + +@pytest.fixture(scope="module") +def sample_with_no_type_params(): + manifest_meta = ManifestMeta.of( + content_length=113629, + record_count=0, + content_type="application/parquet", + content_encoding="identity", + content_type_parameters=[], + ) + return ManifestEntry.of( + url=PARQUET_FILE_PATH_NO_STATS, + uri=PARQUET_FILE_PATH_NO_STATS, + mandatory=True, + uuid="test", + meta=manifest_meta, + ) + + +@pytest.fixture(scope="module") +def sample_with_stats_entry(): + manifest_meta = ManifestMeta.of( + content_length=113629, + record_count=0, + content_type="application/parquet", + content_encoding="identity", + content_type_parameters=[ + PartialParquetParameters.of( + pq_metadata=papq.ParquetFile(PARQUET_FILE_PATH_WITH_STATS).metadata + ) + ], + ) + return ManifestEntry.of( + url=PARQUET_FILE_PATH_WITH_STATS, + uri=PARQUET_FILE_PATH_WITH_STATS, + mandatory=True, + uuid="test", + meta=manifest_meta, + ) + + +class TestEstimateManifestEntryColumnSizeBytes: + def test_when_no_columns_passed_sanity( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 0 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 0 + ) + + def test_when_no_columns_passed_with_intelligent_estimation( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 0 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 0 + ) + + def test_when_one_string_column_passed( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["first_name"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 2988 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + columns=["first_name"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 2989 + ) + + def test_when_invalid_column_passed_assumes_null(self, sample_no_stats_entry): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1 + ) + + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["invalid_column"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 4000 + ) + + def test_when_multiple_columns_passed( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1 + ) + + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["first_name", "id"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 7031 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + columns=["first_name", "id"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 8314 + ) + + def test_when_timestamp_column_passed( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["registration_dttm"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 26540 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + columns=["registration_dttm"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 18602 + ) + + def test_when_intelligent_estimation_enabled_single_column( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["first_name"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 2988 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + columns=["first_name"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 7000 + ) + + def test_when_intelligent_estimation_enabled_timestamp_column( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["registration_dttm"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 12000 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + columns=["registration_dttm"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 8000 + ) + + def test_when_intelligent_estimation_enabled_int_column( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["id"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ), + ) + == 4000 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + columns=["id"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 4000 + ) + + def test_when_intelligent_estimation_enabled_double_column( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["salary"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 8000 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + columns=["salary"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 8000 + ) + + def test_when_intelligent_estimation_enabled_multiple_columns( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["first_name", "id"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 6988 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + columns=["first_name", "id"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 11000 + ) + + def test_when_intelligent_estimation_enabled_with_no_type_params( + self, sample_with_no_type_params + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + + assert ( + estimate_manifest_entry_column_size_bytes( + sample_with_no_type_params, + columns=["first_name"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + is None + ) + + def test_when_previous_inflation_method_with_no_type_params( + self, sample_with_no_type_params + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + resource_estimation_method=ResourceEstimationMethod.PREVIOUS_INFLATION, + ) + + assert ( + estimate_manifest_entry_column_size_bytes( + sample_with_no_type_params, + columns=["first_name"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + is None + ) + + +class TestEstimateManifestEntryNumRows: + def test_sanity(self, sample_no_stats_entry): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + previous_inflation=7, + average_record_size_bytes=1000, + resource_estimation_method=ResourceEstimationMethod.DEFAULT, + ) + + assert ( + estimate_manifest_entry_num_rows( + sample_no_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 1000 + ) + + def test_when_previous_inflation_forced(self, sample_no_stats_entry): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + previous_inflation=7, + average_record_size_bytes=1000, + resource_estimation_method=ResourceEstimationMethod.PREVIOUS_INFLATION, + ) + assert ( + estimate_manifest_entry_num_rows( + sample_no_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 795 + ) + + def test_when_type_params_absent_default_method(self, sample_with_no_type_params): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + previous_inflation=7, + average_record_size_bytes=1000, + resource_estimation_method=ResourceEstimationMethod.DEFAULT, + ) + + assert ( + estimate_manifest_entry_num_rows( + sample_with_no_type_params, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 795 + ) + + def test_when_type_params_absent_intelligent_estimation( + self, sample_with_no_type_params + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + previous_inflation=7, + average_record_size_bytes=1000, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + + assert ( + estimate_manifest_entry_num_rows( + sample_with_no_type_params, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + is None + ) + + def test_when_type_params_absent_content_type_meta( + self, sample_with_no_type_params + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + previous_inflation=7, + average_record_size_bytes=1000, + resource_estimation_method=ResourceEstimationMethod.CONTENT_TYPE_META, + ) + + assert ( + estimate_manifest_entry_num_rows( + sample_with_no_type_params, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + is None + ) + + def test_when_type_params_absent_previous_inflation( + self, sample_with_no_type_params + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + previous_inflation=7, + average_record_size_bytes=1000, + resource_estimation_method=ResourceEstimationMethod.PREVIOUS_INFLATION, + ) + + assert ( + estimate_manifest_entry_num_rows( + sample_with_no_type_params, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 795 + ) + + +class TestEstimateManifestEntrySizeBytes: + def test_sanity(self, sample_no_stats_entry): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.DEFAULT, + ) + + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_no_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 224984 + ) + + def test_when_previous_inflation_forced(self, sample_no_stats_entry): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.PREVIOUS_INFLATION, + ) + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_no_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 795403 + ) + + def test_when_type_params_absent_default(self, sample_with_no_type_params): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.DEFAULT, + ) + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_with_no_type_params, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 795403 + ) + + def test_when_type_params_absent_intelligent_estimation( + self, sample_with_no_type_params + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + assert ( + estimate_manifest_entry_size_bytes( + sample_with_no_type_params, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + is None + ) + + def test_when_type_params_absent_content_meta(self, sample_with_no_type_params): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.CONTENT_TYPE_META, + ) + assert ( + estimate_manifest_entry_size_bytes( + sample_with_no_type_params, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + is None + ) + + def test_when_type_params_absent_previous_inflation( + self, sample_with_no_type_params + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.PREVIOUS_INFLATION, + ) + assert ( + estimate_manifest_entry_size_bytes( + sample_with_no_type_params, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 795403 + ) + + def test_when_intelligent_estimation_sanity(self, sample_no_stats_entry): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_no_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 223096 + ) + + def test_when_type_params_with_stats_default_method(self, sample_with_stats_entry): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.DEFAULT, + ) + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_with_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 227794 + ) + + def test_when_type_params_with_stats_intelligent_method( + self, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + ) + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_with_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 290222 + ) + + def test_when_type_params_with_content_type_meta_method( + self, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.CONTENT_TYPE_META, + ) + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_with_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 227794 + ) diff --git a/deltacat/tests/compute/resource_requirements/data/__init__.py b/deltacat/tests/compute/resource_requirements/data/__init__.py deleted file mode 100644 index e69de29b..00000000 diff --git a/deltacat/tests/compute/resource_requirements/test_utils.py b/deltacat/tests/compute/resource_requirements/test_utils.py deleted file mode 100644 index 9bbcb55c..00000000 --- a/deltacat/tests/compute/resource_requirements/test_utils.py +++ /dev/null @@ -1,377 +0,0 @@ -import pytest -from deltacat.storage import ManifestEntry, ManifestMeta -import pyarrow.parquet as papq -from deltacat.types.partial_download import PartialParquetParameters -from deltacat.compute.resource_requirements.utils import ( - estimate_manifest_entry_column_size_bytes, - estimate_manifest_entry_num_rows, - estimate_manifest_entry_size_bytes, -) - -PARQUET_FILE_PATH_NO_STATS = ( - "deltacat/tests/compute/resource_requirements/data/sample_no_stats.parquet" -) -PARQUET_FILE_PATH_WITH_STATS = ( - "deltacat/tests/compute/resource_requirements/data/sample_with_stats.parquet" -) - - -@pytest.fixture(scope="module") -def sample_no_stats_entry(): - manifest_meta = ManifestMeta.of( - content_length=113629, - record_count=0, - content_type="application/parquet", - content_encoding="identity", - content_type_parameters=[ - PartialParquetParameters.of( - pq_metadata=papq.ParquetFile(PARQUET_FILE_PATH_NO_STATS).metadata - ) - ], - ) - return ManifestEntry.of( - url=PARQUET_FILE_PATH_NO_STATS, - uri=PARQUET_FILE_PATH_NO_STATS, - mandatory=True, - uuid="test", - meta=manifest_meta, - ) - - -@pytest.fixture(scope="module") -def sample_with_no_type_params(): - manifest_meta = ManifestMeta.of( - content_length=113629, - record_count=0, - content_type="application/parquet", - content_encoding="identity", - content_type_parameters=[], - ) - return ManifestEntry.of( - url=PARQUET_FILE_PATH_NO_STATS, - uri=PARQUET_FILE_PATH_NO_STATS, - mandatory=True, - uuid="test", - meta=manifest_meta, - ) - - -@pytest.fixture(scope="module") -def sample_with_stats_entry(): - manifest_meta = ManifestMeta.of( - content_length=113629, - record_count=0, - content_type="application/parquet", - content_encoding="identity", - content_type_parameters=[ - PartialParquetParameters.of( - pq_metadata=papq.ParquetFile(PARQUET_FILE_PATH_WITH_STATS).metadata - ) - ], - ) - return ManifestEntry.of( - url=PARQUET_FILE_PATH_WITH_STATS, - uri=PARQUET_FILE_PATH_WITH_STATS, - mandatory=True, - uuid="test", - meta=manifest_meta, - ) - - -class TestEstimateManifestEntryColumnSizeBytes: - def test_when_no_columns_passed_sanity( - self, sample_no_stats_entry, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 1, False - ) - ) - == 0 - ) - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_with_stats_entry, 1, False - ) - ) - == 0 - ) - - def test_when_no_columns_passed_with_intelligent_estimation( - self, sample_no_stats_entry, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 1, True - ) - ) - == 0 - ) - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_with_stats_entry, 1, True - ) - ) - == 0 - ) - - def test_when_one_string_column_passed( - self, sample_no_stats_entry, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 1, False, ["first_name"] - ) - ) - == 2988 - ) - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_with_stats_entry, 1, False, ["first_name"] - ) - ) - == 2989 - ) - - def test_when_invalid_column_passed_assumes_null(self, sample_no_stats_entry): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 1, False, ["invalid_column"] - ) - ) - == 4000 - ) - - def test_when_multiple_columns_passed( - self, sample_no_stats_entry, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 1, False, ["first_name", "id"] - ) - ) - == 7031 - ) - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_with_stats_entry, 1, False, ["first_name", "id"] - ) - ) - == 8314 - ) - - def test_when_timestamp_column_passed( - self, sample_no_stats_entry, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 2, False, ["registration_dttm"] - ) - ) - == 26540 - ) - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_with_stats_entry, 2, False, ["registration_dttm"] - ) - ) - == 18602 - ) - - def test_when_intelligent_estimation_enabled_single_column( - self, sample_no_stats_entry, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 1, True, ["first_name"] - ) - ) - == 2988 - ) - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_with_stats_entry, 1, True, ["first_name"] - ) - ) - == 7000 - ) - - def test_when_intelligent_estimation_enabled_timestamp_column( - self, sample_no_stats_entry, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 1, True, ["registration_dttm"] - ) - ) - == 12000 - ) - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_with_stats_entry, 1, True, ["registration_dttm"] - ) - ) - == 8000 - ) - - def test_when_intelligent_estimation_enabled_int_column( - self, sample_no_stats_entry, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 1, True, ["id"] - ) - ) - == 4000 - ) - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_with_stats_entry, 1, True, ["id"] - ) - ) - == 4000 - ) - - def test_when_intelligent_estimation_enabled_double_column( - self, sample_no_stats_entry, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 1, True, ["salary"] - ) - ) - == 8000 - ) - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_with_stats_entry, 1, True, ["salary"] - ) - ) - == 8000 - ) - - def test_when_intelligent_estimation_enabled_multiple_columns( - self, sample_no_stats_entry, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_no_stats_entry, 1, True, ["first_name", "id"] - ) - ) - == 6988 - ) - assert ( - int( - estimate_manifest_entry_column_size_bytes( - sample_with_stats_entry, 1, True, ["first_name", "id"] - ) - ) - == 11000 - ) - - def test_when_intelligent_estimation_enabled_with_no_type_params( - self, sample_with_no_type_params - ): - assert ( - estimate_manifest_entry_column_size_bytes( - sample_with_no_type_params, 1, True, ["first_name"] - ) - is None - ) - - -class TestEstimateManifestEntryNumRows: - def test_sanity(self, sample_no_stats_entry): - assert ( - estimate_manifest_entry_num_rows(sample_no_stats_entry, 1000, 7, 1, False) - == 1000 - ) - - def test_when_previous_inflation_forced(self, sample_no_stats_entry): - assert ( - estimate_manifest_entry_num_rows(sample_no_stats_entry, 1000, 7, 1, True) - == 795 - ) - - def test_when_type_params_absent(self, sample_with_no_type_params): - assert ( - estimate_manifest_entry_num_rows( - sample_with_no_type_params, 1000, 7, 1, True - ) - == 795 - ) - - -class TestEstimateManifestEntrySizeBytes: - def test_sanity(self, sample_no_stats_entry): - assert ( - int( - estimate_manifest_entry_size_bytes( - sample_no_stats_entry, 7, 2, False, False - ) - ) - == 224984 - ) - - def test_when_previous_inflation_forced(self, sample_no_stats_entry): - # using previous inflation has turned out to be inaccurate - assert ( - int( - estimate_manifest_entry_size_bytes( - sample_no_stats_entry, 7, 2, True, False - ) - ) - == 795403 - ) - - def test_when_type_params_absent(self, sample_with_no_type_params): - assert ( - int( - estimate_manifest_entry_size_bytes( - sample_with_no_type_params, 7, 2, False, True - ) - ) - == 795403 - ) - - def test_when_intelligent_estimation_is_enabled(self, sample_no_stats_entry): - assert ( - int( - estimate_manifest_entry_size_bytes( - sample_no_stats_entry, 7, 2, False, True - ) - ) - == 223096 - ) - - def test_when_intelligent_estimation_is_enabled_with_stats( - self, sample_with_stats_entry - ): - assert ( - int( - estimate_manifest_entry_size_bytes( - sample_with_stats_entry, 7, 2, False, True - ) - ) - == 290222 - ) From f08e2c1f669a887a1367dc583bb3b32b92ef6c3d Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Thu, 26 Sep 2024 15:28:50 -0700 Subject: [PATCH 11/25] address comments --- .../compute/compactor/model/compact_partition_params.py | 6 +++++- deltacat/compute/compactor_v2/utils/content_type_params.py | 6 +++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/deltacat/compute/compactor/model/compact_partition_params.py b/deltacat/compute/compactor/model/compact_partition_params.py index 96d6623d..4641bfef 100644 --- a/deltacat/compute/compactor/model/compact_partition_params.py +++ b/deltacat/compute/compactor/model/compact_partition_params.py @@ -452,7 +452,7 @@ def enable_input_split(self) -> bool: When this is True, the input split will be always enabled for parquet files. The input split feature will split the parquet files into individual row groups so that we could process them in different nodes in parallel. - By default, input split is enabled for incremental compaction and disabled for rebase. + By default, input split is enabled for incremental compaction and disabled for rebase or backfill. """ return self["enable_input_split"] @@ -462,6 +462,10 @@ def enable_input_split(self, value: bool) -> None: @property def max_parquet_meta_size_bytes(self) -> int: + """ + The maximum size of the parquet metadata in bytes. Used for allocating tasks + to fetch parquet metadata. + """ return self["max_parquet_meta_size_bytes"] @max_parquet_meta_size_bytes.setter diff --git a/deltacat/compute/compactor_v2/utils/content_type_params.py b/deltacat/compute/compactor_v2/utils/content_type_params.py index 6cc2b193..65f3d814 100644 --- a/deltacat/compute/compactor_v2/utils/content_type_params.py +++ b/deltacat/compute/compactor_v2/utils/content_type_params.py @@ -35,7 +35,7 @@ def _contains_partial_parquet_parameters(entry: ManifestEntry) -> bool: @ray.remote -def _download_parquet_meta_for_manifest_entry( +def _download_parquet_metadata_for_manifest_entry( delta: Delta, entry_index: int, deltacat_storage: unimplemented_deltacat_storage, @@ -79,7 +79,7 @@ def append_content_type_params( if not entry_indices_to_download: logger.info( - "No parquet type params to down for " f"delta with locator {delta.locator}." + f"No parquet type params to download for delta with locator {delta.locator}." ) return @@ -101,7 +101,7 @@ def input_provider(index, item) -> Dict: ) pq_files_promise = invoke_parallel( entry_indices_to_download, - ray_task=_download_parquet_meta_for_manifest_entry, + ray_task=_download_parquet_metadata_for_manifest_entry, max_parallelism=task_max_parallelism, options_provider=options_provider, kwargs_provider=input_provider, From f2775abf46dcef26d58308f30744e203d97b0d4b Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Thu, 26 Sep 2024 18:38:08 -0700 Subject: [PATCH 12/25] Adding delta size estimation --- .../compute/resource_estimation/__init__.py | 14 + deltacat/compute/resource_estimation/delta.py | 263 +++++++++++ deltacat/compute/resource_estimation/model.py | 9 + deltacat/compute/resource_estimation/utils.py | 13 +- .../data/date_pk_table.csv | 8 + .../compute/resource_estimation/test_delta.py | 441 ++++++++++++++++++ deltacat/tests/test_utils/pyarrow.py | 20 +- 7 files changed, 756 insertions(+), 12 deletions(-) create mode 100644 deltacat/compute/resource_estimation/delta.py create mode 100644 deltacat/tests/compute/resource_estimation/data/date_pk_table.csv create mode 100644 deltacat/tests/compute/resource_estimation/test_delta.py diff --git a/deltacat/compute/resource_estimation/__init__.py b/deltacat/compute/resource_estimation/__init__.py index f7c2929c..1ebc9b70 100644 --- a/deltacat/compute/resource_estimation/__init__.py +++ b/deltacat/compute/resource_estimation/__init__.py @@ -1,11 +1,25 @@ from deltacat.compute.resource_estimation.model import ( ResourceEstimationMethod, EstimatedResources, + Statistics, EstimateResourcesParams, ) +from deltacat.compute.resource_estimation.utils import ( + estimate_manifest_entry_column_size_bytes, + estimate_manifest_entry_num_rows, + estimate_manifest_entry_size_bytes, +) +from deltacat.compute.resource_estimation.delta import ( + estimate_resources_required_to_process_delta, +) __all__ = [ "ResourceEstimationMethod", "EstimatedResources", "EstimateResourcesParams", + "Statistics", + "estimate_resources_required_to_process_delta", + "estimate_manifest_entry_size_bytes", + "estimate_manifest_entry_num_rows", + "estimate_manifest_entry_column_size_bytes", ] diff --git a/deltacat/compute/resource_estimation/delta.py b/deltacat/compute/resource_estimation/delta.py new file mode 100644 index 00000000..25327630 --- /dev/null +++ b/deltacat/compute/resource_estimation/delta.py @@ -0,0 +1,263 @@ +import logging +from typing import Optional, Dict, Any +from deltacat import logs +from deltacat.storage import ( + Delta, + interface as unimplemented_deltacat_storage, +) +from deltacat.compute.compactor_v2.utils.content_type_params import ( + append_content_type_params, +) +from deltacat.compute.resource_estimation.model import ( + OperationType, + EstimateResourcesParams, + ResourceEstimationMethod, + EstimatedResources, + Statistics, +) +from deltacat.compute.resource_estimation.utils import ( + estimate_manifest_entry_size_bytes, + estimate_manifest_entry_num_rows, +) + +logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) + + +def _estimate_resources_required_to_process_delta_using_previous_inflation( + delta: Delta, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams, + deltacat_storage: unimplemented_deltacat_storage, + deltacat_storage_kwargs: Dict[str, Any], + **kwargs, +) -> Optional[EstimatedResources]: + + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" + assert ( + estimate_resources_params.previous_inflation + ), "Previous inflation must be provided to estimate delta size" + + in_memory_size = ( + delta.meta.content_length * estimate_resources_params.previous_inflation + ) + num_rows = int(in_memory_size / estimate_resources_params.average_record_size_bytes) + + return EstimatedResources.of( + memory_bytes=in_memory_size, + statistics=Statistics.of( + in_memory_size_bytes=in_memory_size, + record_count=num_rows, + on_disk_size_bytes=delta.meta.content_length, + ), + ) + + +def _estimate_resources_required_to_process_delta_using_type_params( + delta: Delta, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams, + deltacat_storage: unimplemented_deltacat_storage, + deltacat_storage_kwargs: Dict[str, Any], + **kwargs, +) -> Optional[EstimatedResources]: + + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" + + if not delta.manifest: + delta.manifest = deltacat_storage.get_delta_manifest( + delta.locator, + **deltacat_storage_kwargs, + ) + + if not delta.manifest or not delta.manifest.entries: + return EstimatedResources.of( + memory_bytes=0, + statistics=Statistics.of( + in_memory_size_bytes=0, + record_count=0, + on_disk_size_bytes=delta.meta.content_length, + ), + ) + + append_content_type_params( + delta=delta, + deltacat_storage=deltacat_storage, + deltacat_storage_kwargs=deltacat_storage_kwargs, + ) + + in_memory_size = 0.0 + num_rows = 0 + + for entry in delta.manifest.entries: + cur_memory = estimate_manifest_entry_size_bytes( + entry=entry, + operation_type=operation_type, + estimate_resources_params=estimate_resources_params, + **kwargs, + ) + cur_num_rows = estimate_manifest_entry_num_rows( + entry=entry, + operation_type=operation_type, + estimate_resources_params=estimate_resources_params, + **kwargs, + ) + + if not cur_memory or not cur_num_rows: + return None + + in_memory_size += cur_memory + num_rows += cur_num_rows + + return EstimatedResources.of( + memory_bytes=in_memory_size, + statistics=Statistics.of( + in_memory_size_bytes=in_memory_size, + record_count=num_rows, + on_disk_size_bytes=delta.meta.content_length, + ), + ) + + +def _estimate_resources_required_to_process_delta_using_file_sampling( + delta: Delta, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams, + deltacat_storage: unimplemented_deltacat_storage, + deltacat_storage_kwargs: Dict[str, Any], + **kwargs, +) -> Optional[EstimatedResources]: + + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" + + if not delta.manifest: + delta.manifest = deltacat_storage.get_delta_manifest( + delta.locator, + **deltacat_storage_kwargs, + ) + + if not delta.manifest or not delta.manifest.entries: + return EstimatedResources.of( + memory_bytes=0, + statistics=Statistics.of( + in_memory_size_bytes=0, + record_count=0, + on_disk_size_bytes=delta.meta.content_length, + ), + ) + + assert ( + estimate_resources_params.max_files_to_sample is not None + ), "Expected max_files_to_sample when resource estimation method is FILE_SAMPLING" + + sampled_in_memory_size = 0.0 + sampled_on_disk_size = 0.0 + sampled_num_rows = 0 + + for entry_index in range( + min(estimate_resources_params.max_files_to_sample, len(delta.manifest.entries)) + ): + tbl = deltacat_storage.download_delta_manifest_entry( + delta, + entry_index, + **deltacat_storage_kwargs, + ) + sampled_in_memory_size += tbl.nbytes + sampled_on_disk_size += delta.manifest.entries[entry_index].meta.content_length + sampled_num_rows += len(tbl) + + if not sampled_on_disk_size: + return EstimatedResources.of( + memory_bytes=0, + statistics=Statistics.of( + in_memory_size_bytes=0, + record_count=0, + on_disk_size_bytes=delta.meta.content_length, + ), + ) + + sampled_inflation = sampled_in_memory_size / sampled_on_disk_size + + in_memory_size = sampled_inflation * delta.meta.content_length + num_rows = int(in_memory_size / sampled_in_memory_size * sampled_num_rows) + + return EstimatedResources.of( + memory_bytes=in_memory_size, + statistics=Statistics.of( + in_memory_size_bytes=in_memory_size, + record_count=num_rows, + on_disk_size_bytes=delta.meta.content_length, + ), + ) + + +RESOURCE_ESTIMATION_METHOD_TO_DELTA_RESOURCE_ESTIMATION_FUNCTIONS = { + ResourceEstimationMethod.PREVIOUS_INFLATION: [ + _estimate_resources_required_to_process_delta_using_previous_inflation + ], + ResourceEstimationMethod.CONTENT_TYPE_META: [ + _estimate_resources_required_to_process_delta_using_type_params + ], + ResourceEstimationMethod.INTELLIGENT_ESTIMATION: [ + _estimate_resources_required_to_process_delta_using_type_params, + ], + ResourceEstimationMethod.FILE_SAMPLING: [ + _estimate_resources_required_to_process_delta_using_file_sampling + ], + ResourceEstimationMethod.DEFAULT: [ + _estimate_resources_required_to_process_delta_using_previous_inflation, + ], +} + + +def estimate_resources_required_to_process_delta( + delta: Delta, + operation_type: OperationType, + estimate_resources_params: EstimateResourcesParams = None, + deltacat_storage=unimplemented_deltacat_storage, + deltacat_storage_kwargs: Dict[str, Any] = {}, + **kwargs, +) -> Optional[EstimatedResources]: + assert ( + operation_type == OperationType.PYARROW_DOWNLOAD + ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" + + if delta.meta.record_count and delta.meta.source_content_length: + # No need to estimate + return EstimatedResources.of( + memory_bytes=delta.meta.source_content_length, + statistics=Statistics.of( + in_memory_size_bytes=delta.meta.source_content_length, + record_count=delta.meta.record_count, + on_disk_size_bytes=delta.meta.content_length, + ), + ) + + if estimate_resources_params is None: + estimate_resources_params = EstimateResourcesParams.of() + + functions = RESOURCE_ESTIMATION_METHOD_TO_DELTA_RESOURCE_ESTIMATION_FUNCTIONS.get( + estimate_resources_params.resource_estimation_method + ) + + for func in functions: + resources = func( + delta=delta, + operation_type=operation_type, + estimate_resources_params=estimate_resources_params, + deltacat_storage=deltacat_storage, + deltacat_storage_kwargs=deltacat_storage_kwargs, + **kwargs, + ) + if resources is not None: + logger.debug( + f"Estimated resources for delta={delta.locator} is {resources} using {func}" + ) + return resources + + return None diff --git a/deltacat/compute/resource_estimation/model.py b/deltacat/compute/resource_estimation/model.py index 877c09ff..c0d05c1c 100644 --- a/deltacat/compute/resource_estimation/model.py +++ b/deltacat/compute/resource_estimation/model.py @@ -14,6 +14,15 @@ class ResourceEstimationMethod(str, Enum): DEFAULT = "DEFAULT" + """ + This approach combines intelligent estimation and inflation based methods + and runs them in the order specified below: + 1. INTELLIGENT_ESTIMATION + 2. FILE_SAMPLING + 3. PREVIOUS_INFLATION + """ + DEFAULT_V2 = "DEFAULT_V2" + """ This approach strictly uses previous inflation and average record size to arrive at a resource estimate. It requires users to pass in previous inflation and average diff --git a/deltacat/compute/resource_estimation/utils.py b/deltacat/compute/resource_estimation/utils.py index 2882c039..36882181 100644 --- a/deltacat/compute/resource_estimation/utils.py +++ b/deltacat/compute/resource_estimation/utils.py @@ -68,7 +68,7 @@ def _calculate_parquet_column_size( def _estimate_manifest_entry_size_bytes_using_previous_inflation( entry: ManifestEntry, operation_type: OperationType, - estimate_resources_params: EstimateResourcesParams = None, + estimate_resources_params: EstimateResourcesParams, **kwargs, ) -> Optional[float]: @@ -91,7 +91,7 @@ def _estimate_manifest_entry_size_bytes_using_previous_inflation( def _estimate_manifest_entry_size_bytes_using_content_type_meta( entry: ManifestEntry, operation_type: OperationType, - estimate_resources_params: EstimateResourcesParams = None, + estimate_resources_params: EstimateResourcesParams, **kwargs, ) -> Optional[float]: @@ -126,7 +126,7 @@ def _estimate_manifest_entry_size_bytes_using_content_type_meta( def _estimate_manifest_entry_size_bytes_using_intelligent_estimation( entry: ManifestEntry, operation_type: OperationType, - estimate_resources_params: EstimateResourcesParams = None, + estimate_resources_params: EstimateResourcesParams, **kwargs, ) -> Optional[float]: @@ -182,7 +182,7 @@ def _estimate_manifest_entry_size_bytes_using_intelligent_estimation( def _estimate_manifest_entry_num_rows_using_previous_inflation( entry: ManifestEntry, operation_type: OperationType, - estimate_resources_params: EstimateResourcesParams = None, + estimate_resources_params: EstimateResourcesParams, **kwargs, ) -> Optional[int]: assert ( @@ -214,7 +214,7 @@ def _estimate_manifest_entry_num_rows_using_previous_inflation( def _estimate_manifest_entry_num_rows_using_content_type_meta( entry: ManifestEntry, operation_type: OperationType, - estimate_resources_params: EstimateResourcesParams = None, + estimate_resources_params: EstimateResourcesParams, **kwargs, ) -> Optional[int]: assert ( @@ -238,7 +238,7 @@ def _estimate_manifest_entry_num_rows_using_content_type_meta( def _estimate_manifest_entry_num_rows_using_intelligent_estimation( entry: ManifestEntry, operation_type: OperationType, - estimate_resources_params: EstimateResourcesParams = None, + estimate_resources_params: EstimateResourcesParams, **kwargs, ) -> Optional[int]: assert ( @@ -348,7 +348,6 @@ def estimate_manifest_entry_num_rows( ) for func in functions: - print(func) num_rows = func( entry=entry, operation_type=operation_type, diff --git a/deltacat/tests/compute/resource_estimation/data/date_pk_table.csv b/deltacat/tests/compute/resource_estimation/data/date_pk_table.csv new file mode 100644 index 00000000..877f7a27 --- /dev/null +++ b/deltacat/tests/compute/resource_estimation/data/date_pk_table.csv @@ -0,0 +1,8 @@ +pk,value +2022-10-21,1 +2022-10-20,2 +2022-10-20,3 +2022-11-24,4 +2022-12-20,5 +2022-10-21,6 +2023-10-23,7 diff --git a/deltacat/tests/compute/resource_estimation/test_delta.py b/deltacat/tests/compute/resource_estimation/test_delta.py new file mode 100644 index 00000000..b94dfdeb --- /dev/null +++ b/deltacat/tests/compute/resource_estimation/test_delta.py @@ -0,0 +1,441 @@ +import deltacat.tests.local_deltacat_storage as ds +from deltacat.types.media import ContentType +import os +import pytest +from deltacat.storage import Delta +from deltacat.compute.resource_estimation.delta import ( + estimate_resources_required_to_process_delta, +) +from deltacat.compute.resource_estimation.model import ( + OperationType, + EstimateResourcesParams, + ResourceEstimationMethod, +) + +DELTA_CSV_FILE_PATH = ( + "deltacat/tests/compute/resource_estimation/data/date_pk_table.csv" +) + +""" +Function scoped fixtures +""" + + +@pytest.fixture(scope="function") +def local_deltacat_storage_kwargs(): + DATABASE_FILE_PATH_KEY, DATABASE_FILE_PATH_VALUE = ( + "db_file_path", + "deltacat/tests/local_deltacat_storage/db_test.sqlite", + ) + # see deltacat/tests/local_deltacat_storage/README.md for documentation + kwargs_for_local_deltacat_storage = { + DATABASE_FILE_PATH_KEY: DATABASE_FILE_PATH_VALUE, + } + yield kwargs_for_local_deltacat_storage + if os.path.exists(DATABASE_FILE_PATH_VALUE): + os.remove(DATABASE_FILE_PATH_VALUE) + + +@pytest.fixture(scope="function") +def parquet_delta_with_manifest(local_deltacat_storage_kwargs): + """ + These fixtures are function scoped as functions can modify the delta. + """ + from deltacat.tests.test_utils.pyarrow import create_delta_from_csv_file + + result = create_delta_from_csv_file( + "test_namespace", + file_paths=[DELTA_CSV_FILE_PATH], + content_type=ContentType.PARQUET, + **local_deltacat_storage_kwargs + ) + + result.meta["source_content_length"] = 0 + result.meta["record_count"] = 0 + for entry in result.manifest.entries: + entry.meta["source_content_length"] = 0 + entry.meta["record_count"] = 0 + + return result + + +@pytest.fixture(scope="function") +def utsv_delta_with_manifest(local_deltacat_storage_kwargs): + from deltacat.tests.test_utils.pyarrow import create_delta_from_csv_file + + result = create_delta_from_csv_file( + "test_namespace", + file_paths=[DELTA_CSV_FILE_PATH], + content_type=ContentType.UNESCAPED_TSV, + **local_deltacat_storage_kwargs + ) + + result.meta["source_content_length"] = 0 + result.meta["record_count"] = 0 + for entry in result.manifest.entries: + entry.meta["source_content_length"] = 0 + entry.meta["record_count"] = 0 + + return result + + +@pytest.fixture(scope="function") +def delta_without_manifest(local_deltacat_storage_kwargs): + from deltacat.tests.test_utils.pyarrow import create_delta_from_csv_file + + delta = create_delta_from_csv_file( + "test_namespace", + file_paths=[DELTA_CSV_FILE_PATH], + content_type=ContentType.PARQUET, + **local_deltacat_storage_kwargs + ) + + # now we intentionally remove manifest + delta.manifest = None + delta.meta["source_content_length"] = 0 + delta.meta["record_count"] = 0 + + return delta + + +@pytest.fixture(scope="function") +def delta_with_populated_meta(local_deltacat_storage_kwargs): + from deltacat.tests.test_utils.pyarrow import create_delta_from_csv_file + + delta = create_delta_from_csv_file( + "test_namespace", + file_paths=[DELTA_CSV_FILE_PATH], + content_type=ContentType.PARQUET, + **local_deltacat_storage_kwargs + ) + + return delta + + +class TestEstimateResourcesRequiredToProcessDelta: + def test_delta_with_prepopulated_meta_returns_directly( + self, local_deltacat_storage_kwargs, delta_with_populated_meta: Delta + ): + + result = estimate_resources_required_to_process_delta( + delta=delta_with_populated_meta, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + ) + + assert ( + result.memory_bytes == delta_with_populated_meta.meta.source_content_length + ) + assert ( + result.statistics.in_memory_size_bytes + == delta_with_populated_meta.meta.source_content_length + ) + assert ( + result.statistics.on_disk_size_bytes + == delta_with_populated_meta.meta.content_length + ) + assert ( + result.statistics.record_count + == delta_with_populated_meta.meta.record_count + ) + + def test_delta_manifest_empty_when_default_method( + self, local_deltacat_storage_kwargs, delta_without_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.DEFAULT, + previous_inflation=7, + average_record_size_bytes=1000, + ) + + result = estimate_resources_required_to_process_delta( + delta=delta_without_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert result.memory_bytes != delta_without_manifest.meta.source_content_length + assert ( + result.memory_bytes + == delta_without_manifest.meta.content_length * params.previous_inflation + ) + assert result.statistics.in_memory_size_bytes == result.memory_bytes + assert ( + result.statistics.on_disk_size_bytes + == delta_without_manifest.meta.content_length + ) + assert result.statistics.record_count == int( + result.memory_bytes / params.average_record_size_bytes + ) + + def test_delta_manifest_exists_when_default_method( + self, local_deltacat_storage_kwargs, parquet_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.DEFAULT, + previous_inflation=7, + average_record_size_bytes=1000, + ) + + result = estimate_resources_required_to_process_delta( + delta=parquet_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert ( + result.memory_bytes + != parquet_delta_with_manifest.meta.source_content_length + ) + assert ( + result.memory_bytes + == parquet_delta_with_manifest.meta.content_length + * params.previous_inflation + ) + assert result.statistics.in_memory_size_bytes == result.memory_bytes + assert ( + result.statistics.on_disk_size_bytes + == parquet_delta_with_manifest.meta.content_length + ) + assert result.statistics.record_count == int( + result.memory_bytes / params.average_record_size_bytes + ) + + def test_previous_inflation_arg_not_passed_when_default_method( + self, local_deltacat_storage_kwargs, parquet_delta_with_manifest: Delta + ): + with pytest.raises(AssertionError): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.DEFAULT, + average_record_size_bytes=1000, + ) + + estimate_resources_required_to_process_delta( + delta=parquet_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + def test_estimate_resources_params_not_passed_assumes_default( + self, local_deltacat_storage_kwargs, parquet_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + previous_inflation=7, + average_record_size_bytes=1000, + ) + + result = estimate_resources_required_to_process_delta( + delta=parquet_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert ( + result.memory_bytes + != parquet_delta_with_manifest.meta.source_content_length + ) + assert ( + result.memory_bytes + == parquet_delta_with_manifest.meta.content_length + * params.previous_inflation + ) + assert result.statistics.in_memory_size_bytes == result.memory_bytes + assert ( + result.statistics.on_disk_size_bytes + == parquet_delta_with_manifest.meta.content_length + ) + assert result.statistics.record_count == int( + result.memory_bytes / params.average_record_size_bytes + ) + + def test_delta_manifest_empty_when_content_type_meta( + self, local_deltacat_storage_kwargs, delta_without_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.CONTENT_TYPE_META, + parquet_to_pyarrow_inflation=2, + ) + + result = estimate_resources_required_to_process_delta( + delta=delta_without_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert delta_without_manifest.manifest is not None + assert int(result.memory_bytes) == 84 + assert int(result.statistics.in_memory_size_bytes) == 84 + assert ( + result.statistics.on_disk_size_bytes + == delta_without_manifest.meta.content_length + ) + assert result.statistics.record_count == 7 + + def test_delta_manifest_exists_when_content_type_meta( + self, local_deltacat_storage_kwargs, parquet_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.CONTENT_TYPE_META, + parquet_to_pyarrow_inflation=2, + ) + + result = estimate_resources_required_to_process_delta( + delta=parquet_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert parquet_delta_with_manifest.manifest is not None + assert int(result.memory_bytes) == 464 + assert int(result.statistics.in_memory_size_bytes) == int(result.memory_bytes) + assert ( + result.statistics.on_disk_size_bytes + == parquet_delta_with_manifest.meta.content_length + ) + assert result.statistics.record_count == 7 + + def test_delta_manifest_empty_when_intelligent_estimation( + self, local_deltacat_storage_kwargs, delta_without_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + parquet_to_pyarrow_inflation=2, + ) + + result = estimate_resources_required_to_process_delta( + delta=delta_without_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert delta_without_manifest.manifest is not None + assert int(result.memory_bytes) == 84 + assert int(result.statistics.in_memory_size_bytes) == 84 + assert ( + result.statistics.on_disk_size_bytes + == delta_without_manifest.meta.content_length + ) + assert result.statistics.record_count == 7 + + def test_delta_manifest_exists_when_intelligent_estimation( + self, local_deltacat_storage_kwargs, parquet_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + parquet_to_pyarrow_inflation=2, + ) + + result = estimate_resources_required_to_process_delta( + delta=parquet_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert parquet_delta_with_manifest.manifest is not None + assert int(result.memory_bytes) == 168 + assert int(result.statistics.in_memory_size_bytes) == int(result.memory_bytes) + assert ( + result.statistics.on_disk_size_bytes + == parquet_delta_with_manifest.meta.content_length + ) + assert result.statistics.record_count == 7 + + def test_delta_utsv_data_when_intelligent_estimation( + self, local_deltacat_storage_kwargs, utsv_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + parquet_to_pyarrow_inflation=2, + ) + + result = estimate_resources_required_to_process_delta( + delta=utsv_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert result is None + + def test_empty_delta_sampled_when_file_sampling( + self, local_deltacat_storage_kwargs, delta_without_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.FILE_SAMPLING, + max_files_to_sample=2, + ) + + result = estimate_resources_required_to_process_delta( + delta=delta_without_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert delta_without_manifest.manifest is not None + assert result.memory_bytes is not None + assert ( + result.statistics.on_disk_size_bytes + == delta_without_manifest.meta.content_length + ) + + def test_delta_manifest_parquet_when_file_sampling( + self, local_deltacat_storage_kwargs, parquet_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.FILE_SAMPLING, + max_files_to_sample=2, + ) + + result = estimate_resources_required_to_process_delta( + delta=parquet_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + assert result.memory_bytes is not None + assert ( + result.statistics.on_disk_size_bytes + == parquet_delta_with_manifest.meta.content_length + ) + + def test_delta_manifest_utsv_when_file_sampling( + self, local_deltacat_storage_kwargs, utsv_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.FILE_SAMPLING, + max_files_to_sample=2, + ) + + result = estimate_resources_required_to_process_delta( + delta=utsv_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + assert result.memory_bytes is not None + assert ( + result.statistics.on_disk_size_bytes + == utsv_delta_with_manifest.meta.content_length + ) diff --git a/deltacat/tests/test_utils/pyarrow.py b/deltacat/tests/test_utils/pyarrow.py index 94910e41..ec0cee6c 100644 --- a/deltacat/tests/test_utils/pyarrow.py +++ b/deltacat/tests/test_utils/pyarrow.py @@ -2,7 +2,7 @@ import pyarrow as pa from deltacat.storage import Delta, Partition, PartitionLocator, DeltaLocator import deltacat.tests.local_deltacat_storage as ds -from deltacat.types.media import StorageType +from deltacat.types.media import StorageType, ContentType def create_delta_from_csv_file( @@ -10,6 +10,7 @@ def create_delta_from_csv_file( file_paths: List[str], table_name: Optional[str] = None, table_version: int = 1, + content_type: ContentType = ContentType.PARQUET, *args, **kwargs, ) -> Delta: @@ -22,7 +23,7 @@ def create_delta_from_csv_file( **kwargs, ) committed_delta = commit_delta_to_staged_partition( - staged_partition, file_paths, *args, **kwargs + staged_partition, file_paths, content_type=content_type, *args, **kwargs ) return committed_delta @@ -45,10 +46,18 @@ def stage_partition_from_file_paths( def commit_delta_to_staged_partition( - staged_partition, file_paths: List[str], *args, **kwargs + staged_partition, + file_paths: List[str], + content_type: ContentType = ContentType.PARQUET, + *args, + **kwargs, ) -> Delta: committed_delta = commit_delta_to_partition( - staged_partition, *args, file_paths=file_paths, **kwargs + staged_partition, + *args, + file_paths=file_paths, + content_type=content_type, + **kwargs, ) ds.commit_partition(staged_partition, **kwargs) return committed_delta @@ -68,6 +77,7 @@ def download_delta(delta_like: Union[Delta, DeltaLocator], *args, **kwargs) -> D def commit_delta_to_partition( partition: Union[Partition, PartitionLocator], file_paths: List[str], + content_type: ContentType = ContentType.PARQUET, *args, **kwargs, ) -> Delta: @@ -83,6 +93,6 @@ def commit_delta_to_partition( tables.append(table) table = pa.concat_tables(tables) - staged_delta = ds.stage_delta(table, partition, **kwargs) + staged_delta = ds.stage_delta(table, partition, content_type=content_type, **kwargs) return ds.commit_delta(staged_delta, **kwargs) From d947ebcf0126d94c79a9e81db19abbc31d361966 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Thu, 26 Sep 2024 18:40:40 -0700 Subject: [PATCH 13/25] rename manifest module --- deltacat/compute/compactor_v2/utils/io.py | 4 +++- deltacat/compute/compactor_v2/utils/task_options.py | 2 +- deltacat/compute/resource_estimation/__init__.py | 2 +- deltacat/compute/resource_estimation/delta.py | 2 +- .../compute/resource_estimation/{utils.py => manifest.py} | 0 .../resource_estimation/{test_utils.py => test_manifest.py} | 2 +- 6 files changed, 7 insertions(+), 5 deletions(-) rename deltacat/compute/resource_estimation/{utils.py => manifest.py} (100%) rename deltacat/tests/compute/resource_estimation/{test_utils.py => test_manifest.py} (99%) diff --git a/deltacat/compute/compactor_v2/utils/io.py b/deltacat/compute/compactor_v2/utils/io.py index 2af79741..b0e10407 100644 --- a/deltacat/compute/compactor_v2/utils/io.py +++ b/deltacat/compute/compactor_v2/utils/io.py @@ -23,7 +23,9 @@ ) from deltacat.utils.metrics import metrics from deltacat.compute.compactor_v2.constants import DISCOVER_DELTAS_METRIC_PREFIX -from deltacat.compute.resource_estimation.utils import does_require_content_type_params +from deltacat.compute.resource_estimation.manifest import ( + does_require_content_type_params, +) from deltacat.compute.resource_estimation.model import OperationType logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) diff --git a/deltacat/compute/compactor_v2/utils/task_options.py b/deltacat/compute/compactor_v2/utils/task_options.py index 4b96a5ff..bce48359 100644 --- a/deltacat/compute/compactor_v2/utils/task_options.py +++ b/deltacat/compute/compactor_v2/utils/task_options.py @@ -13,7 +13,7 @@ from deltacat.compute.compactor_v2.utils.primary_key_index import ( hash_group_index_to_hash_bucket_indices, ) -from deltacat.compute.resource_estimation.utils import ( +from deltacat.compute.resource_estimation.manifest import ( estimate_manifest_entry_num_rows, estimate_manifest_entry_size_bytes, estimate_manifest_entry_column_size_bytes, diff --git a/deltacat/compute/resource_estimation/__init__.py b/deltacat/compute/resource_estimation/__init__.py index 1ebc9b70..0eadaaf0 100644 --- a/deltacat/compute/resource_estimation/__init__.py +++ b/deltacat/compute/resource_estimation/__init__.py @@ -4,7 +4,7 @@ Statistics, EstimateResourcesParams, ) -from deltacat.compute.resource_estimation.utils import ( +from deltacat.compute.resource_estimation.manifest import ( estimate_manifest_entry_column_size_bytes, estimate_manifest_entry_num_rows, estimate_manifest_entry_size_bytes, diff --git a/deltacat/compute/resource_estimation/delta.py b/deltacat/compute/resource_estimation/delta.py index 25327630..8d30e58d 100644 --- a/deltacat/compute/resource_estimation/delta.py +++ b/deltacat/compute/resource_estimation/delta.py @@ -15,7 +15,7 @@ EstimatedResources, Statistics, ) -from deltacat.compute.resource_estimation.utils import ( +from deltacat.compute.resource_estimation.manifest import ( estimate_manifest_entry_size_bytes, estimate_manifest_entry_num_rows, ) diff --git a/deltacat/compute/resource_estimation/utils.py b/deltacat/compute/resource_estimation/manifest.py similarity index 100% rename from deltacat/compute/resource_estimation/utils.py rename to deltacat/compute/resource_estimation/manifest.py diff --git a/deltacat/tests/compute/resource_estimation/test_utils.py b/deltacat/tests/compute/resource_estimation/test_manifest.py similarity index 99% rename from deltacat/tests/compute/resource_estimation/test_utils.py rename to deltacat/tests/compute/resource_estimation/test_manifest.py index 956622cd..4b34dec5 100644 --- a/deltacat/tests/compute/resource_estimation/test_utils.py +++ b/deltacat/tests/compute/resource_estimation/test_manifest.py @@ -2,7 +2,7 @@ from deltacat.storage import ManifestEntry, ManifestMeta import pyarrow.parquet as papq from deltacat.types.partial_download import PartialParquetParameters -from deltacat.compute.resource_estimation.utils import ( +from deltacat.compute.resource_estimation.manifest import ( estimate_manifest_entry_column_size_bytes, estimate_manifest_entry_num_rows, estimate_manifest_entry_size_bytes, From 6535254a1608a0cb17b1d46bab5fde574a282033 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Thu, 26 Sep 2024 19:16:41 -0700 Subject: [PATCH 14/25] Add more tests --- deltacat/compute/resource_estimation/delta.py | 7 +- .../compute/resource_estimation/manifest.py | 42 ++--- .../compute/resource_estimation/test_delta.py | 77 +++++++++ .../resource_estimation/test_manifest.py | 151 ++++++++++++++++++ 4 files changed, 244 insertions(+), 33 deletions(-) diff --git a/deltacat/compute/resource_estimation/delta.py b/deltacat/compute/resource_estimation/delta.py index 8d30e58d..44d1ee8a 100644 --- a/deltacat/compute/resource_estimation/delta.py +++ b/deltacat/compute/resource_estimation/delta.py @@ -106,7 +106,7 @@ def _estimate_resources_required_to_process_delta_using_type_params( **kwargs, ) - if not cur_memory or not cur_num_rows: + if cur_memory is None or cur_num_rows is None: return None in_memory_size += cur_memory @@ -212,6 +212,11 @@ def _estimate_resources_required_to_process_delta_using_file_sampling( ResourceEstimationMethod.DEFAULT: [ _estimate_resources_required_to_process_delta_using_previous_inflation, ], + ResourceEstimationMethod.DEFAULT_V2: [ + _estimate_resources_required_to_process_delta_using_type_params, + _estimate_resources_required_to_process_delta_using_file_sampling, + _estimate_resources_required_to_process_delta_using_previous_inflation, + ], } diff --git a/deltacat/compute/resource_estimation/manifest.py b/deltacat/compute/resource_estimation/manifest.py index 36882181..31ee07cb 100644 --- a/deltacat/compute/resource_estimation/manifest.py +++ b/deltacat/compute/resource_estimation/manifest.py @@ -75,12 +75,6 @@ def _estimate_manifest_entry_size_bytes_using_previous_inflation( assert ( operation_type == OperationType.PYARROW_DOWNLOAD ), "Size can only be estimated for PYARROW_DOWNLOAD operation" - assert ( - estimate_resources_params.resource_estimation_method - == ResourceEstimationMethod.PREVIOUS_INFLATION - or estimate_resources_params.resource_estimation_method - == ResourceEstimationMethod.DEFAULT - ), f"Expected PREVIOUS_INFLATION but found {estimate_resources_params.resource_estimation_method}" assert ( estimate_resources_params.previous_inflation is not None ), "Expected previous_inflation when resource estimation method is PREVIOUS_INFLATION" @@ -98,12 +92,6 @@ def _estimate_manifest_entry_size_bytes_using_content_type_meta( assert ( operation_type == OperationType.PYARROW_DOWNLOAD ), "Size can only be estimated for PYARROW_DOWNLOAD operation" - assert ( - estimate_resources_params.resource_estimation_method - == ResourceEstimationMethod.CONTENT_TYPE_META - or estimate_resources_params.resource_estimation_method - == ResourceEstimationMethod.DEFAULT - ), f"Expected CONTENT_TYPE_META but found {estimate_resources_params.resource_estimation_method}" type_params = _get_parquet_type_params_if_exist(entry=entry) @@ -133,10 +121,6 @@ def _estimate_manifest_entry_size_bytes_using_intelligent_estimation( assert ( operation_type == OperationType.PYARROW_DOWNLOAD ), "Size can only be estimated for PYARROW_DOWNLOAD operation" - assert ( - estimate_resources_params.resource_estimation_method - == ResourceEstimationMethod.INTELLIGENT_ESTIMATION - ), f"Expected INTELLIGENT_ESTIMATION but found {estimate_resources_params.resource_estimation_method}" type_params = _get_parquet_type_params_if_exist(entry=entry) @@ -176,6 +160,10 @@ def _estimate_manifest_entry_size_bytes_using_intelligent_estimation( _estimate_manifest_entry_size_bytes_using_content_type_meta, _estimate_manifest_entry_size_bytes_using_previous_inflation, ], + ResourceEstimationMethod.DEFAULT_V2: [ + _estimate_manifest_entry_size_bytes_using_intelligent_estimation, + _estimate_manifest_entry_size_bytes_using_previous_inflation, + ], } @@ -188,12 +176,6 @@ def _estimate_manifest_entry_num_rows_using_previous_inflation( assert ( operation_type == OperationType.PYARROW_DOWNLOAD ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" - assert ( - estimate_resources_params.resource_estimation_method - == ResourceEstimationMethod.PREVIOUS_INFLATION - or estimate_resources_params.resource_estimation_method - == ResourceEstimationMethod.DEFAULT - ), f"Expected PREVIOUS_INFLATION but found {estimate_resources_params.resource_estimation_method}" assert ( estimate_resources_params.previous_inflation is not None ), "Expected previous_inflation when resource estimation method is PREVIOUS_INFLATION" @@ -220,12 +202,6 @@ def _estimate_manifest_entry_num_rows_using_content_type_meta( assert ( operation_type == OperationType.PYARROW_DOWNLOAD ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" - assert ( - estimate_resources_params.resource_estimation_method - == ResourceEstimationMethod.CONTENT_TYPE_META - or estimate_resources_params.resource_estimation_method - == ResourceEstimationMethod.DEFAULT - ), f"Expected CONTENT_TYPE_META but found {estimate_resources_params.resource_estimation_method}" type_params = _get_parquet_type_params_if_exist(entry=entry) @@ -244,10 +220,6 @@ def _estimate_manifest_entry_num_rows_using_intelligent_estimation( assert ( operation_type == OperationType.PYARROW_DOWNLOAD ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" - assert ( - estimate_resources_params.resource_estimation_method - == ResourceEstimationMethod.INTELLIGENT_ESTIMATION - ), f"Expected INTELLIGENT_ESTIMATION but found {estimate_resources_params.resource_estimation_method}" type_params = _get_parquet_type_params_if_exist(entry=entry) @@ -271,6 +243,10 @@ def _estimate_manifest_entry_num_rows_using_intelligent_estimation( _estimate_manifest_entry_num_rows_using_content_type_meta, _estimate_manifest_entry_num_rows_using_previous_inflation, ], + ResourceEstimationMethod.DEFAULT_V2: [ + _estimate_manifest_entry_num_rows_using_intelligent_estimation, + _estimate_manifest_entry_num_rows_using_previous_inflation, + ], } @@ -392,6 +368,8 @@ def estimate_manifest_entry_column_size_bytes( is_intelligent_estimation = ( estimate_resources_params.resource_estimation_method == ResourceEstimationMethod.INTELLIGENT_ESTIMATION + or estimate_resources_params.resource_estimation_method + == ResourceEstimationMethod.DEFAULT_V2 ) columns_size = 0.0 diff --git a/deltacat/tests/compute/resource_estimation/test_delta.py b/deltacat/tests/compute/resource_estimation/test_delta.py index b94dfdeb..69fbd54c 100644 --- a/deltacat/tests/compute/resource_estimation/test_delta.py +++ b/deltacat/tests/compute/resource_estimation/test_delta.py @@ -439,3 +439,80 @@ def test_delta_manifest_utsv_when_file_sampling( result.statistics.on_disk_size_bytes == utsv_delta_with_manifest.meta.content_length ) + + def test_empty_delta_when_default_v2( + self, local_deltacat_storage_kwargs, delta_without_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + max_files_to_sample=2, + previous_inflation=7, + average_record_size_bytes=1000, + ) + + result = estimate_resources_required_to_process_delta( + delta=delta_without_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert delta_without_manifest.manifest is not None + assert result.memory_bytes is not None + assert ( + result.statistics.on_disk_size_bytes + == delta_without_manifest.meta.content_length + ) + + def test_parquet_delta_when_default_v2( + self, local_deltacat_storage_kwargs, parquet_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + max_files_to_sample=2, + previous_inflation=7, + average_record_size_bytes=1000, + parquet_to_pyarrow_inflation=1, + ) + + result = estimate_resources_required_to_process_delta( + delta=parquet_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert parquet_delta_with_manifest.manifest is not None + assert result.memory_bytes is not None + assert ( + result.statistics.on_disk_size_bytes + == parquet_delta_with_manifest.meta.content_length + ) + + def test_utsv_delta_when_default_v2( + self, local_deltacat_storage_kwargs, utsv_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + max_files_to_sample=2, + previous_inflation=7, + average_record_size_bytes=1000, + parquet_to_pyarrow_inflation=1, + ) + + result = estimate_resources_required_to_process_delta( + delta=utsv_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert utsv_delta_with_manifest.manifest is not None + assert result.memory_bytes is not None + assert ( + result.statistics.on_disk_size_bytes + == utsv_delta_with_manifest.meta.content_length + ) diff --git a/deltacat/tests/compute/resource_estimation/test_manifest.py b/deltacat/tests/compute/resource_estimation/test_manifest.py index 4b34dec5..272a655d 100644 --- a/deltacat/tests/compute/resource_estimation/test_manifest.py +++ b/deltacat/tests/compute/resource_estimation/test_manifest.py @@ -396,6 +396,37 @@ def test_when_intelligent_estimation_enabled_multiple_columns( == 11000 ) + def test_when_default_v2_enabled_multiple_columns( + self, sample_no_stats_entry, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["first_name", "id"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 6988 + ) + assert ( + int( + estimate_manifest_entry_column_size_bytes( + sample_with_stats_entry, + columns=["first_name", "id"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 11000 + ) + def test_when_intelligent_estimation_enabled_with_no_type_params( self, sample_with_no_type_params ): @@ -432,6 +463,24 @@ def test_when_previous_inflation_method_with_no_type_params( is None ) + def test_when_default_v2_method_with_no_type_params( + self, sample_with_no_type_params + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + + assert ( + estimate_manifest_entry_column_size_bytes( + sample_with_no_type_params, + columns=["first_name"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + is None + ) + class TestEstimateManifestEntryNumRows: def test_sanity(self, sample_no_stats_entry): @@ -541,6 +590,55 @@ def test_when_type_params_absent_previous_inflation( == 795 ) + def test_when_type_params_absent_default_v2(self, sample_with_no_type_params): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + previous_inflation=7, + average_record_size_bytes=1000, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + + assert ( + estimate_manifest_entry_num_rows( + sample_with_no_type_params, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 795 # same as previous inflation + ) + + def test_when_type_params_no_stats_with_default_v2(self, sample_no_stats_entry): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + previous_inflation=7, + average_record_size_bytes=1000, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + assert ( + estimate_manifest_entry_num_rows( + sample_no_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 1000 + ) + + def test_when_type_params_with_default_v2(self, sample_with_stats_entry): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=1, + previous_inflation=7, + average_record_size_bytes=1000, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + assert ( + estimate_manifest_entry_num_rows( + sample_with_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 1000 + ) + class TestEstimateManifestEntrySizeBytes: def test_sanity(self, sample_no_stats_entry): @@ -715,3 +813,56 @@ def test_when_type_params_with_content_type_meta_method( ) == 227794 ) + + def test_when_type_params_with_stats_default_v2_method( + self, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_with_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 290222 # same result as intelligent estimation + ) + + def test_when_type_params_without_stats_default_v2_method( + self, sample_no_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + assert ( + int( + estimate_manifest_entry_size_bytes( + sample_no_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + ) + == 223096 + ) + + def test_when_no_type_params_default_v2_method(self, sample_with_no_type_params): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=2, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + assert ( + estimate_manifest_entry_size_bytes( + sample_with_no_type_params, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 795403 # same as previous inflation + ) From 9e02e23a597cc765dcc86dd3473996d06c1e674e Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Fri, 27 Sep 2024 09:15:32 -0700 Subject: [PATCH 15/25] Fix requires content type params --- deltacat/compute/resource_estimation/manifest.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/deltacat/compute/resource_estimation/manifest.py b/deltacat/compute/resource_estimation/manifest.py index 31ee07cb..c13df36c 100644 --- a/deltacat/compute/resource_estimation/manifest.py +++ b/deltacat/compute/resource_estimation/manifest.py @@ -387,6 +387,6 @@ def does_require_content_type_params( resource_estimation_method: ResourceEstimationMethod, ) -> bool: return ( - resource_estimation_method == ResourceEstimationMethod.CONTENT_TYPE_META + resource_estimation_method == ResourceEstimationMethod.DEFAULT_V2 or resource_estimation_method == ResourceEstimationMethod.INTELLIGENT_ESTIMATION ) From c6a1b169b12b66d88082cd8065707458a4fb64c6 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Fri, 27 Sep 2024 09:23:44 -0700 Subject: [PATCH 16/25] Ugrade dependencies and bump version --- deltacat/__init__.py | 2 +- requirements.txt | 2 +- setup.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/deltacat/__init__.py b/deltacat/__init__.py index 167892a4..5a2e2eea 100644 --- a/deltacat/__init__.py +++ b/deltacat/__init__.py @@ -44,7 +44,7 @@ deltacat.logs.configure_deltacat_logger(logging.getLogger(__name__)) -__version__ = "1.1.18" +__version__ = "1.1.19" __all__ = [ diff --git a/requirements.txt b/requirements.txt index 824dcfb5..8707665f 100644 --- a/requirements.txt +++ b/requirements.txt @@ -2,7 +2,7 @@ # any changes here should also be reflected in setup.py "install_requires" aws-embedded-metrics == 3.2.0 boto3 ~= 1.34 -getdaft==0.2.31 +getdaft==0.3.4 numpy == 1.21.5 pandas == 1.3.5 pyarrow == 12.0.1 diff --git a/setup.py b/setup.py index be997147..d0be34df 100644 --- a/setup.py +++ b/setup.py @@ -47,7 +47,7 @@ def find_version(*paths): "typing-extensions == 4.4.0", "pymemcache == 4.0.0", "redis == 4.6.0", - "getdaft == 0.2.31", + "getdaft == 0.3.4", "schedule == 1.2.0", ], setup_requires=["wheel"], From 4fbc4c93efa3275f273493c40bfefd09d74c74a4 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Fri, 27 Sep 2024 10:43:20 -0700 Subject: [PATCH 17/25] Adding a case where files to sample is zero --- deltacat/compute/resource_estimation/delta.py | 6 +-- .../compute/resource_estimation/test_delta.py | 43 +++++++++++++++++++ 2 files changed, 46 insertions(+), 3 deletions(-) diff --git a/deltacat/compute/resource_estimation/delta.py b/deltacat/compute/resource_estimation/delta.py index 44d1ee8a..6b4e9474 100644 --- a/deltacat/compute/resource_estimation/delta.py +++ b/deltacat/compute/resource_estimation/delta.py @@ -151,9 +151,9 @@ def _estimate_resources_required_to_process_delta_using_file_sampling( ), ) - assert ( - estimate_resources_params.max_files_to_sample is not None - ), "Expected max_files_to_sample when resource estimation method is FILE_SAMPLING" + if not estimate_resources_params.max_files_to_sample: + # we cannot calculate if we cannot sample + return None sampled_in_memory_size = 0.0 sampled_on_disk_size = 0.0 diff --git a/deltacat/tests/compute/resource_estimation/test_delta.py b/deltacat/tests/compute/resource_estimation/test_delta.py index 69fbd54c..1b14f8c8 100644 --- a/deltacat/tests/compute/resource_estimation/test_delta.py +++ b/deltacat/tests/compute/resource_estimation/test_delta.py @@ -440,6 +440,23 @@ def test_delta_manifest_utsv_when_file_sampling( == utsv_delta_with_manifest.meta.content_length ) + def test_delta_manifest_utsv_when_file_sampling_zero_files_to_sample( + self, local_deltacat_storage_kwargs, utsv_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.FILE_SAMPLING, + max_files_to_sample=None, + ) + + result = estimate_resources_required_to_process_delta( + delta=utsv_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + assert result is None + def test_empty_delta_when_default_v2( self, local_deltacat_storage_kwargs, delta_without_manifest: Delta ): @@ -491,6 +508,32 @@ def test_parquet_delta_when_default_v2( == parquet_delta_with_manifest.meta.content_length ) + def test_parquet_delta_when_default_v2_and_files_to_sample_zero( + self, local_deltacat_storage_kwargs, parquet_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + max_files_to_sample=0, + previous_inflation=7, + average_record_size_bytes=1000, + parquet_to_pyarrow_inflation=1, + ) + + result = estimate_resources_required_to_process_delta( + delta=parquet_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert parquet_delta_with_manifest.manifest is not None + assert result.memory_bytes is not None + assert ( + result.statistics.on_disk_size_bytes + == parquet_delta_with_manifest.meta.content_length + ) + def test_utsv_delta_when_default_v2( self, local_deltacat_storage_kwargs, utsv_delta_with_manifest: Delta ): From f054c61168ecb4bf2a505ea32fd956bfe3009919 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Fri, 27 Sep 2024 12:04:24 -0700 Subject: [PATCH 18/25] Export operation type --- deltacat/compute/resource_estimation/__init__.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/deltacat/compute/resource_estimation/__init__.py b/deltacat/compute/resource_estimation/__init__.py index 0eadaaf0..c95ad1fa 100644 --- a/deltacat/compute/resource_estimation/__init__.py +++ b/deltacat/compute/resource_estimation/__init__.py @@ -3,6 +3,7 @@ EstimatedResources, Statistics, EstimateResourcesParams, + OperationType, ) from deltacat.compute.resource_estimation.manifest import ( estimate_manifest_entry_column_size_bytes, @@ -22,4 +23,5 @@ "estimate_manifest_entry_size_bytes", "estimate_manifest_entry_num_rows", "estimate_manifest_entry_column_size_bytes", + "OperationType", ] From 150359bbac6579cbd0bf14eb1b20779ff02ae547 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Fri, 27 Sep 2024 12:24:09 -0700 Subject: [PATCH 19/25] Support case when parquet to pyarrow inflation is none --- deltacat/compute/resource_estimation/delta.py | 3 ++ .../compute/resource_estimation/manifest.py | 24 +++++---- .../compute/resource_estimation/test_delta.py | 44 +++++++++++++++ .../resource_estimation/test_manifest.py | 53 +++++++++++++++++++ 4 files changed, 113 insertions(+), 11 deletions(-) diff --git a/deltacat/compute/resource_estimation/delta.py b/deltacat/compute/resource_estimation/delta.py index 6b4e9474..c95d079e 100644 --- a/deltacat/compute/resource_estimation/delta.py +++ b/deltacat/compute/resource_estimation/delta.py @@ -67,6 +67,9 @@ def _estimate_resources_required_to_process_delta_using_type_params( operation_type == OperationType.PYARROW_DOWNLOAD ), "Number of rows can only be estimated for PYARROW_DOWNLOAD operation" + if estimate_resources_params.parquet_to_pyarrow_inflation is None: + return None + if not delta.manifest: delta.manifest = deltacat_storage.get_delta_manifest( delta.locator, diff --git a/deltacat/compute/resource_estimation/manifest.py b/deltacat/compute/resource_estimation/manifest.py index c13df36c..30e8d7f1 100644 --- a/deltacat/compute/resource_estimation/manifest.py +++ b/deltacat/compute/resource_estimation/manifest.py @@ -95,16 +95,15 @@ def _estimate_manifest_entry_size_bytes_using_content_type_meta( type_params = _get_parquet_type_params_if_exist(entry=entry) - if not type_params: + if ( + not type_params + or estimate_resources_params.parquet_to_pyarrow_inflation is None + ): return None if not type_params.row_groups_to_download: return 0 - assert ( - estimate_resources_params.parquet_to_pyarrow_inflation is not None - ), "Expected parquet_to_pyarrow_inflation when resource estimation method is CONTENT_TYPE_META" - return ( type_params.in_memory_size_bytes * estimate_resources_params.parquet_to_pyarrow_inflation @@ -124,16 +123,15 @@ def _estimate_manifest_entry_size_bytes_using_intelligent_estimation( type_params = _get_parquet_type_params_if_exist(entry=entry) - if not type_params: + if ( + not type_params + or estimate_resources_params.parquet_to_pyarrow_inflation is None + ): return None if not type_params.row_groups_to_download: return 0 - assert ( - estimate_resources_params.parquet_to_pyarrow_inflation is not None - ), "Expected parquet_to_pyarrow_inflation when resource estimation method is INTELLIGENT_ESTIMATION" - column_names = [ type_params.pq_metadata.row_group(0).column(col).path_in_schema for col in range(type_params.pq_metadata.num_columns) @@ -356,7 +354,11 @@ def estimate_manifest_entry_column_size_bytes( type_params = _get_parquet_type_params_if_exist(entry=entry) - if not type_params or not type_params.pq_metadata: + if ( + not type_params + or not type_params.pq_metadata + or not estimate_resources_params.parquet_to_pyarrow_inflation + ): return None if not columns or not type_params.row_groups_to_download: diff --git a/deltacat/tests/compute/resource_estimation/test_delta.py b/deltacat/tests/compute/resource_estimation/test_delta.py index 1b14f8c8..8b17537c 100644 --- a/deltacat/tests/compute/resource_estimation/test_delta.py +++ b/deltacat/tests/compute/resource_estimation/test_delta.py @@ -357,6 +357,24 @@ def test_delta_manifest_exists_when_intelligent_estimation( ) assert result.statistics.record_count == 7 + def test_delta_manifest_exists_inflation_absent_when_intelligent_estimation( + self, local_deltacat_storage_kwargs, parquet_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.INTELLIGENT_ESTIMATION, + parquet_to_pyarrow_inflation=None, + ) + + result = estimate_resources_required_to_process_delta( + delta=parquet_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert result is None + def test_delta_utsv_data_when_intelligent_estimation( self, local_deltacat_storage_kwargs, utsv_delta_with_manifest: Delta ): @@ -559,3 +577,29 @@ def test_utsv_delta_when_default_v2( result.statistics.on_disk_size_bytes == utsv_delta_with_manifest.meta.content_length ) + + def test_parquet_delta_without_inflation_when_default_v2( + self, local_deltacat_storage_kwargs, parquet_delta_with_manifest: Delta + ): + params = EstimateResourcesParams.of( + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + max_files_to_sample=2, + previous_inflation=7, + average_record_size_bytes=1000, + parquet_to_pyarrow_inflation=None, # inflation is None + ) + + result = estimate_resources_required_to_process_delta( + delta=parquet_delta_with_manifest, + operation_type=OperationType.PYARROW_DOWNLOAD, + deltacat_storage=ds, + deltacat_storage_kwargs=local_deltacat_storage_kwargs, + estimate_resources_params=params, + ) + + assert parquet_delta_with_manifest.manifest is not None + assert result.memory_bytes is not None + assert ( + result.statistics.on_disk_size_bytes + == parquet_delta_with_manifest.meta.content_length + ) diff --git a/deltacat/tests/compute/resource_estimation/test_manifest.py b/deltacat/tests/compute/resource_estimation/test_manifest.py index 272a655d..997d250e 100644 --- a/deltacat/tests/compute/resource_estimation/test_manifest.py +++ b/deltacat/tests/compute/resource_estimation/test_manifest.py @@ -427,6 +427,24 @@ def test_when_default_v2_enabled_multiple_columns( == 11000 ) + def test_when_default_v2_enabled_multiple_columns_and_inflation_not_passed( + self, sample_no_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=None, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + + assert ( + estimate_manifest_entry_column_size_bytes( + sample_no_stats_entry, + columns=["first_name", "id"], + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + is None + ) + def test_when_intelligent_estimation_enabled_with_no_type_params( self, sample_with_no_type_params ): @@ -623,6 +641,24 @@ def test_when_type_params_no_stats_with_default_v2(self, sample_no_stats_entry): == 1000 ) + def test_when_type_params_parquet_inflation_absent_with_default_v2( + self, sample_no_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=None, + previous_inflation=7, + average_record_size_bytes=1000, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + assert ( + estimate_manifest_entry_num_rows( + sample_no_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 1000 + ) + def test_when_type_params_with_default_v2(self, sample_with_stats_entry): estimate_resources_params = EstimateResourcesParams.of( parquet_to_pyarrow_inflation=1, @@ -866,3 +902,20 @@ def test_when_no_type_params_default_v2_method(self, sample_with_no_type_params) ) == 795403 # same as previous inflation ) + + def test_when_type_params_but_inflation_absent_default_v2_method( + self, sample_with_stats_entry + ): + estimate_resources_params = EstimateResourcesParams.of( + parquet_to_pyarrow_inflation=None, + previous_inflation=7, + resource_estimation_method=ResourceEstimationMethod.DEFAULT_V2, + ) + assert ( + estimate_manifest_entry_size_bytes( + sample_with_stats_entry, + operation_type=OperationType.PYARROW_DOWNLOAD, + estimate_resources_params=estimate_resources_params, + ) + == 795403 # same as previous inflation + ) From 0bba5f6e230a45587666ced51eb1965fabb9e988 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Fri, 27 Sep 2024 17:14:37 -0700 Subject: [PATCH 20/25] Add caching in append_content_type_params to avoid redownloading parquet meta --- .../compactor_v2/utils/content_type_params.py | 39 ++++++++++++++++++- 1 file changed, 38 insertions(+), 1 deletion(-) diff --git a/deltacat/compute/compactor_v2/utils/content_type_params.py b/deltacat/compute/compactor_v2/utils/content_type_params.py index 65f3d814..130e242e 100644 --- a/deltacat/compute/compactor_v2/utils/content_type_params.py +++ b/deltacat/compute/compactor_v2/utils/content_type_params.py @@ -34,6 +34,25 @@ def _contains_partial_parquet_parameters(entry: ManifestEntry) -> bool: ) +APPEND_CONTENT_TYPE_PARAMS_CACHE = "append_content_type_params_cache" + + +@ray.remote +class AppendContentTypeParamsCache: + """ + This actor caches the delta that contains content type meta. + """ + + def __init__(self): + self.cache = {} + + def get(self, key): + return self.cache.get(key) + + def put(self, key, value): + self.cache[key] = value + + @ray.remote def _download_parquet_metadata_for_manifest_entry( delta: Delta, @@ -58,11 +77,15 @@ def _download_parquet_metadata_for_manifest_entry( def append_content_type_params( delta: Delta, - task_max_parallelism: int = TASK_MAX_PARALLELISM, + task_max_parallelism: int = TASK_MAX_PARALLELISM * 2, max_parquet_meta_size_bytes: Optional[int] = MAX_PARQUET_METADATA_SIZE, deltacat_storage=unimplemented_deltacat_storage, deltacat_storage_kwargs: Optional[Dict[str, Any]] = {}, ) -> None: + """ + This operation appends content type params into the delta entry. Note + that this operation can be time consuming, hence we cache it in a Ray actor. + """ if not delta.meta: logger.warning(f"Delta with locator {delta.locator} doesn't contain meta.") @@ -81,6 +104,18 @@ def append_content_type_params( logger.info( f"No parquet type params to download for delta with locator {delta.locator}." ) + return None + + cache = AppendContentTypeParamsCache.options( + name=APPEND_CONTENT_TYPE_PARAMS_CACHE, get_if_exists=True + ).remote() + + cached_value = ray.get(cache.get.remote(delta.locator.hexdigest())) + if cached_value is not None: + logger.info( + f"Using cached parquet meta for delta with locator {delta.locator}." + ) + delta.manifest = cached_value.manifest return options_provider = functools.partial( @@ -132,3 +167,5 @@ def input_provider(index, item) -> Dict: assert _contains_partial_parquet_parameters( entry ), "partial parquet params validation failed." + + ray.get(cache.put.remote(delta.locator.hexdigest(), delta)) From 8ab22d1e77778033cece81b0818a8cf029b16c46 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Fri, 27 Sep 2024 18:00:43 -0700 Subject: [PATCH 21/25] Only cache when the number of entries is high to avoid constant calls to actor --- .../compactor_v2/utils/content_type_params.py | 32 +++++++++++++------ 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/deltacat/compute/compactor_v2/utils/content_type_params.py b/deltacat/compute/compactor_v2/utils/content_type_params.py index 130e242e..309486b5 100644 --- a/deltacat/compute/compactor_v2/utils/content_type_params.py +++ b/deltacat/compute/compactor_v2/utils/content_type_params.py @@ -35,6 +35,9 @@ def _contains_partial_parquet_parameters(entry: ManifestEntry) -> bool: APPEND_CONTENT_TYPE_PARAMS_CACHE = "append_content_type_params_cache" +# At this point, it's better to fetch all parquet than to cache and +# call actor which is not expected to support high throughput. +MINIMUM_ENTRIES_TO_CACHE = 10 @ray.remote @@ -106,17 +109,25 @@ def append_content_type_params( ) return None - cache = AppendContentTypeParamsCache.options( - name=APPEND_CONTENT_TYPE_PARAMS_CACHE, get_if_exists=True - ).remote() - - cached_value = ray.get(cache.get.remote(delta.locator.hexdigest())) - if cached_value is not None: + if len(entry_indices_to_download) >= MINIMUM_ENTRIES_TO_CACHE: logger.info( - f"Using cached parquet meta for delta with locator {delta.locator}." + "Checking if cache contains parquet meta for " + f"delta locator {delta.locator}..." + ) + cache = AppendContentTypeParamsCache.options( + name=APPEND_CONTENT_TYPE_PARAMS_CACHE, get_if_exists=True + ).remote() + + cached_value = ray.get(cache.get.remote(delta.locator.hexdigest())) + if cached_value is not None: + logger.info( + f"Using cached parquet meta for delta with locator {delta.locator}." + ) + delta.manifest = cached_value.manifest + return + logger.info( + f"Cache doesn't contain parquet meta for delta with locator {delta.locator}." ) - delta.manifest = cached_value.manifest - return options_provider = functools.partial( append_content_type_params_options_provider, @@ -168,4 +179,5 @@ def input_provider(index, item) -> Dict: entry ), "partial parquet params validation failed." - ray.get(cache.put.remote(delta.locator.hexdigest(), delta)) + if len(entry_indices_to_download) >= MINIMUM_ENTRIES_TO_CACHE: + ray.get(cache.put.remote(delta.locator.hexdigest(), delta)) From 1a7a55932d87f373d5df24a124b16c1430bc8e84 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Fri, 27 Sep 2024 18:27:42 -0700 Subject: [PATCH 22/25] Add json context to logs --- deltacat/constants.py | 2 ++ deltacat/logs.py | 8 ++++++++ deltacat/tests/test_logs.py | 34 ++++++++++++++++++++++++++++++++++ 3 files changed, 44 insertions(+) diff --git a/deltacat/constants.py b/deltacat/constants.py index 73f02657..91799d4d 100644 --- a/deltacat/constants.py +++ b/deltacat/constants.py @@ -28,6 +28,8 @@ "DELTACAT_APP_DEBUG_LOG_BASE_FILE_NAME", "application.debug.log", ) +# A json context which will be logged along with other context args. +DELTACAT_LOGGER_CONTEXT = env_string("DELTACAT_LOGGER_CONTEXT", None) # Byte Units BYTES_PER_KIBIBYTE = 2**10 diff --git a/deltacat/logs.py b/deltacat/logs.py index a2e934f8..524079d5 100644 --- a/deltacat/logs.py +++ b/deltacat/logs.py @@ -17,6 +17,7 @@ DELTACAT_SYS_INFO_LOG_BASE_FILE_NAME, DELTACAT_APP_DEBUG_LOG_BASE_FILE_NAME, DELTACAT_SYS_DEBUG_LOG_BASE_FILE_NAME, + DELTACAT_LOGGER_CONTEXT, ) DEFAULT_LOG_LEVEL = "INFO" @@ -66,6 +67,13 @@ def __init__( self.ray_runtime_ctx = None self.context = {} + if DELTACAT_LOGGER_CONTEXT is not None: + try: + env_context = json.loads(DELTACAT_LOGGER_CONTEXT) + self.additional_context.update(env_context) + except Exception: + pass + def usesTime(self) -> bool: """ Overwritten to look for the attribute in the format dict values instead of the fmt string. diff --git a/deltacat/tests/test_logs.py b/deltacat/tests/test_logs.py index db8b19e5..27b6a8c4 100644 --- a/deltacat/tests/test_logs.py +++ b/deltacat/tests/test_logs.py @@ -1,6 +1,7 @@ import unittest import json import ray +from unittest import mock from logging import LogRecord from deltacat.logs import JsonFormatter @@ -190,3 +191,36 @@ def test_format_with_context_kwargs(self): ) self.assertFalse(ray.is_initialized()) self.assertNotIn("ray_runtime_context", json.loads(result)) + + @mock.patch("deltacat.logs.DELTACAT_LOGGER_CONTEXT", '{"DATABASE_URL": "mytemp"}') + def test_format_with_env_context_kwargs(self): + ray.shutdown() + formatter = JsonFormatter( + {"message": "msg"}, context_kwargs={"custom_key": "custom_val"} + ) + + record = LogRecord( + level="INFO", + name="test", + pathname="test", + lineno=0, + message="test_message", + msg="test_message", + args=None, + exc_info=None, + ) + + result = formatter.format(record) + + self.assertEqual( + { + "message": "test_message", + "additional_context": { + "custom_key": "custom_val", + "DATABASE_URL": "mytemp", + }, + }, + json.loads(result), + ) + self.assertFalse(ray.is_initialized()) + self.assertNotIn("ray_runtime_context", json.loads(result)) From 173116dedb7db1d42c56096e1b341001bb4b2ad1 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Fri, 27 Sep 2024 21:56:34 -0700 Subject: [PATCH 23/25] Ensure appropriate log level --- deltacat/compute/compactor/model/delta_annotated.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/deltacat/compute/compactor/model/delta_annotated.py b/deltacat/compute/compactor/model/delta_annotated.py index 051228ae..155713b8 100644 --- a/deltacat/compute/compactor/model/delta_annotated.py +++ b/deltacat/compute/compactor/model/delta_annotated.py @@ -114,7 +114,7 @@ def rebatch( # (i.e. the previous compaction round ran a rebase) if new_da and src_da.locator != new_da.locator: groups.append(new_da) - logger.info( + logger.debug( f"Due to different delta locator, Appending group of {da_group_entry_count} elements " f"and {new_da_bytes} bytes" ) @@ -133,12 +133,12 @@ def rebatch( or da_group_entry_count >= min_file_counts ): if new_da_bytes >= min_delta_bytes: - logger.info( + logger.debug( f"Appending group of {da_group_entry_count} elements " f"and {new_da_bytes} bytes to meet file size limit" ) if da_group_entry_count >= min_file_counts: - logger.info( + logger.debug( f"Appending group of {da_group_entry_count} elements " f"and {da_group_entry_count} files to meet file count limit" ) From 047f2cf1cbb3b91113e1e2050be3e9a56eb8f735 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Sat, 28 Sep 2024 01:12:19 -0700 Subject: [PATCH 24/25] Fix circular imports --- .../compactor_v2/utils/content_type_params.py | 22 ++++++++++++++++--- .../compactor_v2/utils/task_options.py | 8 ------- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/deltacat/compute/compactor_v2/utils/content_type_params.py b/deltacat/compute/compactor_v2/utils/content_type_params.py index 309486b5..32991130 100644 --- a/deltacat/compute/compactor_v2/utils/content_type_params.py +++ b/deltacat/compute/compactor_v2/utils/content_type_params.py @@ -16,13 +16,26 @@ from deltacat.types.media import TableType from deltacat.types.media import ContentType from deltacat.types.partial_download import PartialParquetParameters -from deltacat.compute.compactor_v2.utils.task_options import ( - append_content_type_params_options_provider, -) +from deltacat.exceptions import RetryableError logger = logs.configure_deltacat_logger(logging.getLogger(__name__)) +def append_content_type_params_options_provider( + index: int, item: Any, max_parquet_meta_size_bytes: int, **kwargs +) -> Dict: + task_opts = { + "num_cpus": 0.01, + "memory": max_parquet_meta_size_bytes, + "scheduling_strategy": "DEFAULT", + } + + task_opts["max_retries"] = 3 + task_opts["retry_exceptions"] = [RetryableError] + + return task_opts + + def _contains_partial_parquet_parameters(entry: ManifestEntry) -> bool: return ( entry.meta @@ -180,4 +193,7 @@ def input_provider(index, item) -> Dict: ), "partial parquet params validation failed." if len(entry_indices_to_download) >= MINIMUM_ENTRIES_TO_CACHE: + cache = AppendContentTypeParamsCache.options( + name=APPEND_CONTENT_TYPE_PARAMS_CACHE, get_if_exists=True + ).remote() ray.get(cache.put.remote(delta.locator.hexdigest(), delta)) diff --git a/deltacat/compute/compactor_v2/utils/task_options.py b/deltacat/compute/compactor_v2/utils/task_options.py index bce48359..53567f5d 100644 --- a/deltacat/compute/compactor_v2/utils/task_options.py +++ b/deltacat/compute/compactor_v2/utils/task_options.py @@ -163,14 +163,6 @@ def _get_merge_task_options( return _get_task_options(0.01, total_memory, ray_custom_resources) -def append_content_type_params_options_provider( - index: int, item: Any, max_parquet_meta_size_bytes: int, **kwargs -) -> Dict: - return _get_task_options( - 0.01, max_parquet_meta_size_bytes, scheduling_strategy="DEFAULT" - ) - - def hash_bucket_resource_options_provider( index: int, item: DeltaAnnotated, From 86c5d111b4b0c165ed116eb603357f2336916c19 Mon Sep 17 00:00:00 2001 From: Raghavendra Dani Date: Sat, 28 Sep 2024 13:43:50 -0700 Subject: [PATCH 25/25] Adding --- .../compactor_v2/utils/content_type_params.py | 30 +++++++++++++++---- 1 file changed, 24 insertions(+), 6 deletions(-) diff --git a/deltacat/compute/compactor_v2/utils/content_type_params.py b/deltacat/compute/compactor_v2/utils/content_type_params.py index 32991130..b206baa5 100644 --- a/deltacat/compute/compactor_v2/utils/content_type_params.py +++ b/deltacat/compute/compactor_v2/utils/content_type_params.py @@ -93,7 +93,7 @@ def _download_parquet_metadata_for_manifest_entry( def append_content_type_params( delta: Delta, - task_max_parallelism: int = TASK_MAX_PARALLELISM * 2, + task_max_parallelism: int = TASK_MAX_PARALLELISM, max_parquet_meta_size_bytes: Optional[int] = MAX_PARQUET_METADATA_SIZE, deltacat_storage=unimplemented_deltacat_storage, deltacat_storage_kwargs: Optional[Dict[str, Any]] = {}, @@ -122,19 +122,29 @@ def append_content_type_params( ) return None + ray_namespace = ray.get_runtime_context().namespace + logger.info( + f"Got Ray namespace: {ray_namespace}. " + "Note that caching only works with non-anonymous namespace." + "To set a non-anonymous namespace, call ray.init(namespace='X')." + ) if len(entry_indices_to_download) >= MINIMUM_ENTRIES_TO_CACHE: logger.info( - "Checking if cache contains parquet meta for " - f"delta locator {delta.locator}..." + f"Checking if cache contains parquet meta in namespace {ray_namespace} for " + f"delta locator {delta.locator} and digest {delta.locator.hexdigest()}..." ) cache = AppendContentTypeParamsCache.options( - name=APPEND_CONTENT_TYPE_PARAMS_CACHE, get_if_exists=True + name=APPEND_CONTENT_TYPE_PARAMS_CACHE, + namespace=ray_namespace, + get_if_exists=True, ).remote() + logger.info(f"Got cache actor: {cache}") cached_value = ray.get(cache.get.remote(delta.locator.hexdigest())) if cached_value is not None: logger.info( - f"Using cached parquet meta for delta with locator {delta.locator}." + "Using cached parquet meta for delta with locator" + f" {delta.locator} and digest {delta.locator.hexdigest()}." ) delta.manifest = cached_value.manifest return @@ -194,6 +204,14 @@ def input_provider(index, item) -> Dict: if len(entry_indices_to_download) >= MINIMUM_ENTRIES_TO_CACHE: cache = AppendContentTypeParamsCache.options( - name=APPEND_CONTENT_TYPE_PARAMS_CACHE, get_if_exists=True + name=APPEND_CONTENT_TYPE_PARAMS_CACHE, + namespace=ray_namespace, + get_if_exists=True, ).remote() + logger.info(f"Got cache actor when writing: {cache}") + logger.info( + f"Caching parquet meta for delta with locator {delta.locator} " + f"and digest {delta.locator.hexdigest()}..." + ) ray.get(cache.put.remote(delta.locator.hexdigest(), delta)) + assert ray.get(cache.get.remote(delta.locator.hexdigest())) is not None