diff --git a/.coveragerc b/.coveragerc index 1a9432923..080bef7e3 100644 --- a/.coveragerc +++ b/.coveragerc @@ -2,5 +2,5 @@ omit = tests/* vendor_loads_migration/* - libsys_airflow/plugins/folio/encumbrances/fix_encumbrances.py + libsys_airflow/plugins/folio/encumbrances/fix_encumbrances*.py libsys_airflow/dags/* \ No newline at end of file diff --git a/.gitignore b/.gitignore index 830aed927..e6aa27277 100644 --- a/.gitignore +++ b/.gitignore @@ -17,4 +17,5 @@ orafin-files/ vendor-data/ virtual-env/ vendor-keys/ -digital-bookplates/ \ No newline at end of file +digital-bookplates/ +authorities/ \ No newline at end of file diff --git a/docker-compose.yaml b/docker-compose.yaml index 7d21a201a..9d9a94599 100644 --- a/docker-compose.yaml +++ b/docker-compose.yaml @@ -94,6 +94,7 @@ x-airflow-common: - ${AIRFLOW_PROJ_DIR:-.}/data-export-files:/opt/airflow/data-export-files - ${AIRFLOW_PROJ_DIR:-.}/fix_encumbrances:/opt/airflow/fix_encumbrances - ${AIRFLOW_PROJ_DIR:-.}/orafin-files:/opt/airflow/orafin-files + - ${AIRFLOW_PROJ_DIR:-.}/authorities:/opt/airflow/authorities - ${AIRFLOW_PROJ_DIR:-.}/.aws:/home/airflow/.aws user: "${AIRFLOW_UID:-50000}:0" depends_on: diff --git a/libsys_airflow/dags/authority_control/load_records.py b/libsys_airflow/dags/authority_control/load_records.py new file mode 100644 index 000000000..00b815eb9 --- /dev/null +++ b/libsys_airflow/dags/authority_control/load_records.py @@ -0,0 +1,67 @@ +"""Load MARC Authority and Bibiliographic Records into FOLIO.""" + +import logging +from datetime import datetime + +from airflow.decorators import dag, task +from airflow.operators.python import get_current_context + +from libsys_airflow.plugins.authority_control import ( + clean_up, + email_report, + run_folio_data_import, +) + +logger = logging.getLogger(__name__) + + +@dag( + schedule=None, + start_date=datetime(2025, 2, 7), + catchup=False, + tags=["authorities", "folio"], +) +def load_marc_file(**kwargs): + """DAG loads an Authority or Bib File into FOLIO.""" + + @task + def prepare_file_upload(*args, **kwargs): + task_instance = kwargs["ti"] + context = get_current_context() + params = context.get("params", {}) + file_path = params["kwargs"].get("file") + if file_path is None: + raise ValueError("File path is required") + task_instance.xcom_push(key="file_path", value=file_path) + profile_name = params["kwargs"].get("profile") + if profile_name is None: + raise ValueError("Profile name is required") + task_instance.xcom_push(key="profile_name", value=profile_name) + return {"file_path": file_path, "profile_name": profile_name} + + @task + def initiate_folio_data_import(file_path, profile_name): + context = get_current_context() + bash_operator = run_folio_data_import(file_path, profile_name) + return bash_operator.execute(context) + + @task + def email_load_report(**kwargs): + return email_report(**kwargs) + + @task + def clean_up_dag(*args, **kwargs): + task_instance = kwargs["ti"] + marc_file_path = task_instance.xcom_pull(key="file_path") + return clean_up(marc_file_path) + + dag_params = prepare_file_upload() + bash_result = initiate_folio_data_import( + dag_params["file_path"], dag_params["profile_name"] + ) + + email_load_report(bash_result=bash_result) + bash_result >> clean_up_dag() + + +load_marc_file() diff --git a/libsys_airflow/dags/data_exports/full_dump_retrieval.py b/libsys_airflow/dags/data_exports/full_dump_retrieval.py index 469922efb..714ad37fc 100644 --- a/libsys_airflow/dags/data_exports/full_dump_retrieval.py +++ b/libsys_airflow/dags/data_exports/full_dump_retrieval.py @@ -6,11 +6,13 @@ from airflow import DAG from airflow.models.param import Param +from airflow.models import Variable from airflow.operators.empty import EmptyOperator from airflow.operators.python import get_current_context from airflow.decorators import task, task_group from libsys_airflow.plugins.data_exports.full_dump_marc import ( + create_materialized_view, fetch_number_of_records, fetch_full_dump_marc, reset_s3, @@ -54,11 +56,37 @@ type="integer", description="Number of batch processing jobs to run in parallel.", ), + "exclude_tags": Param( + True, + type="boolean", + description="Remove excluded tags listed in marc/excluded_tags.pyfrom incoming record.", + ), + "from_date": Param( + Variable.get("FOLIO_EPOCH_DATE", "2023-08-23"), + format="date", + type="string", + description="The earliest date to select record IDs from FOLIO.", + ), + "to_date": Param( + f"{(datetime.now()).strftime('%Y-%m-%d')}", + format="date", + type="string", + description="The latest date to select record IDs from FOLIO.", + ), + "recreate_view": Param( + False, + type="boolean", + description="Recreate the materialized view with the original FOLIO marc records to process.", + ), }, ) as dag: start = EmptyOperator(task_id='start') + @task + def create_full_selection_matrerialized_view(): + create_materialized_view() + @task def reset_s3_bucket(): reset_s3() @@ -131,8 +159,13 @@ def transform_marc_records_add_holdings(marc_files: list): @task def transform_marc_records_clean_serialize(marc_files: list): + context = get_current_context() + params = context.get("params", {}) # type: ignore + exclude_tags = params.get("exclude_tags", True) for marc_file in marc_files: - marc_clean_serialize(marc_file, full_dump=True) + marc_clean_serialize( + marc_file, full_dump=True, exclude_tags=exclude_tags + ) @task def compress_marc_files(marc_files: list): @@ -161,6 +194,8 @@ def compress_marc_files(marc_files: list): number_in_batch=batch_size, ) + create_view = create_full_selection_matrerialized_view() + delete_s3_files = reset_s3_bucket() start_stop = calculate_start_stop.partial(div=record_div).expand(job=number_of_jobs) @@ -175,5 +210,5 @@ def compress_marc_files(marc_files: list): task_id="finish_marc", ) - start >> delete_s3_files >> total_records + start >> create_view >> delete_s3_files >> total_records finish_transforms >> finish_processing_marc diff --git a/libsys_airflow/dags/folio_finance/lane_fix_encumbrances.py b/libsys_airflow/dags/folio_finance/lane_fix_encumbrances.py index c77d20e26..447538f93 100644 --- a/libsys_airflow/dags/folio_finance/lane_fix_encumbrances.py +++ b/libsys_airflow/dags/folio_finance/lane_fix_encumbrances.py @@ -22,7 +22,7 @@ "retry_delay": timedelta(minutes=1), } -FY_CODE = Variable.get("FISCAL_YEAR_CODE_LANE", "LANE2025") +FY_CODE = Variable.get("FISCAL_YEAR_CODE_LANE", "") with DAG( diff --git a/libsys_airflow/dags/folio_finance/law_fix_encumbrances.py b/libsys_airflow/dags/folio_finance/law_fix_encumbrances.py index d71c44ae5..ab05d6e16 100644 --- a/libsys_airflow/dags/folio_finance/law_fix_encumbrances.py +++ b/libsys_airflow/dags/folio_finance/law_fix_encumbrances.py @@ -22,7 +22,7 @@ "retry_delay": timedelta(minutes=1), } -FY_CODE = Variable.get("FISCAL_YEAR_CODE_LAW", "LAW2025") +FY_CODE = Variable.get("FISCAL_YEAR_CODE_LAW", "") with DAG( "fix_encumbrances_law", diff --git a/libsys_airflow/dags/folio_finance/sul_fix_encumbrances.py b/libsys_airflow/dags/folio_finance/sul_fix_encumbrances.py index 7a4dc923f..1243fa025 100644 --- a/libsys_airflow/dags/folio_finance/sul_fix_encumbrances.py +++ b/libsys_airflow/dags/folio_finance/sul_fix_encumbrances.py @@ -22,7 +22,7 @@ "retry_delay": timedelta(minutes=1), } -FY_CODE = Variable.get("FISCAL_YEAR_CODE_SUL", "SUL2025") +FY_CODE = Variable.get("FISCAL_YEAR_CODE_SUL", "") with DAG( diff --git a/libsys_airflow/plugins/authority_control/__init__.py b/libsys_airflow/plugins/authority_control/__init__.py new file mode 100644 index 000000000..1e1db85b0 --- /dev/null +++ b/libsys_airflow/plugins/authority_control/__init__.py @@ -0,0 +1,5 @@ +from libsys_airflow.plugins.authority_control.data_import import ( # noqa + run_folio_data_import, +) +from libsys_airflow.plugins.authority_control.email import email_report # noqa +from libsys_airflow.plugins.authority_control.helpers import clean_up # noqa diff --git a/libsys_airflow/plugins/authority_control/data_import.py b/libsys_airflow/plugins/authority_control/data_import.py new file mode 100644 index 000000000..adcd7ddc9 --- /dev/null +++ b/libsys_airflow/plugins/authority_control/data_import.py @@ -0,0 +1,34 @@ +import logging + +from airflow.models import Variable +from airflow.operators.bash import BashOperator + +logger = logging.getLogger(__name__) + + +def run_folio_data_import(file_path: str, profile_name: str): + """ + Run the folio data import + """ + args = [ + "python3 -m folio_data_import", + "--record-type MARC21", + "--gateway_url $gateway_url", + "--tenant_id sul", + "--username $username", + "--password $password", + "--marc_file_path $marc_file_path", + "--import_profile_name \"$profile_name\"", + ] + bash_operator = BashOperator( + task_id="run_folio_data_import", + bash_command=" ".join(args), + env={ + "gateway_url": Variable.get("OKAPI_URL"), + "username": Variable.get("FOLIO_USER"), + "password": Variable.get("FOLIO_PASSWORD"), + "marc_file_path": file_path, + "profile_name": profile_name, + }, + ) + return bash_operator diff --git a/libsys_airflow/plugins/authority_control/email.py b/libsys_airflow/plugins/authority_control/email.py new file mode 100644 index 000000000..fe2703f53 --- /dev/null +++ b/libsys_airflow/plugins/authority_control/email.py @@ -0,0 +1,38 @@ +import logging + +from airflow.models import Variable + +from libsys_airflow.plugins.shared.utils import ( + is_production, + send_email_with_server_name, + dag_run_url, +) + +logger = logging.getLogger(__name__) + + +def email_report(**kwargs): + """ + Emails result of folio-data-import to developers and SUL + + Args: + - bash_result (str): result of the bash command + """ + bash_result = kwargs.get("bash_result") + devs_email = Variable.get("EMAIL_DEVS") + to_emails = [devs_email] + + if is_production(): + sul_email = Variable.get("OCLC_EMAIL_SUL") + to_emails.append(sul_email) + + url_dag_run = dag_run_url(**kwargs) + body = ( + f"""
{bash_result}
DAG Run{url_dag_run}""" + ) + send_email_with_server_name( + to=to_emails, + subject="Folio Data Import", + html_content=body, + ) + logger.info(f"Emailing load report: {bash_result}") diff --git a/libsys_airflow/plugins/authority_control/helpers.py b/libsys_airflow/plugins/authority_control/helpers.py new file mode 100644 index 000000000..54c020195 --- /dev/null +++ b/libsys_airflow/plugins/authority_control/helpers.py @@ -0,0 +1,19 @@ +import logging +import pathlib + +logger = logging.getLogger(__name__) + + +def clean_up(marc_file: str, airflow: str = '/opt/airflow'): + """ + Moves marc file after running folio data import + """ + marc_file_path = pathlib.Path(marc_file) + archive_dir = pathlib.Path(airflow) / "authorities/archive" + archive_dir.mkdir(parents=True, exist_ok=True) + + archive_file = archive_dir / marc_file_path.name + marc_file_path.rename(archive_file) + + logger.info(f"Moved {marc_file_path} to archive") + return True diff --git a/libsys_airflow/plugins/data_exports/apps/data_export_upload_view.py b/libsys_airflow/plugins/data_exports/apps/data_export_upload_view.py index adb615373..1bde08adc 100644 --- a/libsys_airflow/plugins/data_exports/apps/data_export_upload_view.py +++ b/libsys_airflow/plugins/data_exports/apps/data_export_upload_view.py @@ -93,7 +93,7 @@ def run_data_export_upload(self): except Exception as e: flash(f"Error: {e}") finally: - return default_rendered_page(self) + return default_rendered_page(self) # noqa @expose("/") def data_export_upload_home(self): diff --git a/libsys_airflow/plugins/data_exports/full_dump_marc.py b/libsys_airflow/plugins/data_exports/full_dump_marc.py index 924032481..a48b9be2a 100644 --- a/libsys_airflow/plugins/data_exports/full_dump_marc.py +++ b/libsys_airflow/plugins/data_exports/full_dump_marc.py @@ -1,6 +1,9 @@ import logging +from datetime import datetime, timedelta +from pathlib import Path from s3path import S3Path +from typing import Union from airflow.models import Variable from airflow.operators.python import get_current_context @@ -10,6 +13,45 @@ logger = logging.getLogger(__name__) +def create_materialized_view(**kwargs) -> Union[str, None]: + context = get_current_context() + params = context.get("params", {}) # type: ignore + recreate = params.get("recreate_view", False) + from_date = params.get("from_date") + to_date = params.get( + "to_date", (datetime.now() + timedelta(1)).strftime('%Y-%m-%d') + ) + + query = None + if recreate: + with open(materialized_view_sql_file()) as sqf: + query = sqf.read() + + SQLExecuteQueryOperator( + task_id="postgres_full_count_query", + conn_id="postgres_folio", + database=kwargs.get("database", "okapi"), + sql=query, + parameters={ + "from_date": from_date, + "to_date": to_date, + }, + ).execute(context) + else: + logger.info("Skipping refresh of materialized view") + + return query + + +def materialized_view_sql_file(**kwargs) -> Path: + sql_path = ( + Path(kwargs.get("airflow", "/opt/airflow")) + / "libsys_airflow/plugins/data_exports/sql/materialized_view.sql" + ) + + return sql_path + + def fetch_full_dump_marc(**kwargs) -> str: offset = kwargs.get("offset") batch_size = kwargs.get("batch_size", 1000) diff --git a/libsys_airflow/plugins/data_exports/marc/excluded_tags.py b/libsys_airflow/plugins/data_exports/marc/excluded_tags.py index 697beda87..6244ff5f2 100644 --- a/libsys_airflow/plugins/data_exports/marc/excluded_tags.py +++ b/libsys_airflow/plugins/data_exports/marc/excluded_tags.py @@ -5,6 +5,7 @@ '005', '029', '049', + '079', '066', '099', '265', diff --git a/libsys_airflow/plugins/data_exports/marc/transforms.py b/libsys_airflow/plugins/data_exports/marc/transforms.py index 7e48c6256..1e6b7c9b9 100644 --- a/libsys_airflow/plugins/data_exports/marc/transforms.py +++ b/libsys_airflow/plugins/data_exports/marc/transforms.py @@ -3,7 +3,7 @@ import pathlib import pymarc -import xml.etree.ElementTree as etree +import xml.etree.ElementTree as etree # noqa from libsys_airflow.plugins.data_exports.marc.excluded_tags import excluded_tags from libsys_airflow.plugins.data_exports.marc.transformer import Transformer @@ -83,13 +83,13 @@ def leader_for_deletes(marc_file: str, full_dump: bool): def clean_and_serialize_marc_files(marc_file_list: dict): for kind, file_list in marc_file_list.items(): for filepath in file_list: - marc_clean_serialize(filepath, False) + marc_clean_serialize(filepath, False, True) logger.info( f"Removed MARC fields and serialized records for '{kind}' files: {filepath}" ) -def marc_clean_serialize(marc_file: str, full_dump: bool): +def marc_clean_serialize(marc_file: str, full_dump: bool, exclude_tags: bool): """ Removes MARC fields from export MARC21 file """ @@ -101,16 +101,16 @@ def marc_clean_serialize(marc_file: str, full_dump: bool): with marc_path.open('rb') as fo: marc_records = [record for record in pymarc.MARCReader(fo)] - logger.info(f"Removing MARC fields for {len(marc_records):,} records") - - for i, record in enumerate(marc_records): - try: - record.remove_fields(*excluded_tags) - if not i % 100: - logger.info(f"{i:,} records processed") - except AttributeError as e: - logger.warning(e) - continue + if exclude_tags: + logger.info(f"Removing MARC fields for {len(marc_records):,} records") + for i, record in enumerate(marc_records): + try: + record.remove_fields(*excluded_tags) + if not i % 100: + logger.info(f"{i:,} records processed") + except AttributeError as e: + logger.warning(e) + continue """ Writes the records back to the filesystem @@ -133,7 +133,7 @@ def marc_clean_serialize(marc_file: str, full_dump: bool): for record in marc_records: try: xml_element = pymarc.record_to_xml_node(record, namespace=True) - etree.fromstring(etree.tostring(xml_element)) + etree.fromstring(etree.tostring(xml_element)) # noqa xml_writer.write(record) except AttributeError as e: diff --git a/libsys_airflow/plugins/data_exports/oclc_api.py b/libsys_airflow/plugins/data_exports/oclc_api.py index 6207829d5..8577a3dd5 100644 --- a/libsys_airflow/plugins/data_exports/oclc_api.py +++ b/libsys_airflow/plugins/data_exports/oclc_api.py @@ -66,9 +66,9 @@ def oclc_records_operation(**kwargs) -> dict: logger.info(f"No {function_name} records for {library}") else: for library, records in type_of_records.items(): - for record in records: + for _record in records: logger.info( - f"Skipping OCLC API {function_name} record operation not in production" + f"Skipping OCLC API {function_name} record operation for {library} not in production" ) return {"success": success, "failures": failures} diff --git a/libsys_airflow/plugins/data_exports/oclc_reports.py b/libsys_airflow/plugins/data_exports/oclc_reports.py index 43f3abc74..d51a678ed 100644 --- a/libsys_airflow/plugins/data_exports/oclc_reports.py +++ b/libsys_airflow/plugins/data_exports/oclc_reports.py @@ -183,7 +183,8 @@ "new-oclc-marc-errors.html": new_oclc_invalid_records, "oclc-payload-template.html": oclc_payload_template, } - ) + ), + autoescape=True, ) diff --git a/libsys_airflow/plugins/data_exports/materialized_view.sql b/libsys_airflow/plugins/data_exports/sql/materialized_view.sql similarity index 84% rename from libsys_airflow/plugins/data_exports/materialized_view.sql rename to libsys_airflow/plugins/data_exports/sql/materialized_view.sql index d7c35e8ee..392cbd2de 100644 --- a/libsys_airflow/plugins/data_exports/materialized_view.sql +++ b/libsys_airflow/plugins/data_exports/sql/materialized_view.sql @@ -1,3 +1,5 @@ +DROP MATERIALIZED VIEW IF EXISTS data_export_marc +; create materialized view data_export_marc as select I.id, I.jsonb->'hrid' as hrid, M.content from sul_mod_inventory_storage.instance I @@ -12,7 +14,7 @@ and (I.jsonb->>'statusId')::uuid in ( from sul_mod_inventory_storage.instance_status where jsonb->>'name' = 'Cataloged' ) - and I.jsonb->>'catalogedDate' is not null + and I.jsonb->>'catalogedDate' between %(from_date)s and %(to_date)s and (I.jsonb->>'discoverySuppress')::boolean is false and I.jsonb->>'source' = 'MARC' join sul_mod_source_record_storage.marc_records_lb M diff --git a/libsys_airflow/plugins/folio/circ_rules.py b/libsys_airflow/plugins/folio/circ_rules.py index 4ce3cd100..f6b286231 100644 --- a/libsys_airflow/plugins/folio/circ_rules.py +++ b/libsys_airflow/plugins/folio/circ_rules.py @@ -3,7 +3,7 @@ import pathlib import urllib.parse -import requests +import httpx from typing import Union from airflow.operators.python import get_current_context @@ -39,8 +39,10 @@ def _friendly_name(**kwargs): query = kwargs["query"] json_path = kwargs["json_path"] fallback = kwargs["fallback"] - folio_result = requests.get( - f"{folio_client.okapi_url}/{query}", headers=folio_client.okapi_headers + folio_result = httpx.get( + f"{folio_client.okapi_url}/{query}", + headers=folio_client.okapi_headers, + timeout=15, ) expression = parse(json_path) matches = expression.find(folio_result.json()) @@ -68,9 +70,10 @@ def _library_location_names(**kwargs): instance = kwargs["task_instance"] location_id = kwargs["location_id"] row_count = kwargs.get("row_count", "") - library_location_result = requests.get( + library_location_result = httpx.get( f"""{folio_client.okapi_url}/locations?query=(id=="{location_id}")""", headers=folio_client.okapi_headers, + timeout=15, ) library_location_payload = library_location_result.json() lib_name_expr = parse("$.locations[0].name") @@ -87,9 +90,10 @@ def _library_location_names(**kwargs): if len(lib_id_match) > 0: # Second call to Okapi library_id = lib_id_match[0].value - location_units_results = requests.get( + location_units_results = httpx.get( f"""{folio_client.okapi_url}/location-units/libraries?query=(id=="{library_id}")""", headers=folio_client.okapi_headers, + timeout=15, ) location_code_expr = parse("$.loclibs[0].code") location_code_matches = location_code_expr.find(location_units_results.json()) @@ -295,7 +299,9 @@ def policy_report(**kwargs): policy_url = f"{folio_client.okapi_url}/{endpoint}?limit=2000" - policies_result = requests.get(policy_url, headers=folio_client.okapi_headers) + policies_result = httpx.get( + policy_url, headers=folio_client.okapi_headers, timeout=15 + ) policies = policies_result.json() @@ -364,8 +370,10 @@ def retrieve_policies(**kwargs): task_ids=task_id, key=f"all-policies-url{row_count}" ) - single_policy_result = requests.get( - single_policy_url, headers=folio_client.okapi_headers + single_policy_result = httpx.get( + single_policy_url, + headers=folio_client.okapi_headers, + timeout=15, ) if single_policy_result.status_code == 200: @@ -377,8 +385,10 @@ def retrieve_policies(**kwargs): f"Cannot retrieve {single_policy_url}\n{single_policy_result.text}" ) - all_policies_result = requests.get( - all_policies_url, headers=folio_client.okapi_headers + all_policies_result = httpx.get( + all_policies_url, + headers=folio_client.okapi_headers, + timeout=15, ) if all_policies_result.status_code == 200: instance.xcom_push( diff --git a/libsys_airflow/plugins/folio/encumbrances/fix_encumbrances_quesnelia.py b/libsys_airflow/plugins/folio/encumbrances/fix_encumbrances_quesnelia.py new file mode 100644 index 000000000..2eaba6103 --- /dev/null +++ b/libsys_airflow/plugins/folio/encumbrances/fix_encumbrances_quesnelia.py @@ -0,0 +1,1192 @@ +# type: ignore + +import asyncio +import getpass +import json +import sys +import time +from datetime import datetime +from decimal import Decimal +from http import HTTPStatus + +import httpx +import requests + +import logging +from airflow.models import Variable + +logger = logging.getLogger(__name__) + +ITEM_MAX = 2147483647 +MAX_BY_CHUNK = 1000 + +okapi_url = f"{Variable.get('OKAPI_URL')}/" +headers = {} +client = httpx.AsyncClient() +dryrun = False + +# request timeout in seconds +ASYNC_CLIENT_TIMEOUT = 30 + +# limit the number of parallel threads. +# Try different values. Bigger values - for increasing performance, but could produce "Connection timeout exception" +MAX_ACTIVE_THREADS = Variable.get("FIX_ENC_THREADS", 1) + + +# --------------------------------------------------- +# Utility functions + + +def raise_exception_for_reply(r): + raise Exception(f'Status code: {r.status_code}. Response: "{r.text}"') + + +def login(tenant, username, password): + login_headers = {'x-okapi-tenant': tenant, 'Content-Type': 'application/json'} + data = {'username': username, 'password': password} + try: + r = requests.post(okapi_url + 'authn/login', headers=login_headers, json=data) + if r.status_code != 201: + raise_exception_for_reply(r) + logger.info('Logged in successfully.') + okapi_token = r.json()['okapiToken'] + return { + 'x-okapi-tenant': tenant, + 'x-okapi-token': okapi_token, + 'Content-Type': 'application/json', + } + except Exception as err: + print('Error during login:', err) + logger.error('Error during login:', err) + raise Exception("Exiting Fix Encumbrances script.") + + +async def get_request_without_query(url: str) -> dict: + try: + resp = await client.get(url, headers=headers, timeout=ASYNC_CLIENT_TIMEOUT) + + if resp.status_code == HTTPStatus.OK: + return resp.json() + else: + print(f'Error getting record with url {url} : \n{resp.text} ') + logger.error(f'Error getting record with url {url} : \n{resp.text} ') + raise Exception("Exiting Fix Encumbrances script.") + except Exception as err: + print(f'Error getting record with url {url} : {err=}') + logger.error(f'Error getting record with url {url} : {err=}') + raise Exception("Exiting Fix Encumbrances script.") + + +async def get_request(url: str, query: str) -> dict: + params = {'query': query, 'offset': '0', 'limit': ITEM_MAX} + + try: + resp = await client.get( + url, headers=headers, params=params, timeout=ASYNC_CLIENT_TIMEOUT + ) + + if resp.status_code == HTTPStatus.OK: + return resp.json() + else: + print(f'Error getting records by {url} ?query= "{query}": \n{resp.text} ') + logger.error( + f'Error getting records by {url} ?query= "{query}": \n{resp.text} ' + ) + raise Exception("Exiting Fix Encumbrances script.") + except Exception as err: + print(f'Error getting records by {url}?query={query}: {err=}') + logger.error(f'Error getting records by {url}?query={query}: {err=}') + raise Exception("Exiting Fix Encumbrances script.") + + +async def post_request(url: str, data): + if dryrun: + return + try: + resp = await client.post( + url, headers=headers, data=json.dumps(data), timeout=ASYNC_CLIENT_TIMEOUT + ) + if ( + resp.status_code == HTTPStatus.CREATED + or resp.status_code == HTTPStatus.NO_CONTENT + ): + return + print(f'Error in POST request {url} "{data}": {resp.text}') + logger.error(f'Error in POST request {url} "{data}": {resp.text}') + raise Exception("Exiting Fix Encumbrances script.") + + except Exception as err: + print(f'Error in POST request {url} "{data}": {err=}') + logger.error(f'Error in POST request {url} "{data}": {err=}') + raise Exception("Exiting Fix Encumbrances script.") + + +async def put_request(url: str, data): + if dryrun: + return + try: + resp = await client.put( + url, headers=headers, data=json.dumps(data), timeout=ASYNC_CLIENT_TIMEOUT + ) + if resp.status_code == HTTPStatus.NO_CONTENT: + return + print(f'Error updating record {url} "{data}": {resp.text}') + logger.error(f'Error updating record {url} "{data}": {resp.text}') + raise Exception("Exiting Fix Encumbrances script.") + + except Exception as err: + print(f'Error updating record {url} "{data}": {err=}') + logger.error(f'Error updating record {url} "{data}": {err=}') + raise Exception("Exiting Fix Encumbrances script.") + + +def get_fiscal_years_by_query(query) -> dict: + params = {'query': query, 'offset': '0', 'limit': ITEM_MAX} + try: + r = requests.get( + okapi_url + 'finance-storage/fiscal-years', headers=headers, params=params + ) + if r.status_code != 200: + raise_exception_for_reply(r) + return r.json()['fiscalYears'] + except Exception as err: + print(f'Error getting fiscal years with query "{query}": {err}') + logger.error(f'Error getting fiscal years with query "{query}": {err}') + raise Exception("Exiting Fix Encumbrances script.") + + +def get_by_chunks(url, query, key) -> list: + # See https://github.com/folio-org/raml-module-builder#implement-chunked-bulk-download + records = [] + last_id = None + while True: + if last_id is None: + modified_query = query + ' AND cql.allRecords=1 sortBy id' + else: + modified_query = query + f' AND id > {last_id} sortBy id' + params = {'query': modified_query, 'offset': 0, 'limit': MAX_BY_CHUNK} + r = requests.get(url, headers=headers, params=params) + if r.status_code != 200: + raise_exception_for_reply(r) + j = r.json() + if key not in j.keys(): + raise Exception( + f'Could not find key when retrieving by chunks; url={url}, key={key}' + ) + records_in_chunk = j[key] + if len(records_in_chunk) == 0: + break + records.extend(records_in_chunk) + last_id = records_in_chunk[-1]['id'] + return records + + +def get_order_ids_by_query(query) -> list: + try: + orders = get_by_chunks( + okapi_url + 'orders-storage/purchase-orders', query, 'purchaseOrders' + ) + ids = [] + for order in orders: + ids.append(order['id']) + except Exception as err: + print(f'Error getting order ids with query "{query}": {err}') + logger.error(f'Error getting order ids with query "{query}": {err}') + raise Exception("Exiting Fix Encumbrances script.") + return ids + + +async def get_encumbrances_by_query(query) -> list: + url = okapi_url + 'finance-storage/transactions' + response = await get_request(url, query) + return response['transactions'] + + +async def get_encumbrance_by_ids(encumbrance_ids) -> list: + query = '' + for idx, enc_id in enumerate(encumbrance_ids): + if len(encumbrance_ids) != idx + 1: + query = query + f"id=={enc_id} OR " + else: + query = query + f"id=={enc_id}" + resp = await get_request(okapi_url + 'finance-storage/transactions', query) + + return resp['transactions'] + + +def get_fiscal_year(fiscal_year_code) -> dict: + query = f'code=="{fiscal_year_code}"' + fiscal_years = get_fiscal_years_by_query(query) + if len(fiscal_years) == 0: + print(f'Could not find fiscal year "{fiscal_year_code}".') + logger.error(f'Could not find fiscal year "{fiscal_year_code}".') + raise Exception("Exiting Fix Encumbrances script.") + return fiscal_years[0] + + +def test_fiscal_year_current(fiscal_year) -> bool: + start = datetime.fromisoformat(fiscal_year['periodStart']) + end = datetime.fromisoformat(fiscal_year['periodEnd']) + now = datetime.now().astimezone() + return start < now < end + + +def get_closed_orders_ids() -> list: + print('Retrieving closed order ids...') + query = 'workflowStatus=="Closed"' + closed_orders_ids = get_order_ids_by_query(query) + print(' Closed orders:', len(closed_orders_ids)) + return closed_orders_ids + + +def get_open_orders_ids() -> list: + print('Retrieving open order ids...') + query = 'workflowStatus=="Open"' + open_orders_ids = get_order_ids_by_query(query) + print(' Open orders:', len(open_orders_ids)) + return open_orders_ids + + +async def batch_update(transactions: list): + batch = {'transactionsToUpdate': transactions} + url = f'{okapi_url}finance-storage/transactions/batch-all-or-nothing' + await post_request(url, batch) + + +async def batch_delete(transaction_ids: list): + batch = {'idsOfTransactionsToDelete': transaction_ids} + url = f'{okapi_url}finance-storage/transactions/batch-all-or-nothing' + await post_request(url, batch) + + +async def get_budgets_by_query(query) -> list: + budget_collection = await get_request(okapi_url + 'finance-storage/budgets', query) + return budget_collection['budgets'] + + +async def get_budget_by_fund_id(fund_id, fiscal_year_id) -> dict: + query = f'fundId=={fund_id} AND fiscalYearId=={fiscal_year_id}' + budgets = await get_budgets_by_query(query) + if len(budgets) == 0: + print( + f'Could not find budget for fund "{fund_id}" and fiscal year "{fiscal_year_id}".' + ) + logger.error( + f'Could not find budget for fund "{fund_id}" and fiscal year "{fiscal_year_id}".' + ) + raise Exception("Exiting Fix Encumbrances script.") + return budgets[0] + + +async def get_budgets_by_fiscal_year(fiscal_year_id) -> list: + query = f'fiscalYearId=={fiscal_year_id}' + return await get_budgets_by_query(query) + + +async def get_order_encumbrances(order_id, fiscal_year_id, sem=None) -> list: + url = okapi_url + 'finance-storage/transactions' + query = f'encumbrance.sourcePurchaseOrderId=={order_id} AND fiscalYearId=={fiscal_year_id}' + response = await get_request(url, query) + if sem is not None: + sem.release() + return response['transactions'] + + +def progress(index, total_elements, label=''): + if total_elements == 0: + return + progress_length = 80 + current_progress_length = int( + round(progress_length * index / float(total_elements)) + ) + + percents_completed = round(100.0 * index / float(total_elements), 1) + bar = '=' * current_progress_length + '-' * ( + progress_length - current_progress_length + ) + + sys.stdout.write('%s - [%s] %s%s \r' % (label, bar, percents_completed, '%')) + sys.stdout.flush() + + if index == total_elements: + print() + + +# --------------------------------------------------- +# Remove duplicate encumbrances + + +def find_encumbrances_to_remove(encumbrances) -> list: + encumbrance_changes = [] + unreleased_encumbrances = [] + released_encumbrances = [] + for enc in encumbrances: + if enc['encumbrance']['status'] == 'Unreleased': + unreleased_encumbrances.append(enc) + if enc['encumbrance']['status'] == 'Released': + released_encumbrances.append(enc) + for enc1 in unreleased_encumbrances: + from_fund_id = enc1['fromFundId'] + source_po_line_id = enc1['encumbrance']['sourcePoLineId'] + if 'expenseClassId' in enc1: + expense_class_id = enc1['expenseClassId'] + else: + expense_class_id = None + fiscal_year_id = enc1['fiscalYearId'] + for enc2 in released_encumbrances: + ec_test = (expense_class_id is None and 'expenseClassId' not in enc2) or ( + expense_class_id is not None + and 'expenseClassId' in enc2 + and enc2['expenseClassId'] == expense_class_id + ) + if ( + enc2['fromFundId'] == from_fund_id + and enc2['encumbrance']['sourcePoLineId'] == source_po_line_id + and ec_test + and enc2['fiscalYearId'] == fiscal_year_id + ): + encumbrance_changes.append({'remove': enc1, 'replace_by': enc2}) + break + return encumbrance_changes + + +async def update_poline_encumbrance(encumbrance_to_remove, replace_by, poline=None): + url = ( + okapi_url + + f"orders-storage/po-lines/{encumbrance_to_remove['encumbrance']['sourcePoLineId']}" + ) + if poline is None: + poline = await get_request_without_query(url) + for fd in poline['fundDistribution']: + if 'encumbrance' in fd and fd['encumbrance'] == encumbrance_to_remove['id']: + fd['encumbrance'] = replace_by['id'] + await put_request(url, poline) + break + + +async def remove_encumbrances_and_update_polines(encumbrance_changes): + futures = [] + ids_to_delete = [] + for change in encumbrance_changes: + encumbrance_to_remove = change['remove'] + replace_by = change['replace_by'] + futures.append( + asyncio.ensure_future( + update_poline_encumbrance(encumbrance_to_remove, replace_by) + ) + ) + ids_to_delete.append(encumbrance_to_remove['id']) + await asyncio.gather(*futures) + await batch_delete(ids_to_delete) + + +async def remove_duplicate_encumbrances_in_order(order_id, fiscal_year_id, sem) -> int: + order_encumbrances = await get_order_encumbrances(order_id, fiscal_year_id) + if len(order_encumbrances) == 0: + sem.release() + return 0 + encumbrance_changes = find_encumbrances_to_remove(order_encumbrances) + if len(encumbrance_changes) == 0: + sem.release() + return 0 + print(f" Removing the following encumbrances for order {order_id}:") + for change in encumbrance_changes: + print(f" {change['remove']['id']}") + await remove_encumbrances_and_update_polines(encumbrance_changes) + sem.release() + return len(encumbrance_changes) + + +async def remove_duplicate_encumbrances(open_and_closed_orders_ids, fiscal_year_id): + print('Removing duplicate encumbrances...') + futures = [] + sem = asyncio.Semaphore(MAX_ACTIVE_THREADS) + for idx, order_id in enumerate(open_and_closed_orders_ids): + await sem.acquire() + # progress(idx, len(open_and_closed_orders_ids)) + futures.append( + asyncio.ensure_future( + remove_duplicate_encumbrances_in_order(order_id, fiscal_year_id, sem) + ) + ) + + nb_removed_encumbrances = await asyncio.gather(*futures) + # progress(len(open_and_closed_orders_ids), len(open_and_closed_orders_ids)) + print(f' Removed {sum(nb_removed_encumbrances)} encumbrance(s).') + + +# --------------------------------------------------- +# Fix poline-encumbrance relations + + +async def get_polines_by_order_id(order_id) -> list: + query = f'purchaseOrderId=={order_id}' + resp = await get_request(okapi_url + 'orders-storage/po-lines', query) + po_lines = resp['poLines'] + return po_lines + + +async def update_encumbrance_fund_id(encumbrance, new_fund_id, poline): + encumbrance['fromFundId'] = new_fund_id + encumbrance_id = encumbrance['id'] + print( + f" Fixing fromFundId for po line {poline['id']} ({poline['poLineNumber']}) encumbrance {encumbrance_id}" + ) + await batch_update([encumbrance]) + + +# Remove a duplicate encumbrance if it has a wrong fromFundId, and update the poline fd if needed +async def fix_fund_id_with_duplicate_encumbrances(encumbrances, fd_fund_id, poline): + encumbrances_with_right_fund = [] + encumbrances_with_bad_fund = [] + for encumbrance in encumbrances: + if encumbrance['fromFundId'] == fd_fund_id: + encumbrances_with_right_fund.append(encumbrance) + else: + encumbrances_with_bad_fund.append(encumbrance) + if len(encumbrances_with_bad_fund) == 0: + print( + f" Warning: there is a remaining duplicate encumbrance for poline {poline['id']} " + f"({poline['poLineNumber']})." + ) + return + if len(encumbrances_with_right_fund) != 1: + print( + f" Problem fixing encumbrances for poline {poline['id']} ({poline['poLineNumber']}), " + "please fix by hand." + ) + return + replace_by = encumbrances_with_right_fund[0] + ids_to_delete = [] + for encumbrance_to_remove in encumbrances_with_bad_fund: + print( + f" Removing encumbrance {encumbrance_to_remove['id']} for po line {poline['id']} " + f"({poline['poLineNumber']})" + ) + await update_poline_encumbrance(encumbrance_to_remove, replace_by, poline) + ids_to_delete.append(encumbrance_to_remove['id']) + await batch_delete(ids_to_delete) + + +# Fix encumbrance fromFundId if it doesn't match the po line fund distribution (see MODFISTO-384, MODFISTO-385) +async def fix_poline_encumbrance_fund_id(poline, order_encumbrances): + fds = poline['fundDistribution'] + # we can't fix the fundId if there is more than 1 fund distribution in the po line + if len(fds) != 1: + return + + fd_fund_id = fds[0]['fundId'] + encumbrances = [] + for enc in order_encumbrances: + if enc['encumbrance']['sourcePoLineId'] == poline['id']: + encumbrances.append(enc) + if len(encumbrances) == 0: + return + if len(encumbrances) == 1: + if encumbrances[0]['fromFundId'] == fd_fund_id: + return + await update_encumbrance_fund_id(encumbrances[0], fd_fund_id, poline) + return + await fix_fund_id_with_duplicate_encumbrances(encumbrances, fd_fund_id, poline) + + +def check_if_fd_needs_updates_and_update_fd(poline, order_encumbrances, fd) -> bool: + poline_id = poline['id'] + fd_fund_id = fd['fundId'] + for enc in order_encumbrances: + # if ( + # enc['encumbrance']['sourcePoLineId'] == poline_id + # and float(enc['amount']) != 0.0 + # and enc['fromFundId'] == fd_fund_id + # ): + if ( + enc['encumbrance']['sourcePoLineId'] == poline_id + and enc['fromFundId'] == fd_fund_id + ): + fd_encumbrance_id = fd['encumbrance'] + if enc['id'] == fd_encumbrance_id: + return False + # print( + # f" Updating poline {poline_id} ({poline['poLineNumber']}) encumbrance {fd_encumbrance_id} " + # f"with new value {enc['id']}" + # ) + fd['encumbrance'] = enc['id'] + return True + return False + + +# for each fund distribution check encumbrance relationship and modify if needed - +# in case if encumbrance id specified in fund distribution: +# get encumbrance by poline id and current FY<>transaction.FY and amount <> 0 +# if fd.encumbrance != transaction.id --> set new encumbrance reference +# update poline if modified +# (feature added with MODFISTO-350) +async def fix_poline_encumbrance_link(poline, order_encumbrances): + poline_needs_updates = False + for fd in poline['fundDistribution']: + if 'encumbrance' in fd: + if check_if_fd_needs_updates_and_update_fd(poline, order_encumbrances, fd): + poline_needs_updates = True + + # update poline if one or more fund distributions modified + if poline_needs_updates: + url = f"{okapi_url}orders-storage/po-lines/{poline['id']}" + await put_request(url, poline) + + +async def process_poline_encumbrances_relations(poline, order_encumbrances): + await fix_poline_encumbrance_fund_id(poline, order_encumbrances) + await fix_poline_encumbrance_link(poline, order_encumbrances) + + +# Get encumbrances for the fiscal year and call process_po_line_encumbrances_relations() for each po line +async def process_order_encumbrances_relations(order_id, fiscal_year_id, order_sem): + po_lines = await get_polines_by_order_id(order_id) + if len(po_lines) == 0: + order_sem.release() + return + order_encumbrances = await get_order_encumbrances(order_id, fiscal_year_id) + if len(order_encumbrances) == 0: + order_sem.release() + return + + for po_line in po_lines: + await process_poline_encumbrances_relations(po_line, order_encumbrances) + + order_sem.release() + + +# Call process_order_encumbrances_relations() for each order +async def fix_poline_encumbrances_relations( + open_orders_ids, fiscal_year_id, fy_is_current +): + print('Fixing poline-encumbrance links...') + if len(open_orders_ids) == 0: + print(' Found no open orders.') + return + if not fy_is_current: + print( + ' Fiscal year is not current, the step to fix po line encumbrance relations will be skipped.' + ) + return + orders_futures = [] + order_sem = asyncio.Semaphore(MAX_ACTIVE_THREADS) + for idx, order_id in enumerate(open_orders_ids): + await order_sem.acquire() + # progress(idx, len(open_orders_ids)) + orders_futures.append( + asyncio.ensure_future( + process_order_encumbrances_relations( + order_id, fiscal_year_id, order_sem + ) + ) + ) + await asyncio.gather(*orders_futures) + # progress(len(open_orders_ids), len(open_orders_ids)) + + +# --------------------------------------------------- +# Fix encumbrance orderStatus for closed orders + + +async def get_order_encumbrances_to_fix(order_id, fiscal_year_id) -> dict: + query = ( + f'encumbrance.orderStatus<>"Closed" AND encumbrance.sourcePurchaseOrderId=={order_id} AND ' + f'fiscalYearId=={fiscal_year_id}' + ) + url = okapi_url + 'finance-storage/transactions' + + return await get_request(url, query) + + +async def unrelease_order_encumbrances(encumbrances) -> list: + for encumbrance in encumbrances: + encumbrance['encumbrance']['status'] = 'Unreleased' + await batch_update(encumbrances) + + # the encumbrance amounts get modified (and the version in MG), so we need to get a fresh version + modified_encumbrance_futures = [] + + # split retrieving encumbrances by small id lists + # reasons: + # - retrieving one by one will slow down performance + # - retrieving all at once will generate too long query and fail the request due to RMB restrictions + enc_ids = build_ids_2d_array(encumbrances) + for enc_ids_row in enc_ids: + modified_encumbrance_futures.append(get_encumbrance_by_ids(enc_ids_row)) + modified_encumbrances = await asyncio.gather(*modified_encumbrance_futures) + + flatten_list_of_modified_encumbrances = sum(modified_encumbrances, []) + return flatten_list_of_modified_encumbrances + + +def build_ids_2d_array(entities) -> list: + # prepare two-dimensional array of ids for requesting the records by ids in bulks + ids_2d_array = [] + index = 0 + for row in range(ITEM_MAX): + inner_list = [] + for col in range(20): + if index == len(entities): + break + inner_list.append(entities[index]['id']) + index = index + 1 + ids_2d_array.append(inner_list) + if index == len(entities): + break + return ids_2d_array + + +async def fix_order_status_and_release_encumbrances(order_id, encumbrances): + try: + for encumbrance in encumbrances: + encumbrance['encumbrance']['status'] = 'Released' + encumbrance['encumbrance']['orderStatus'] = 'Closed' + await batch_update(encumbrances) + + except Exception as err: + print( + f'Error when fixing order status in encumbrances for order {order_id}:', err + ) + logger.error( + f'Error when fixing order status in encumbrances for order {order_id}:', err + ) + raise Exception("Exiting Fix Encumbrances script.") + + +async def fix_order_encumbrances_order_status(order_id, encumbrances): + # We can't just PUT the encumbrance with a modified orderStatus, because + # mod-finance-storage's EncumbranceService ignores changes to released encumbrances + # unless it's to unrelease them. So we have to unrelease the encumbrances first. + # Eventually we could rely on the post-MODFISTO-328 implementation to change orderStatus directly + # (Morning Glory bug fix). + try: + # print(f'\n Fixing the following encumbrance(s) for order {order_id} :') + for encumbrance in encumbrances: + print(f" {encumbrance['id']}") + modified_encumbrances = await unrelease_order_encumbrances(encumbrances) + if len(modified_encumbrances) != 0: + await fix_order_status_and_release_encumbrances( + order_id, modified_encumbrances + ) + except Exception as err: + print( + f'Error when fixing order status in encumbrances for order {order_id}:', err + ) + logger.error( + f'Error when fixing order status in encumbrances for order {order_id}:', err + ) + raise Exception("Exiting Fix Encumbrances script.") + + +async def fix_encumbrance_order_status_for_closed_order( + order_id, fiscal_year_id, sem +) -> int: + encumbrances = await get_order_encumbrances_to_fix(order_id, fiscal_year_id) + if len(encumbrances['transactions']) != 0: + await fix_order_encumbrances_order_status( + order_id, encumbrances['transactions'] + ) + sem.release() + return len(encumbrances['transactions']) + + +async def fix_encumbrance_order_status_for_closed_orders( + closed_orders_ids, fiscal_year_id +): + print('Fixing encumbrance order status for closed orders...') + if len(closed_orders_ids) == 0: + print(' Found no closed orders.') + return + fix_encumbrance_futures = [] + max_active_order_threads = 5 + sem = asyncio.Semaphore(max_active_order_threads) + for idx, order_id in enumerate(closed_orders_ids): + await sem.acquire() + # progress(idx, len(closed_orders_ids)) + fixed_encumbrance_future = asyncio.ensure_future( + fix_encumbrance_order_status_for_closed_order(order_id, fiscal_year_id, sem) + ) + fix_encumbrance_futures.append(fixed_encumbrance_future) + nb_fixed_encumbrances = await asyncio.gather(*fix_encumbrance_futures) + # progress(len(closed_orders_ids), len(closed_orders_ids)) + + print(f' Fixed order status for {sum(nb_fixed_encumbrances)} encumbrance(s).') + + +# --------------------------------------------------- +# Unrelease open orders encumbrances with non-zero amounts + + +async def unrelease_encumbrances(order_id, encumbrances): + # print(f'\n Unreleasing the following encumbrance(s) for order {order_id} :') + for encumbrance in encumbrances: + print(f" {encumbrance['id']}") + encumbrance['encumbrance']['status'] = 'Unreleased' + await batch_update(encumbrances) + + +async def unrelease_encumbrances_with_non_zero_amounts( + order_id, fiscal_year_id, sem +) -> int: + query = ( + f'amount<>0.0 AND encumbrance.status=="Released" AND encumbrance.sourcePurchaseOrderId=={order_id} AND ' + f'fiscalYearId=={fiscal_year_id}' + ) + transactions_response = await get_request( + okapi_url + 'finance-storage/transactions', query + ) + + order_encumbrances = transactions_response['transactions'] + # unrelease encumbrances by order id + if len(order_encumbrances) != 0: + await unrelease_encumbrances(order_id, order_encumbrances) + + sem.release() + return len(order_encumbrances) + + +async def unrelease_open_orders_encumbrances_with_nonzero_amounts( + fiscal_year_id, open_orders_ids +): + print('Unreleasing open orders encumbrances with non-zero amounts...') + if len(open_orders_ids) == 0: + print(' Found no open orders.') + return + enc_futures = [] + sem = asyncio.Semaphore(MAX_ACTIVE_THREADS) + for idx, order_id in enumerate(open_orders_ids): + await sem.acquire() + # progress(idx, len(open_orders_ids)) + enc_futures.append( + asyncio.ensure_future( + unrelease_encumbrances_with_non_zero_amounts( + order_id, fiscal_year_id, sem + ) + ) + ) + unreleased_encumbrances_amounts = await asyncio.gather(*enc_futures) + # progress(len(open_orders_ids), len(open_orders_ids)) + + print( + f' Unreleased {sum(unreleased_encumbrances_amounts)} open order encumbrance(s) with non-zero amounts.' + ) + + +# --------------------------------------------------- +# Release open orders encumbrances with negative amounts (see MODFISTO-368) + + +async def release_encumbrances(order_id, encumbrances): + # print(f'\n Releasing the following encumbrances for order {order_id} :') + for encumbrance in encumbrances: + print(f" {encumbrance['id']}") + encumbrance['encumbrance']['status'] = 'Released' + await batch_update(encumbrances) + + +async def release_encumbrances_with_negative_amounts( + order_id, fiscal_year_id, sem +) -> int: + query = ( + 'amount /number 0 OR encumbrance.amountExpended >/number 0) AND ' + f'encumbrance.sourcePurchaseOrderId=={order_id} AND fiscalYearId=={fiscal_year_id}' + ) + transactions_response = await get_request( + okapi_url + 'finance-storage/transactions', query + ) + + order_encumbrances = transactions_response['transactions'] + # release encumbrances by order id + if len(order_encumbrances) != 0: + await release_encumbrances(order_id, order_encumbrances) + + sem.release() + return len(order_encumbrances) + + +async def release_open_orders_encumbrances_with_negative_amounts( + fiscal_year_id, open_orders_ids +): + print('Releasing open orders encumbrances with negative amounts...') + if len(open_orders_ids) == 0: + print(' Found no open orders.') + return + enc_futures = [] + sem = asyncio.Semaphore(MAX_ACTIVE_THREADS) + for idx, order_id in enumerate(open_orders_ids): + await sem.acquire() + # progress(idx, len(open_orders_ids)) + enc_futures.append( + asyncio.ensure_future( + release_encumbrances_with_negative_amounts( + order_id, fiscal_year_id, sem + ) + ) + ) + released_encumbrances_amounts = await asyncio.gather(*enc_futures) + # progress(len(open_orders_ids), len(open_orders_ids)) + + print( + f' Released {sum(released_encumbrances_amounts)} open order encumbrance(s) with negative amounts.' + ) + + +# --------------------------------------------------- +# Release cancelled order line encumbrances (see MODFISTO-383) + + +def find_encumbrances_to_release(po_lines, order_encumbrances) -> list: + encumbrances_to_release = [] + for pol in po_lines: + if pol['paymentStatus'] == 'Cancelled': + for enc in order_encumbrances: + if ( + enc['encumbrance']['sourcePoLineId'] == pol['id'] + and enc['encumbrance']['status'] == 'Unreleased' + ): + encumbrances_to_release.append(enc) + return encumbrances_to_release + + +async def release_cancelled_pol_encumbrances(order_id, fiscal_year_id, sem) -> int: + po_lines = await get_polines_by_order_id(order_id) + if len(po_lines) == 0: + sem.release() + return 0 + order_encumbrances = await get_order_encumbrances(order_id, fiscal_year_id) + if len(order_encumbrances) == 0: + sem.release() + return 0 + encumbrances_to_release = find_encumbrances_to_release(po_lines, order_encumbrances) + if len(encumbrances_to_release) == 0: + sem.release() + return 0 + await release_encumbrances(order_id, encumbrances_to_release) + sem.release() + return len(encumbrances_to_release) + + +async def release_cancelled_order_line_encumbrances(fiscal_year_id, open_orders_ids): + print('Releasing cancelled order line encumbrances...') + if len(open_orders_ids) == 0: + print(' Found no open orders.') + return + enc_futures = [] + sem = asyncio.Semaphore(MAX_ACTIVE_THREADS) + for idx, order_id in enumerate(open_orders_ids): + await sem.acquire() + # progress(idx, len(open_orders_ids)) + enc_futures.append( + asyncio.ensure_future( + release_cancelled_pol_encumbrances(order_id, fiscal_year_id, sem) + ) + ) + released_encumbrances_amounts = await asyncio.gather(*enc_futures) + # progress(len(open_orders_ids), len(open_orders_ids)) + + print( + f' Released {sum(released_encumbrances_amounts)} cancelled order line encumbrance(s).' + ) + + +# --------------------------------------------------- +# Recalculate budget encumbered + + +async def update_budgets(encumbered, fund_id, fiscal_year_id, sem) -> int: + nb_modified = 0 + budget = await get_budget_by_fund_id(fund_id, fiscal_year_id) + + # Cast into decimal values, so 0 == 0.0 == 0.00 will return true + if Decimal(str(budget['encumbered'])) != Decimal(encumbered): + # print( + # f" Budget \"{budget['name']}\": changing encumbered from {budget['encumbered']} to {encumbered}" + # ) + budget['encumbered'] = encumbered + + url = f"{okapi_url}finance-storage/budgets/{budget['id']}" + await put_request(url, budget) + nb_modified = 1 + sem.release() + return nb_modified + + +async def recalculate_budget_encumbered(open_and_closed_orders_ids, fiscal_year_id): + # Recalculate the encumbered property for all the budgets related to these encumbrances + # Take closed orders into account because we might have to set a budget encumbered to 0 + print( + f'Recalculating budget encumbered for {len(open_and_closed_orders_ids)} orders ...' + ) + enc_future = [] + sem = asyncio.Semaphore(MAX_ACTIVE_THREADS) + for idx, order_id in enumerate(open_and_closed_orders_ids): + await sem.acquire() + # progress(idx, len(open_and_closed_orders_ids)) + enc_future.append( + asyncio.ensure_future(get_order_encumbrances(order_id, fiscal_year_id, sem)) + ) + + encumbrances = sum(await asyncio.gather(*enc_future), []) + # progress(len(open_and_closed_orders_ids), len(open_and_closed_orders_ids)) + + encumbered_for_fund = {} + budgets = await get_budgets_by_fiscal_year(fiscal_year_id) + for budget in budgets: + fund_id = budget['fundId'] + if fund_id not in encumbered_for_fund: + encumbered_for_fund[fund_id] = 0 + + for encumbrance in encumbrances: + fund_id = encumbrance['fromFundId'] + if fund_id in encumbered_for_fund: + encumbered_for_fund[fund_id] += Decimal(str(encumbrance['amount'])) + + print(' Updating budgets...') + + update_budget_futures = [] + for fund_id, encumbered in encumbered_for_fund.items(): + await sem.acquire() + update_budget_futures.append( + asyncio.ensure_future( + update_budgets(str(encumbered), fund_id, fiscal_year_id, sem) + ) + ) + nb_modified = sum(await asyncio.gather(*update_budget_futures)) + + print(f' Edited {nb_modified} budget(s).') + print(' Done recalculating budget encumbered.') + + +# --------------------------------------------------- +# Release unreleased encumbrances for closed orders + + +async def get_order_encumbrances_to_release(order_id, fiscal_year_id) -> list: + query = ( + f'encumbrance.status=="Unreleased" AND encumbrance.sourcePurchaseOrderId=={order_id} AND ' + f'fiscalYearId=={fiscal_year_id}' + ) + return await get_encumbrances_by_query(query) + + +async def release_order_encumbrances(order_id, fiscal_year_id, sem) -> int: + encumbrances = await get_order_encumbrances_to_release(order_id, fiscal_year_id) + if len(encumbrances) != 0: + await release_encumbrances(order_id, encumbrances) + sem.release() + return len(encumbrances) + + +async def release_unreleased_encumbrances_for_closed_orders( + closed_orders_ids, fiscal_year_id +): + print('Releasing unreleased encumbrances for closed orders...') + if len(closed_orders_ids) == 0: + print(' Found no closed orders.') + return + nb_released_encumbrance_futures = [] + sem = asyncio.Semaphore(MAX_ACTIVE_THREADS) + + for idx, order_id in enumerate(closed_orders_ids): + await sem.acquire() + # progress(idx, len(closed_orders_ids)) + nb_released_encumbrance_futures.append( + asyncio.ensure_future( + release_order_encumbrances(order_id, fiscal_year_id, sem) + ) + ) + nb_released_encumbrances = await asyncio.gather(*nb_released_encumbrance_futures) + # progress(len(closed_orders_ids), len(closed_orders_ids)) + + print(f' Released {sum(nb_released_encumbrances)} encumbrance(s).') + + +# --------------------------------------------------- +# All operations + + +async def all_operations( + closed_orders_ids, + open_orders_ids, + open_and_closed_orders_ids, + fiscal_year_id, + fy_is_current, +): + await remove_duplicate_encumbrances(open_and_closed_orders_ids, fiscal_year_id) + await fix_poline_encumbrances_relations( + open_orders_ids, fiscal_year_id, fy_is_current + ) + if fy_is_current: + await fix_encumbrance_order_status_for_closed_orders( + closed_orders_ids, fiscal_year_id + ) + await unrelease_open_orders_encumbrances_with_nonzero_amounts( + fiscal_year_id, open_orders_ids + ) + await release_open_orders_encumbrances_with_negative_amounts( + fiscal_year_id, open_orders_ids + ) + await release_cancelled_order_line_encumbrances(fiscal_year_id, open_orders_ids) + await recalculate_budget_encumbered(open_and_closed_orders_ids, fiscal_year_id) + await release_unreleased_encumbrances_for_closed_orders( + closed_orders_ids, fiscal_year_id + ) + + +# --------------------------------------------------- +# Dry-run mode selection + + +def dryrun_mode_selection(): + global dryrun + + choice_i = 0 + while choice_i < 1 or choice_i > 2: + print('1) Dry-run mode (read-only, will not apply fixes)') + print('2) Normal mode (will apply fixes)') + choice_s = input('Choose an option: ') + try: + choice_i = int(choice_s) + if choice_i < 1 or choice_i > 2: + print('Invalid option.') + except ValueError: + print('Invalid option.') + if choice_i == 1: + dryrun = True + print("Dry-run mode enabled. Fixes *will not* actually be applied.") + else: + print("Normal mode. All fixes *will* be applied.") + print() + + +# --------------------------------------------------- +# Menu and running operations + + +async def run_operation(choice, fiscal_year_code, tenant, username, password): + global headers + initial_time = time.time() + headers = login(tenant, username, password) + fiscal_year = get_fiscal_year(fiscal_year_code) + fy_is_current = test_fiscal_year_current(fiscal_year) + fiscal_year_id = fiscal_year['id'] + + if choice == 1: + closed_orders_ids = get_closed_orders_ids() + open_orders_ids = get_open_orders_ids() + open_and_closed_orders_ids = closed_orders_ids + open_orders_ids + await all_operations( + closed_orders_ids, + open_orders_ids, + open_and_closed_orders_ids, + fiscal_year_id, + fy_is_current, + ) + elif choice == 2: + closed_orders_ids = get_closed_orders_ids() + open_orders_ids = get_open_orders_ids() + open_and_closed_orders_ids = closed_orders_ids + open_orders_ids + await remove_duplicate_encumbrances(open_and_closed_orders_ids, fiscal_year_id) + elif choice == 3: + open_orders_ids = get_open_orders_ids() + await fix_poline_encumbrances_relations( + open_orders_ids, fiscal_year_id, fy_is_current + ) + elif choice == 4: + if not fy_is_current: + print( + 'Fiscal year is not current - fixing encumbrance order status is not needed.' + ) + else: + closed_orders_ids = get_closed_orders_ids() + await fix_encumbrance_order_status_for_closed_orders( + closed_orders_ids, fiscal_year_id + ) + elif choice == 5: + open_orders_ids = get_open_orders_ids() + await unrelease_open_orders_encumbrances_with_nonzero_amounts( + fiscal_year_id, open_orders_ids + ) + elif choice == 6: + open_orders_ids = get_open_orders_ids() + await release_open_orders_encumbrances_with_negative_amounts( + fiscal_year_id, open_orders_ids + ) + elif choice == 7: + open_orders_ids = get_open_orders_ids() + await release_cancelled_order_line_encumbrances(fiscal_year_id, open_orders_ids) + elif choice == 8: + closed_orders_ids = get_closed_orders_ids() + open_orders_ids = get_open_orders_ids() + open_and_closed_orders_ids = closed_orders_ids + open_orders_ids + await recalculate_budget_encumbered(open_and_closed_orders_ids, fiscal_year_id) + elif choice == 9: + closed_orders_ids = get_closed_orders_ids() + await release_unreleased_encumbrances_for_closed_orders( + closed_orders_ids, fiscal_year_id + ) + delta = round(time.time() - initial_time) + hours, remainder = divmod(delta, 3600) + minutes, seconds = divmod(remainder, 60) + print( + f'Elapsed time: {hours} hour(s), {minutes} minute(s) and {seconds} second(s).' + ) + + +def menu(fiscal_year_code, tenant, username): + print('1) Run all fixes (can be long)') + print('2) Remove duplicate encumbrances') + print('3) Fix order line - encumbrance relations') + print( + '4) Fix encumbrance order status for closed orders (current fiscal year only)' + ) + print('5) Unrelease open order encumbrances with nonzero amounts') + print('6) Release open order encumbrances with negative amounts') + print('7) Release cancelled order line encumbrances') + print( + '8) Recalculate all budget encumbered amounts (avoid any transaction while this is running!)' + ) + print('9) Release unreleased encumbrances for closed orders') + print('10) Quit') + choice_s = input('Choose an option: ') + try: + choice_i = int(choice_s) + except ValueError: + print('Invalid option.') + return + if choice_i < 1 or choice_i > 10: + print('Invalid option.') + return + if choice_i == 10: + return + if choice_i == 1 and dryrun: + print( + "Note that, because dry-run mode is enabled, some operations will behave differently because they " + "depend on the execution of previous ones, such as when recalculating the budget encumbrances." + ) + password = getpass.getpass('Password:') + asyncio.run(run_operation(choice_i, fiscal_year_code, tenant, username, password)) + + +# --------------------------------------------------- +# Main + + +# def main(): +# global okapi_url +# if len(sys.argv) != 5: +# print( +# "Syntax: ./fix_encumbrances.py 'fiscal_year_code' 'okapi_url' 'tenant' 'username'" +# ) +# raise SystemExit(1) +# fiscal_year_code = sys.argv[1] +# okapi_url = sys.argv[2] +# tenant = sys.argv[3] +# username = sys.argv[4] +# dryrun_mode_selection() +# menu(fiscal_year_code, tenant, username) + + +# main() diff --git a/libsys_airflow/plugins/folio/encumbrances/fix_encumbrances_run.py b/libsys_airflow/plugins/folio/encumbrances/fix_encumbrances_run.py index 4aff9f6b7..10ec37cd2 100644 --- a/libsys_airflow/plugins/folio/encumbrances/fix_encumbrances_run.py +++ b/libsys_airflow/plugins/folio/encumbrances/fix_encumbrances_run.py @@ -2,7 +2,7 @@ import contextlib import logging import pathlib -import libsys_airflow.plugins.folio.encumbrances.fix_encumbrances as fix_encumbrances_script +import libsys_airflow.plugins.folio.encumbrances.fix_encumbrances_quesnelia as fix_encumbrances_script logger = logging.getLogger(__name__) @@ -21,6 +21,12 @@ def fix_encumbrances_run(*args, **kwargs): log_path = pathlib.Path(airflow) / f"fix_encumbrances/{library}-{run_id}.log" log_path.parent.mkdir(parents=True, exist_ok=True) + if not fiscal_year_code: + logger.warning( + f"fix_encumbrance_script exiting: Set Airflow Variable FISCAL_YEAR_CODE_{library.upper()}" + ) + return None + with log_path.open("w+", 1) as log: with contextlib.redirect_stdout(log): try: diff --git a/libsys_airflow/plugins/folio/helpers/bw.py b/libsys_airflow/plugins/folio/helpers/bw.py index 33b104291..75a403470 100644 --- a/libsys_airflow/plugins/folio/helpers/bw.py +++ b/libsys_airflow/plugins/folio/helpers/bw.py @@ -53,7 +53,7 @@ def _bw_summary_body(task_instance, file_name) -> str: ): errors.append(row) total_success = 0 - for row in task_instance.xcom_pull( + for _ in task_instance.xcom_pull( task_ids="new_bw_record", key="success", default=[] ): total_success += 1 diff --git a/libsys_airflow/plugins/orafin/emails.py b/libsys_airflow/plugins/orafin/emails.py index 21ebe187a..ef1043f2a 100644 --- a/libsys_airflow/plugins/orafin/emails.py +++ b/libsys_airflow/plugins/orafin/emails.py @@ -117,7 +117,7 @@ def _ap_report_paid_email_body( def _excluded_email_body(grouped_reasons: dict, folio_url: str) -> str: - jinja_env = Environment() + jinja_env = Environment() # noqa jinja_env.filters["invoice_line_links"] = _invoice_line_links template = jinja_env.from_string( @@ -149,7 +149,7 @@ def _summary_email_body(invoices: list, folio_url: str): return "" converter = models_converter() - jinja_env = Environment() + jinja_env = Environment(autoescape=True) template = jinja_env.from_string( """ diff --git a/libsys_airflow/plugins/vendor/download.py b/libsys_airflow/plugins/vendor/download.py index 77a16e12c..768a25f0f 100644 --- a/libsys_airflow/plugins/vendor/download.py +++ b/libsys_airflow/plugins/vendor/download.py @@ -1,5 +1,5 @@ import logging -import ftplib +import ftplib # noqa import pathlib import re diff --git a/libsys_airflow/plugins/vendor/extract.py b/libsys_airflow/plugins/vendor/extract.py index cc890704f..7abdfb796 100644 --- a/libsys_airflow/plugins/vendor/extract.py +++ b/libsys_airflow/plugins/vendor/extract.py @@ -58,7 +58,7 @@ def _filter_filenames(filenames: list[str], regex: Optional[str]) -> str: f for f in filenames if re.compile(regex, flags=re.IGNORECASE).match(f) ] if len(filtered_filenames) != 1: - raise Exception( + raise ValueError( f"Expected to extract 1 file, but found {len(filtered_filenames)}" ) return filtered_filenames[0] diff --git a/poetry.lock b/poetry.lock index 74c712f28..66162e164 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,15 +1,27 @@ -# This file is automatically @generated by Poetry 2.0.1 and should not be changed by hand. +# This file is automatically @generated by Poetry 2.0.0 and should not be changed by hand. + +[[package]] +name = "aiofiles" +version = "24.1.0" +description = "File support for asyncio." +optional = false +python-versions = ">=3.8" +groups = ["main"] +files = [ + {file = "aiofiles-24.1.0-py3-none-any.whl", hash = "sha256:b4ec55f4195e3eb5d7abd1bf7e061763e864dd4954231fb8539a0ef8bb8260e5"}, + {file = "aiofiles-24.1.0.tar.gz", hash = "sha256:22a075c9e5a3810f0c2e48f3008c94d68c65d763b9b03857924c99e57355166c"}, +] [[package]] name = "aiohappyeyeballs" -version = "2.4.4" +version = "2.4.6" description = "Happy Eyeballs for asyncio" optional = false -python-versions = ">=3.8" +python-versions = ">=3.9" groups = ["main", "test"] files = [ - {file = "aiohappyeyeballs-2.4.4-py3-none-any.whl", hash = "sha256:a980909d50efcd44795c4afeca523296716d50cd756ddca6af8c65b996e27de8"}, - {file = "aiohappyeyeballs-2.4.4.tar.gz", hash = "sha256:5fdd7d87889c63183afc18ce9271f9b0a7d32c2303e394468dd45d514a757745"}, + {file = "aiohappyeyeballs-2.4.6-py3-none-any.whl", hash = "sha256:147ec992cf873d74f5062644332c539fcd42956dc69453fe5204195e560517e1"}, + {file = "aiohappyeyeballs-2.4.6.tar.gz", hash = "sha256:9b05052f9042985d32ecbe4b59a77ae19c006a78f1344d7fdad69d28ded3d0b0"}, ] [[package]] @@ -160,6 +172,19 @@ typing-extensions = ">=4" [package.extras] tz = ["backports.zoneinfo", "tzdata"] +[[package]] +name = "ansicon" +version = "1.89.0" +description = "Python wrapper for loading Jason Hood's ANSICON" +optional = false +python-versions = "*" +groups = ["main"] +markers = "platform_system == \"Windows\"" +files = [ + {file = "ansicon-1.89.0-py2.py3-none-any.whl", hash = "sha256:f1def52d17f65c2c9682cf8370c03f541f410c1752d6a14029f97318e4b9dfec"}, + {file = "ansicon-1.89.0.tar.gz", hash = "sha256:e4d039def5768a47e4afec8e89e83ec3ae5a26bf00ad851f914d1240b444d2b1"}, +] + [[package]] name = "anyio" version = "4.8.0" @@ -790,18 +815,43 @@ tests-mypy = ["mypy (>=1.11.1)", "pytest-mypy-plugins"] [[package]] name = "babel" -version = "2.16.0" +version = "2.17.0" description = "Internationalization utilities" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "babel-2.16.0-py3-none-any.whl", hash = "sha256:368b5b98b37c06b7daf6696391c3240c938b37767d4584413e8438c5c435fa8b"}, - {file = "babel-2.16.0.tar.gz", hash = "sha256:d1f3554ca26605fe173f3de0c65f750f5a42f924499bf134de6423582298e316"}, + {file = "babel-2.17.0-py3-none-any.whl", hash = "sha256:4d0b53093fdfb4b21c92b5213dba5a1b23885afa8383709427046b21c366e5f2"}, + {file = "babel-2.17.0.tar.gz", hash = "sha256:0c54cffb19f690cdcc52a3b50bcbf71e07a808d1c80d549f2459b9d2cf0afb9d"}, ] [package.extras] -dev = ["freezegun (>=1.0,<2.0)", "pytest (>=6.0)", "pytest-cov"] +dev = ["backports.zoneinfo", "freezegun (>=1.0,<2.0)", "jinja2 (>=3.0)", "pytest (>=6.0)", "pytest-cov", "pytz", "setuptools", "tzdata"] + +[[package]] +name = "bandit" +version = "1.8.2" +description = "Security oriented static analyser for python code." +optional = false +python-versions = ">=3.9" +groups = ["main"] +files = [ + {file = "bandit-1.8.2-py3-none-any.whl", hash = "sha256:df6146ad73dd30e8cbda4e29689ddda48364e36ff655dbfc86998401fcf1721f"}, + {file = "bandit-1.8.2.tar.gz", hash = "sha256:e00ad5a6bc676c0954669fe13818024d66b70e42cf5adb971480cf3b671e835f"}, +] + +[package.dependencies] +colorama = {version = ">=0.3.9", markers = "platform_system == \"Windows\""} +PyYAML = ">=5.3.1" +rich = "*" +stevedore = ">=1.20.0" + +[package.extras] +baseline = ["GitPython (>=3.1.30)"] +sarif = ["jschema-to-python (>=1.2.3)", "sarif-om (>=1.0.4)"] +test = ["beautifulsoup4 (>=4.8.0)", "coverage (>=4.5.4)", "fixtures (>=3.0.0)", "flake8 (>=4.0.0)", "pylint (==1.9.4)", "stestr (>=2.5.0)", "testscenarios (>=0.5.0)", "testtools (>=2.3.0)"] +toml = ["tomli (>=1.1.0)"] +yaml = ["PyYAML"] [[package]] name = "bcrypt" @@ -844,18 +894,19 @@ typecheck = ["mypy"] [[package]] name = "beautifulsoup4" -version = "4.12.3" +version = "4.13.3" description = "Screen-scraping library" optional = false -python-versions = ">=3.6.0" +python-versions = ">=3.7.0" groups = ["main", "test"] files = [ - {file = "beautifulsoup4-4.12.3-py3-none-any.whl", hash = "sha256:b80878c9f40111313e55da8ba20bdba06d8fa3969fc68304167741bbf9e082ed"}, - {file = "beautifulsoup4-4.12.3.tar.gz", hash = "sha256:74e3d1928edc070d21748185c46e3fb33490f22f52a3addee9aee0f4f7781051"}, + {file = "beautifulsoup4-4.13.3-py3-none-any.whl", hash = "sha256:99045d7d3f08f91f0d656bc9b7efbae189426cd913d830294a15eefa0ea4df16"}, + {file = "beautifulsoup4-4.13.3.tar.gz", hash = "sha256:1bd32405dacc920b42b83ba01644747ed77456a65760e285fbc47633ceddaf8b"}, ] [package.dependencies] soupsieve = ">1.2" +typing-extensions = ">=4.0.0" [package.extras] cchardet = ["cchardet"] @@ -870,7 +921,7 @@ version = "24.3.0" description = "The uncompromising code formatter." optional = false python-versions = ">=3.8" -groups = ["test"] +groups = ["main", "test"] files = [ {file = "black-24.3.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:7d5e026f8da0322b5662fa7a8e752b3fa2dac1c1cbc213c3d7ff9bdd0ab12395"}, {file = "black-24.3.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9f50ea1132e2189d8dff0115ab75b65590a3e97de1e143795adb4ce317934995"}, @@ -911,6 +962,23 @@ d = ["aiohttp (>=3.7.4)", "aiohttp (>=3.7.4,!=3.9.0)"] jupyter = ["ipython (>=7.8.0)", "tokenize-rt (>=3.2.0)"] uvloop = ["uvloop (>=0.15.2)"] +[[package]] +name = "blessed" +version = "1.20.0" +description = "Easy, practical library for making terminal apps, by providing an elegant, well-documented interface to Colors, Keyboard input, and screen Positioning capabilities." +optional = false +python-versions = ">=2.7" +groups = ["main"] +files = [ + {file = "blessed-1.20.0-py2.py3-none-any.whl", hash = "sha256:0c542922586a265e699188e52d5f5ac5ec0dd517e5a1041d90d2bbf23f906058"}, + {file = "blessed-1.20.0.tar.gz", hash = "sha256:2cdd67f8746e048f00df47a2880f4d6acbcdb399031b604e34ba8f71d5787680"}, +] + +[package.dependencies] +jinxed = {version = ">=1.1.0", markers = "platform_system == \"Windows\""} +six = ">=1.9.0" +wcwidth = ">=0.1.4" + [[package]] name = "blinker" version = "1.9.0" @@ -940,18 +1008,18 @@ requests = ">=2.31,<3.0" [[package]] name = "boto3" -version = "1.36.6" +version = "1.36.16" description = "The AWS SDK for Python" optional = false python-versions = ">=3.8" groups = ["main"] files = [ - {file = "boto3-1.36.6-py3-none-any.whl", hash = "sha256:6d473f0f340d02b4e9ad5b8e68786a09728101a8b950231b89ebdaf72b6dca21"}, - {file = "boto3-1.36.6.tar.gz", hash = "sha256:b36feae061dc0793cf311468956a0a9e99215ce38bc99a1a4e55a5b105f16297"}, + {file = "boto3-1.36.16-py3-none-any.whl", hash = "sha256:b10583bf8bd35be1b4027ee7e26b7cdf2078c79eab18357fd602cecb6d39400b"}, + {file = "boto3-1.36.16.tar.gz", hash = "sha256:0cf92ca0538ab115447e1c58050d43e1273e88c58ddfea2b6f133fdc508b400a"}, ] [package.dependencies] -botocore = ">=1.36.6,<1.37.0" +botocore = ">=1.36.16,<1.37.0" jmespath = ">=0.7.1,<2.0.0" s3transfer = ">=0.11.0,<0.12.0" @@ -960,14 +1028,14 @@ crt = ["botocore[crt] (>=1.21.0,<2.0a0)"] [[package]] name = "botocore" -version = "1.36.6" +version = "1.36.16" description = "Low-level, data-driven core of boto 3." optional = false python-versions = ">=3.8" groups = ["main"] files = [ - {file = "botocore-1.36.6-py3-none-any.whl", hash = "sha256:f77bbbb03fb420e260174650fb5c0cc142ec20a96967734eed2b0ef24334ef34"}, - {file = "botocore-1.36.6.tar.gz", hash = "sha256:4864c53d638da191a34daf3ede3ff1371a3719d952cc0c6bd24ce2836a38dd77"}, + {file = "botocore-1.36.16-py3-none-any.whl", hash = "sha256:aca0348ccd730332082489b6817fdf89e1526049adcf6e9c8c11c96dd9f42c03"}, + {file = "botocore-1.36.16.tar.gz", hash = "sha256:10c6aa386ba1a9a0faef6bb5dbfc58fc2563a3c6b95352e86a583cd5f14b11f3"}, ] [package.dependencies] @@ -976,7 +1044,7 @@ python-dateutil = ">=2.1,<3.0.0" urllib3 = {version = ">=1.25.4,<2.2.0 || >2.2.0,<3", markers = "python_version >= \"3.10\""} [package.extras] -crt = ["awscrt (==0.23.4)"] +crt = ["awscrt (==0.23.8)"] [[package]] name = "cachelib" @@ -1018,14 +1086,14 @@ ujson = ["ujson (>=5.7.0)"] [[package]] name = "certifi" -version = "2024.12.14" +version = "2025.1.31" description = "Python package for providing Mozilla's CA Bundle." optional = false python-versions = ">=3.6" groups = ["main", "test"] files = [ - {file = "certifi-2024.12.14-py3-none-any.whl", hash = "sha256:1275f7a45be9464efc1173084eaa30f866fe2e47d389406136d332ed4967ec56"}, - {file = "certifi-2024.12.14.tar.gz", hash = "sha256:b650d30f370c2b724812bee08008be0c4163b163ddaec3f2546c1caf65f191db"}, + {file = "certifi-2025.1.31-py3-none-any.whl", hash = "sha256:ca78db4565a652026a4db2bcdf68f2fb589ea80d0be70e03929ed730746b84fe"}, + {file = "certifi-2025.1.31.tar.gz", hash = "sha256:3d5da6925056f6f18f119200434a4780a94263f10d1c21d032a6f6b2baa20651"}, ] [[package]] @@ -1514,14 +1582,14 @@ files = [ [[package]] name = "deprecated" -version = "1.2.17" +version = "1.2.18" description = "Python @deprecated decorator to deprecate old python classes, functions or methods." optional = false python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,>=2.7" groups = ["main", "test"] files = [ - {file = "Deprecated-1.2.17-py2.py3-none-any.whl", hash = "sha256:69cdc0a751671183f569495e2efb14baee4344b0236342eec29f1fde25d61818"}, - {file = "deprecated-1.2.17.tar.gz", hash = "sha256:0114a10f0bbb750b90b2c2296c90cf7e9eaeb0abb5cf06c80de2c60138de0a82"}, + {file = "Deprecated-1.2.18-py2.py3-none-any.whl", hash = "sha256:bd5011788200372a32418f888e326a09ff80d0214bd961147cfed01b5c018eec"}, + {file = "deprecated-1.2.18.tar.gz", hash = "sha256:422b6f6d859da6f2ef57857761bfb392480502a64c3028ca9bbe86085d72115d"}, ] [package.dependencies] @@ -1578,6 +1646,22 @@ files = [ {file = "docopt-0.6.2.tar.gz", hash = "sha256:49b3a825280bd66b3aa83585ef59c4a8c82f2c8a522dbe754a8bc8d08c85c491"}, ] +[[package]] +name = "editor" +version = "1.6.6" +description = "🖋 Open the default text editor 🖋" +optional = false +python-versions = ">=3.8" +groups = ["main"] +files = [ + {file = "editor-1.6.6-py3-none-any.whl", hash = "sha256:e818e6913f26c2a81eadef503a2741d7cca7f235d20e217274a009ecd5a74abf"}, + {file = "editor-1.6.6.tar.gz", hash = "sha256:bb6989e872638cd119db9a4fce284cd8e13c553886a1c044c6b8d8a160c871f8"}, +] + +[package.dependencies] +runs = "*" +xmod = "*" + [[package]] name = "email-validator" version = "1.3.1" @@ -1616,7 +1700,7 @@ version = "6.1.0" description = "the modular source code checker: pep8 pyflakes and co" optional = false python-versions = ">=3.8.1" -groups = ["test"] +groups = ["main", "test"] files = [ {file = "flake8-6.1.0-py2.py3-none-any.whl", hash = "sha256:ffdfce58ea94c6580c77888a86506937f9a1a227dfcd15f245d694ae20a6b6e5"}, {file = "flake8-6.1.0.tar.gz", hash = "sha256:d5b3857f07c030bdb5bf41c7f53799571d75c4491748a3adcd47de929e34cd23"}, @@ -1627,6 +1711,96 @@ mccabe = ">=0.7.0,<0.8.0" pycodestyle = ">=2.11.0,<2.12.0" pyflakes = ">=3.1.0,<3.2.0" +[[package]] +name = "flake8-bandit" +version = "4.1.1" +description = "Automated security testing with bandit and flake8." +optional = false +python-versions = ">=3.6" +groups = ["main"] +files = [ + {file = "flake8_bandit-4.1.1-py3-none-any.whl", hash = "sha256:4c8a53eb48f23d4ef1e59293657181a3c989d0077c9952717e98a0eace43e06d"}, + {file = "flake8_bandit-4.1.1.tar.gz", hash = "sha256:068e09287189cbfd7f986e92605adea2067630b75380c6b5733dab7d87f9a84e"}, +] + +[package.dependencies] +bandit = ">=1.7.3" +flake8 = ">=5.0.0" + +[[package]] +name = "flake8-black" +version = "0.3.6" +description = "flake8 plugin to call black as a code style validator" +optional = false +python-versions = ">=3.7" +groups = ["main"] +files = [ + {file = "flake8-black-0.3.6.tar.gz", hash = "sha256:0dfbca3274777792a5bcb2af887a4cad72c72d0e86c94e08e3a3de151bb41c34"}, + {file = "flake8_black-0.3.6-py3-none-any.whl", hash = "sha256:fe8ea2eca98d8a504f22040d9117347f6b367458366952862ac3586e7d4eeaca"}, +] + +[package.dependencies] +black = ">=22.1.0" +flake8 = ">=3" +tomli = {version = "*", markers = "python_version < \"3.11\""} + +[package.extras] +develop = ["build", "twine"] + +[[package]] +name = "flake8-bugbear" +version = "24.12.12" +description = "A plugin for flake8 finding likely bugs and design problems in your program. Contains warnings that don't belong in pyflakes and pycodestyle." +optional = false +python-versions = ">=3.8.1" +groups = ["main"] +files = [ + {file = "flake8_bugbear-24.12.12-py3-none-any.whl", hash = "sha256:1b6967436f65ca22a42e5373aaa6f2d87966ade9aa38d4baf2a1be550767545e"}, + {file = "flake8_bugbear-24.12.12.tar.gz", hash = "sha256:46273cef0a6b6ff48ca2d69e472f41420a42a46e24b2a8972e4f0d6733d12a64"}, +] + +[package.dependencies] +attrs = ">=22.2.0" +flake8 = ">=6.0.0" + +[package.extras] +dev = ["coverage", "hypothesis", "hypothesmith (>=0.2)", "pre-commit", "pytest", "tox"] + +[[package]] +name = "flake8-docstrings" +version = "1.7.0" +description = "Extension for flake8 which uses pydocstyle to check docstrings" +optional = false +python-versions = ">=3.7" +groups = ["main"] +files = [ + {file = "flake8_docstrings-1.7.0-py2.py3-none-any.whl", hash = "sha256:51f2344026da083fc084166a9353f5082b01f72901df422f74b4d953ae88ac75"}, + {file = "flake8_docstrings-1.7.0.tar.gz", hash = "sha256:4c8cc748dc16e6869728699e5d0d685da9a10b0ea718e090b1ba088e67a941af"}, +] + +[package.dependencies] +flake8 = ">=3" +pydocstyle = ">=2.1" + +[[package]] +name = "flake8-isort" +version = "6.1.2" +description = "flake8 plugin that integrates isort" +optional = false +python-versions = ">=3.9" +groups = ["main"] +files = [ + {file = "flake8_isort-6.1.2-py3-none-any.whl", hash = "sha256:549197dedf0273502fb74f04c080beed9e62a7eb70244610413d27052e78bd3b"}, + {file = "flake8_isort-6.1.2.tar.gz", hash = "sha256:9d0452acdf0e1cd6f2d6848e3605e66b54d920e73471fb4744eef0f93df62d5d"}, +] + +[package.dependencies] +flake8 = "*" +isort = ">=5.0.0,<7" + +[package.extras] +test = ["pytest"] + [[package]] name = "flask" version = "2.2.5" @@ -1839,6 +2013,33 @@ wtforms = "*" [package.extras] email = ["email-validator"] +[[package]] +name = "folio-data-import" +version = "0.2.7" +description = "A python module to interact with the data importing capabilities of the open-source FOLIO ILS" +optional = false +python-versions = "<4.0,>=3.9" +groups = ["main"] +files = [ + {file = "folio_data_import-0.2.7-py3-none-any.whl", hash = "sha256:78944cc090408c004c1323a39194597d5261e90aa3ef763b1c2a6d8720ca1ae7"}, + {file = "folio_data_import-0.2.7.tar.gz", hash = "sha256:9b38a90d4ba8a441ab6975bab63227726266d0206e7b8dccc6b7b72baa6efc70"}, +] + +[package.dependencies] +aiofiles = ">=24.1.0,<25.0.0" +flake8-bandit = ">=4.1.1,<5.0.0" +flake8-black = ">=0.3.6,<0.4.0" +flake8-bugbear = ">=24.8.19,<25.0.0" +flake8-docstrings = ">=1.7.0,<2.0.0" +flake8-isort = ">=6.1.1,<7.0.0" +folioclient = ">=0.61.0,<0.62.0" +httpx = ">=0.27.2,<0.28.0" +inquirer = ">=3.4.0,<4.0.0" +pyhumps = ">=3.8.0,<4.0.0" +pymarc = ">=5.2.2,<6.0.0" +tabulate = ">=0.9.0,<0.10.0" +tqdm = ">=4.66.5,<5.0.0" + [[package]] name = "folio-migration-tools" version = "1.8.18" @@ -2000,14 +2201,14 @@ files = [ [[package]] name = "fsspec" -version = "2024.12.0" +version = "2025.2.0" description = "File-system specification" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "fsspec-2024.12.0-py3-none-any.whl", hash = "sha256:b520aed47ad9804237ff878b504267a3b0b441e97508bd6d2d8774e3db85cee2"}, - {file = "fsspec-2024.12.0.tar.gz", hash = "sha256:670700c977ed2fb51e0d9f9253177ed20cbde4a3e5c0283cc5385b5870c8533f"}, + {file = "fsspec-2025.2.0-py3-none-any.whl", hash = "sha256:9de2ad9ce1f85e1931858535bc882543171d197001a0a5eb2ddc04f1781ab95b"}, + {file = "fsspec-2025.2.0.tar.gz", hash = "sha256:1c24b16eaa0a1798afa0337aa0db9b256718ab2a89c425371f5628d22c3b6afd"}, ] [package.extras] @@ -2034,7 +2235,7 @@ sftp = ["paramiko"] smb = ["smbprotocol"] ssh = ["paramiko"] test = ["aiohttp (!=4.0.0a0,!=4.0.0a1)", "numpy", "pytest", "pytest-asyncio (!=0.22.0)", "pytest-benchmark", "pytest-cov", "pytest-mock", "pytest-recording", "pytest-rerunfailures", "requests"] -test-downstream = ["aiobotocore (>=2.5.4,<3.0.0)", "dask-expr", "dask[dataframe,test]", "moto[server] (>4,<5)", "pytest-timeout", "xarray"] +test-downstream = ["aiobotocore (>=2.5.4,<3.0.0)", "dask[dataframe,test]", "moto[server] (>4,<5)", "pytest-timeout", "xarray"] test-full = ["adlfs", "aiohttp (!=4.0.0a0,!=4.0.0a1)", "cloudpickle", "dask", "distributed", "dropbox", "dropboxdrivefs", "fastparquet", "fusepy", "gcsfs", "jinja2", "kerchunk", "libarchive-c", "lz4", "notebook", "numpy", "ocifs", "pandas", "panel", "paramiko", "pyarrow", "pyarrow (>=1)", "pyftpdlib", "pygit2", "pytest", "pytest-asyncio (!=0.22.0)", "pytest-benchmark", "pytest-cov", "pytest-mock", "pytest-recording", "pytest-rerunfailures", "python-snappy", "requests", "smbprotocol", "tqdm", "urllib3", "zarr", "zstandard"] tqdm = ["tqdm"] @@ -2433,6 +2634,23 @@ files = [ {file = "iniconfig-2.0.0.tar.gz", hash = "sha256:2d91e135bf72d31a410b17c16da610a82cb55f6b0477d1a902134b24a455b8b3"}, ] +[[package]] +name = "inquirer" +version = "3.4.0" +description = "Collection of common interactive command line user interfaces, based on Inquirer.js" +optional = false +python-versions = ">=3.8.1" +groups = ["main"] +files = [ + {file = "inquirer-3.4.0-py3-none-any.whl", hash = "sha256:bb0ec93c833e4ce7b51b98b1644b0a4d2bb39755c39787f6a504e4fee7a11b60"}, + {file = "inquirer-3.4.0.tar.gz", hash = "sha256:8edc99c076386ee2d2204e5e3653c2488244e82cb197b2d498b3c1b5ffb25d0b"}, +] + +[package.dependencies] +blessed = ">=1.19.0" +editor = ">=1.6.0" +readchar = ">=4.2.0" + [[package]] name = "isodate" version = "0.7.2" @@ -2445,6 +2663,22 @@ files = [ {file = "isodate-0.7.2.tar.gz", hash = "sha256:4cd1aa0f43ca76f4a6c6c0292a85f40b35ec2e43e315b59f06e6d32171a953e6"}, ] +[[package]] +name = "isort" +version = "6.0.0" +description = "A Python utility / library to sort Python imports." +optional = false +python-versions = ">=3.9.0" +groups = ["main"] +files = [ + {file = "isort-6.0.0-py3-none-any.whl", hash = "sha256:567954102bb47bb12e0fae62606570faacddd441e45683968c8d1734fb1af892"}, + {file = "isort-6.0.0.tar.gz", hash = "sha256:75d9d8a1438a9432a7d7b54f2d3b45cad9a4a0fdba43617d9873379704a8bdf1"}, +] + +[package.extras] +colors = ["colorama"] +plugins = ["setuptools"] + [[package]] name = "itsdangerous" version = "2.2.0" @@ -2475,6 +2709,22 @@ MarkupSafe = ">=2.0" [package.extras] i18n = ["Babel (>=2.7)"] +[[package]] +name = "jinxed" +version = "1.3.0" +description = "Jinxed Terminal Library" +optional = false +python-versions = "*" +groups = ["main"] +markers = "platform_system == \"Windows\"" +files = [ + {file = "jinxed-1.3.0-py2.py3-none-any.whl", hash = "sha256:b993189f39dc2d7504d802152671535b06d380b26d78070559551cbf92df4fc5"}, + {file = "jinxed-1.3.0.tar.gz", hash = "sha256:1593124b18a41b7a3da3b078471442e51dbad3d77b4d4f2b0c26ab6f7d660dbf"}, +] + +[package.dependencies] +ansicon = {version = "*", markers = "platform_system == \"Windows\""} + [[package]] name = "jmespath" version = "1.0.1" @@ -2818,14 +3068,14 @@ source = ["Cython (>=3.0.11)"] [[package]] name = "mako" -version = "1.3.8" +version = "1.3.9" description = "A super-fast templating language that borrows the best ideas from the existing templating languages." optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "Mako-1.3.8-py3-none-any.whl", hash = "sha256:42f48953c7eb91332040ff567eb7eea69b22e7a4affbc5ba8e845e8f730f6627"}, - {file = "mako-1.3.8.tar.gz", hash = "sha256:577b97e414580d3e088d47c2dbbe9594aa7a5146ed2875d4dfa9075af2dd3cc8"}, + {file = "Mako-1.3.9-py3-none-any.whl", hash = "sha256:95920acccb578427a9aa38e37a186b1e43156c87260d7ba18ca63aa4c7cbd3a1"}, + {file = "mako-1.3.9.tar.gz", hash = "sha256:b5d65ff3462870feec922dbccf38f6efb44e5714d7b593a656be86663d8600ac"}, ] [package.dependencies] @@ -2950,14 +3200,14 @@ files = [ [[package]] name = "marshmallow" -version = "3.26.0" +version = "3.26.1" description = "A lightweight library for converting complex datatypes to and from native Python datatypes." optional = false python-versions = ">=3.9" groups = ["main", "test"] files = [ - {file = "marshmallow-3.26.0-py3-none-any.whl", hash = "sha256:1287bca04e6a5f4094822ac153c03da5e214a0a60bcd557b140f3e66991b8ca1"}, - {file = "marshmallow-3.26.0.tar.gz", hash = "sha256:eb36762a1cc76d7abf831e18a3a1b26d3d481bbc74581b8e532a3d3a8115e1cb"}, + {file = "marshmallow-3.26.1-py3-none-any.whl", hash = "sha256:3350409f20a70a7e4e11a27661187b77cdcaeb20abca41c1454fe33636bea09c"}, + {file = "marshmallow-3.26.1.tar.gz", hash = "sha256:e6d8affb6cb61d39d26402096dc0aee12d5a26d490a121f118d2e81dc0719dc6"}, ] [package.dependencies] @@ -3015,7 +3265,7 @@ version = "0.7.0" description = "McCabe checker, plugin for flake8" optional = false python-versions = ">=3.6" -groups = ["test"] +groups = ["main", "test"] files = [ {file = "mccabe-0.7.0-py2.py3-none-any.whl", hash = "sha256:6c2d30ab6be0e4a46919781807b4f0d834ebdd6c6e3dca0bda5a15f863427b6e"}, {file = "mccabe-0.7.0.tar.gz", hash = "sha256:348e0240c33b60bbdf4e523192ef919f28cb2c3d7d5c7794f74009290f236325"}, @@ -3172,50 +3422,44 @@ typing-extensions = {version = ">=4.1.0", markers = "python_version < \"3.11\""} [[package]] name = "mypy" -version = "1.14.1" +version = "1.15.0" description = "Optional static typing for Python" optional = false -python-versions = ">=3.8" +python-versions = ">=3.9" groups = ["test"] files = [ - {file = "mypy-1.14.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:52686e37cf13d559f668aa398dd7ddf1f92c5d613e4f8cb262be2fb4fedb0fcb"}, - {file = "mypy-1.14.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1fb545ca340537d4b45d3eecdb3def05e913299ca72c290326be19b3804b39c0"}, - {file = "mypy-1.14.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:90716d8b2d1f4cd503309788e51366f07c56635a3309b0f6a32547eaaa36a64d"}, - {file = "mypy-1.14.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:2ae753f5c9fef278bcf12e1a564351764f2a6da579d4a81347e1d5a15819997b"}, - {file = "mypy-1.14.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:e0fe0f5feaafcb04505bcf439e991c6d8f1bf8b15f12b05feeed96e9e7bf1427"}, - {file = "mypy-1.14.1-cp310-cp310-win_amd64.whl", hash = "sha256:7d54bd85b925e501c555a3227f3ec0cfc54ee8b6930bd6141ec872d1c572f81f"}, - {file = "mypy-1.14.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:f995e511de847791c3b11ed90084a7a0aafdc074ab88c5a9711622fe4751138c"}, - {file = "mypy-1.14.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:d64169ec3b8461311f8ce2fd2eb5d33e2d0f2c7b49116259c51d0d96edee48d1"}, - {file = "mypy-1.14.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:ba24549de7b89b6381b91fbc068d798192b1b5201987070319889e93038967a8"}, - {file = "mypy-1.14.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:183cf0a45457d28ff9d758730cd0210419ac27d4d3f285beda038c9083363b1f"}, - {file = "mypy-1.14.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:f2a0ecc86378f45347f586e4163d1769dd81c5a223d577fe351f26b179e148b1"}, - {file = "mypy-1.14.1-cp311-cp311-win_amd64.whl", hash = "sha256:ad3301ebebec9e8ee7135d8e3109ca76c23752bac1e717bc84cd3836b4bf3eae"}, - {file = "mypy-1.14.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:30ff5ef8519bbc2e18b3b54521ec319513a26f1bba19a7582e7b1f58a6e69f14"}, - {file = "mypy-1.14.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:cb9f255c18052343c70234907e2e532bc7e55a62565d64536dbc7706a20b78b9"}, - {file = "mypy-1.14.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:8b4e3413e0bddea671012b063e27591b953d653209e7a4fa5e48759cda77ca11"}, - {file = "mypy-1.14.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:553c293b1fbdebb6c3c4030589dab9fafb6dfa768995a453d8a5d3b23784af2e"}, - {file = "mypy-1.14.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:fad79bfe3b65fe6a1efaed97b445c3d37f7be9fdc348bdb2d7cac75579607c89"}, - {file = "mypy-1.14.1-cp312-cp312-win_amd64.whl", hash = "sha256:8fa2220e54d2946e94ab6dbb3ba0a992795bd68b16dc852db33028df2b00191b"}, - {file = "mypy-1.14.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:92c3ed5afb06c3a8e188cb5da4984cab9ec9a77ba956ee419c68a388b4595255"}, - {file = "mypy-1.14.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:dbec574648b3e25f43d23577309b16534431db4ddc09fda50841f1e34e64ed34"}, - {file = "mypy-1.14.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:8c6d94b16d62eb3e947281aa7347d78236688e21081f11de976376cf010eb31a"}, - {file = "mypy-1.14.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:d4b19b03fdf54f3c5b2fa474c56b4c13c9dbfb9a2db4370ede7ec11a2c5927d9"}, - {file = "mypy-1.14.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:0c911fde686394753fff899c409fd4e16e9b294c24bfd5e1ea4675deae1ac6fd"}, - {file = "mypy-1.14.1-cp313-cp313-win_amd64.whl", hash = "sha256:8b21525cb51671219f5307be85f7e646a153e5acc656e5cebf64bfa076c50107"}, - {file = "mypy-1.14.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:7084fb8f1128c76cd9cf68fe5971b37072598e7c31b2f9f95586b65c741a9d31"}, - {file = "mypy-1.14.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:8f845a00b4f420f693f870eaee5f3e2692fa84cc8514496114649cfa8fd5e2c6"}, - {file = "mypy-1.14.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:44bf464499f0e3a2d14d58b54674dee25c031703b2ffc35064bd0df2e0fac319"}, - {file = "mypy-1.14.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:c99f27732c0b7dc847adb21c9d47ce57eb48fa33a17bc6d7d5c5e9f9e7ae5bac"}, - {file = "mypy-1.14.1-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:bce23c7377b43602baa0bd22ea3265c49b9ff0b76eb315d6c34721af4cdf1d9b"}, - {file = "mypy-1.14.1-cp38-cp38-win_amd64.whl", hash = "sha256:8edc07eeade7ebc771ff9cf6b211b9a7d93687ff892150cb5692e4f4272b0837"}, - {file = "mypy-1.14.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3888a1816d69f7ab92092f785a462944b3ca16d7c470d564165fe703b0970c35"}, - {file = "mypy-1.14.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:46c756a444117c43ee984bd055db99e498bc613a70bbbc120272bd13ca579fbc"}, - {file = "mypy-1.14.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:27fc248022907e72abfd8e22ab1f10e903915ff69961174784a3900a8cba9ad9"}, - {file = "mypy-1.14.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:499d6a72fb7e5de92218db961f1a66d5f11783f9ae549d214617edab5d4dbdbb"}, - {file = "mypy-1.14.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:57961db9795eb566dc1d1b4e9139ebc4c6b0cb6e7254ecde69d1552bf7613f60"}, - {file = "mypy-1.14.1-cp39-cp39-win_amd64.whl", hash = "sha256:07ba89fdcc9451f2ebb02853deb6aaaa3d2239a236669a63ab3801bbf923ef5c"}, - {file = "mypy-1.14.1-py3-none-any.whl", hash = "sha256:b66a60cc4073aeb8ae00057f9c1f64d49e90f918fbcef9a977eb121da8b8f1d1"}, - {file = "mypy-1.14.1.tar.gz", hash = "sha256:7ec88144fe9b510e8475ec2f5f251992690fcf89ccb4500b214b4226abcd32d6"}, + {file = "mypy-1.15.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:979e4e1a006511dacf628e36fadfecbcc0160a8af6ca7dad2f5025529e082c13"}, + {file = "mypy-1.15.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c4bb0e1bd29f7d34efcccd71cf733580191e9a264a2202b0239da95984c5b559"}, + {file = "mypy-1.15.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:be68172e9fd9ad8fb876c6389f16d1c1b5f100ffa779f77b1fb2176fcc9ab95b"}, + {file = "mypy-1.15.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:c7be1e46525adfa0d97681432ee9fcd61a3964c2446795714699a998d193f1a3"}, + {file = "mypy-1.15.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:2e2c2e6d3593f6451b18588848e66260ff62ccca522dd231cd4dd59b0160668b"}, + {file = "mypy-1.15.0-cp310-cp310-win_amd64.whl", hash = "sha256:6983aae8b2f653e098edb77f893f7b6aca69f6cffb19b2cc7443f23cce5f4828"}, + {file = "mypy-1.15.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:2922d42e16d6de288022e5ca321cd0618b238cfc5570e0263e5ba0a77dbef56f"}, + {file = "mypy-1.15.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:2ee2d57e01a7c35de00f4634ba1bbf015185b219e4dc5909e281016df43f5ee5"}, + {file = "mypy-1.15.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:973500e0774b85d9689715feeffcc980193086551110fd678ebe1f4342fb7c5e"}, + {file = "mypy-1.15.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:5a95fb17c13e29d2d5195869262f8125dfdb5c134dc8d9a9d0aecf7525b10c2c"}, + {file = "mypy-1.15.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:1905f494bfd7d85a23a88c5d97840888a7bd516545fc5aaedff0267e0bb54e2f"}, + {file = "mypy-1.15.0-cp311-cp311-win_amd64.whl", hash = "sha256:c9817fa23833ff189db061e6d2eff49b2f3b6ed9856b4a0a73046e41932d744f"}, + {file = "mypy-1.15.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:aea39e0583d05124836ea645f412e88a5c7d0fd77a6d694b60d9b6b2d9f184fd"}, + {file = "mypy-1.15.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:2f2147ab812b75e5b5499b01ade1f4a81489a147c01585cda36019102538615f"}, + {file = "mypy-1.15.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:ce436f4c6d218a070048ed6a44c0bbb10cd2cc5e272b29e7845f6a2f57ee4464"}, + {file = "mypy-1.15.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:8023ff13985661b50a5928fc7a5ca15f3d1affb41e5f0a9952cb68ef090b31ee"}, + {file = "mypy-1.15.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:1124a18bc11a6a62887e3e137f37f53fbae476dc36c185d549d4f837a2a6a14e"}, + {file = "mypy-1.15.0-cp312-cp312-win_amd64.whl", hash = "sha256:171a9ca9a40cd1843abeca0e405bc1940cd9b305eaeea2dda769ba096932bb22"}, + {file = "mypy-1.15.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:93faf3fdb04768d44bf28693293f3904bbb555d076b781ad2530214ee53e3445"}, + {file = "mypy-1.15.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:811aeccadfb730024c5d3e326b2fbe9249bb7413553f15499a4050f7c30e801d"}, + {file = "mypy-1.15.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:98b7b9b9aedb65fe628c62a6dc57f6d5088ef2dfca37903a7d9ee374d03acca5"}, + {file = "mypy-1.15.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:c43a7682e24b4f576d93072216bf56eeff70d9140241f9edec0c104d0c515036"}, + {file = "mypy-1.15.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:baefc32840a9f00babd83251560e0ae1573e2f9d1b067719479bfb0e987c6357"}, + {file = "mypy-1.15.0-cp313-cp313-win_amd64.whl", hash = "sha256:b9378e2c00146c44793c98b8d5a61039a048e31f429fb0eb546d93f4b000bedf"}, + {file = "mypy-1.15.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:e601a7fa172c2131bff456bb3ee08a88360760d0d2f8cbd7a75a65497e2df078"}, + {file = "mypy-1.15.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:712e962a6357634fef20412699a3655c610110e01cdaa6180acec7fc9f8513ba"}, + {file = "mypy-1.15.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:f95579473af29ab73a10bada2f9722856792a36ec5af5399b653aa28360290a5"}, + {file = "mypy-1.15.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:8f8722560a14cde92fdb1e31597760dc35f9f5524cce17836c0d22841830fd5b"}, + {file = "mypy-1.15.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:1fbb8da62dc352133d7d7ca90ed2fb0e9d42bb1a32724c287d3c76c58cbaa9c2"}, + {file = "mypy-1.15.0-cp39-cp39-win_amd64.whl", hash = "sha256:d10d994b41fb3497719bbf866f227b3489048ea4bbbb5015357db306249f7980"}, + {file = "mypy-1.15.0-py3-none-any.whl", hash = "sha256:5469affef548bd1895d86d3bf10ce2b44e33d86923c29e4d675b3e323437ea3e"}, + {file = "mypy-1.15.0.tar.gz", hash = "sha256:404534629d51d3efea5c800ee7c42b72a6554d6c400e6a79eafe15d11341fd43"}, ] [package.dependencies] @@ -3236,7 +3480,7 @@ version = "1.0.0" description = "Type system extensions for programs checked with the mypy type checker." optional = false python-versions = ">=3.5" -groups = ["test"] +groups = ["main", "test"] files = [ {file = "mypy_extensions-1.0.0-py3-none-any.whl", hash = "sha256:4392f6c0eb8a5668a69e23d168ffa70f0be9ccfd32b5cc2d26a34ae5b844552d"}, {file = "mypy_extensions-1.0.0.tar.gz", hash = "sha256:75dbf8955dc00442a438fc4d0666508a9a97b6bd41aa2f0ffe9d2f2725af0782"}, @@ -3302,14 +3546,14 @@ files = [ [[package]] name = "opentelemetry-api" -version = "1.29.0" +version = "1.30.0" description = "OpenTelemetry Python API" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "opentelemetry_api-1.29.0-py3-none-any.whl", hash = "sha256:5fcd94c4141cc49c736271f3e1efb777bebe9cc535759c54c936cca4f1b312b8"}, - {file = "opentelemetry_api-1.29.0.tar.gz", hash = "sha256:d04a6cf78aad09614f52964ecb38021e248f5714dc32c2e0d8fd99517b4d69cf"}, + {file = "opentelemetry_api-1.30.0-py3-none-any.whl", hash = "sha256:d5f5284890d73fdf47f843dda3210edf37a38d66f44f2b5aedc1e89ed455dc09"}, + {file = "opentelemetry_api-1.30.0.tar.gz", hash = "sha256:375893400c1435bf623f7dfb3bcd44825fe6b56c34d0667c542ea8257b1a1240"}, ] [package.dependencies] @@ -3318,45 +3562,45 @@ importlib-metadata = ">=6.0,<=8.5.0" [[package]] name = "opentelemetry-exporter-otlp" -version = "1.29.0" +version = "1.30.0" description = "OpenTelemetry Collector Exporters" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "opentelemetry_exporter_otlp-1.29.0-py3-none-any.whl", hash = "sha256:b8da6e20f5b0ffe604154b1e16a407eade17ce310c42fb85bb4e1246fc3688ad"}, - {file = "opentelemetry_exporter_otlp-1.29.0.tar.gz", hash = "sha256:ee7dfcccbb5e87ad9b389908452e10b7beeab55f70a83f41ce5b8c4efbde6544"}, + {file = "opentelemetry_exporter_otlp-1.30.0-py3-none-any.whl", hash = "sha256:44e11054ec571ccfed73a83c6429dee5d334d061d0e0572e3160d6de97156dbc"}, + {file = "opentelemetry_exporter_otlp-1.30.0.tar.gz", hash = "sha256:da7769f95cd5be5b09dd4188ac153a33709eda652217f2d10aed6518c8e60f0d"}, ] [package.dependencies] -opentelemetry-exporter-otlp-proto-grpc = "1.29.0" -opentelemetry-exporter-otlp-proto-http = "1.29.0" +opentelemetry-exporter-otlp-proto-grpc = "1.30.0" +opentelemetry-exporter-otlp-proto-http = "1.30.0" [[package]] name = "opentelemetry-exporter-otlp-proto-common" -version = "1.29.0" +version = "1.30.0" description = "OpenTelemetry Protobuf encoding" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "opentelemetry_exporter_otlp_proto_common-1.29.0-py3-none-any.whl", hash = "sha256:a9d7376c06b4da9cf350677bcddb9618ed4b8255c3f6476975f5e38274ecd3aa"}, - {file = "opentelemetry_exporter_otlp_proto_common-1.29.0.tar.gz", hash = "sha256:e7c39b5dbd1b78fe199e40ddfe477e6983cb61aa74ba836df09c3869a3e3e163"}, + {file = "opentelemetry_exporter_otlp_proto_common-1.30.0-py3-none-any.whl", hash = "sha256:5468007c81aa9c44dc961ab2cf368a29d3475977df83b4e30aeed42aa7bc3b38"}, + {file = "opentelemetry_exporter_otlp_proto_common-1.30.0.tar.gz", hash = "sha256:ddbfbf797e518411857d0ca062c957080279320d6235a279f7b64ced73c13897"}, ] [package.dependencies] -opentelemetry-proto = "1.29.0" +opentelemetry-proto = "1.30.0" [[package]] name = "opentelemetry-exporter-otlp-proto-grpc" -version = "1.29.0" +version = "1.30.0" description = "OpenTelemetry Collector Protobuf over gRPC Exporter" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "opentelemetry_exporter_otlp_proto_grpc-1.29.0-py3-none-any.whl", hash = "sha256:5a2a3a741a2543ed162676cf3eefc2b4150e6f4f0a193187afb0d0e65039c69c"}, - {file = "opentelemetry_exporter_otlp_proto_grpc-1.29.0.tar.gz", hash = "sha256:3d324d07d64574d72ed178698de3d717f62a059a93b6b7685ee3e303384e73ea"}, + {file = "opentelemetry_exporter_otlp_proto_grpc-1.30.0-py3-none-any.whl", hash = "sha256:2906bcae3d80acc54fd1ffcb9e44d324e8631058b502ebe4643ca71d1ff30830"}, + {file = "opentelemetry_exporter_otlp_proto_grpc-1.30.0.tar.gz", hash = "sha256:d0f10f0b9b9a383b7d04a144d01cb280e70362cccc613987e234183fd1f01177"}, ] [package.dependencies] @@ -3364,41 +3608,41 @@ deprecated = ">=1.2.6" googleapis-common-protos = ">=1.52,<2.0" grpcio = ">=1.63.2,<2.0.0" opentelemetry-api = ">=1.15,<2.0" -opentelemetry-exporter-otlp-proto-common = "1.29.0" -opentelemetry-proto = "1.29.0" -opentelemetry-sdk = ">=1.29.0,<1.30.0" +opentelemetry-exporter-otlp-proto-common = "1.30.0" +opentelemetry-proto = "1.30.0" +opentelemetry-sdk = ">=1.30.0,<1.31.0" [[package]] name = "opentelemetry-exporter-otlp-proto-http" -version = "1.29.0" +version = "1.30.0" description = "OpenTelemetry Collector Protobuf over HTTP Exporter" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "opentelemetry_exporter_otlp_proto_http-1.29.0-py3-none-any.whl", hash = "sha256:b228bdc0f0cfab82eeea834a7f0ffdd2a258b26aa33d89fb426c29e8e934d9d0"}, - {file = "opentelemetry_exporter_otlp_proto_http-1.29.0.tar.gz", hash = "sha256:b10d174e3189716f49d386d66361fbcf6f2b9ad81e05404acdee3f65c8214204"}, + {file = "opentelemetry_exporter_otlp_proto_http-1.30.0-py3-none-any.whl", hash = "sha256:9578e790e579931c5ffd50f1e6975cbdefb6a0a0a5dea127a6ae87df10e0a589"}, + {file = "opentelemetry_exporter_otlp_proto_http-1.30.0.tar.gz", hash = "sha256:c3ae75d4181b1e34a60662a6814d0b94dd33b628bee5588a878bed92cee6abdc"}, ] [package.dependencies] deprecated = ">=1.2.6" googleapis-common-protos = ">=1.52,<2.0" opentelemetry-api = ">=1.15,<2.0" -opentelemetry-exporter-otlp-proto-common = "1.29.0" -opentelemetry-proto = "1.29.0" -opentelemetry-sdk = ">=1.29.0,<1.30.0" +opentelemetry-exporter-otlp-proto-common = "1.30.0" +opentelemetry-proto = "1.30.0" +opentelemetry-sdk = ">=1.30.0,<1.31.0" requests = ">=2.7,<3.0" [[package]] name = "opentelemetry-proto" -version = "1.29.0" +version = "1.30.0" description = "OpenTelemetry Python Proto" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "opentelemetry_proto-1.29.0-py3-none-any.whl", hash = "sha256:495069c6f5495cbf732501cdcd3b7f60fda2b9d3d4255706ca99b7ca8dec53ff"}, - {file = "opentelemetry_proto-1.29.0.tar.gz", hash = "sha256:3c136aa293782e9b44978c738fff72877a4b78b5d21a64e879898db7b2d93e5d"}, + {file = "opentelemetry_proto-1.30.0-py3-none-any.whl", hash = "sha256:c6290958ff3ddacc826ca5abbeb377a31c2334387352a259ba0df37c243adc11"}, + {file = "opentelemetry_proto-1.30.0.tar.gz", hash = "sha256:afe5c9c15e8b68d7c469596e5b32e8fc085eb9febdd6fb4e20924a93a0389179"}, ] [package.dependencies] @@ -3406,36 +3650,36 @@ protobuf = ">=5.0,<6.0" [[package]] name = "opentelemetry-sdk" -version = "1.29.0" +version = "1.30.0" description = "OpenTelemetry Python SDK" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "opentelemetry_sdk-1.29.0-py3-none-any.whl", hash = "sha256:173be3b5d3f8f7d671f20ea37056710217959e774e2749d984355d1f9391a30a"}, - {file = "opentelemetry_sdk-1.29.0.tar.gz", hash = "sha256:b0787ce6aade6ab84315302e72bd7a7f2f014b0fb1b7c3295b88afe014ed0643"}, + {file = "opentelemetry_sdk-1.30.0-py3-none-any.whl", hash = "sha256:14fe7afc090caad881addb6926cec967129bd9260c4d33ae6a217359f6b61091"}, + {file = "opentelemetry_sdk-1.30.0.tar.gz", hash = "sha256:c9287a9e4a7614b9946e933a67168450b9ab35f08797eb9bc77d998fa480fa18"}, ] [package.dependencies] -opentelemetry-api = "1.29.0" -opentelemetry-semantic-conventions = "0.50b0" +opentelemetry-api = "1.30.0" +opentelemetry-semantic-conventions = "0.51b0" typing-extensions = ">=3.7.4" [[package]] name = "opentelemetry-semantic-conventions" -version = "0.50b0" +version = "0.51b0" description = "OpenTelemetry Semantic Conventions" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "opentelemetry_semantic_conventions-0.50b0-py3-none-any.whl", hash = "sha256:e87efba8fdb67fb38113efea6a349531e75ed7ffc01562f65b802fcecb5e115e"}, - {file = "opentelemetry_semantic_conventions-0.50b0.tar.gz", hash = "sha256:02dc6dbcb62f082de9b877ff19a3f1ffaa3c306300fa53bfac761c4567c83d38"}, + {file = "opentelemetry_semantic_conventions-0.51b0-py3-none-any.whl", hash = "sha256:fdc777359418e8d06c86012c3dc92c88a6453ba662e941593adb062e48c2eeae"}, + {file = "opentelemetry_semantic_conventions-0.51b0.tar.gz", hash = "sha256:3fabf47f35d1fd9aebcdca7e6802d86bd5ebc3bc3408b7e3248dde6e87a18c47"}, ] [package.dependencies] deprecated = ">=1.2.6" -opentelemetry-api = "1.29.0" +opentelemetry-api = "1.30.0" [[package]] name = "ordered-set" @@ -3514,14 +3758,14 @@ test = ["hypothesis (>=5.5.3)", "pytest (>=6.0)", "pytest-xdist (>=1.31)"] [[package]] name = "paramiko" -version = "3.5.0" +version = "3.5.1" description = "SSH2 protocol library" optional = false python-versions = ">=3.6" groups = ["test"] files = [ - {file = "paramiko-3.5.0-py3-none-any.whl", hash = "sha256:1fedf06b085359051cd7d0d270cebe19e755a8a921cc2ddbfa647fb0cd7d68f9"}, - {file = "paramiko-3.5.0.tar.gz", hash = "sha256:ad11e540da4f55cedda52931f1a3f812a8238a7af7f62a60de538cd80bb28124"}, + {file = "paramiko-3.5.1-py3-none-any.whl", hash = "sha256:43b9a0501fc2b5e70680388d9346cf252cfb7d00b0667c39e80eb43a408b8f61"}, + {file = "paramiko-3.5.1.tar.gz", hash = "sha256:b2c665bc45b2b215bd7d7f039901b14b067da00f3a11e6640995fd58f2664822"}, ] [package.dependencies] @@ -3546,6 +3790,21 @@ files = [ {file = "pathspec-0.12.1.tar.gz", hash = "sha256:a482d51503a1ab33b1c67a6c3813a26953dbdc71c31dacaef9a838c4e29f5712"}, ] +[[package]] +name = "pbr" +version = "6.1.1" +description = "Python Build Reasonableness" +optional = false +python-versions = ">=2.6" +groups = ["main"] +files = [ + {file = "pbr-6.1.1-py2.py3-none-any.whl", hash = "sha256:38d4daea5d9fa63b3f626131b9d34947fd0c8be9b05a29276870580050a25a76"}, + {file = "pbr-6.1.1.tar.gz", hash = "sha256:93ea72ce6989eb2eed99d0f75721474f69ad88128afdef5ac377eb797c4bf76b"}, +] + +[package.dependencies] +setuptools = "*" + [[package]] name = "pendulum" version = "2.1.2" @@ -3587,7 +3846,7 @@ version = "4.3.6" description = "A small Python package for determining appropriate platform-specific dirs, e.g. a `user data dir`." optional = false python-versions = ">=3.8" -groups = ["test"] +groups = ["main", "test"] files = [ {file = "platformdirs-4.3.6-py3-none-any.whl", hash = "sha256:73e575e1408ab8103900836b97580d5307456908a03e92031bab39e4554cc3fb"}, {file = "platformdirs-4.3.6.tar.gz", hash = "sha256:357fb2acbc885b0419afd3ce3ed34564c13c9b95c89360cd9563f73aa5e2b907"}, @@ -3924,7 +4183,7 @@ version = "2.11.1" description = "Python style guide checker" optional = false python-versions = ">=3.8" -groups = ["test"] +groups = ["main", "test"] files = [ {file = "pycodestyle-2.11.1-py2.py3-none-any.whl", hash = "sha256:44fe31000b2d866f2e41841b18528a505fbd7fef9017b04eff4e2648a0fadc67"}, {file = "pycodestyle-2.11.1.tar.gz", hash = "sha256:41ba0e7afc9752dfb53ced5489e89f8186be00e599e712660695b7a75ff2663f"}, @@ -4010,13 +4269,31 @@ typing-extensions = ">=4.2.0" dotenv = ["python-dotenv (>=0.10.4)"] email = ["email-validator (>=1.0.3)"] +[[package]] +name = "pydocstyle" +version = "6.3.0" +description = "Python docstring style checker" +optional = false +python-versions = ">=3.6" +groups = ["main"] +files = [ + {file = "pydocstyle-6.3.0-py3-none-any.whl", hash = "sha256:118762d452a49d6b05e194ef344a55822987a462831ade91ec5c06fd2169d019"}, + {file = "pydocstyle-6.3.0.tar.gz", hash = "sha256:7ce43f0c0ac87b07494eb9c0b462c0b73e6ff276807f204d6b53edc72b7e44e1"}, +] + +[package.dependencies] +snowballstemmer = ">=2.2.0" + +[package.extras] +toml = ["tomli (>=1.2.3)"] + [[package]] name = "pyflakes" version = "3.1.0" description = "passive checker of Python programs" optional = false python-versions = ">=3.8" -groups = ["test"] +groups = ["main", "test"] files = [ {file = "pyflakes-3.1.0-py2.py3-none-any.whl", hash = "sha256:4132f6d49cb4dae6819e5379898f2b8cce3c5f23994194c24b77d5da2e36f774"}, {file = "pyflakes-3.1.0.tar.gz", hash = "sha256:a0aae034c444db0071aa077972ba4768d40c830d9539fd45bf4cd3f8f6992efc"}, @@ -4329,14 +4606,14 @@ test = ["coverage (>=4.5.2)", "flake8 (>=3.6.0,<=5.0.0)", "freezegun (>=0.3.11,< [[package]] name = "pytz" -version = "2024.2" +version = "2025.1" description = "World timezone definitions, modern and historical" optional = false python-versions = "*" groups = ["main", "test"] files = [ - {file = "pytz-2024.2-py2.py3-none-any.whl", hash = "sha256:31c7c1817eb7fae7ca4b8c7ee50c72f93aa2dd863de768e1ef4245d426aa0725"}, - {file = "pytz-2024.2.tar.gz", hash = "sha256:2aa355083c50a0f93fa581709deac0c9ad65cca8a9e9beac660adcbd493c798a"}, + {file = "pytz-2025.1-py2.py3-none-any.whl", hash = "sha256:89dd22dca55b46eac6eda23b2d72721bf1bdfef212645d81513ef5d03038de57"}, + {file = "pytz-2025.1.tar.gz", hash = "sha256:c2db42be2a2518b28e65f9207c4d05e6ff547d1efa4086469ef855e4ab70178e"}, ] [[package]] @@ -4414,6 +4691,18 @@ files = [ {file = "pyyaml-6.0.2.tar.gz", hash = "sha256:d584d9ec91ad65861cc08d42e834324ef890a082e591037abe114850ff7bbc3e"}, ] +[[package]] +name = "readchar" +version = "4.2.1" +description = "Library to easily read single chars and key strokes" +optional = false +python-versions = ">=3.8" +groups = ["main"] +files = [ + {file = "readchar-4.2.1-py3-none-any.whl", hash = "sha256:a769305cd3994bb5fa2764aa4073452dc105a4ec39068ffe6efd3c20c60acc77"}, + {file = "readchar-4.2.1.tar.gz", hash = "sha256:91ce3faf07688de14d800592951e5575e9c7a3213738ed01d394dcc949b79adb"}, +] + [[package]] name = "redshift-connector" version = "2.1.5" @@ -4548,14 +4837,14 @@ jupyter = ["ipywidgets (>=7.5.1,<9)"] [[package]] name = "rich-argparse" -version = "1.6.0" +version = "1.7.0" description = "Rich help formatters for argparse and optparse" optional = false python-versions = ">=3.8" groups = ["main", "test"] files = [ - {file = "rich_argparse-1.6.0-py3-none-any.whl", hash = "sha256:fbe70a1d821b3f2fa8958cddf0cae131870a6e9faa04ab52b409cb1eda809bd7"}, - {file = "rich_argparse-1.6.0.tar.gz", hash = "sha256:092083c30da186f25bcdff8b1d47fdfb571288510fb051e0488a72cc3128de13"}, + {file = "rich_argparse-1.7.0-py3-none-any.whl", hash = "sha256:b8ec8943588e9731967f4f97b735b03dc127c416f480a083060433a97baf2fd3"}, + {file = "rich_argparse-1.7.0.tar.gz", hash = "sha256:f31d809c465ee43f367d599ccaf88b73bc2c4d75d74ed43f2d538838c53544ba"}, ] [package.dependencies] @@ -4674,6 +4963,21 @@ files = [ {file = "rpds_py-0.22.3.tar.gz", hash = "sha256:e32fee8ab45d3c2db6da19a5323bc3362237c8b653c70194414b892fd06a080d"}, ] +[[package]] +name = "runs" +version = "1.2.2" +description = "🏃 Run a block of text as a subprocess 🏃" +optional = false +python-versions = ">=3.8" +groups = ["main"] +files = [ + {file = "runs-1.2.2-py3-none-any.whl", hash = "sha256:0980dcbc25aba1505f307ac4f0e9e92cbd0be2a15a1e983ee86c24c87b839dfd"}, + {file = "runs-1.2.2.tar.gz", hash = "sha256:9dc1815e2895cfb3a48317b173b9f1eac9ba5549b36a847b5cc60c3bf82ecef1"}, +] + +[package.dependencies] +xmod = "*" + [[package]] name = "s3path" version = "0.5.8" @@ -4892,6 +5196,18 @@ files = [ {file = "sniffio-1.3.1.tar.gz", hash = "sha256:f4324edc670a0f49750a81b895f35c3adb843cca46f0530f79fc1babb23789dc"}, ] +[[package]] +name = "snowballstemmer" +version = "2.2.0" +description = "This package provides 29 stemmers for 28 languages generated from Snowball algorithms." +optional = false +python-versions = "*" +groups = ["main"] +files = [ + {file = "snowballstemmer-2.2.0-py2.py3-none-any.whl", hash = "sha256:c8e1716e83cc398ae16824e5572ae04e0d9fc2c6b985fb0f900f5f0c96ecba1a"}, + {file = "snowballstemmer-2.2.0.tar.gz", hash = "sha256:09b16deb8547d3412ad7b590689584cd0fe25ec8db3be37788be3810cbf19cb1"}, +] + [[package]] name = "soupsieve" version = "2.6" @@ -5062,6 +5378,21 @@ build-sphinx = ["sphinx", "sphinxcontrib-napoleon"] dev = ["check-manifest"] test = ["tox (>=1.8.1)"] +[[package]] +name = "stevedore" +version = "5.4.0" +description = "Manage dynamic plugins for Python applications" +optional = false +python-versions = ">=3.9" +groups = ["main"] +files = [ + {file = "stevedore-5.4.0-py3-none-any.whl", hash = "sha256:b0be3c4748b3ea7b854b265dcb4caa891015e442416422be16f8b31756107857"}, + {file = "stevedore-5.4.0.tar.gz", hash = "sha256:79e92235ecb828fe952b6b8b0c6c87863248631922c8e8e0fa5b17b232c4514d"}, +] + +[package.dependencies] +pbr = ">=2.0.0" + [[package]] name = "tabulate" version = "0.9.0" @@ -5126,8 +5457,8 @@ version = "2.2.1" description = "A lil' TOML parser" optional = false python-versions = ">=3.8" -groups = ["test"] -markers = "python_full_version <= \"3.11.0a6\"" +groups = ["main", "test"] +markers = "python_version < \"3.11\"" files = [ {file = "tomli-2.2.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:678e4fa69e4575eb77d103de3df8a895e1591b48e740211bd1067378c69e8249"}, {file = "tomli-2.2.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:023aa114dd824ade0100497eb2318602af309e5a55595f76b626d6d9f3b7b0a6"}, @@ -5163,6 +5494,28 @@ files = [ {file = "tomli-2.2.1.tar.gz", hash = "sha256:cd45e1dc79c835ce60f7404ec8119f2eb06d38b1deba146f07ced3bbc44505ff"}, ] +[[package]] +name = "tqdm" +version = "4.67.1" +description = "Fast, Extensible Progress Meter" +optional = false +python-versions = ">=3.7" +groups = ["main"] +files = [ + {file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"}, + {file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"}, +] + +[package.dependencies] +colorama = {version = "*", markers = "platform_system == \"Windows\""} + +[package.extras] +dev = ["nbval", "pytest (>=6)", "pytest-asyncio (>=0.24)", "pytest-cov", "pytest-timeout"] +discord = ["requests"] +notebook = ["ipywidgets (>=6)"] +slack = ["slack-sdk"] +telegram = ["requests"] + [[package]] name = "types-markdown" version = "3.7.0.20241204" @@ -5283,6 +5636,18 @@ boto3 = ">=1.9.253,<2" [package.extras] tests = ["build", "coverage", "mypy", "pyyaml", "ruff", "wheel"] +[[package]] +name = "wcwidth" +version = "0.2.13" +description = "Measures the displayed width of unicode strings in a terminal" +optional = false +python-versions = "*" +groups = ["main"] +files = [ + {file = "wcwidth-0.2.13-py2.py3-none-any.whl", hash = "sha256:3da69048e4540d84af32131829ff948f1e022c1c6bdb8d6102117aac784f6859"}, + {file = "wcwidth-0.2.13.tar.gz", hash = "sha256:72ea0c06399eb286d978fdedb6923a9eb47e1c486ce63e9b4e64fc18303972b5"}, +] + [[package]] name = "werkzeug" version = "2.3.8" @@ -5479,6 +5844,18 @@ files = [ [package.dependencies] lxml = ">=3.8" +[[package]] +name = "xmod" +version = "1.8.1" +description = "🌱 Turn any object into a module 🌱" +optional = false +python-versions = ">=3.8" +groups = ["main"] +files = [ + {file = "xmod-1.8.1-py3-none-any.whl", hash = "sha256:a24e9458a4853489042522bdca9e50ee2eac5ab75c809a91150a8a7f40670d48"}, + {file = "xmod-1.8.1.tar.gz", hash = "sha256:38c76486b9d672c546d57d8035df0beb7f4a9b088bc3fb2de5431ae821444377"}, +] + [[package]] name = "yarl" version = "1.18.3" @@ -5599,4 +5976,4 @@ type = ["pytest-mypy"] [metadata] lock-version = "2.1" python-versions = ">=3.10,<3.12" -content-hash = "af05f0caba06a69fd96d9b569d996cd8388c46766036de51720477b5f558202e" +content-hash = "207c07aae4f035ab3d39f60d7669d3b1fe06837cbfaf922307fb30edd2817256" diff --git a/pyproject.toml b/pyproject.toml index 242c09aab..3edbeccec 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -40,6 +40,7 @@ bookops-worldcat = "^1.0.1" # Pinning wtforms because version 3.2.1 is breaking Flask AppBuilder # should remove when we upgrade Airflow wtforms = "3.0.1" +folio-data-import = "^0.2.7" [tool.poetry.group.test.dependencies] black = "~24.3.0" diff --git a/setup.cfg b/setup.cfg index 18eed4131..b50b8073c 100644 --- a/setup.cfg +++ b/setup.cfg @@ -4,9 +4,12 @@ addopts = --cov=libsys_airflow --cov-report=xml --cov-report=term [flake8] # Flake8 cannot be configured via pyproject.toml see https://pypi.org/project/Flake8-pyproject/ max-line-length = 88 +# D Various docstring related checks # E203 "Whitespace before ':'" conflicts with black # E225 "Missing whitespace around operator" # E501 "Line too long" +# I Various import related checks +# S101 "Use of assert detected" # W503 "Line break before binary operator" -extend-ignore = E203,E225,E501,W503 -exclude = .git,__pycache__,circ,logs,old,vendor-data,vendor_loads_migration,digital_bookplates_migration +extend-ignore = D,E203,E225,E501,I,S101,W503 +exclude = .git,__pycache__,circ,logs,old,vendor-data,vendor_loads_migration,digital_bookplates_migration,libsys_airflow/plugins/folio/encumbrances/fix_encumbrances*.py diff --git a/tests/apps/data_exports/test_data_exports_download_view.py b/tests/apps/data_exports/test_data_exports_download_view.py index 2b03f5329..db318ce61 100644 --- a/tests/apps/data_exports/test_data_exports_download_view.py +++ b/tests/apps/data_exports/test_data_exports_download_view.py @@ -16,7 +16,7 @@ def test_airflow_client(): app = application.create_app(testing=True) app.config['WTF_CSRF_ENABLED'] = False - setattr(DataExportDownloadView, "files_base", files_base) + setattr(DataExportDownloadView, "files_base", files_base) # noqa app.appbuilder.add_view( DataExportDownloadView, "DataExport", category="Data export" ) diff --git a/tests/apps/data_exports/test_exports_oclc_reports_view.py b/tests/apps/data_exports/test_exports_oclc_reports_view.py index 61dfa4346..07f9b2ff3 100644 --- a/tests/apps/data_exports/test_exports_oclc_reports_view.py +++ b/tests/apps/data_exports/test_exports_oclc_reports_view.py @@ -16,7 +16,7 @@ def test_airflow_client(): app = application.create_app(testing=True) app.config['WTF_CSRF_ENABLED'] = False - setattr(DataExportOCLCReportsView, "files_base", files_base) + setattr(DataExportOCLCReportsView, "files_base", files_base) # noqa app.appbuilder.add_view( DataExportOCLCReportsView, "DataExport", category="Data export" ) diff --git a/tests/apps/digital_bookplates/test_digital_bookplates_download_view.py b/tests/apps/digital_bookplates/test_digital_bookplates_download_view.py index 7dbd461b4..f799b7968 100644 --- a/tests/apps/digital_bookplates/test_digital_bookplates_download_view.py +++ b/tests/apps/digital_bookplates/test_digital_bookplates_download_view.py @@ -18,7 +18,7 @@ def test_airflow_client(): app = application.create_app(testing=True) app.config['WTF_CSRF_ENABLED'] = False - setattr(DigitalBookplatesDownloadView, "files_base", files_base) + setattr(DigitalBookplatesDownloadView, "files_base", files_base) # noqa app.appbuilder.add_view( DigitalBookplatesDownloadView, "DigitalBookplates", diff --git a/tests/apps/orafin/test_orafin_files_view.py b/tests/apps/orafin/test_orafin_files_view.py index a7ac65985..78290bfe4 100644 --- a/tests/apps/orafin/test_orafin_files_view.py +++ b/tests/apps/orafin/test_orafin_files_view.py @@ -19,7 +19,7 @@ def test_airflow_client(): app = application.create_app(testing=True) app.config['WTF_CSRF_ENABLED'] = False - setattr(OrafinFilesView, "files_base", files_base) + setattr(OrafinFilesView, "files_base", files_base) # noqa app.appbuilder.add_view(OrafinFilesView, "Orafin", category="Folio") app.blueprints['OrafinFilesView'].template_folder = templates_folder app.response_class = HTMLResponse diff --git a/tests/authority_control/test_authority_email.py b/tests/authority_control/test_authority_email.py new file mode 100644 index 000000000..b68d389bb --- /dev/null +++ b/tests/authority_control/test_authority_email.py @@ -0,0 +1,56 @@ +import pytest + + +from airflow.models import Variable +from pydantic import BaseModel + +from libsys_airflow.plugins.authority_control import email_report + + +@pytest.fixture +def mock_folio_variables(monkeypatch): + + def mock_get(key, *args): + value = None + match key: + case "OKAPI_URL": + value = "folio-test" + + case "EMAIL_DEVS": + value = "sul-unicorn-devs@lists.stanford.edu" + + case "FOLIO_URL": + value = "https://folio-test.edu" + + return value + + monkeypatch.setattr(Variable, "get", mock_get) + + +class MockDag(BaseModel): + dag_id: str = "load_marc_file" + + +class MockDagRun(BaseModel): + run_id: str = "marc-import-2025-02-12T00:00:00+00:00" + dag: MockDag = MockDag() + + +def test_email_report(mocker, mock_folio_variables): + + mock_send_email = mocker.MagicMock() + + mocker.patch.multiple( + "libsys_airflow.plugins.shared.utils", + send_email=mock_send_email, + is_production=lambda: False, + ) + + mocker.patch( + "libsys_airflow.plugins.data_exports.email.is_production", + return_value=False, + ) + + email_report(batch_report="Ran folio-data-import", dag_run=MockDagRun()) + + assert mock_send_email.call_count == 1 diff --git a/tests/authority_control/test_authority_helpers.py b/tests/authority_control/test_authority_helpers.py new file mode 100644 index 000000000..ff2040e8f --- /dev/null +++ b/tests/authority_control/test_authority_helpers.py @@ -0,0 +1,14 @@ +import pytest # noqa + + +from libsys_airflow.plugins.authority_control.helpers import clean_up + + +def test_clean_up(mocker, tmp_path): + authority_marc_file = tmp_path / "authority.mrc" + authority_marc_file.touch() + + clean_up(str(authority_marc_file), airflow=str(tmp_path)) + + assert authority_marc_file.exists() is False + assert (tmp_path / "authorities/archive/authority.mrc").exists() diff --git a/tests/authority_control/test_folio_data_import.py b/tests/authority_control/test_folio_data_import.py new file mode 100644 index 000000000..63588bea3 --- /dev/null +++ b/tests/authority_control/test_folio_data_import.py @@ -0,0 +1,32 @@ +import pytest + +from airflow.models import Variable +from libsys_airflow.plugins.authority_control.data_import import run_folio_data_import + + +@pytest.fixture +def mock_folio_variables(monkeypatch): + def mock_get(key, *args): + value = None + match key: + case "OKAPI_URL": + value = "folio-test" + + case "FOLIO_USER": + value = "libsys" + + case "FOLIO_PASSWORD": + value = "password" + return value + + monkeypatch.setattr(Variable, "get", mock_get) + + +def test_data_import(mock_folio_variables): + bash_operator = run_folio_data_import("test.mrc", "test_profile") + + assert bash_operator.task_id == "run_folio_data_import" + assert bash_operator.bash_command.startswith( + "python3 -m folio_data_import --record-type MARC21 --gateway_url $gateway_url" + ) + assert bash_operator.env["gateway_url"] == "folio-test" diff --git a/tests/data_exports/test_full_dump_selections.py b/tests/data_exports/test_full_dump_selections.py index d2e0669ad..ce1a87746 100644 --- a/tests/data_exports/test_full_dump_selections.py +++ b/tests/data_exports/test_full_dump_selections.py @@ -6,6 +6,15 @@ from libsys_airflow.plugins.data_exports.marc import exporter +class MockSQLExecuteQueryOperator(pydantic.BaseModel): + recreate: bool = False + from_date: str = "2023-09-01" + to_date: str = "2025-02-12" + + def execute(self, sql): + return None + + class MockCursor(pydantic.BaseModel): batch_size: int = 0 offset: int = 0 @@ -47,7 +56,7 @@ def mock_marc_records(): @pytest.fixture def mock_airflow_connection(): - return Connection( + return Connection( # noqa conn_id="postgres-folio", conn_type="postgres", host="example.com", @@ -152,10 +161,68 @@ def mock_result_set(): @pytest.fixture -def mock_get_current_context(mocker): - context = mocker.stub(name="context") - context.get = lambda arg: {} - return context +def mock_get_current_context_no_recreate(monkeypatch, mocker): + def _context(): + context = mocker.stub(name="context") + context.get = lambda *args: { + "recreate_view": False, + "from_date": "2023-09-01", + "to_date": "2025-02-01", + } + return context + + monkeypatch.setattr( + 'libsys_airflow.plugins.data_exports.full_dump_marc.get_current_context', + _context, + ) + + +@pytest.fixture +def mock_get_current_context_no_from_date(monkeypatch, mocker): + def _context(): + context = mocker.stub(name="context") + context.get = lambda *args: { + "recreate_view": True, + "to_date": "2025-02-01", + } + return context + + monkeypatch.setattr( + 'libsys_airflow.plugins.data_exports.full_dump_marc.get_current_context', + _context, + ) + + +@pytest.fixture +def mock_get_current_context_recreate(monkeypatch, mocker): + def _context(): + context = mocker.stub(name="context") + context.get = lambda *args: { + "recreate_view": True, + "from_date": "2023-09-01", + "to_date": "2025-02-01", + } + return context + + monkeypatch.setattr( + 'libsys_airflow.plugins.data_exports.full_dump_marc.get_current_context', + _context, + ) + + +def setup_recreate_tests(mocker, mock_airflow_connection): + mocker.patch( + 'libsys_airflow.plugins.data_exports.sql_pool.Connection.get_connection_from_secrets', + return_value=mock_airflow_connection, + ) + mocker.patch( + 'libsys_airflow.plugins.data_exports.full_dump_marc.SQLExecuteQueryOperator', + return_value=MockSQLExecuteQueryOperator(), + ) + mocker.patch( + 'libsys_airflow.plugins.data_exports.full_dump_marc.materialized_view_sql_file', + return_value='libsys_airflow/plugins/data_exports/sql/materialized_view.sql', + ) def test_fetch_full_dump(tmp_path, mocker, mock_airflow_connection, caplog): @@ -175,3 +242,41 @@ def test_fetch_full_dump(tmp_path, mocker, mock_airflow_connection, caplog): offset=3, batch_size=3, connection=MockConnection() ) assert "Saving 3 marc records to 3_6.mrc in bucket" in caplog.text + + +def test_no_recreate_materialized_view( + mocker, mock_get_current_context_no_recreate, mock_airflow_connection, caplog +): + setup_recreate_tests(mocker, mock_airflow_connection) + + query = full_dump_marc.create_materialized_view() + + if query is None: + assert True + + assert "Skipping refresh of materialized view" in caplog.text + + +def test_no_from_date_materialized_view( + mocker, mock_get_current_context_no_from_date, mock_airflow_connection, caplog +): + setup_recreate_tests(mocker, mock_airflow_connection) + + query = full_dump_marc.create_materialized_view() + + if query is None: + assert True + + assert query.startswith("DROP MATERIALIZED VIEW IF EXISTS data_export_marc") + assert "Skipping refresh of materialized view" not in caplog.text + + +def test_recreate_materialized_view( + mocker, mock_get_current_context_recreate, mock_airflow_connection, caplog +): + setup_recreate_tests(mocker, mock_airflow_connection) + + query = full_dump_marc.create_materialized_view() + + assert query.startswith("DROP MATERIALIZED VIEW IF EXISTS data_export_marc") + assert "Skipping refresh of materialized view" not in caplog.text diff --git a/tests/data_exports/test_marc_transformations.py b/tests/data_exports/test_marc_transformations.py index 7f7567ceb..909232ea9 100644 --- a/tests/data_exports/test_marc_transformations.py +++ b/tests/data_exports/test_marc_transformations.py @@ -439,8 +439,7 @@ def test_clean_and_serialize_marc_files(mock_marc_dir, caplog): assert f"Removed MARC fields and serialized records for updates files: {str(marc_file)}" -@pytest.mark.parametrize("mock_marc_dir", ["vendor"], indirect=True) -def test_marc_clean_serialize(mock_marc_dir): +def setup_marc_file_for_clean_serialize(mock_marc_dir): record = pymarc.Record() record.add_field( pymarc.Field( @@ -471,7 +470,15 @@ def test_marc_clean_serialize(mock_marc_dir): marc_writer = pymarc.MARCWriter(fo) marc_writer.write(record) - marc_clean_serialize(str(marc_file.absolute()), full_dump=False) + return marc_file + + +@pytest.mark.parametrize("mock_marc_dir", ["vendor"], indirect=True) +def test_marc_clean_serialize(mock_marc_dir): + + marc_file = setup_marc_file_for_clean_serialize(mock_marc_dir) + + marc_clean_serialize(str(marc_file.absolute()), full_dump=False, exclude_tags=True) with marc_file.open('rb') as fo: marc_reader = pymarc.MARCReader(fo) @@ -489,6 +496,27 @@ def test_marc_clean_serialize(mock_marc_dir): assert pathlib.Path(xml_file).stat().st_size > 0 +@pytest.mark.parametrize("mock_marc_dir", ["vendor"], indirect=True) +def test_marc_no_clean_serialize(mock_marc_dir): + + marc_file = setup_marc_file_for_clean_serialize(mock_marc_dir) + + marc_clean_serialize(str(marc_file.absolute()), full_dump=False, exclude_tags=False) + + with marc_file.open('rb') as fo: + marc_reader = pymarc.MARCReader(fo) + modified_marc_record = next(marc_reader) + + current_fields = [field.tag for field in modified_marc_record.fields] + + assert "598" in current_fields + assert "699" in current_fields + + xml_file = mock_marc_dir / "20240228.xml" + + assert pathlib.Path(xml_file).stat().st_size > 0 + + @pytest.mark.parametrize("mock_marc_dir", ["vendor"], indirect=True) def test_change_leader(mock_marc_dir): marc_file = mock_marc_dir / "20240509.mrc" diff --git a/tests/data_exports/test_oclc_api.py b/tests/data_exports/test_oclc_api.py index 50251ac95..412bc1330 100644 --- a/tests/data_exports/test_oclc_api.py +++ b/tests/data_exports/test_oclc_api.py @@ -345,7 +345,7 @@ def mock_holdings_unset(**kwargs): mock_session._url_manage_bibs_create = ( lambda: 'https://metadata.api.oclc.org/worldcat' ) - mock_session.authorization.token_str = "ab3565ackelas" + mock_session.authorization.token_str = "ab3565ackelas" # noqa: S105 mock_session.bib_create = mock_bib_create mock_session.bib_match = mock_bib_match mock_session.holdings_set = mock_holdings_set @@ -514,18 +514,21 @@ def mock_oclc_api(mocker): def test_oclc_api_class_init(mock_oclc_api): - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) assert oclc_api_instance.folio_client.okapi_url == "https://okapi.stanford.edu" - assert oclc_api_instance.oclc_token == "tk_6e302a204c2bfa4d266813cO647d62a77b10" + assert ( + oclc_api_instance.oclc_token + == "tk_6e302a204c2bfa4d266813cO647d62a77b10" # noqa + ) def test_oclc_api_class_no_new_records(mock_oclc_api, caplog): - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -544,7 +547,7 @@ def test_oclc_api_class_new_records(tmp_path, mock_oclc_api): for record in [marc_record, no_srs_record]: marc_writer.write(record) - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -576,7 +579,7 @@ def test_oclc_api_class_updated_records(tmp_path, mock_oclc_api): for record in [marc_record, no_srs_record]: marc_writer.write(record) - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -590,7 +593,7 @@ def test_oclc_api_class_updated_records(tmp_path, mock_oclc_api): def test_oclc_api_failed_authentication(mock_oclc_api): with pytest.raises(Exception, match="Unable to Retrieve Worldcat Access Token"): - oclc_api.OCLCAPIWrapper( + oclc_api.OCLCAPIWrapper( # noqa client_id="n0taVal1dC1i3nt", secret="c867b1dd75e6490f99d1cd1c9252ef22" ) @@ -628,10 +631,10 @@ def test_failed_oclc_new_record(tmp_path, mock_oclc_api): with marc_file.open('wb+') as fo: marc_writer = pymarc.MARCWriter(fo) - for record in [error_records[1], error_records[4], error_records[5], record]: - marc_writer.write(record) + for rec in [error_records[1], error_records[4], error_records[5], record]: + marc_writer.write(rec) - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -678,7 +681,7 @@ def test_failed_oclc_new_record(tmp_path, mock_oclc_api): def test_new_no_control_number(mock_oclc_api, tmp_path): - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -718,7 +721,7 @@ def test_new_no_control_number(mock_oclc_api, tmp_path): def test_bad_srs_put_in_new_context(tmp_path, mock_oclc_api): - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -742,7 +745,7 @@ def test_bad_srs_put_in_new_context(tmp_path, mock_oclc_api): def test_no_update_records(mock_oclc_api, caplog): - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -762,7 +765,7 @@ def test_already_exists_control_number(tmp_path, mock_oclc_api): ), ) - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -775,7 +778,7 @@ def test_already_exists_control_number(tmp_path, mock_oclc_api): def test_missing_srs_record_id(mock_oclc_api, caplog): - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -822,7 +825,7 @@ def test_oclc_update_errors(mock_oclc_api, caplog, tmp_path): for record in missing_or_multiple_oclc_records(): marc_writer.write(record) - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -868,7 +871,7 @@ def test_oclc_update_errors(mock_oclc_api, caplog, tmp_path): def test_failed_folio_put(mock_oclc_api, caplog): - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -888,7 +891,7 @@ def test_delete_missing_or_multiple_oclc_numbers(mock_oclc_api, tmp_path): for record in missing_or_multiple_oclc_records(): marc_writer.write(record) - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -921,7 +924,7 @@ def test_delete_result_success_and_errors(mock_oclc_api, tmp_path, caplog): for record in [error_records[0], error_records[2], good_record]: marc_writer.write(record) - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -977,7 +980,7 @@ def test_match_oclc_number(mock_oclc_api, tmp_path, caplog): for row in [existing_record, new_record, bad_oclc_num, bad_srs_record]: marc_writer.write(row) - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) @@ -1109,7 +1112,7 @@ def test_worldcat_error(mocker, mock_oclc_api, tmp_path): marc_writer = pymarc.MARCWriter(fo) marc_writer.write(bad_records()[0]) - oclc_api_instance = oclc_api.OCLCAPIWrapper( + oclc_api_instance = oclc_api.OCLCAPIWrapper( # noqa client_id="EDIoHuhLbdRvOHDjpEBtcEnBHneNtLUDiPRYtAqfTlpOThrxzUwHDUjMGEakoIJSObKpICwsmYZlmpYK", secret="c867b1dd75e6490f99d1cd1c9252ef22", ) diff --git a/tests/data_exports/test_save_instance_ids.py b/tests/data_exports/test_save_instance_ids.py index 1c92a4f0e..560d089c7 100644 --- a/tests/data_exports/test_save_instance_ids.py +++ b/tests/data_exports/test_save_instance_ids.py @@ -65,7 +65,7 @@ def test_save_ids_to_fs(tmp_path, mock_task_instance): assert "updates" in save_path[1] assert "deletes" in save_path[2] - for i, path in enumerate(save_path): + for _, path in enumerate(save_path): file = pathlib.Path(path) assert file.exists() with file.open('r') as fo: diff --git a/tests/data_exports/test_transmission_tasks.py b/tests/data_exports/test_transmission_tasks.py index 133fb97cb..5283bc674 100644 --- a/tests/data_exports/test_transmission_tasks.py +++ b/tests/data_exports/test_transmission_tasks.py @@ -105,7 +105,7 @@ def mock_httpx_connection(): @pytest.fixture def mock_ftphook_connection(): - return Connection( + return Connection( # noqa conn_id="ftp-example.com", conn_type="ftp", host="ftp://www.example.com", @@ -118,7 +118,7 @@ def mock_ftphook_connection(): @pytest.fixture def mock_oclc_connection(): - return Connection( + return Connection( # noqa conn_id="http.oclc-LIB", conn_type="http", host=None, diff --git a/tests/encumbrances/test_fix_encumbrances_run.py b/tests/encumbrances/test_fix_encumbrances_run.py index 5062af521..540fbae69 100644 --- a/tests/encumbrances/test_fix_encumbrances_run.py +++ b/tests/encumbrances/test_fix_encumbrances_run.py @@ -76,6 +76,42 @@ def test_fix_encumbrances_log_file_params( assert log_path.endswith("foo-scheduled__2024-07-29T19:00:00:00:00.log") +def test_fix_encumbrances_fiscal_code( + mocker, tmp_path, mock_task_instance, mock_folio_variables, monkeypatch, caplog +): + mocker.patch( + 'libsys_airflow.plugins.folio.encumbrances.fix_encumbrances.Variable.get', + return_value=mock_folio_variables, + ) + + async_mock = AsyncMock() + mocker.patch( + 'libsys_airflow.plugins.folio.encumbrances.fix_encumbrances.run_operation', + side_effect=async_mock, + return_value=None, + ) + + from libsys_airflow.plugins.folio.encumbrances.fix_encumbrances_run import ( + fix_encumbrances_run, + ) + + fix_encumbrances_run( + 1, + None, + "sul", + "username", + "password", + airflow=tmp_path, + task_instance=mock_task_instance, + library="foo", + ) + + assert ( + "fix_encumbrance_script exiting: Set Airflow Variable FISCAL_YEAR_CODE_" + in caplog.text + ) + + def test_fix_encumbrances_email_subject(): from libsys_airflow.plugins.shared.utils import _subject_with_server_name from libsys_airflow.plugins.folio.encumbrances.email import subject diff --git a/tests/orafin/test_ap_reports.py b/tests/orafin/test_ap_reports.py index c5bcad6eb..006275675 100644 --- a/tests/orafin/test_ap_reports.py +++ b/tests/orafin/test_ap_reports.py @@ -154,7 +154,7 @@ def test_extract_rows_large_file(tmp_path): existing_csv = orafin_reports / "xxdl_ap_payment_112820231000.csv" with existing_csv.open('w+') as fo: fo.write(f"{report[0]}\n") - for i in range(1200): + for _ in range(1200): fo.write(f"{report[1]}\n") assert existing_csv.exists() diff --git a/tests/test_circ_rules.py b/tests/test_circ_rules.py index b52aa8cbb..bb7742320 100644 --- a/tests/test_circ_rules.py +++ b/tests/test_circ_rules.py @@ -1,5 +1,5 @@ import pytest -import requests +import httpx from pytest_mock import MockerFixture @@ -125,7 +125,7 @@ def mock_get(*args, **kwargs): get_response.json = json_func return get_response - monkeypatch.setattr(requests, "get", mock_get) + monkeypatch.setattr(httpx, "get", mock_get) @pytest.fixture diff --git a/tests/test_connections.py b/tests/test_connections.py index bd8ed00f3..229d1c174 100644 --- a/tests/test_connections.py +++ b/tests/test_connections.py @@ -51,7 +51,7 @@ def test_connection_does_not_exist(db_session): def test_connection_already_exists(db_session): delete_connection("ftp-example.com-user", db_session) - prev_conn = Connection( + prev_conn = Connection( # noqa conn_id="ftp-example.com", conn_type="ftp", host="prev.example.com", diff --git a/tests/vendor/test_download.py b/tests/vendor/test_download.py index 047cd0b45..89ed0304b 100644 --- a/tests/vendor/test_download.py +++ b/tests/vendor/test_download.py @@ -1,4 +1,4 @@ -import ftplib +import ftplib # noqa import pytest # noqa from datetime import datetime from pytest_mock_resources import create_sqlite_fixture, Rows @@ -228,9 +228,9 @@ def test_sftp_download(sftp_hook, download_path, pg_hook): def test_download_error(ftp_hook, download_path, pg_hook): - ftp_hook.retrieve_file.side_effect = Exception("Error") + ftp_hook.retrieve_file.side_effect = Exception("Server Error") - with pytest.raises(Exception): + with pytest.raises(Exception, match="Server Error"): download( ftp_hook, "oclc", diff --git a/tests/vendor/test_extract.py b/tests/vendor/test_extract.py index bdcf960f0..422801910 100644 --- a/tests/vendor/test_extract.py +++ b/tests/vendor/test_extract.py @@ -42,10 +42,10 @@ def test_extract_regex(download_path): def test_extract_no_file(download_path): - with pytest.raises(Exception): + with pytest.raises(ValueError): extract(pathlib.Path(download_path) / "0720230118.tar.gz", r"^.*\.EDI$") def test_extract_multiple_files(download_path): - with pytest.raises(Exception): + with pytest.raises(ValueError): extract(pathlib.Path(download_path) / "download.tar.gz", None) diff --git a/tests/vendor/test_marc.py b/tests/vendor/test_marc.py index 2992b63ba..47d73d67d 100644 --- a/tests/vendor/test_marc.py +++ b/tests/vendor/test_marc.py @@ -67,7 +67,7 @@ def test_batch(tmp_path, marc_path): count = 0 with batch_path.open("rb") as fo: marc_reader = pymarc.MARCReader(fo) - for record in marc_reader: + for _ in marc_reader: count += 1 assert count == 10