diff --git a/.github/workflows/cd-docs.yaml b/.github/workflows/cd-docs.yaml index 3ebbb8fd1..721f0c692 100644 --- a/.github/workflows/cd-docs.yaml +++ b/.github/workflows/cd-docs.yaml @@ -1,6 +1,9 @@ name: CD-docs on: + pull_request: + branches: + - main push: branches: - main @@ -11,6 +14,7 @@ env: GKE_CLUSTER: ${{ secrets.GKE_APP_CLUSTER_NAME }} GKE_ZONE: ${{ secrets.GKE_CLUSTER_ZONE }} IMAGE_NAME: gcr.io/rj-smtr/dbt-docs + DBT_USER: ${{ github.actor }} jobs: build-container: diff --git a/pipelines/CHANGELOG.md b/pipelines/CHANGELOG.md index 9f3886942..3cfbc4a67 100644 --- a/pipelines/CHANGELOG.md +++ b/pipelines/CHANGELOG.md @@ -1,5 +1,11 @@ # Changelog - pipelines +## [1.0.1] - 2024-12-13 + +### Adicionado + +- Adiciona task `check_fail` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) + ## [1.0.0] - 2024-09-09 ### Alterado diff --git a/pipelines/capture/jae/CHANGELOG.md b/pipelines/capture/jae/CHANGELOG.md new file mode 100644 index 000000000..aa349444b --- /dev/null +++ b/pipelines/capture/jae/CHANGELOG.md @@ -0,0 +1,18 @@ +# Changelog - source_jae + +## [1.1.0] - 2024-12-19 + +### Adicionado + +- Cria flow `verificacao_ip` para alertar sobre falhas de conexão com o banco de dados da Jaé (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/366) + +## [1.0.1] - 2024-12-16 + +### Alterado +- Altera IP do banco de tracking da Jaé (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/362) + +## [1.0.0] - 2024-11-25 + +### Adicionado + +- Cria flow de captura da relação entre transação e ordem de pagamento \ No newline at end of file diff --git a/pipelines/capture/jae/__init__.py b/pipelines/capture/jae/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/pipelines/capture/jae/constants.py b/pipelines/capture/jae/constants.py new file mode 100644 index 000000000..291e09add --- /dev/null +++ b/pipelines/capture/jae/constants.py @@ -0,0 +1,89 @@ +# -*- coding: utf-8 -*- +""" +Valores constantes para captura de dados da Jaé +""" + +from datetime import datetime +from enum import Enum + +from pipelines.schedules import create_daily_cron +from pipelines.utils.gcp.bigquery import SourceTable + + +class constants(Enum): # pylint: disable=c0103 + """ + Valores constantes para captura de dados da Jaé + """ + + JAE_SOURCE_NAME = "jae" + + JAE_DATABASE_SETTINGS = { + "principal_db": { + "engine": "mysql", + "host": "10.5.114.227", + }, + "tarifa_db": { + "engine": "postgresql", + "host": "10.5.113.254", + }, + "transacao_db": { + "engine": "postgresql", + "host": "10.5.115.1", + }, + "tracking_db": { + "engine": "postgresql", + "host": "10.5.12.67", + }, + "ressarcimento_db": { + "engine": "postgresql", + "host": "10.5.12.50", + }, + "gratuidade_db": { + "engine": "postgresql", + "host": "10.5.12.107", + }, + "fiscalizacao_db": { + "engine": "postgresql", + "host": "10.5.115.29", + }, + } + + JAE_SECRET_PATH = "smtr_jae_access_data" + + TRANSACAO_ORDEM_TABLE_ID = "transacao_ordem" + + JAE_TABLE_CAPTURE_PARAMS = { + TRANSACAO_ORDEM_TABLE_ID: { + "query": """ + SELECT + id, + id_ordem_ressarcimento, + data_processamento, + data_transacao + FROM + transacao + WHERE + DATE(data_processamento) >= DATE('{start}') + AND DATE(data_processamento) <= DATE('{end}') + AND id_ordem_ressarcimento IS NOT NULL + """, + "database": "transacao_db", + } + } + + TRANSACAO_ORDEM_SOURCE = SourceTable( + source_name=JAE_SOURCE_NAME, + table_id=TRANSACAO_ORDEM_TABLE_ID, + first_timestamp=datetime(2024, 11, 21, 0, 0, 0), + schedule_cron=create_daily_cron(hour=6), + partition_date_only=True, + max_recaptures=5, + primary_keys=[ + "id", + "id_ordem_ressarcimento", + "data_processamento", + "data_transacao", + ], + ) + + ALERT_WEBHOOK = "alertas_bilhetagem" diff --git a/pipelines/capture/jae/flows.py b/pipelines/capture/jae/flows.py new file mode 100644 index 000000000..3ccee6f69 --- /dev/null +++ b/pipelines/capture/jae/flows.py @@ -0,0 +1,49 @@ +# -*- coding: utf-8 -*- +"""Flows de captura dos dados da Jaé""" +from prefect import case +from prefect.run_configs import KubernetesRun +from prefect.storage import GCS +from prefeitura_rio.pipelines_utils.custom import Flow +from prefeitura_rio.pipelines_utils.state_handlers import ( + handler_initialize_sentry, + handler_inject_bd_credentials, +) + +from pipelines.capture.jae.constants import constants +from pipelines.capture.jae.tasks import ( + create_database_error_discord_message, + create_jae_general_extractor, + test_jae_databases_connections, +) +from pipelines.capture.templates.flows import create_default_capture_flow +from pipelines.constants import constants as smtr_constants +from pipelines.schedules import every_hour +from pipelines.tasks import log_discord +from pipelines.utils.prefect import set_default_parameters + +CAPTURA_TRANSACAO_ORDEM = create_default_capture_flow( + flow_name="jae: transacao_ordem - captura", + source=constants.TRANSACAO_ORDEM_SOURCE.value, + create_extractor_task=create_jae_general_extractor, + agent_label=smtr_constants.RJ_SMTR_AGENT_LABEL.value, +) +set_default_parameters(CAPTURA_TRANSACAO_ORDEM, {"recapture": True}) + +with Flow("jae: verifica ip do banco de dados") as verificacao_ip: + success, failed_connections = test_jae_databases_connections() + with case(success, False): + message = create_database_error_discord_message(failed_connections=failed_connections) + send_discord_message = log_discord( + message=message, + key=constants.ALERT_WEBHOOK.value, + dados_tag=True, + ) + verificacao_ip.set_reference_tasks(tasks=[send_discord_message, success]) + +verificacao_ip.storage = GCS(smtr_constants.GCS_FLOWS_BUCKET.value) +verificacao_ip.run_config = KubernetesRun( + image=smtr_constants.DOCKER_IMAGE.value, + labels=[smtr_constants.RJ_SMTR_AGENT_LABEL.value], +) +verificacao_ip.state_handlers = [handler_inject_bd_credentials, handler_initialize_sentry] +verificacao_ip.schedule = every_hour diff --git a/pipelines/capture/jae/tasks.py b/pipelines/capture/jae/tasks.py new file mode 100644 index 000000000..a59164d3b --- /dev/null +++ b/pipelines/capture/jae/tasks.py @@ -0,0 +1,88 @@ +# -*- coding: utf-8 -*- +"""Tasks de captura dos dados da Jaé""" +from datetime import datetime, timedelta +from functools import partial + +from prefect import task +from pytz import timezone + +from pipelines.capture.jae.constants import constants +from pipelines.constants import constants as smtr_constants +from pipelines.utils.database import test_database_connection +from pipelines.utils.extractors.db import get_raw_db +from pipelines.utils.gcp.bigquery import SourceTable +from pipelines.utils.secret import get_secret + + +@task( + max_retries=smtr_constants.MAX_RETRIES.value, + retry_delay=timedelta(seconds=smtr_constants.RETRY_DELAY.value), +) +def create_jae_general_extractor(source: SourceTable, timestamp: datetime): + """Cria a extração de tabelas da Jaé""" + + credentials = get_secret(constants.JAE_SECRET_PATH.value) + params = constants.JAE_TABLE_CAPTURE_PARAMS.value[source.table_id] + + start = ( + source.get_last_scheduled_timestamp(timestamp=timestamp) + .astimezone(tz=timezone("UTC")) + .strftime("%Y-%m-%d %H:%M:%S") + ) + end = timestamp.astimezone(tz=timezone("UTC")).strftime("%Y-%m-%d %H:%M:%S") + + query = params["query"].format(start=start, end=end) + database_name = params["database"] + database = constants.JAE_DATABASE_SETTINGS.value[database_name] + + return partial( + get_raw_db, + query=query, + engine=database["engine"], + host=database["host"], + user=credentials["user"], + password=credentials["password"], + database=database_name, + ) + + +@task(nout=2) +def test_jae_databases_connections() -> tuple[bool, list[str]]: + """ + Testa a conexão com os bancos de dados da Jaé + + Returns: + bool: Se todas as conexões foram bem-sucedidas ou não + list[str]: Lista com os nomes dos bancos de dados com falha de conexão + """ + credentials = get_secret(constants.JAE_SECRET_PATH.value) + failed_connections = [] + for database_name, database in constants.JAE_DATABASE_SETTINGS.value.items(): + success, _ = test_database_connection( + engine=database["engine"], + host=database["host"], + user=credentials["user"], + password=credentials["password"], + database=database_name, + ) + if not success: + failed_connections.append(database_name) + + return len(failed_connections) == 0, failed_connections + + +@task +def create_database_error_discord_message(failed_connections: list[str]) -> str: + """ + Cria a mensagem para ser enviada no Discord caso haja + problemas de conexão com os bancos da Jaé + + Args: + failed_connections (list[str]): Lista com os nomes dos bancos de dados com falha de conexão + Returns: + str: Mensagem + """ + message = "Falha de conexão com o(s) banco(s) de dados:\n" + failed_connections = "\n".join(failed_connections) + message += failed_connections + return message + "\n" diff --git a/pipelines/capture/rioonibus/CHANGELOG.md b/pipelines/capture/rioonibus/CHANGELOG.md index f792fcac8..33341a5f1 100644 --- a/pipelines/capture/rioonibus/CHANGELOG.md +++ b/pipelines/capture/rioonibus/CHANGELOG.md @@ -1,5 +1,16 @@ # Changelog - source_rioonibus +## [1.2.0] - 2024-11-28 + +### Alterado +- Altera a captura da viagem_informada para pegar o range D-2 até D+0 (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/337) + +## [1.0.1] - 2024-11-25 + +### Alterado + +- Substitui variavel de expressão cron pela função `create_daily_cron` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/333) + ## [1.0.0] - 2024-10-21 ### Adicionado diff --git a/pipelines/capture/rioonibus/constants.py b/pipelines/capture/rioonibus/constants.py index de60699df..3e834491a 100644 --- a/pipelines/capture/rioonibus/constants.py +++ b/pipelines/capture/rioonibus/constants.py @@ -6,7 +6,7 @@ from datetime import datetime from enum import Enum -from pipelines.schedules import cron_every_day_hour_7 +from pipelines.schedules import create_daily_cron from pipelines.utils.gcp.bigquery import SourceTable @@ -23,7 +23,7 @@ class constants(Enum): # pylint: disable=c0103 source_name=RIO_ONIBUS_SOURCE_NAME, table_id="viagem_informada", first_timestamp=datetime(2024, 10, 16, 0, 0, 0), - schedule_cron=cron_every_day_hour_7, + schedule_cron=create_daily_cron(hour=7), partition_date_only=True, max_recaptures=5, primary_keys=["id_viagem"], diff --git a/pipelines/capture/rioonibus/tasks.py b/pipelines/capture/rioonibus/tasks.py index c32937e18..76b115dc9 100644 --- a/pipelines/capture/rioonibus/tasks.py +++ b/pipelines/capture/rioonibus/tasks.py @@ -3,11 +3,12 @@ from datetime import datetime, timedelta from functools import partial +import pandas as pd from prefect import task from pipelines.capture.rioonibus.constants import constants from pipelines.constants import constants as smtr_constants -from pipelines.utils.extractors.api import get_raw_api +from pipelines.utils.extractors.api import get_raw_api_params_list from pipelines.utils.gcp.bigquery import SourceTable from pipelines.utils.secret import get_secret @@ -16,20 +17,26 @@ max_retries=smtr_constants.MAX_RETRIES.value, retry_delay=timedelta(seconds=smtr_constants.RETRY_DELAY.value), ) -def create_viagem_informada_extractor(source: SourceTable, timestamp: datetime): +def create_viagem_informada_extractor( + source: SourceTable, # pylint: disable=W0613 + timestamp: datetime, +): """Cria a extração de viagens informadas na api da Rio Ônibus""" - extraction_day = timestamp.date() - timedelta(days=2) - params = { - "guidIdentificacao": get_secret(constants.RIO_ONIBUS_SECRET_PATH.value)[ - "guididentificacao" - ], - "datetime_processamento_inicio": extraction_day.isoformat() + "T00:00:00", - "datetime_processamento_fim": extraction_day.isoformat() + "T23:59:59", - } + end_date = timestamp.date() + start_date = end_date - timedelta(days=2) + api_key = get_secret(constants.RIO_ONIBUS_SECRET_PATH.value)["guididentificacao"] + params = [ + { + "guidIdentificacao": api_key, + "datetime_processamento_inicio": d.date().isoformat() + "T00:00:00", + "datetime_processamento_fim": d.date().isoformat() + "T23:59:59", + } + for d in pd.date_range(start_date, end_date) + ] + return partial( - get_raw_api, + get_raw_api_params_list, url=constants.VIAGEM_INFORMADA_BASE_URL.value, - params=params, - raw_filetype=source.raw_filetype, + params_list=params, ) diff --git a/pipelines/capture/sonda/CHANGELOG.md b/pipelines/capture/sonda/CHANGELOG.md new file mode 100644 index 000000000..4853bbc76 --- /dev/null +++ b/pipelines/capture/sonda/CHANGELOG.md @@ -0,0 +1,7 @@ +# Changelog - source_sonda + +## [1.0.0] - 2024-11-28 + +### Adicionado + +- Cria flow de captura de viagens informadas do BRT (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/337) \ No newline at end of file diff --git a/pipelines/capture/sonda/__init__.py b/pipelines/capture/sonda/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/pipelines/capture/sonda/constants.py b/pipelines/capture/sonda/constants.py new file mode 100644 index 000000000..9be022883 --- /dev/null +++ b/pipelines/capture/sonda/constants.py @@ -0,0 +1,31 @@ +# -*- coding: utf-8 -*- +""" +Valores constantes para captura de dados da SONDA +""" + +from datetime import datetime +from enum import Enum + +from pipelines.schedules import create_daily_cron +from pipelines.utils.gcp.bigquery import SourceTable + + +class constants(Enum): # pylint: disable=c0103 + """ + Valores constantes para captura de dados da SONDA + """ + + SONDA_SOURCE_NAME = "sonda" + SONDA_SECRET_PATH = "sonda_api" + VIAGEM_INFORMADA_LOGIN_URL = "http://consultaviagem.m2mfrota.com.br/AutenticarUsuario" + VIAGEM_INFORMADA_BASE_URL = "https://zn4.sinopticoplus.com/servico-dados/api/v1/obterDadosGTFS" + VIAGEM_INFORMADA_TABLE_ID = "viagem_informada" + VIAGEM_INFORMADA_SOURCE = SourceTable( + source_name=SONDA_SOURCE_NAME, + table_id=VIAGEM_INFORMADA_TABLE_ID, + first_timestamp=datetime(2024, 9, 10, 0, 0, 0), + schedule_cron=create_daily_cron(hour=7, minute=10), + partition_date_only=True, + max_recaptures=5, + primary_keys=["id_viagem"], + ) diff --git a/pipelines/capture/sonda/flows.py b/pipelines/capture/sonda/flows.py new file mode 100644 index 000000000..b50957627 --- /dev/null +++ b/pipelines/capture/sonda/flows.py @@ -0,0 +1,15 @@ +# -*- coding: utf-8 -*- +"""Flows de captura dos dados da SONDA""" +from pipelines.capture.sonda.constants import constants +from pipelines.capture.sonda.tasks import create_viagem_informada_extractor +from pipelines.capture.templates.flows import create_default_capture_flow +from pipelines.constants import constants as smtr_constants +from pipelines.utils.prefect import set_default_parameters + +CAPTURA_VIAGEM_INFORMADA_BRT = create_default_capture_flow( + flow_name="sonda: viagem_informada - captura", + source=constants.VIAGEM_INFORMADA_SOURCE.value, + create_extractor_task=create_viagem_informada_extractor, + agent_label=smtr_constants.RJ_SMTR_AGENT_LABEL.value, +) +set_default_parameters(CAPTURA_VIAGEM_INFORMADA_BRT, {"recapture": True}) diff --git a/pipelines/capture/sonda/tasks.py b/pipelines/capture/sonda/tasks.py new file mode 100644 index 000000000..287d3179b --- /dev/null +++ b/pipelines/capture/sonda/tasks.py @@ -0,0 +1,54 @@ +# -*- coding: utf-8 -*- +"""Tasks de captura dos dados da SONDA""" +from datetime import datetime, timedelta +from functools import partial + +import pandas as pd +import requests +from prefect import task + +from pipelines.capture.sonda.constants import constants +from pipelines.constants import constants as smtr_constants +from pipelines.utils.extractors.api import get_raw_api_params_list +from pipelines.utils.gcp.bigquery import SourceTable +from pipelines.utils.secret import get_secret + + +@task( + max_retries=smtr_constants.MAX_RETRIES.value, + retry_delay=timedelta(seconds=smtr_constants.RETRY_DELAY.value), +) +def create_viagem_informada_extractor( + source: SourceTable, # pylint: disable=W0613 + timestamp: datetime, +): + """Cria a extração de viagens informadas na api da SONDA""" + + end_date = timestamp.date() + start_date = end_date - timedelta(days=2) + + loging_response = requests.post( + constants.VIAGEM_INFORMADA_LOGIN_URL.value, + data=get_secret(constants.SONDA_SECRET_PATH.value), + timeout=120, + ) + + loging_response.raise_for_status() + + key = loging_response.json()["IdentificacaoLogin"] + + params = [ + { + "datetime_processamento": d.strftime("%d/%m/%Y 00:00:00"), + } + for d in pd.date_range(start_date, end_date) + ] + + headers = {"Authorization": key} + + return partial( + get_raw_api_params_list, + url=constants.VIAGEM_INFORMADA_BASE_URL.value, + params_list=params, + headers=headers, + ) diff --git a/pipelines/capture/templates/CHANGELOG.md b/pipelines/capture/templates/CHANGELOG.md index 88ae0579c..f7a964c32 100644 --- a/pipelines/capture/templates/CHANGELOG.md +++ b/pipelines/capture/templates/CHANGELOG.md @@ -1,5 +1,11 @@ # Changelog - capture +## [1.0.1] - 2024-11-25 + +### Alterado + +- Altera função de pre-tratamento caso o número de primary_keys for igual ao de colunas (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/333) + ## [1.0.0] - 2024-10-21 ### Adicionado diff --git a/pipelines/capture/templates/tasks.py b/pipelines/capture/templates/tasks.py index aa11d8e2c..e6da0014f 100644 --- a/pipelines/capture/templates/tasks.py +++ b/pipelines/capture/templates/tasks.py @@ -225,7 +225,7 @@ def upload_source_data_to_gcs(source: SourceTable, partition: str, filepaths: di if not source.exists(): log("Staging Table does not exist, creating table...") source.append(source_filepath=filepaths["source"], partition=partition) - source.create() + source.create(sample_filepath=filepaths["source"]) else: log("Staging Table already exists, appending to it...") source.append(source_filepath=filepaths["source"], partition=partition) @@ -275,7 +275,8 @@ def transform_raw_to_nested_structure( for step in pretreat_funcs: data = step(data=data, timestamp=timestamp, primary_keys=primary_keys) - data = transform_to_nested_structure(data=data, primary_keys=primary_keys) + if len(primary_keys) < len(data.columns): + data = transform_to_nested_structure(data=data, primary_keys=primary_keys) timestamp = create_timestamp_captura(timestamp=timestamp) data["timestamp_captura"] = timestamp diff --git a/pipelines/constants.py b/pipelines/constants.py index 263de48a7..d61a8a17c 100644 --- a/pipelines/constants.py +++ b/pipelines/constants.py @@ -2,7 +2,7 @@ """ Valores constantes gerais para pipelines da rj-smtr -DBT 2024-09-11 +DBT 2024-11-11 """ from enum import Enum @@ -160,7 +160,7 @@ class constants(Enum): # pylint: disable=c0103 "databases": { "principal_db": { "engine": "mysql", - "host": "10.5.114.121", + "host": "10.5.114.227", }, "tarifa_db": { "engine": "postgresql", @@ -172,7 +172,7 @@ class constants(Enum): # pylint: disable=c0103 }, "tracking_db": { "engine": "postgresql", - "host": "10.5.15.25", + "host": "10.5.12.67", }, "ressarcimento_db": { "engine": "postgresql", diff --git a/pipelines/flows.py b/pipelines/flows.py index 109e40f80..7f9448767 100644 --- a/pipelines/flows.py +++ b/pipelines/flows.py @@ -2,7 +2,9 @@ """ Imports all flows for every project so we can register all of them. """ +from pipelines.capture.jae.flows import * # noqa from pipelines.capture.rioonibus.flows import * # noqa +from pipelines.capture.sonda.flows import * # noqa from pipelines.exemplo import * # noqa from pipelines.janitor.flows import * # noqa from pipelines.migration.br_rj_riodejaneiro_bilhetagem.flows import * # noqa @@ -20,4 +22,7 @@ from pipelines.migration.projeto_subsidio_sppo.flows import * # noqa from pipelines.migration.veiculo.flows import * # noqa from pipelines.serpro.flows import * # noqa +from pipelines.treatment.bilhetagem.flows import * # noqa +from pipelines.treatment.datario.flows import * # noqa from pipelines.treatment.monitoramento.flows import * # noqa +from pipelines.treatment.planejamento.flows import * # noqa diff --git a/pipelines/janitor/tasks.py b/pipelines/janitor/tasks.py index 5a9390826..93dc24e58 100644 --- a/pipelines/janitor/tasks.py +++ b/pipelines/janitor/tasks.py @@ -14,12 +14,11 @@ @task -def query_active_flow_names(prefix="%SMTR%", prefect_client=None, prefect_project="production"): +def query_active_flow_names(prefect_client=None, prefect_project="production"): query = """ -query ($prefix: String, $offset: Int, $project_name: String){ +query ($offset: Int, $project_name: String){ flow( where: { - name: {_like: $prefix}, archived: {_eq: false}, project: {name:{_eq: $project_name}} } @@ -32,7 +31,7 @@ def query_active_flow_names(prefix="%SMTR%", prefect_client=None, prefect_projec """ if not prefect_client: prefect_client = Client() - variables = {"prefix": prefix, "offset": 0, "project_name": prefect_project} + variables = {"offset": 0, "project_name": prefect_project} # flow_names = [] response = prefect_client.graphql(query=query, variables=variables)["data"] active_flows = [] @@ -137,8 +136,8 @@ def get_prefect_client(): @task -def get_active_flow_names(prefix="%SMTR%"): - flow_names = query_active_flow_names(prefix=prefix) +def get_active_flow_names(): + flow_names = query_active_flow_names() log(f"Got flow_names\n{flow_names[:10]}\n...\n{flow_names[-10:-1]}") return flow_names diff --git a/pipelines/migration/CHANGELOG.md b/pipelines/migration/CHANGELOG.md index 1910d2d0d..063e2aea4 100644 --- a/pipelines/migration/CHANGELOG.md +++ b/pipelines/migration/CHANGELOG.md @@ -1,5 +1,12 @@ # Changelog - migration +## [1.0.3] - 2024-12-13 + +### Adicionado + +- Adiciona a task `transform_raw_to_nested_structure_chunked` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) +- Adiciona parâmetros `log_param` e `args` na função `save_treated_local_func` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) + ## [1.0.2] - 2024-09-11 ### Adicionado diff --git a/pipelines/migration/br_rj_riodejaneiro_bilhetagem/CHANGELOG.md b/pipelines/migration/br_rj_riodejaneiro_bilhetagem/CHANGELOG.md index 384fddf61..eaa4262b9 100644 --- a/pipelines/migration/br_rj_riodejaneiro_bilhetagem/CHANGELOG.md +++ b/pipelines/migration/br_rj_riodejaneiro_bilhetagem/CHANGELOG.md @@ -1,5 +1,10 @@ # Changelog - br_rj_riodejaneiro_bilhetagem +## [1.4.8] - 2024-12-16 + +### Alterado +- Altera IP do banco de tracking da Jaé (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/362) + ## [1.4.7] - 2024-09-16 ### Alterado diff --git a/pipelines/migration/br_rj_riodejaneiro_bilhetagem/constants.py b/pipelines/migration/br_rj_riodejaneiro_bilhetagem/constants.py index 7aae158ea..b925ba713 100644 --- a/pipelines/migration/br_rj_riodejaneiro_bilhetagem/constants.py +++ b/pipelines/migration/br_rj_riodejaneiro_bilhetagem/constants.py @@ -538,7 +538,8 @@ class constants(Enum): # pylint: disable=c0103 ordem_pagamento_dia ordem_pagamento_consorcio_dia ordem_pagamento_consorcio_operador_dia \ staging_ordem_pagamento_consorcio staging_ordem_pagamento \ ordem_pagamento_servico_operador_dia staging_ordem_pagamento_consorcio_operadora \ -aux_retorno_ordem_pagamento staging_arquivo_retorno", +aux_retorno_ordem_pagamento staging_arquivo_retorno aux_transacao_id_ordem_pagamento \ +staging_transacao_ordem", } BILHETAGEM_MATERIALIZACAO_PASSAGEIROS_HORA_PARAMS = { diff --git a/pipelines/migration/br_rj_riodejaneiro_bilhetagem/flows.py b/pipelines/migration/br_rj_riodejaneiro_bilhetagem/flows.py index 9b41064fc..4c60accc4 100644 --- a/pipelines/migration/br_rj_riodejaneiro_bilhetagem/flows.py +++ b/pipelines/migration/br_rj_riodejaneiro_bilhetagem/flows.py @@ -2,7 +2,7 @@ """ Flows for br_rj_riodejaneiro_bilhetagem -DBT: 2024-10-09 +DBT: 2024-12-12 """ from copy import deepcopy diff --git a/pipelines/migration/br_rj_riodejaneiro_gtfs/CHANGELOG.md b/pipelines/migration/br_rj_riodejaneiro_gtfs/CHANGELOG.md index 5aa706342..2e423bb3f 100644 --- a/pipelines/migration/br_rj_riodejaneiro_gtfs/CHANGELOG.md +++ b/pipelines/migration/br_rj_riodejaneiro_gtfs/CHANGELOG.md @@ -1,5 +1,47 @@ # Changelog - gtfs + +## [1.1.7] - 2024-12-13 + +### Adicionado + +- Cria arquivo `constants.py` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) +- Adiciona automação dos testes do DBT no arquivo `flows.py` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) +- Adiciona compatibilidade com padrão "KM" na função `processa_ordem_servico` no arquivo `utils.py` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) + +### Alterado + +- Remove parâmetros personalizados de execução no arquivo `flows.py` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) +- Troca task `transform_raw_to_nested_structure` pela `transform_raw_to_nested_structure_chunked` no arquivo `flows.py` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) + +### Corrigido + +- Corrige parâmetro `supportsAllDrives` na função `download_xlsx` arquivo `utils.py` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) + +## [1.1.6] - 2024-12-04 + +- Adiciona o modelo `viagem_planejada_planejamento` no exclude da materialização (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/352) + +## [1.1.5] - 2024-12-03 + +### Corrigido + +- Corrige a conversão de valores para float na OS por faixa horaria (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/350) + +- Corrige a captura dos arquivos do GTFS no drive compartilhado (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/350) + +## [1.1.4] - 2024-11-07 + +### Alterado + +- Refatora função `processa_ordem_servico_faixa_horaria` no arquivo `utils.py` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/306) + +## [1.1.3] - 2024-10-30 + +### Alterado + +- Alterado arquivo `utils.py` em razão das novas faixas horárias (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/295) + ## [1.1.2] - 2024-10-21 ### Alterado diff --git a/pipelines/migration/br_rj_riodejaneiro_gtfs/constants.py b/pipelines/migration/br_rj_riodejaneiro_gtfs/constants.py new file mode 100644 index 000000000..ec139f454 --- /dev/null +++ b/pipelines/migration/br_rj_riodejaneiro_gtfs/constants.py @@ -0,0 +1,36 @@ +# -*- coding: utf-8 -*- +""" +Constant values for rj_smtr br_rj_riodejaneiro_gtfs +""" + +from enum import Enum + + +class constants(Enum): # pylint: disable=c0103 + """ + Constant values for rj_smtr br_rj_riodejaneiro_gtfs + """ + + GTFS_DATA_CHECKS_LIST = { + "calendar_gtfs": { + "dbt_expectations.expect_column_values_to_match_regex__service_id__calendar_gtfs": { + "description": "Todos os 'service\\_id' começam com 'U\\_', 'S\\_', 'D\\_' ou 'EXCEP'." # noqa + }, + }, + "ordem_servico_trajeto_alternativo_gtfs": { + "dbt_expectations.expect_table_aggregation_to_equal_other_table__ordem_servico_trajeto_alternativo_gtfs": { # noqa + "description": "Todos os dados de 'feed_start_date' e 'tipo_os' correspondem 1:1 entre as tabelas 'ordem_servico_trajeto_alternativo_gtfs' e 'ordem_servico_gtfs'." # noqa + }, + }, + "ordem_servico_trips_shapes_gtfs": { + "dbt_expectations.expect_table_aggregation_to_equal_other_table__ordem_servico_trips_shapes_gtfs": { # noqa + "description": "Todos os dados de 'feed_start_date', 'tipo_os', 'tipo_dia', 'servico' e 'faixa_horaria_inicio' correspondem 1:1 entre as tabelas 'ordem_servico_trips_shapes_gtfs' e 'ordem_servico_faixa_horaria'." # noqa + }, + "dbt_utils.unique_combination_of_columns__ordem_servico_trips_shapes_gtfs": { + "description": "Todos os dados de 'feed_start_date', 'tipo_dia', 'tipo_os', 'servico', 'sentido', 'faixa_horaria_inicio' e 'shape_id' são únicos." # noqa + }, + "dbt_expectations.expect_table_row_count_to_be_between__ordem_servico_trips_shapes_gtfs": { # noqa + "description": "A quantidade de registros de 'feed_start_date', 'tipo_dia', 'tipo_os', 'servico', 'faixa_horaria_inicio' e 'shape_id' está dentro do intervalo esperado." # noqa + }, + }, + } diff --git a/pipelines/migration/br_rj_riodejaneiro_gtfs/flows.py b/pipelines/migration/br_rj_riodejaneiro_gtfs/flows.py index 3767a1ed0..21836870d 100644 --- a/pipelines/migration/br_rj_riodejaneiro_gtfs/flows.py +++ b/pipelines/migration/br_rj_riodejaneiro_gtfs/flows.py @@ -2,7 +2,7 @@ """ Flows for gtfs -DBT 2024-09-24 +DBT 2024-12-04a """ from prefect import Parameter, case, task @@ -20,6 +20,9 @@ ) from pipelines.constants import constants +from pipelines.migration.br_rj_riodejaneiro_gtfs.constants import ( + constants as gtfs_constants, +) # SMTR Imports # from pipelines.migration.br_rj_riodejaneiro_gtfs.tasks import ( @@ -37,13 +40,21 @@ get_join_dict, rename_current_flow_run_now_time, run_dbt_model, - transform_raw_to_nested_structure, + transform_raw_to_nested_structure_chunked, unpack_mapped_results_nout2, upload_raw_data_to_gcs, upload_staging_data_to_gcs, ) -from pipelines.schedules import every_5_minutes -from pipelines.tasks import get_scheduled_timestamp, parse_timestamp_to_string + +# from pipelines.schedules import every_5_minutes +from pipelines.tasks import ( + check_fail, + get_scheduled_timestamp, + log_discord, + parse_timestamp_to_string, + task_value_is_none, +) +from pipelines.treatment.templates.tasks import dbt_data_quality_checks, run_dbt_tests # from pipelines.capture.templates.flows import create_default_capture_flow @@ -128,6 +139,12 @@ data_versao_gtfs_task = get_current_timestamp(data_versao_gtfs_task) + data_versao_gtfs_str = parse_timestamp_to_string( + timestamp=data_versao_gtfs_task, pattern="%Y-%m-%d" + ) + + log_discord("Captura do GTFS " + data_versao_gtfs_str + " iniciada", "gtfs") + partition = create_date_hour_partition( timestamp=data_versao_gtfs_task, partition_date_name="data_versao", @@ -150,14 +167,18 @@ local_filepath=local_filepaths, regular_sheet_index=regular_sheet_index, upload_from_gcs=upload_from_gcs, + data_versao_gtfs=data_versao_gtfs_str, ) - transform_raw_to_nested_structure_results = transform_raw_to_nested_structure.map( - raw_filepath=raw_filepaths, - filepath=local_filepaths, - primary_key=primary_keys, - timestamp=unmapped(data_versao_gtfs_task), - error=unmapped(None), + transform_raw_to_nested_structure_results = ( + transform_raw_to_nested_structure_chunked.map( + raw_filepath=raw_filepaths, + filepath=local_filepaths, + primary_key=primary_keys, + timestamp=unmapped(data_versao_gtfs_task), + error=unmapped(None), + chunksize=unmapped(50000), + ) ) errors, treated_filepaths = unpack_mapped_results_nout2( @@ -180,6 +201,14 @@ timestamp=unmapped(data_versao_gtfs_task), error=errors, ) + + upload_failed = check_fail(wait_captura_true) + + with case(upload_failed, True): + log_discord( + "Falha na subida dos dados do GTFS " + data_versao_gtfs_str, "gtfs", True + ) + with case(materialize_only, True): wait_captura_false = task() @@ -209,14 +238,39 @@ + " " + constants.PLANEJAMENTO_MATERIALIZACAO_DATASET_ID.value, _vars=dbt_vars, + exclude="calendario aux_calendario_manual viagem_planejada_planejamento", ).set_upstream(task=wait_captura) + run_dbt_failed = task_value_is_none(wait_run_dbt_model) + + with case(run_dbt_failed, False): + log_discord( + "Falha na materialização dos dados do GTFS " + data_versao_gtfs, "gtfs", True + ) + + with case(run_dbt_failed, True): + log_discord( + "Captura e materialização do GTFS " + data_versao_gtfs + " finalizada com sucesso!", + "gtfs", + ) + wait_materialize_true = update_last_captured_os( dataset_id=constants.GTFS_DATASET_ID.value, data_index=data_index, mode=mode, ).set_upstream(task=wait_run_dbt_model) + gtfs_data_quality = run_dbt_tests( + dataset_id=constants.GTFS_MATERIALIZACAO_DATASET_ID.value + + " " + + constants.PLANEJAMENTO_MATERIALIZACAO_DATASET_ID.value, + _vars=dbt_vars, + ).set_upstream(task=wait_run_dbt_model) + + gtfs_data_quality_results = dbt_data_quality_checks( + gtfs_data_quality, gtfs_constants.GTFS_DATA_CHECKS_LIST.value, dbt_vars + ) + with case(verifica_materialize, False): wait_materialize_false = task() @@ -232,17 +286,13 @@ gtfs_captura_nova.run_config = KubernetesRun( image=constants.DOCKER_IMAGE.value, labels=[constants.RJ_SMTR_AGENT_LABEL.value], - cpu_limit="1000m", - memory_limit="4600Mi", - cpu_request="500m", - memory_request="1000Mi", ) gtfs_captura_nova.state_handlers = [ handler_inject_bd_credentials, handler_initialize_sentry, handler_skip_if_running, ] -gtfs_captura_nova.schedule = every_5_minutes +# gtfs_captura_nova.schedule = every_5_minutes # with Flow( diff --git a/pipelines/migration/br_rj_riodejaneiro_gtfs/tasks.py b/pipelines/migration/br_rj_riodejaneiro_gtfs/tasks.py index 01fbdbdad..641866094 100644 --- a/pipelines/migration/br_rj_riodejaneiro_gtfs/tasks.py +++ b/pipelines/migration/br_rj_riodejaneiro_gtfs/tasks.py @@ -155,7 +155,11 @@ def get_os_info(last_captured_os: str = None, data_versao_gtfs: str = None) -> d @task(nout=2) def get_raw_gtfs_files( - os_control, local_filepath: list, regular_sheet_index: int = None, upload_from_gcs: bool = False + os_control, + local_filepath: list, + regular_sheet_index: int = None, + upload_from_gcs: bool = False, + data_versao_gtfs: str = None, ): """ Downloads raw files and processes them. @@ -241,6 +245,7 @@ def get_raw_gtfs_files( file_bytes=file_bytes_os, local_filepath=local_filepath, raw_filepaths=raw_filepaths, + data_versao_gtfs=data_versao_gtfs, ) else: diff --git a/pipelines/migration/br_rj_riodejaneiro_gtfs/utils.py b/pipelines/migration/br_rj_riodejaneiro_gtfs/utils.py index cb16dcda7..eef9515d6 100644 --- a/pipelines/migration/br_rj_riodejaneiro_gtfs/utils.py +++ b/pipelines/migration/br_rj_riodejaneiro_gtfs/utils.py @@ -9,6 +9,7 @@ import requests from googleapiclient.http import MediaIoBaseDownload from prefeitura_rio.pipelines_utils.logging import log +from unidecode import unidecode from pipelines.constants import constants from pipelines.migration.utils import save_raw_local_func @@ -130,7 +131,9 @@ def download_xlsx(file_link, drive_service): """ file_id = file_link.split("/")[-2] - file = drive_service.files().get(fileId=file_id).execute() # pylint: disable=E1101 + file = ( + drive_service.files().get(fileId=file_id, supportsAllDrives=True).execute() + ) # pylint: disable=E1101 mime_type = file.get("mimeType") if "google-apps" in mime_type: @@ -228,18 +231,22 @@ def processa_ordem_servico( "Partidas Volta Dia Útil": "partidas_volta_du", "Viagens Dia Útil": "viagens_du", "Quilometragem Dia Útil": "km_dia_util", + "KM Dia Útil": "km_dia_util", "Partidas Ida Sábado": "partidas_ida_sabado", "Partidas Volta Sábado": "partidas_volta_sabado", "Viagens Sábado": "viagens_sabado", "Quilometragem Sábado": "km_sabado", + "KM Sábado": "km_sabado", "Partidas Ida Domingo": "partidas_ida_domingo", "Partidas Volta Domingo": "partidas_volta_domingo", "Viagens Domingo": "viagens_domingo", "Quilometragem Domingo": "km_domingo", + "KM Domingo": "km_domingo", "Partidas Ida Ponto Facultativo": "partidas_ida_pf", "Partidas Volta Ponto Facultativo": "partidas_volta_pf", "Viagens Ponto Facultativo": "viagens_pf", "Quilometragem Ponto Facultativo": "km_pf", + "KM Ponto Facultativo": "km_pf", "tipo_os": "tipo_os", } @@ -418,7 +425,7 @@ def download_file(file_link, drive_service): """ file_id = file_link.split("/")[-2] - request = drive_service.files().get_media(fileId=file_id) # pylint: disable=E1101 + request = drive_service.files().get_media(fileId=file_id, supportsAllDrives=True) file_bytes = io.BytesIO() downloader = MediaIoBaseDownload(file_bytes, request) done = False @@ -427,7 +434,9 @@ def download_file(file_link, drive_service): return file_bytes -def processa_ordem_servico_faixa_horaria(sheetnames, file_bytes, local_filepath, raw_filepaths): +def processa_ordem_servico_faixa_horaria( + sheetnames, file_bytes, local_filepath, raw_filepaths, data_versao_gtfs +): """ Process 'Faixa Horária' from an Excel file. @@ -451,95 +460,59 @@ def processa_ordem_servico_faixa_horaria(sheetnames, file_bytes, local_filepath, file_bytes, sheet_name=sheetnames[sheet], dtype=object ) + metricas = ["Partidas", "Quilometragem", "KM"] + dias = ["Dias Úteis", "Sábado", "Domingo", "Ponto Facultativo"] + formatos = ["{metrica} entre {intervalo} — {dia}", "{metrica} entre {intervalo} ({dia})"] + + if data_versao_gtfs >= "2024-11-06": + intervalos = [ + "00h e 03h", + "03h e 06h", + "06h e 09h", + "09h e 12h", + "12h e 15h", + "15h e 18h", + "18h e 21h", + "21h e 24h", + "24h e 03h (dia seguinte)", + ] + else: + intervalos = [ + "00h e 03h", + "03h e 12h", + "12h e 21h", + "21h e 24h", + "24h e 03h (dia seguinte)", + ] + fh_columns = { - "Serviço": "servico", - "Consórcio": "consorcio", - "Partidas entre 00h e 03h — Dias Úteis": "partidas_entre_00h_e_03h_dias_uteis", - "Quilometragem entre 00h e 03h — Dias Úteis": "quilometragem_entre_00h_e_03h_dias_uteis", - "Partidas entre 03h e 12h — Dias Úteis": "partidas_entre_03h_e_12h_dias_uteis", - "Quilometragem entre 03h e 12h — Dias Úteis": "quilometragem_entre_03h_e_12h_dias_uteis", - "Partidas entre 12h e 21h — Dias Úteis": "partidas_entre_12h_e_21h_dias_uteis", - "Quilometragem entre 12h e 21h — Dias Úteis": "quilometragem_entre_12h_e_21h_dias_uteis", - "Partidas entre 21h e 24h — Dias Úteis": "partidas_entre_21h_e_24h_dias_uteis", - "Quilometragem entre 21h e 24h — Dias Úteis": "quilometragem_entre_21h_e_24h_dias_uteis", - "Partidas entre 24h e 03h (dia seguinte) — Dias Úteis": "partidas_entre_24h_e_03h_diaseguinte_dias_uteis", # noqa - "Quilometragem entre 24h e 03h (dia seguinte) — Dias Úteis": "quilometragem_entre_24h_e_03h_diaseguinte_dias_uteis", # noqa - "Partidas entre 00h e 03h — Sábado": "partidas_entre_00h_e_03h_sabado", - "Quilometragem entre 00h e 03h — Sábado": "quilometragem_entre_00h_e_03h_sabado", - "Partidas entre 03h e 12h — Sábado": "partidas_entre_03h_e_12h_sabado", - "Quilometragem entre 03h e 12h — Sábado": "quilometragem_entre_03h_e_12h_sabado", - "Partidas entre 12h e 21h — Sábado": "partidas_entre_12h_e_21h_sabado", - "Quilometragem entre 12h e 21h — Sábado": "quilometragem_entre_12h_e_21h_sabado", - "Partidas entre 21h e 24h — Sábado": "partidas_entre_21h_e_24h_sabado", - "Quilometragem entre 21h e 24h — Sábado": "quilometragem_entre_21h_e_24h_sabado", - "Partidas entre 24h e 03h (dia seguinte) — Sábado": "partidas_entre_24h_e_03h_diaseguinte_sabado", # noqa - "Quilometragem entre 24h e 03h (dia seguinte) — Sábado": "quilometragem_entre_24h_e_03h_diaseguinte_sabado", # noqa - "Partidas entre 00h e 03h — Domingo": "partidas_entre_00h_e_03h_domingo", - "Quilometragem entre 00h e 03h — Domingo": "quilometragem_entre_00h_e_03h_domingo", - "Partidas entre 03h e 12h — Domingo": "partidas_entre_03h_e_12h_domingo", - "Quilometragem entre 03h e 12h — Domingo": "quilometragem_entre_03h_e_12h_domingo", - "Partidas entre 12h e 21h — Domingo": "partidas_entre_12h_e_21h_domingo", - "Quilometragem entre 12h e 21h — Domingo": "quilometragem_entre_12h_e_21h_domingo", - "Partidas entre 21h e 24h — Domingo": "partidas_entre_21h_e_24h_domingo", - "Quilometragem entre 21h e 24h — Domingo": "quilometragem_entre_21h_e_24h_domingo", - "Partidas entre 24h e 03h (dia seguinte) — Domingo": "partidas_entre_24h_e_03h_diaseguinte_domingo", # noqa - "Quilometragem entre 24h e 03h (dia seguinte) — Domingo": "quilometragem_entre_24h_e_03h_diaseguinte_domingo", # noqa - "Partidas entre 00h e 03h — Ponto Facultativo": "partidas_entre_00h_e_03h_ponto_facultativo", # noqa - "Quilometragem entre 00h e 03h — Ponto Facultativo": "quilometragem_entre_00h_e_03h_ponto_facultativo", # noqa - "Partidas entre 03h e 12h — Ponto Facultativo": "partidas_entre_03h_e_12h_ponto_facultativo", # noqa - "Quilometragem entre 03h e 12h — Ponto Facultativo": "quilometragem_entre_03h_e_12h_ponto_facultativo", # noqa - "Partidas entre 12h e 21h — Ponto Facultativo": "partidas_entre_12h_e_21h_ponto_facultativo", # noqa - "Quilometragem entre 12h e 21h — Ponto Facultativo": "quilometragem_entre_12h_e_21h_ponto_facultativo", # noqa - "Partidas entre 21h e 24h — Ponto Facultativo": "partidas_entre_21h_e_24h_ponto_facultativo", # noqa - "Quilometragem entre 21h e 24h — Ponto Facultativo": "quilometragem_entre_21h_e_24h_ponto_facultativo", # noqa - "Partidas entre 24h e 03h (dia seguinte) — Ponto Facultativo": "partidas_entre_24h_e_03h_diaseguinte_ponto_facultativo", # noqa - "Quilometragem entre 24h e 03h (dia seguinte) — Ponto Facultativo": "quilometragem_entre_24h_e_03h_diaseguinte_ponto_facultativo", # noqa - "Partidas entre 00h e 03h (Dias Úteis)": "partidas_entre_00h_e_03h_dias_uteis", - "Quilometragem entre 00h e 03h (Dias Úteis)": "quilometragem_entre_00h_e_03h_dias_uteis", - "Partidas entre 03h e 12h (Dias Úteis)": "partidas_entre_03h_e_12h_dias_uteis", - "Quilometragem entre 03h e 12h (Dias Úteis)": "quilometragem_entre_03h_e_12h_dias_uteis", - "Partidas entre 12h e 21h (Dias Úteis)": "partidas_entre_12h_e_21h_dias_uteis", - "Quilometragem entre 12h e 21h (Dias Úteis)": "quilometragem_entre_12h_e_21h_dias_uteis", - "Partidas entre 21h e 24h (Dias Úteis)": "partidas_entre_21h_e_24h_dias_uteis", - "Quilometragem entre 21h e 24h (Dias Úteis)": "quilometragem_entre_21h_e_24h_dias_uteis", - "Partidas entre 24h e 03h (dia seguinte) (Dias Úteis)": "partidas_entre_24h_e_03h_diaseguinte_dias_uteis", # noqa - "Quilometragem entre 24h e 03h (dia seguinte) (Dias Úteis)": "quilometragem_entre_24h_e_03h_diaseguinte_dias_uteis", # noqa - "Partidas entre 00h e 03h (Sábado)": "partidas_entre_00h_e_03h_sabado", - "Quilometragem entre 00h e 03h (Sábado)": "quilometragem_entre_00h_e_03h_sabado", - "Partidas entre 03h e 12h (Sábado)": "partidas_entre_03h_e_12h_sabado", - "Quilometragem entre 03h e 12h (Sábado)": "quilometragem_entre_03h_e_12h_sabado", - "Partidas entre 12h e 21h (Sábado)": "partidas_entre_12h_e_21h_sabado", - "Quilometragem entre 12h e 21h (Sábado)": "quilometragem_entre_12h_e_21h_sabado", - "Partidas entre 21h e 24h (Sábado)": "partidas_entre_21h_e_24h_sabado", - "Quilometragem entre 21h e 24h (Sábado)": "quilometragem_entre_21h_e_24h_sabado", - "Partidas entre 24h e 03h (dia seguinte) (Sábado)": "partidas_entre_24h_e_03h_diaseguinte_sabado", # noqa - "Quilometragem entre 24h e 03h (dia seguinte) (Sábado)": "quilometragem_entre_24h_e_03h_diaseguinte_sabado", # noqa - "Partidas entre 00h e 03h (Domingo)": "partidas_entre_00h_e_03h_domingo", - "Quilometragem entre 00h e 03h (Domingo)": "quilometragem_entre_00h_e_03h_domingo", - "Partidas entre 03h e 12h (Domingo)": "partidas_entre_03h_e_12h_domingo", - "Quilometragem entre 03h e 12h (Domingo)": "quilometragem_entre_03h_e_12h_domingo", - "Partidas entre 12h e 21h (Domingo)": "partidas_entre_12h_e_21h_domingo", - "Quilometragem entre 12h e 21h (Domingo)": "quilometragem_entre_12h_e_21h_domingo", - "Partidas entre 21h e 24h (Domingo)": "partidas_entre_21h_e_24h_domingo", - "Quilometragem entre 21h e 24h (Domingo)": "quilometragem_entre_21h_e_24h_domingo", - "Partidas entre 24h e 03h (dia seguinte) (Domingo)": "partidas_entre_24h_e_03h_diaseguinte_domingo", # noqa - "Quilometragem entre 24h e 03h (dia seguinte) (Domingo)": "quilometragem_entre_24h_e_03h_diaseguinte_domingo", # noqa - "Partidas entre 00h e 03h (Ponto Facultativo)": "partidas_entre_00h_e_03h_ponto_facultativo", # noqa - "Quilometragem entre 00h e 03h (Ponto Facultativo)": "quilometragem_entre_00h_e_03h_ponto_facultativo", # noqa - "Partidas entre 03h e 12h (Ponto Facultativo)": "partidas_entre_03h_e_12h_ponto_facultativo", # noqa - "Quilometragem entre 03h e 12h (Ponto Facultativo)": "quilometragem_entre_03h_e_12h_ponto_facultativo", # noqa - "Partidas entre 12h e 21h (Ponto Facultativo)": "partidas_entre_12h_e_21h_ponto_facultativo", # noqa - "Quilometragem entre 12h e 21h (Ponto Facultativo)": "quilometragem_entre_12h_e_21h_ponto_facultativo", # noqa - "Partidas entre 21h e 24h (Ponto Facultativo)": "partidas_entre_21h_e_24h_ponto_facultativo", # noqa - "Quilometragem entre 21h e 24h (Ponto Facultativo)": "quilometragem_entre_21h_e_24h_ponto_facultativo", # noqa - "Partidas entre 24h e 03h (dia seguinte) (Ponto Facultativo)": "partidas_entre_24h_e_03h_diaseguinte_ponto_facultativo", # noqa - "Quilometragem entre 24h e 03h (dia seguinte) (Ponto Facultativo)": "quilometragem_entre_24h_e_03h_diaseguinte_ponto_facultativo", # noqa - "tipo_os": "tipo_os", + formato.format(metrica=metrica, intervalo=intervalo, dia=dia): unidecode( + ("quilometragem" if metrica in ["Quilometragem", "KM"] else "partidas") + + f"_entre_{intervalo.replace(' ', '_').replace('(', '').replace(')', '').replace('-', '_')}_{dia.lower().replace(' ', '_')}" # noqa + ) + for metrica in metricas + for intervalo in intervalos + for dia in dias + for formato in formatos } + fh_columns["Serviço"] = "servico" + fh_columns["Consórcio"] = "consorcio" + fh_columns["tipo_os"] = "tipo_os" + + ordem_servico_faixa_horaria.columns = ( + ordem_servico_faixa_horaria.columns.str.replace("\n", " ") + .str.strip() + .str.replace(r"\s+", " ", regex=True) + ) - ordem_servico_faixa_horaria.columns = ordem_servico_faixa_horaria.columns.str.replace("\n", " ") ordem_servico_faixa_horaria = ordem_servico_faixa_horaria.rename(columns=fh_columns) + for col in ordem_servico_faixa_horaria.columns: + if "quilometragem" in col: + ordem_servico_faixa_horaria[col] = ( + ordem_servico_faixa_horaria[col].astype(str).apply(convert_to_float).astype(float) + ) + if "tipo_os" not in ordem_servico_faixa_horaria.columns: ordem_servico_faixa_horaria["tipo_os"] = "Regular" diff --git a/pipelines/migration/br_rj_riodejaneiro_onibus_gps/CHANGELOG.md b/pipelines/migration/br_rj_riodejaneiro_onibus_gps/CHANGELOG.md index 2a34272d7..3b9eb8f36 100644 --- a/pipelines/migration/br_rj_riodejaneiro_onibus_gps/CHANGELOG.md +++ b/pipelines/migration/br_rj_riodejaneiro_onibus_gps/CHANGELOG.md @@ -1,5 +1,11 @@ # Changelog - br_rj_riodejaneiro_onibus_gps +## [1.0.3] - 2024-10-29 + +### Alterado + +- Altera o flow `materialize_sppo` para utilizar as tasks que rodam os testes do DBT (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/288) + ## [1.0.2] - 2024-08-25 ### Adicionado diff --git a/pipelines/migration/br_rj_riodejaneiro_onibus_gps/flows.py b/pipelines/migration/br_rj_riodejaneiro_onibus_gps/flows.py index a6ecd62b6..f241ac7c5 100644 --- a/pipelines/migration/br_rj_riodejaneiro_onibus_gps/flows.py +++ b/pipelines/migration/br_rj_riodejaneiro_onibus_gps/flows.py @@ -2,7 +2,7 @@ """ Flows for br_rj_riodejaneiro_onibus_gps -DBT 2024-08-20 +DBT 2024-08-20 2 """ from copy import deepcopy @@ -25,6 +25,9 @@ from pipelines.constants import constants from pipelines.constants import constants as emd_constants +from pipelines.migration.br_rj_riodejaneiro_onibus_gps.constants import ( + constants as gps_constants, +) from pipelines.migration.br_rj_riodejaneiro_onibus_gps.tasks import ( clean_br_rj_riodejaneiro_onibus_gps, create_api_url_onibus_gps, @@ -61,6 +64,11 @@ every_hour_minute_six, every_minute, ) +from pipelines.treatment.templates.tasks import ( + check_dbt_test_run, + dbt_data_quality_checks, + run_dbt_tests, +) # from pipelines.utils.execute_dbt_model.tasks import get_k8s_dbt_client @@ -156,14 +164,16 @@ table_id = Parameter("table_id", default=constants.GPS_SPPO_TREATED_TABLE_ID.value) rebuild = Parameter("rebuild", False) rematerialization = Parameter("rematerialization", default=False) - date_range_start = Parameter("date_range_start", default=None) - date_range_end = Parameter("date_range_end", default=None) + date_range_start_param = Parameter("date_range_start", default=None) + date_range_end_param = Parameter("date_range_end", default=None) fifteen_minutes = Parameter("fifteen_minutes", default="") materialize_delay_hours = Parameter( "materialize_delay_hours", default=constants.GPS_SPPO_MATERIALIZE_DELAY_HOURS.value, ) truncate_minutes = Parameter("truncate_minutes", default=True) + test_only = Parameter("test_only", default=False) + run_time_test = Parameter("run_time_test", default="01:00:00") LABELS = get_current_flow_labels() MODE = get_current_flow_mode() @@ -174,85 +184,121 @@ # dbt_client = get_local_dbt_client(host="localhost", port=3001) # Set specific run parameters # - with case(rematerialization, False): - date_range_false = get_materialization_date_range( + with case(test_only, False): + with case(rematerialization, False): + date_range_false = get_materialization_date_range( + dataset_id=dataset_id, + table_id=table_id, + raw_dataset_id=raw_dataset_id, + raw_table_id=raw_table_id, + table_run_datetime_column_name="timestamp_gps", + mode=MODE, + delay_hours=materialize_delay_hours, + truncate_minutes=truncate_minutes, + ) + + RUN_CLEAN_FALSE = task( + lambda: [None], + checkpoint=False, + name="assign_none_to_previous_runs", + )() + with case(rematerialization, True): + date_range_true = task( + lambda start, end: { + "date_range_start": start, + "date_range_end": end, + } + )(start=date_range_start_param, end=date_range_end_param) + + RUN_CLEAN_TRUE = clean_br_rj_riodejaneiro_onibus_gps(date_range_true) + + RUN_CLEAN = merge(RUN_CLEAN_TRUE, RUN_CLEAN_FALSE) + + date_range = merge(date_range_true, date_range_false) + + dataset_sha = fetch_dataset_sha( dataset_id=dataset_id, - table_id=table_id, - raw_dataset_id=raw_dataset_id, - raw_table_id=raw_table_id, - table_run_datetime_column_name="timestamp_gps", - mode=MODE, - delay_hours=materialize_delay_hours, - truncate_minutes=truncate_minutes, + upstream_tasks=[RUN_CLEAN], ) - RUN_CLEAN_FALSE = task( - lambda: [None], - checkpoint=False, - name="assign_none_to_previous_runs", - )() - with case(rematerialization, True): - date_range_true = task( - lambda start, end: { - "date_range_start": start, - "date_range_end": end, - } - )(start=date_range_start, end=date_range_end) + # Run materialization # + with case(rebuild, True): + RUN_TRUE = run_dbt_model( + # dbt_client=dbt_client, + dataset_id=dataset_id, + table_id=table_id, + upstream=True, + exclude="+data_versao_efetiva", + _vars=[date_range, dataset_sha, {"fifteen_minutes": fifteen_minutes}], + flags="--full-refresh", + ) - RUN_CLEAN_TRUE = clean_br_rj_riodejaneiro_onibus_gps(date_range_true) + with case(rebuild, False): + RUN_FALSE = run_dbt_model( + # dbt_client=dbt_client, + dataset_id=dataset_id, + table_id=table_id, + exclude="+data_versao_efetiva", + _vars=[date_range, dataset_sha, {"fifteen_minutes": fifteen_minutes}], + upstream=True, + ) - RUN_CLEAN = merge(RUN_CLEAN_TRUE, RUN_CLEAN_FALSE) + date_range_start = date_range["date_range_start"] + date_range_end = date_range["date_range_end"] - date_range = merge(date_range_true, date_range_false) + RUN_TEST, datetime_start, datetime_end = check_dbt_test_run( + date_range_start, date_range_end, run_time_test, upstream_tasks=[RUN_FALSE] + ) - dataset_sha = fetch_dataset_sha( - dataset_id=dataset_id, - upstream_tasks=[RUN_CLEAN], - ) + _vars = {"date_range_start": datetime_start, "date_range_end": datetime_end} + + with case(RUN_TEST, True): + gps_sppo_data_quality = run_dbt_tests( + dataset_id=dataset_id, + table_id=table_id, + _vars=_vars, + ) + GPS_SPPO_DATA_QUALITY_RESULTS = dbt_data_quality_checks( + gps_sppo_data_quality, + gps_constants.GPS_DATA_CHECKS_LIST.value, + _vars, + ) + + RUN = merge(RUN_TRUE, RUN_FALSE) + + with case(rematerialization, False): + SET_FALSE = set_last_run_timestamp( + dataset_id=dataset_id, + table_id=table_id, + timestamp=date_range["date_range_end"], + wait=RUN, + mode=MODE, + ) - # Run materialization # - with case(rebuild, True): - RUN_TRUE = run_dbt_model( - # dbt_client=dbt_client, - dataset_id=dataset_id, - table_id=table_id, - upstream=True, - exclude="+data_versao_efetiva", - _vars=[date_range, dataset_sha, {"fifteen_minutes": fifteen_minutes}], - flags="--full-refresh", - ) + with case(rematerialization, True): + SET_TRUE = task( + lambda: [None], + checkpoint=False, + name="assign_none_to_previous_runs", + )() - with case(rebuild, False): - RUN_FALSE = run_dbt_model( - # dbt_client=dbt_client, - dataset_id=dataset_id, - table_id=table_id, - exclude="+data_versao_efetiva", - _vars=[date_range, dataset_sha, {"fifteen_minutes": fifteen_minutes}], - upstream=True, - ) + SET = merge(SET_TRUE, SET_FALSE) + + materialize_sppo.set_reference_tasks([RUN, RUN_CLEAN, SET]) + with case(test_only, True): - RUN = merge(RUN_TRUE, RUN_FALSE) + _vars = {"date_range_start": date_range_start_param, "date_range_end": date_range_end_param} - with case(rematerialization, False): - SET_FALSE = set_last_run_timestamp( + gps_sppo_data_quality = run_dbt_tests( dataset_id=dataset_id, table_id=table_id, - timestamp=date_range["date_range_end"], - wait=RUN, - mode=MODE, + _vars=_vars, + ) + GPS_SPPO_DATA_QUALITY_RESULTS = dbt_data_quality_checks( + gps_sppo_data_quality, + gps_constants.GPS_DATA_CHECKS_LIST.value, + _vars, ) - - with case(rematerialization, True): - SET_TRUE = task( - lambda: [None], - checkpoint=False, - name="assign_none_to_previous_runs", - )() - - SET = merge(SET_TRUE, SET_FALSE) - - materialize_sppo.set_reference_tasks([RUN, RUN_CLEAN, SET]) materialize_sppo.storage = GCS(emd_constants.GCS_FLOWS_BUCKET.value) materialize_sppo.run_config = KubernetesRun( diff --git a/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/CHANGELOG.md b/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/CHANGELOG.md index 32b5f3e07..b09072594 100644 --- a/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/CHANGELOG.md +++ b/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/CHANGELOG.md @@ -1,5 +1,13 @@ # Changelog - br_rj_riodejaneiro_onibus_gps_zirix +## [1.1.0] - 2024-11-12 + +### Adicionado +- Cria flow de recaptura da realocação `recaptura_realocacao_sppo_zirix` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/316) + +### Alterado +- Cria lógica de rematerialização do gps_sppo_zirix (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/316) + ## [1.0.0] - 2024-09-02 ### Corrigido diff --git a/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/constants.py b/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/constants.py index 99322697c..af1f58f73 100644 --- a/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/constants.py +++ b/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/constants.py @@ -13,3 +13,4 @@ class constants(Enum): # pylint: disable=c0103 GPS_SPPO_ZIRIX_RAW_DATASET_ID = "br_rj_riodejaneiro_onibus_gps_zirix" GPS_SPPO_ZIRIX_TREATED_TABLE_ID = "gps_sppo_zirix" + GPS_SPPO_MATERIALIZE_DELAY_HOURS = 1 diff --git a/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/flows.py b/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/flows.py index 2face7cc9..8450918d0 100644 --- a/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/flows.py +++ b/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/flows.py @@ -4,9 +4,10 @@ DBT 2024-08-26 """ -from prefect import Parameter, case +from prefect import Parameter, case, task from prefect.run_configs import KubernetesRun from prefect.storage import GCS +from prefect.tasks.control_flow import merge from prefect.tasks.prefect import create_flow_run, wait_for_flow_run from prefect.utilities.edges import unmapped from prefeitura_rio.pipelines_utils.custom import Flow @@ -18,6 +19,7 @@ from pipelines.constants import constants as smtr_constants from pipelines.migration.br_rj_riodejaneiro_onibus_gps_zirix.constants import constants from pipelines.migration.br_rj_riodejaneiro_onibus_gps_zirix.tasks import ( + clean_br_rj_riodejaneiro_onibus_gps_zirix, create_api_url_onibus_gps, create_api_url_onibus_realocacao, pre_treatment_br_rj_riodejaneiro_onibus_gps, @@ -35,6 +37,7 @@ get_materialization_date_range, get_now_time, get_raw, + get_rounded_timestamp, parse_timestamp_to_string, query_logs, rename_current_flow_run_now_time, @@ -109,6 +112,85 @@ realocacao_sppo_zirix.state_handlers = [handler_initialize_sentry, handler_inject_bd_credentials] realocacao_sppo_zirix.schedule = every_10_minutes +with Flow( + "SMTR: GPS SPPO Zirix Realocação - Recaptura (subflow)", +) as recaptura_realocacao_sppo_zirix: + timestamp = Parameter("timestamp", default=None) + recapture_window_days = Parameter("recapture_window_days", default=1) + + # SETUP # + LABELS = get_current_flow_labels() + + # Consulta de logs para verificar erros + errors, timestamps, previous_errors = query_logs( + dataset_id=constants.GPS_SPPO_ZIRIX_RAW_DATASET_ID.value, + table_id=smtr_constants.GPS_SPPO_REALOCACAO_RAW_TABLE_ID.value, + datetime_filter=get_rounded_timestamp(timestamp=timestamp, interval_minutes=10), + interval_minutes=10, + recapture_window_days=recapture_window_days, + ) + + rename_flow_run = rename_current_flow_run_now_time( + prefix=recaptura_realocacao_sppo_zirix.name + ": ", + now_time=get_now_time(), + wait=timestamps, + ) + + # Em caso de erros, executa a recaptura + with case(errors, True): + # SETUP # + partitions = create_date_hour_partition.map(timestamps) + filename = parse_timestamp_to_string.map(timestamps) + + filepath = create_local_partition_path.map( + dataset_id=unmapped(constants.GPS_SPPO_ZIRIX_RAW_DATASET_ID.value), + table_id=unmapped(smtr_constants.GPS_SPPO_REALOCACAO_RAW_TABLE_ID.value), + filename=filename, + partitions=partitions, + ) + + url = create_api_url_onibus_realocacao.map(timestamp=timestamps) + + # EXTRACT # + raw_status = get_raw.map(url) + + raw_filepath = save_raw_local.map(status=raw_status, file_path=filepath) + + # CLEAN # + treated_status = pre_treatment_br_rj_riodejaneiro_onibus_realocacao.map( + status=raw_status, timestamp=timestamps + ) + + treated_filepath = save_treated_local.map(status=treated_status, file_path=filepath) + + # LOAD # + error = bq_upload.map( + dataset_id=unmapped(constants.GPS_SPPO_ZIRIX_RAW_DATASET_ID.value), + table_id=unmapped(smtr_constants.GPS_SPPO_REALOCACAO_RAW_TABLE_ID.value), + filepath=treated_filepath, + raw_filepath=raw_filepath, + partitions=partitions, + status=treated_status, + ) + + upload_logs_to_bq.map( + dataset_id=unmapped(constants.GPS_SPPO_ZIRIX_RAW_DATASET_ID.value), + parent_table_id=unmapped(smtr_constants.GPS_SPPO_REALOCACAO_RAW_TABLE_ID.value), + error=error, + previous_error=previous_errors, + timestamp=timestamps, + recapture=unmapped(True), + ) + +recaptura_realocacao_sppo_zirix.storage = GCS(smtr_constants.GCS_FLOWS_BUCKET.value) +recaptura_realocacao_sppo_zirix.run_config = KubernetesRun( + image=smtr_constants.DOCKER_IMAGE.value, + labels=[smtr_constants.RJ_SMTR_AGENT_LABEL.value], +) +recaptura_realocacao_sppo_zirix.state_handlers = [ + handler_initialize_sentry, + handler_inject_bd_credentials, +] with Flow( "SMTR: GPS SPPO Zirix - Materialização (subflow)", @@ -127,22 +209,52 @@ dataset_id = Parameter("dataset_id", default=constants.GPS_SPPO_ZIRIX_RAW_DATASET_ID.value) table_id = Parameter("table_id", default=constants.GPS_SPPO_ZIRIX_TREATED_TABLE_ID.value) rebuild = Parameter("rebuild", False) + rematerialization = Parameter("rematerialization", default=False) + date_range_start_param = Parameter("date_range_start", default=None) + date_range_end_param = Parameter("date_range_end", default=None) + materialize_delay_hours = Parameter( + "materialize_delay_hours", + default=constants.GPS_SPPO_MATERIALIZE_DELAY_HOURS.value, + ) LABELS = get_current_flow_labels() MODE = get_current_flow_mode() # Set specific run parameters # - date_range = get_materialization_date_range( - dataset_id=dataset_id, - table_id="gps_sppo", - raw_dataset_id=raw_dataset_id, - raw_table_id=raw_table_id, - table_run_datetime_column_name="timestamp_gps", - mode=MODE, - delay_hours=smtr_constants.GPS_SPPO_MATERIALIZE_DELAY_HOURS.value, - ) + with case(rematerialization, False): + date_range_false = get_materialization_date_range( + dataset_id=dataset_id, + table_id="gps_sppo", + raw_dataset_id=raw_dataset_id, + raw_table_id=raw_table_id, + table_run_datetime_column_name="timestamp_gps", + mode=MODE, + delay_hours=materialize_delay_hours, + truncate_minutes=True, + ) + + RUN_CLEAN_FALSE = task( + lambda: [None], + checkpoint=False, + name="assign_none_to_previous_runs", + )() + with case(rematerialization, True): + date_range_true = task( + lambda start, end: { + "date_range_start": start, + "date_range_end": end, + } + )(start=date_range_start_param, end=date_range_end_param) + + RUN_CLEAN_TRUE = clean_br_rj_riodejaneiro_onibus_gps_zirix(date_range_true) + + RUN_CLEAN = merge(RUN_CLEAN_TRUE, RUN_CLEAN_FALSE) + + date_range = merge(date_range_true, date_range_false) + dataset_sha = fetch_dataset_sha( dataset_id=dataset_id, + upstream_tasks=[RUN_CLEAN], ) # Run materialization # @@ -253,9 +365,33 @@ ) as recaptura_zirix: datetime_filter = Parameter("datetime_filter", default=None) materialize = Parameter("materialize", default=True) + # SETUP # LABELS = get_current_flow_labels() PROJECT = get_flow_project() + + rounded_timestamp = get_rounded_timestamp(interval_minutes=60) + rounded_timestamp_str = parse_timestamp_to_string( + timestamp=rounded_timestamp, + pattern="%Y-%m-%d %H:%M:%S", + ) + + # roda o subflow de recaptura da realocação + run_recaptura_realocacao_sppo_zirix = create_flow_run( + flow_name=recaptura_realocacao_sppo_zirix.name, + project_name=PROJECT, + labels=LABELS, + run_name=recaptura_realocacao_sppo_zirix.name, + parameters={"timestamp": rounded_timestamp_str}, + ) + + wait_recaptura_realocacao_sppo_zirix = wait_for_flow_run( + run_recaptura_realocacao_sppo_zirix, + stream_states=True, + stream_logs=True, + raise_final_state=True, + ) + errors, timestamps, previous_errors = query_logs( dataset_id=constants.GPS_SPPO_ZIRIX_RAW_DATASET_ID.value, table_id=smtr_constants.GPS_SPPO_RAW_TABLE_ID.value, diff --git a/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/tasks.py b/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/tasks.py index 67a9db5fa..75b77f887 100644 --- a/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/tasks.py +++ b/pipelines/migration/br_rj_riodejaneiro_onibus_gps_zirix/tasks.py @@ -5,8 +5,9 @@ import traceback from datetime import datetime, timedelta -from typing import Dict +from typing import Dict, Union +import basedosdados as bd import pandas as pd import pendulum from prefect import task @@ -237,3 +238,71 @@ def pre_treatment_br_rj_riodejaneiro_onibus_gps( log(f"[CATCHED] Task failed with error: \n{error}", level="error") return {"data": df_gps, "error": error} + + +@task +def clean_br_rj_riodejaneiro_onibus_gps_zirix(date_range: dict) -> Union[str, None]: + """ + Clean GPS data for a given date range. + + This function deletes records from three different tables in the database: + - `rj-smtr.br_rj_riodejaneiro_onibus_gps_zirix.sppo_aux_registro_filtrada` + - `rj-smtr.br_rj_riodejaneiro_onibus_gps_zirix.sppo_aux_registros_realocacao` + - `rj-smtr.br_rj_riodejaneiro_onibus_gps_zirix.gps_sppo` + + The records to be deleted are determined by the provided + date range and the timestamp_gps column. + + Parameters: + - date_range (dict): A dictionary containing the start + and end dates for the data to be cleaned. + + Returns: + - str or None: If an error occurs during the cleaning process, + the error message is returned. Otherwise, None is returned. + + """ + error = None + + try: + q = f""" + DELETE + FROM + `rj-smtr.br_rj_riodejaneiro_onibus_gps_zirix.sppo_aux_registro_filtrada` + WHERE + (data BETWEEN DATE("{date_range['date_range_start']}") + AND DATE("{date_range['date_range_end']}")) + AND (timestamp_gps > "{date_range['date_range_start']}" + AND timestamp_gps <= "{date_range['date_range_end']}"); + DELETE + FROM + `rj-smtr.br_rj_riodejaneiro_onibus_gps_zirix.sppo_aux_registros_realocacao` + WHERE + (data BETWEEN DATE("{date_range['date_range_start']}") + AND DATE("{date_range['date_range_end']}")) + AND (timestamp_gps > "{date_range['date_range_start']}" + AND timestamp_gps <= "{date_range['date_range_end']}"); + DELETE + FROM + `rj-smtr.br_rj_riodejaneiro_onibus_gps_zirix.gps_sppo` + WHERE + (data BETWEEN DATE("{date_range['date_range_start']}") + AND DATE("{date_range['date_range_end']}")) + AND (timestamp_gps > "{date_range['date_range_start']}" + AND timestamp_gps <= "{date_range['date_range_end']}"); + """ + log(q) + + results = bd.read_sql(q) + + log( + f"""Cleaned GPS data for + {date_range['date_range_start']} to {date_range['date_range_end']}\n + Resulting:\n + {results}""" + ) + except Exception: # pylint: disable = W0703 + error = traceback.format_exc() + log(f"[CATCHED] Task failed with error: \n{error}", level="error") + + return error diff --git a/pipelines/migration/controle_financeiro/tasks.py b/pipelines/migration/controle_financeiro/tasks.py index f8d1aa50d..c7c4a41ef 100644 --- a/pipelines/migration/controle_financeiro/tasks.py +++ b/pipelines/migration/controle_financeiro/tasks.py @@ -63,12 +63,10 @@ def create_cct_arquivo_retorno_params( headers = {"Authorization": f"Bearer {auth_resp.json()['token']}"} if start_date is not None and end_date is not None: - return headers, [ - { - "dt_inicio": start_date, - "dt_fim": end_date, - } - ] + return headers, { + "dt_inicio": start_date, + "dt_fim": end_date, + } redis_client = get_redis_client() diff --git a/pipelines/migration/projeto_subsidio_sppo/CHANGELOG.md b/pipelines/migration/projeto_subsidio_sppo/CHANGELOG.md index 65d0a4e07..b44ffb47a 100644 --- a/pipelines/migration/projeto_subsidio_sppo/CHANGELOG.md +++ b/pipelines/migration/projeto_subsidio_sppo/CHANGELOG.md @@ -1,5 +1,11 @@ # Changelog - projeto_subsidio_sppo +## [1.0.6] - 2024-12-17 + +### Adicionado + +- Adiciona automação dos testes do DBT no arquivo `flows.py` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/256) + ## [1.0.5] - 2024-08-29 ### Alterado diff --git a/pipelines/migration/projeto_subsidio_sppo/flows.py b/pipelines/migration/projeto_subsidio_sppo/flows.py index a6139c664..152fede9d 100644 --- a/pipelines/migration/projeto_subsidio_sppo/flows.py +++ b/pipelines/migration/projeto_subsidio_sppo/flows.py @@ -3,7 +3,7 @@ """ Flows for projeto_subsidio_sppo -DBT: 2024-10-24 +DBT: 2024-12-18 """ from prefect import Parameter, case, task @@ -23,10 +23,7 @@ from pipelines.constants import constants as smtr_constants from pipelines.migration.projeto_subsidio_sppo.constants import constants -from pipelines.migration.projeto_subsidio_sppo.tasks import ( - check_param, - subsidio_data_quality_check, -) +from pipelines.migration.projeto_subsidio_sppo.tasks import check_param from pipelines.migration.tasks import ( check_date_in_range, fetch_dataset_sha, @@ -44,7 +41,12 @@ ) from pipelines.migration.veiculo.flows import sppo_veiculo_dia from pipelines.schedules import every_day_hour_five, every_day_hour_seven_minute_five -from pipelines.treatment.templates.tasks import run_dbt_selector +from pipelines.tasks import check_fail +from pipelines.treatment.templates.tasks import ( + dbt_data_quality_checks, + run_dbt_selector, + run_dbt_tests, +) # from pipelines.materialize_to_datario.flows import ( # smtr_materialize_to_datario_viagem_sppo_flow, @@ -207,13 +209,23 @@ ) # 3. PRE-DATA QUALITY CHECK # - SUBSIDIO_SPPO_DATA_QUALITY_PRE = subsidio_data_quality_check( - mode="pre", - params=_vars, - upstream_tasks=[SPPO_VEICULO_DIA_RUN_WAIT], + dbt_vars = {"date_range_start": start_date, "date_range_end": end_date} + + SUBSIDIO_SPPO_DATA_QUALITY_PRE = run_dbt_tests( + dataset_id="sppo_registros sppo_realocacao check_gps_treatment__gps_sppo sppo_veiculo_dia", # noqa + _vars=dbt_vars, + ).set_upstream(task=SPPO_VEICULO_DIA_RUN_WAIT) + + DATA_QUALITY_PRE = dbt_data_quality_checks( + dbt_logs=SUBSIDIO_SPPO_DATA_QUALITY_PRE, + checks_list={}, + webhook_key="subsidio_data_check", + params=dbt_vars, ) - with case(SUBSIDIO_SPPO_DATA_QUALITY_PRE, True): + test_failed = check_fail(DATA_QUALITY_PRE) + + with case(test_failed, False): # 4. CALCULATE # date_in_range = check_date_in_range( _vars["start_date"], _vars["end_date"], constants.DATA_SUBSIDIO_V9_INICIO.value @@ -234,10 +246,22 @@ ) # POST-DATA QUALITY CHECK # - SUBSIDIO_SPPO_DATA_QUALITY_POS = subsidio_data_quality_check( - mode="pos", - params=dbt_vars_1, - upstream_tasks=[SUBSIDIO_SPPO_APURACAO_RUN], + SUBSIDIO_SPPO_DATA_QUALITY_POS = run_dbt_tests( + dataset_id="dashboard_subsidio_sppo", + _vars={ + "date_range_start": date_intervals["first_range"]["start_date"], + "date_range_end": date_intervals["first_range"]["end_date"], + }, + ).set_upstream(task=SUBSIDIO_SPPO_APURACAO_RUN) + + DATA_QUALITY_POS = dbt_data_quality_checks( + dbt_logs=SUBSIDIO_SPPO_DATA_QUALITY_POS, + checks_list={}, + webhook_key="subsidio_data_check", + params={ + "date_range_start": date_intervals["first_range"]["start_date"], + "date_range_end": date_intervals["first_range"]["end_date"], + }, ) dbt_vars_2 = get_join_dict( @@ -253,10 +277,22 @@ ) # POST-DATA QUALITY CHECK # - SUBSIDIO_SPPO_DATA_QUALITY_POS_2 = subsidio_data_quality_check( - mode="pos", - params=dbt_vars_2, - upstream_tasks=[SUBSIDIO_SPPO_APURACAO_RUN_2], + SUBSIDIO_SPPO_DATA_QUALITY_POS_2 = run_dbt_tests( + dataset_id="viagens_remuneradas sumario_servico_dia_pagamento", + _vars={ + "date_range_start": date_intervals["second_range"]["start_date"], + "date_range_end": date_intervals["second_range"]["end_date"], + }, + ).set_upstream(task=SUBSIDIO_SPPO_APURACAO_RUN_2) + + DATA_QUALITY_POS_2 = dbt_data_quality_checks( + dbt_logs=SUBSIDIO_SPPO_DATA_QUALITY_POS_2, + checks_list={}, + webhook_key="subsidio_data_check", + params={ + "date_range_start": date_intervals["second_range"]["start_date"], + "date_range_end": date_intervals["second_range"]["end_date"], + }, ) with case(date_in_range, False): @@ -268,6 +304,18 @@ selector_name="apuracao_subsidio_v8", _vars=_vars, ) + # POST-DATA QUALITY CHECK # + SUBSIDIO_SPPO_DATA_QUALITY_POS = run_dbt_tests( + dataset_id="dashboard_subsidio_sppo", + _vars=dbt_vars, + ).set_upstream(task=SUBSIDIO_SPPO_APURACAO_RUN) + + DATA_QUALITY_POS = dbt_data_quality_checks( + dbt_logs=SUBSIDIO_SPPO_DATA_QUALITY_POS, + checks_list={}, + webhook_key="subsidio_data_check", + params=dbt_vars, + ) with case(gte_result, True): SUBSIDIO_SPPO_APURACAO_RUN = run_dbt_selector( @@ -275,12 +323,18 @@ _vars=_vars, ) - # POST-DATA QUALITY CHECK # - SUBSIDIO_SPPO_DATA_QUALITY_POS = subsidio_data_quality_check( - mode="pos", - params=_vars, - upstream_tasks=[SUBSIDIO_SPPO_APURACAO_RUN], - ) + # POST-DATA QUALITY CHECK # + SUBSIDIO_SPPO_DATA_QUALITY_POS = run_dbt_tests( + dataset_id="viagens_remuneradas sumario_servico_dia_pagamento", + _vars=dbt_vars, + ).set_upstream(task=SUBSIDIO_SPPO_APURACAO_RUN) + + DATA_QUALITY_POS = dbt_data_quality_checks( + dbt_logs=SUBSIDIO_SPPO_DATA_QUALITY_POS, + checks_list={}, + webhook_key="subsidio_data_check", + params=dbt_vars, + ) # TODO: test upstream_tasks=[SUBSIDIO_SPPO_DASHBOARD_RUN] # 6. PUBLISH # @@ -313,17 +367,94 @@ # SUBSIDIO_SPPO_DASHBOARD_RUN # ) with case(test_only, True): - SUBSIDIO_SPPO_DATA_QUALITY_PRE = subsidio_data_quality_check( - mode="pre", - params=_vars, + dbt_vars = {"date_range_start": start_date, "date_range_end": end_date} + + SUBSIDIO_SPPO_DATA_QUALITY_PRE = run_dbt_tests( + dataset_id="sppo_registros sppo_realocacao check_gps_treatment__gps_sppo sppo_veiculo_dia", # noqa + _vars=dbt_vars, + ) + DATA_QUALITY_PRE = dbt_data_quality_checks( + dbt_logs=SUBSIDIO_SPPO_DATA_QUALITY_PRE, + checks_list={}, + webhook_key="subsidio_data_check", + params=dbt_vars, ) - SUBSIDIO_SPPO_DATA_QUALITY_POS = subsidio_data_quality_check( - mode="pos", - params=_vars, - upstream_tasks=[SUBSIDIO_SPPO_DATA_QUALITY_PRE], + date_in_range = check_date_in_range( + _vars["start_date"], _vars["end_date"], constants.DATA_SUBSIDIO_V9_INICIO.value ) + with case(date_in_range, True): + date_intervals = split_date_range( + _vars["start_date"], _vars["end_date"], constants.DATA_SUBSIDIO_V9_INICIO.value + ) + + SUBSIDIO_SPPO_DATA_QUALITY_POS = run_dbt_tests( + dataset_id="dashboard_subsidio_sppo", + _vars={ + "date_range_start": date_intervals["first_range"]["start_date"], + "date_range_end": date_intervals["first_range"]["end_date"], + }, + ) + + DATA_QUALITY_POS = dbt_data_quality_checks( + dbt_logs=SUBSIDIO_SPPO_DATA_QUALITY_POS, + checks_list={}, + webhook_key="subsidio_data_check", + params={ + "date_range_start": date_intervals["first_range"]["start_date"], + "date_range_end": date_intervals["first_range"]["end_date"], + }, + ) + + SUBSIDIO_SPPO_DATA_QUALITY_POS_2 = run_dbt_tests( + dataset_id="viagens_remuneradas sumario_servico_dia_pagamento", + _vars={ + "date_range_start": date_intervals["second_range"]["start_date"], + "date_range_end": date_intervals["second_range"]["end_date"], + }, + ).set_upstream(task=SUBSIDIO_SPPO_DATA_QUALITY_POS) + + DATA_QUALITY_POS_2 = dbt_data_quality_checks( + dbt_logs=SUBSIDIO_SPPO_DATA_QUALITY_POS_2, + checks_list={}, + webhook_key="subsidio_data_check", + params={ + "date_range_start": date_intervals["second_range"]["start_date"], + "date_range_end": date_intervals["second_range"]["end_date"], + }, + ) + + with case(date_in_range, False): + gte = GreaterThanOrEqual() + gte_result = gte.run(_vars["start_date"], constants.DATA_SUBSIDIO_V9_INICIO.value) + + with case(gte_result, False): + SUBSIDIO_SPPO_DATA_QUALITY_POS = run_dbt_tests( + dataset_id="dashboard_subsidio_sppo", + _vars=dbt_vars, + ) + + DATA_QUALITY_POS = dbt_data_quality_checks( + dbt_logs=SUBSIDIO_SPPO_DATA_QUALITY_POS, + checks_list={}, + webhook_key="subsidio_data_check", + params=dbt_vars, + ) + + with case(gte_result, True): + SUBSIDIO_SPPO_DATA_QUALITY_POS = run_dbt_tests( + dataset_id="viagens_remuneradas sumario_servico_dia_pagamento", + _vars=dbt_vars, + ) + + DATA_QUALITY_POS = dbt_data_quality_checks( + dbt_logs=SUBSIDIO_SPPO_DATA_QUALITY_POS, + checks_list={}, + webhook_key="subsidio_data_check", + params=dbt_vars, + ) + subsidio_sppo_apuracao.storage = GCS(smtr_constants.GCS_FLOWS_BUCKET.value) subsidio_sppo_apuracao.run_config = KubernetesRun( image=smtr_constants.DOCKER_IMAGE.value, labels=[smtr_constants.RJ_SMTR_AGENT_LABEL.value] diff --git a/pipelines/migration/projeto_subsidio_sppo/tasks.py b/pipelines/migration/projeto_subsidio_sppo/tasks.py index 1b540a90e..867c09ffb 100644 --- a/pipelines/migration/projeto_subsidio_sppo/tasks.py +++ b/pipelines/migration/projeto_subsidio_sppo/tasks.py @@ -10,10 +10,8 @@ from pipelines.constants import constants as smtr_constants from pipelines.migration.projeto_subsidio_sppo.constants import constants -from pipelines.migration.tasks import ( # perform_check, - format_send_discord_message, - perform_checks_for_table, -) +from pipelines.migration.tasks import perform_checks_for_table # perform_check, +from pipelines.utils.discord import format_send_discord_message from pipelines.utils.secret import get_secret from pipelines.utils.utils import log diff --git a/pipelines/migration/tasks.py b/pipelines/migration/tasks.py index d1c53e376..329f90a25 100644 --- a/pipelines/migration/tasks.py +++ b/pipelines/migration/tasks.py @@ -22,6 +22,7 @@ from prefect.backend import FlowRunView from prefeitura_rio.pipelines_utils.dbt import run_dbt_model as run_dbt_model_func from prefeitura_rio.pipelines_utils.infisical import inject_bd_credentials +from prefeitura_rio.pipelines_utils.io import get_root_path from prefeitura_rio.pipelines_utils.logging import log from prefeitura_rio.pipelines_utils.redis_pal import get_redis_client from pytz import timezone @@ -43,9 +44,9 @@ read_raw_data, save_raw_local_func, save_treated_local_func, - send_discord_message, upload_run_logs_to_bq, ) +from pipelines.utils.pretreatment import transform_to_nested_structure from pipelines.utils.secret import get_secret @@ -405,7 +406,8 @@ def create_local_partition_path( either to save raw or staging files. """ data_folder = os.getenv("DATA_FOLDER", "data") - file_path = f"{os.getcwd()}/{data_folder}/{{mode}}/{dataset_id}/{table_id}" + root = str(get_root_path()) + file_path = f"{root}/{data_folder}/{{mode}}/{dataset_id}/{table_id}" file_path += f"/{partitions}/{filename}.{{filetype}}" log(f"Creating file path: {file_path}") return file_path @@ -1537,13 +1539,10 @@ def transform_raw_to_nested_structure( else: log(f"Raw data:\n{data_info_str(data)}", level="info") - log("Adding captured timestamp column...", level="info") - data["timestamp_captura"] = timestamp - if "customFieldValues" not in data: log("Striping string columns...", level="info") - for col in data.columns[data.dtypes == "object"].to_list(): - data[col] = data[col].str.strip() + object_cols = data.select_dtypes(include=["object"]).columns + data[object_cols] = data[object_cols].apply(lambda x: x.str.strip()) if ( constants.GTFS_DATASET_ID.value in raw_filepath @@ -1555,23 +1554,25 @@ def transform_raw_to_nested_structure( log(f"Finished cleaning! Data:\n{data_info_str(data)}", level="info") log("Creating nested structure...", level="info") - pk_cols = primary_key + ["timestamp_captura"] - - data = ( - data.groupby(pk_cols) - .apply( - lambda x: x[data.columns.difference(pk_cols)].to_json( - orient="records", - force_ascii=( - constants.CONTROLE_FINANCEIRO_DATASET_ID.value - not in raw_filepath - ), - ) - ) - .str.strip("[]") - .reset_index(name="content")[primary_key + ["content", "timestamp_captura"]] + + content_columns = [c for c in data.columns if c not in primary_key] + data["content"] = data.apply( + lambda row: json.dumps( + { + key: value if not pd.isna(value) else None + for key, value in row[content_columns].to_dict().items() + }, + ensure_ascii=( + constants.CONTROLE_FINANCEIRO_DATASET_ID.value not in raw_filepath + ), + ), + axis=1, ) + log("Adding captured timestamp column...", level="info") + data["timestamp_captura"] = timestamp + data = data[primary_key + ["content", "timestamp_captura"]] + log( f"Finished nested structure! Data:\n{data_info_str(data)}", level="info", @@ -1587,6 +1588,88 @@ def transform_raw_to_nested_structure( return error, filepath +@task(nout=2) +def transform_raw_to_nested_structure_chunked( + raw_filepath: str, + filepath: str, + error: str, + timestamp: datetime, + chunksize: int, + primary_key: list = None, + reader_args: dict = None, +) -> tuple[str, str]: + """ + Task to transform raw data to nested structure + + Args: + raw_filepath (str): Path to the saved raw .json file + filepath (str): Path to the saved treated .csv file + error (str): Error catched from upstream tasks + timestamp (datetime): timestamp for flow run + chunksize (int): Number of lines to read from the file per chunk + primary_key (list, optional): Primary key to be used on nested structure + reader_args (dict): arguments to pass to pandas.read_csv or read_json + + Returns: + str: Error traceback + str: Path to the saved treated .csv file + """ + if error is None: + try: + # leitura do dado raw + error, data_chunks = read_raw_data( + filepath=raw_filepath, reader_args={"chunksize": chunksize} + ) + + if primary_key is None: + primary_key = [] + + if error is None: + + log("Creating nested structure...", level="info") + + index = 0 + for chunk in data_chunks: + + if "customFieldValues" not in chunk: + object_cols = chunk.select_dtypes(include=["object"]).columns + chunk[object_cols] = chunk[object_cols].apply(lambda x: x.str.strip()) + + if ( + constants.GTFS_DATASET_ID.value in raw_filepath + and "ordem_servico" in raw_filepath + and "tipo_os" not in chunk.columns + ): + chunk["tipo_os"] = "Regular" + + transformed_chunk = transform_to_nested_structure(chunk, primary_key) + transformed_chunk["timestamp_captura"] = timestamp + if index == 0: + filepath = save_treated_local_func( + data=transformed_chunk, + error=error, + filepath=filepath, + args={"header": True, "mode": "w"}, + ) + else: + filepath = save_treated_local_func( + data=transformed_chunk, + error=error, + filepath=filepath, + log_param=False, + args={"header": False, "mode": "a"}, + ) + index += 1 + + log("Finished nested structure!", level="info") + + except Exception: # pylint: disable=W0703 + error = traceback.format_exc() + log(f"[CATCHED] Task failed with error: \n{error}", level="error") + + return error, filepath + + # SUBSIDIO CHECKS @@ -1658,44 +1741,6 @@ def perform_checks_for_table( return checks -def format_send_discord_message(formatted_messages: list, webhook_url: str): - """ - Format and send a message to discord - - Args: - formatted_messages (list): The formatted messages - webhook_url (str): The webhook url - - Returns: - None - """ - formatted_message = "".join(formatted_messages) - log(formatted_message) - msg_ext = len(formatted_message) - if msg_ext > 2000: - log(f"** Message too long ({msg_ext} characters), will be split into multiple messages **") - # Split message into lines - lines = formatted_message.split("\n") - message_chunks = [] - chunk = "" - for line in lines: - if len(chunk) + len(line) + 1 > 2000: # +1 for the newline character - message_chunks.append(chunk) - chunk = "" - chunk += line + "\n" - message_chunks.append(chunk) # Append the last chunk - for chunk in message_chunks: - send_discord_message( - message=chunk, - webhook_url=webhook_url, - ) - else: - send_discord_message( - message=formatted_message, - webhook_url=webhook_url, - ) - - ############### # # Utilitary tasks diff --git a/pipelines/migration/utils.py b/pipelines/migration/utils.py index a44e27fd0..d7d7c41e1 100644 --- a/pipelines/migration/utils.py +++ b/pipelines/migration/utils.py @@ -37,6 +37,7 @@ from pytz import timezone from pipelines.constants import constants +from pipelines.utils.discord import send_discord_message from pipelines.utils.implicit_ftp import ImplicitFtpTls from pipelines.utils.secret import get_secret @@ -54,19 +55,6 @@ def set_default_parameters(flow: prefect.Flow, default_parameters: dict) -> pref return flow -def send_discord_message( - message: str, - webhook_url: str, -) -> None: - """ - Sends a message to a Discord channel. - """ - requests.post( - webhook_url, - data={"content": message}, - ) - - def log_critical(message: str, secret_path: str = constants.CRITICAL_SECRET_PATH.value): """Logs message to critical discord channel specified @@ -908,7 +896,12 @@ def get_raw_data_db( def save_treated_local_func( - filepath: str, data: pd.DataFrame, error: str, mode: str = "staging" + filepath: str, + data: pd.DataFrame, + error: str, + mode: str = "staging", + log_param: bool = True, + args: dict = None, ) -> str: """ Save treated file to CSV. @@ -918,15 +911,20 @@ def save_treated_local_func( data (pd.DataFrame): Dataframe to save error (str): Error catched during execution mode (str, optional): Folder to save locally, later folder which to upload to GCS. + log_param (bool, optional): Whether to log the path of the saved file. Defaults to True. + args (dict, optional): arguments to pass to pandas.to_csv. Defaults to None. Returns: str: Path to the saved file """ + if args is None: + args = {} _filepath = filepath.format(mode=mode, filetype="csv") Path(_filepath).parent.mkdir(parents=True, exist_ok=True) if error is None: - data.to_csv(_filepath, index=False) - log(f"Treated data saved to: {_filepath}") + data.to_csv(_filepath, index=False, **args) + if log_param: + log(f"Treated data saved to: {_filepath}") return _filepath diff --git a/pipelines/schedules.py b/pipelines/schedules.py index b48c594d2..2c06b0238 100644 --- a/pipelines/schedules.py +++ b/pipelines/schedules.py @@ -12,8 +12,32 @@ from pipelines.constants import constants from pipelines.constants import constants as emd_constants -cron_every_day_hour_7 = "0 7 * * *" -cron_every_day_hour_7_minute_10 = "10 7 * * *" + +def create_daily_cron(hour: int, minute: int = 0) -> str: + """ + Cria uma expressão cron de execução diário na hora e minuto definido + + Args: + hour (int): Hora de execução + minute (int): Minuto da execução + + Returns: + str: expressão cron + """ + return f"{minute} {hour} * * *" + + +def create_hourly_cron(minute: int = 0) -> str: + """ + Cria uma expressão cron de execução de hora em hora no minuto definido + + Args: + minute (int): Minuto da execução + + Returns: + str: expressão cron + """ + return f"{minute} * * * *" def generate_interval_schedule( diff --git a/pipelines/tasks.py b/pipelines/tasks.py index 40cf5991d..e6cb8bc58 100644 --- a/pipelines/tasks.py +++ b/pipelines/tasks.py @@ -5,16 +5,20 @@ import prefect from prefect import task +from prefect.engine.signals import FAIL +from prefect.triggers import all_finished from prefeitura_rio.pipelines_utils.logging import log from prefeitura_rio.pipelines_utils.prefect import get_flow_run_mode from pytz import timezone from pipelines.constants import constants +from pipelines.utils.discord import send_discord_message from pipelines.utils.prefect import FailedSubFlow, create_subflow_run, wait_subflow_run +from pipelines.utils.secret import get_secret from pipelines.utils.utils import convert_timezone -@task +@task(trigger=all_finished) def task_value_is_none(task_value: Union[Any, None]) -> bool: """Testa se o valor retornado por uma Task é None @@ -214,3 +218,37 @@ def run_subflow( if flag_failed_runs: raise FailedSubFlow(failed_message) + + +@task(trigger=all_finished) +def check_fail(results: Union[list, str]): + """ + Checks if any task result indicates failure. + + Args: + results (Union[list, str]): A result or list of results to check. + + Returns: + bool: True if any result is an instance of `FAIL`, otherwise False. + """ + if isinstance(results, list): + return any(isinstance(result, FAIL) for result in results) + else: + return isinstance(results, FAIL) + + +@task +def log_discord(message: str, key: str, dados_tag: bool = False): + """Logs message to discord channel specified + + Args: + message (str): Message to post on the channel + key (str): Key to secret path storing the webhook to channel. + dados_tag (bool): Indicates whether the message will tag the data team + """ + if dados_tag: + message = ( + message + f" - <@&{constants.OWNERS_DISCORD_MENTIONS.value['dados_smtr']['user_id']}>\n" + ) + url = get_secret(secret_path=constants.WEBHOOKS_SECRET_PATH.value)[key] + send_discord_message(message=message, webhook_url=url) diff --git a/pipelines/treatment/bilhetagem/CHANGELOG.md b/pipelines/treatment/bilhetagem/CHANGELOG.md new file mode 100644 index 000000000..54dce5cb9 --- /dev/null +++ b/pipelines/treatment/bilhetagem/CHANGELOG.md @@ -0,0 +1,7 @@ +# Changelog - bilhetagem + +## [1.0.0] - 2024-11-25 + +### Adicionado + +- Cria flow de tratamento da tabela auxiliar `aux_transacao_id_ordem_pagamento` \ No newline at end of file diff --git a/pipelines/treatment/bilhetagem/__init__.py b/pipelines/treatment/bilhetagem/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/pipelines/treatment/bilhetagem/constants.py b/pipelines/treatment/bilhetagem/constants.py new file mode 100644 index 000000000..47b951a65 --- /dev/null +++ b/pipelines/treatment/bilhetagem/constants.py @@ -0,0 +1,22 @@ +# -*- coding: utf-8 -*- +""" +Valores constantes para materialização dos dados de bilhetagem +""" + +from datetime import datetime +from enum import Enum + +from pipelines.schedules import create_daily_cron +from pipelines.treatment.templates.utils import DBTSelector + + +class constants(Enum): # pylint: disable=c0103 + """ + Valores constantes para materialização dos dados de bilhetagem + """ + + TRANSACAO_ORDEM_SELECTOR = DBTSelector( + name="transacao_ordem", + schedule_cron=create_daily_cron(hour=6, minute=10), + initial_datetime=datetime(2024, 11, 21, 0, 0, 0), + ) diff --git a/pipelines/treatment/bilhetagem/flows.py b/pipelines/treatment/bilhetagem/flows.py new file mode 100644 index 000000000..8c6db8c5f --- /dev/null +++ b/pipelines/treatment/bilhetagem/flows.py @@ -0,0 +1,36 @@ +# -*- coding: utf-8 -*- +""" +Flows de tratamento dos dados de bilhetagem + +DBT: 2024-11-27 2 +""" + +from pipelines.constants import constants as smtr_constants +from pipelines.migration.br_rj_riodejaneiro_bilhetagem.constants import ( + constants as old_constants, +) +from pipelines.schedules import create_daily_cron +from pipelines.treatment.bilhetagem.constants import constants +from pipelines.treatment.templates.flows import create_default_materialization_flow + +ordem_pagamento_materialize_params = ( + old_constants.BILHETAGEM_MATERIALIZACAO_ORDEM_PAGAMENTO_PARAMS.value +) + +TRANSACAO_ORDEM_MATERIALIZACAO = create_default_materialization_flow( + flow_name="transacao_ordem - materializacao", + selector=constants.TRANSACAO_ORDEM_SELECTOR.value, + agent_label=smtr_constants.RJ_SMTR_AGENT_LABEL.value, + wait=[ + { + "redis_key": f"{ordem_pagamento_materialize_params['dataset_id']}\ +.{ordem_pagamento_materialize_params['table_id']}", + "dict_key": "last_run_timestamp", + "datetime_format": "%Y-%m-%dT%H:%M:%S", + "delay_hours": ordem_pagamento_materialize_params["dbt_vars"]["date_range"][ + "delay_hours" + ], + "schedule_cron": create_daily_cron(hour=5), + } + ], +) diff --git a/pipelines/treatment/datario/CHANGELOG.md b/pipelines/treatment/datario/CHANGELOG.md new file mode 100644 index 000000000..200360ed6 --- /dev/null +++ b/pipelines/treatment/datario/CHANGELOG.md @@ -0,0 +1,7 @@ +# Changelog - datario + +## [1.0.0] - 2024-12-16 + +### Adicionado + +- Cria flow de tratamento do datario (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/361) \ No newline at end of file diff --git a/pipelines/treatment/datario/__init__.py b/pipelines/treatment/datario/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/pipelines/treatment/datario/constants.py b/pipelines/treatment/datario/constants.py new file mode 100644 index 000000000..17824f115 --- /dev/null +++ b/pipelines/treatment/datario/constants.py @@ -0,0 +1,22 @@ +# -*- coding: utf-8 -*- +""" +Valores constantes para materialização dos dados de monitoramento +""" + +from datetime import datetime +from enum import Enum + +from pipelines.schedules import create_daily_cron +from pipelines.treatment.templates.utils import DBTSelector + + +class constants(Enum): # pylint: disable=c0103 + """ + Valores constantes para materialização do datario + """ + + DATARIO_SELECTOR = DBTSelector( + name="datario", + schedule_cron=create_daily_cron(hour=0), + initial_datetime=datetime(2024, 12, 16), + ) diff --git a/pipelines/treatment/datario/flows.py b/pipelines/treatment/datario/flows.py new file mode 100644 index 000000000..219fa671a --- /dev/null +++ b/pipelines/treatment/datario/flows.py @@ -0,0 +1,16 @@ +# -*- coding: utf-8 -*- +""" +Flows de tratamento do datario +""" + + +from pipelines.constants import constants as smtr_constants +from pipelines.treatment.datario.constants import constants +from pipelines.treatment.templates.flows import create_default_materialization_flow + +VIAGEM_INFORMADA_MATERIALIZACAO = create_default_materialization_flow( + flow_name="datario - materializacao", + selector=constants.DATARIO_SELECTOR.value, + agent_label=smtr_constants.RJ_SMTR_AGENT_LABEL.value, + generate_schedule=False, +) diff --git a/pipelines/treatment/monitoramento/CHANGELOG.md b/pipelines/treatment/monitoramento/CHANGELOG.md index ee739f0c8..1ca5f2c64 100644 --- a/pipelines/treatment/monitoramento/CHANGELOG.md +++ b/pipelines/treatment/monitoramento/CHANGELOG.md @@ -1,5 +1,27 @@ # Changelog - monitoramento +## [1.1.2] - 2024-11-28 + +### Alterado + +- Altera hora de execução da validação de viagens e da materialização das viagens informadas (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/337) + +### Corrigido + +- Ajusta espera pelos dados de viagem informada na materialização da validação (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/337) + +## [1.1.1] - 2024-11-25 + +### Alterado + +- Substitui variavel de expressão cron pela função `create_daily_cron` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/333) + +## [1.1.0] - 2024-11-08 + +### Adicionado + +- Cria flow de tratamento de validação de viagens informadas (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/237) + ## [1.0.0] - 2024-10-21 ### Adicionado diff --git a/pipelines/treatment/monitoramento/constants.py b/pipelines/treatment/monitoramento/constants.py index 50873ecb2..09018e117 100644 --- a/pipelines/treatment/monitoramento/constants.py +++ b/pipelines/treatment/monitoramento/constants.py @@ -6,7 +6,7 @@ from datetime import datetime from enum import Enum -from pipelines.schedules import cron_every_day_hour_7_minute_10 +from pipelines.schedules import create_daily_cron from pipelines.treatment.templates.utils import DBTSelector @@ -17,6 +17,13 @@ class constants(Enum): # pylint: disable=c0103 VIAGEM_INFORMADA_SELECTOR = DBTSelector( name="viagem_informada", - schedule_cron=cron_every_day_hour_7_minute_10, + schedule_cron=create_daily_cron(hour=7, minute=30), initial_datetime=datetime(2024, 10, 16, 0, 0, 0), ) + + VIAGEM_VALIDACAO_SELECTOR = DBTSelector( + name="viagem_validacao", + schedule_cron=create_daily_cron(hour=8), + initial_datetime=datetime(2024, 10, 12, 0, 0, 0), + incremental_delay_hours=48, + ) diff --git a/pipelines/treatment/monitoramento/flows.py b/pipelines/treatment/monitoramento/flows.py index 80e591435..326651874 100644 --- a/pipelines/treatment/monitoramento/flows.py +++ b/pipelines/treatment/monitoramento/flows.py @@ -2,19 +2,64 @@ """ Flows de tratamento dos dados de monitoramento -DBT: 2024-10-23 +DBT: 2024-11-21 """ +from copy import deepcopy + from pipelines.capture.rioonibus.constants import ( constants as rioonibus_source_constants, ) from pipelines.constants import constants as smtr_constants +from pipelines.migration.br_rj_riodejaneiro_onibus_gps_zirix.constants import ( + constants as gps_zirix_constants, +) +from pipelines.schedules import create_hourly_cron from pipelines.treatment.monitoramento.constants import constants +from pipelines.treatment.planejamento.constants import ( + constants as planejamento_constants, +) from pipelines.treatment.templates.flows import create_default_materialization_flow +cron_every_hour_minute_6 = create_hourly_cron(minute=6) + VIAGEM_INFORMADA_MATERIALIZACAO = create_default_materialization_flow( flow_name="viagem_informada - materializacao", selector=constants.VIAGEM_INFORMADA_SELECTOR.value, agent_label=smtr_constants.RJ_SMTR_AGENT_LABEL.value, - wait=[rioonibus_source_constants.VIAGEM_INFORMADA_SOURCE.value], + wait=[ + planejamento_constants.PLANEJAMENTO_DIARIO_SELECTOR.value, + rioonibus_source_constants.VIAGEM_INFORMADA_SOURCE.value, + ], +) + +wait_viagem_informada = deepcopy(constants.VIAGEM_INFORMADA_SELECTOR.value) +wait_viagem_informada.incremental_delay_hours = ( + -constants.VIAGEM_VALIDACAO_SELECTOR.value.incremental_delay_hours +) + +VIAGEM_VALIDACAO_MATERIALIZACAO = create_default_materialization_flow( + flow_name="viagem_validacao - materializacao", + selector=constants.VIAGEM_VALIDACAO_SELECTOR.value, + agent_label=smtr_constants.RJ_SMTR_AGENT_LABEL.value, + wait=[ + wait_viagem_informada, + planejamento_constants.PLANEJAMENTO_DIARIO_SELECTOR.value, + { + "redis_key": f"{smtr_constants.GPS_SPPO_DATASET_ID.value}\ +.{smtr_constants.GPS_SPPO_TREATED_TABLE_ID.value}", + "dict_key": "last_run_timestamp", + "datetime_format": "%Y-%m-%dT%H:%M:%S", + "delay_hours": smtr_constants.GPS_SPPO_MATERIALIZE_DELAY_HOURS.value, + "schedule_cron": cron_every_hour_minute_6, + }, + { + "redis_key": f"{gps_zirix_constants.GPS_SPPO_ZIRIX_RAW_DATASET_ID.value}\ +.{smtr_constants.GPS_SPPO_TREATED_TABLE_ID.value}", + "dict_key": "last_run_timestamp", + "datetime_format": "%Y-%m-%dT%H:%M:%S", + "delay_hours": smtr_constants.GPS_SPPO_MATERIALIZE_DELAY_HOURS.value, + "schedule_cron": cron_every_hour_minute_6, + }, + ], ) diff --git a/pipelines/treatment/planejamento/CHANGELOG.md b/pipelines/treatment/planejamento/CHANGELOG.md new file mode 100644 index 000000000..9d467be29 --- /dev/null +++ b/pipelines/treatment/planejamento/CHANGELOG.md @@ -0,0 +1,13 @@ +# Changelog - planejamento + +## [1.0.1] - 2024-11-25 + +### Alterado + +- Substitui variavel de expressão cron pela função `create_daily_cron` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/333) + +## [1.0.0] - 2024-11-08 + +### Adicionado + +- Cria flow de tratamento diário de dados de planejamento (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/237) \ No newline at end of file diff --git a/pipelines/treatment/planejamento/__init__.py b/pipelines/treatment/planejamento/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/pipelines/treatment/planejamento/constants.py b/pipelines/treatment/planejamento/constants.py new file mode 100644 index 000000000..d83a73334 --- /dev/null +++ b/pipelines/treatment/planejamento/constants.py @@ -0,0 +1,22 @@ +# -*- coding: utf-8 -*- +""" +Valores constantes para materialização dos dados de planejamento +""" + +from datetime import datetime +from enum import Enum + +from pipelines.schedules import create_daily_cron +from pipelines.treatment.templates.utils import DBTSelector + + +class constants(Enum): # pylint: disable=c0103 + """ + Valores constantes para materialização dos dados de planejamento + """ + + PLANEJAMENTO_DIARIO_SELECTOR = DBTSelector( + name="planejamento_diario", + schedule_cron=create_daily_cron(hour=1), + initial_datetime=datetime(2024, 9, 1, 0, 0, 0), + ) diff --git a/pipelines/treatment/planejamento/flows.py b/pipelines/treatment/planejamento/flows.py new file mode 100644 index 000000000..82922bedc --- /dev/null +++ b/pipelines/treatment/planejamento/flows.py @@ -0,0 +1,16 @@ +# -*- coding: utf-8 -*- +""" +Flows de tratamento dos dados de planejamento + +DBT 2024-12-19 +""" + +from pipelines.constants import constants as smtr_constants +from pipelines.treatment.planejamento.constants import constants +from pipelines.treatment.templates.flows import create_default_materialization_flow + +PLANEJAMENTO_DIARIO_MATERIALIZACAO = create_default_materialization_flow( + flow_name="planejamento_diario - materializacao", + selector=constants.PLANEJAMENTO_DIARIO_SELECTOR.value, + agent_label=smtr_constants.RJ_SMTR_AGENT_LABEL.value, +) diff --git a/pipelines/treatment/templates/CHANGELOG.md b/pipelines/treatment/templates/CHANGELOG.md index 3050b5d77..fa7887177 100644 --- a/pipelines/treatment/templates/CHANGELOG.md +++ b/pipelines/treatment/templates/CHANGELOG.md @@ -1,5 +1,41 @@ # Changelog - treatment +## [1.0.4] - 2024-12-17 + +### Adicionado + +- Adiciona raise FAIL caso os testes do DBT falhem na task `dbt_data_quality_checks` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/256) +- Adiciona parametro `webhook_key` na task `dbt_data_quality_checks` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/256) + +## [1.0.3] - 2024-12-13 + +### Adicionado + +- Adiciona trigger `all_finished` na task `dbt_data_quality_checks` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) +- Adiciona task `log_discord` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) + +### Alterado + +- Refatora task `dbt_data_quality_checks` para informar o nome do teste quando a descrição não estiver definida no arquivo `constants.py` e garantir compatibilidade com diferentes parâmetros de data dos modelos DBT (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) + +### Corrigido + +- Corrige problema de construção incorreta de caminhos ao acessar arquivos na função `parse_dbt_test_output` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/359) + +## [1.0.2] - 2024-11-08 + +### Alterado + +- Adiciona lógica para verificar fontes de dados no padrão antigo (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/237) + +## [1.0.1] - 2024-10-29 + +### Adicionado + +- Adiciona as tasks `check_dbt_test_run`, `run_dbt_tests` e `dbt_data_quality_checks` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/288) +- Adiciona a função `parse_dbt_test_output` no `utils.py` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/288) + + ## [1.0.0] - 2024-10-21 ### Adicionado diff --git a/pipelines/treatment/templates/tasks.py b/pipelines/treatment/templates/tasks.py index f31ec1045..55785c407 100644 --- a/pipelines/treatment/templates/tasks.py +++ b/pipelines/treatment/templates/tasks.py @@ -1,12 +1,16 @@ # -*- coding: utf-8 -*- import time from datetime import datetime, timedelta -from typing import Union +from typing import Dict, List, Union import basedosdados as bd +import prefect import requests from prefect import task +from prefect.engine.signals import FAIL +from prefect.triggers import all_finished from prefeitura_rio.pipelines_utils.logging import log +from prefeitura_rio.pipelines_utils.redis_pal import get_redis_client from pytz import timezone from pipelines.constants import constants @@ -14,12 +18,15 @@ DBTSelector, IncompleteDataError, create_dataplex_log_message, + parse_dbt_test_output, send_dataplex_discord_message, ) from pipelines.utils.dataplex import DataQuality, DataQualityCheckArgs +from pipelines.utils.discord import format_send_discord_message from pipelines.utils.gcp.bigquery import SourceTable from pipelines.utils.prefect import flow_is_running_local, rename_current_flow_run -from pipelines.utils.utils import convert_timezone +from pipelines.utils.secret import get_secret +from pipelines.utils.utils import convert_timezone, cron_get_last_date # from pipelines.utils.utils import get_last_materialization_redis_key @@ -119,7 +126,7 @@ def wait_data_sources( env: str, datetime_start: datetime, datetime_end: datetime, - data_sources: list[Union[SourceTable, DBTSelector]], + data_sources: list[Union[SourceTable, DBTSelector, dict]], skip: bool, ): """ @@ -129,7 +136,7 @@ def wait_data_sources( env (str): prod ou dev datetime_start (datetime): Datetime inicial da materialização datetime_end (datetime): Datetime final da materialização - data_sources (list[Union[SourceTable, DBTSelector]]): Fontes de dados para esperar + data_sources (list[Union[SourceTable, DBTSelector, dict]]): Fontes de dados para esperar skip (bool): se a verificação deve ser pulada ou não """ if skip: @@ -146,10 +153,29 @@ def wait_data_sources( timestamp=datetime_end, retroactive_days=max(2, (datetime_end - datetime_start).days), ) + complete = len(uncaptured_timestamps) == 0 elif isinstance(ds, DBTSelector): name = f"{ds.name}" complete = ds.is_up_to_date(env=env, timestamp=datetime_end) + elif isinstance(ds, dict): + # source dicionário utilizado para compatibilização com flows antigos + name = ds["redis_key"] + redis_client = get_redis_client() + last_materialization = datetime.strptime( + redis_client.get(name)[ds["dict_key"]], + ds["datetime_format"], + ) + last_schedule = cron_get_last_date( + cron_expr=ds["schedule_cron"], + timestamp=datetime_end, + ) + last_materialization = convert_timezone(timestamp=last_materialization) + + complete = last_materialization >= last_schedule - timedelta( + hours=ds.get("delay_hours", 0) + ) + else: raise NotImplementedError(f"Espera por fontes do tipo {type(ds)} não implementada") @@ -338,3 +364,240 @@ def run_data_quality_checks( timestamp=datetime.now(tz=timezone(constants.TIMEZONE.value)), partitions=partitions, ) + + +@task(nout=3) +def check_dbt_test_run( + date_range_start: str, date_range_end: str, run_time: str +) -> tuple[bool, str, str]: + """ + Compares the specified run time with the start date's time component. + If they match, it calculates and returns the start and end date strings + for the previous day in ISO format. + + Args: + date_range_start (str): The start date of the range. + date_range_end (str): The end date of the range. + run_time (str): The time to check against in the format "HH:MM:SS". + + Returns: + Tuple[bool, str, str]: A tuple containing the following elements: + - bool: True if the run time matches the start date's time; otherwise, False. + - str: The start date of the previous day in ISO format if the time matches. + - str: The end date of the previous day in ISO format if the time matches. + """ + + datetime_start = datetime.fromisoformat(date_range_start) + datetime_end = datetime.fromisoformat(date_range_end) + + run_time = datetime.strptime(run_time, "%H:%M:%S").time() + + if datetime_start.time() == run_time: + datetime_start_str = (datetime_start - timedelta(days=1)).strftime("%Y-%m-%dT%H:%M:%S") + datetime_end_str = (datetime_end - timedelta(days=1)).strftime("%Y-%m-%dT%H:%M:%S") + return True, datetime_start_str, datetime_end_str + return False, None, None + + +@task +def run_dbt_tests( + dataset_id: str = None, + table_id: str = None, + model: str = None, + upstream: bool = None, + downstream: bool = None, + test_name: str = None, + exclude: str = None, + flags: str = None, + _vars: Union[dict, List[Dict]] = None, +) -> str: + """ + Runs a DBT test + + Args: + dataset_id (str, optional): Dataset ID of the dbt model. Defaults to None. + table_id (str, optional): Table ID of the dbt model. Defaults to None. + model (str, optional): model to be tested. Defaults to None. + upstream (bool, optional): If True, includes upstream models. Defaults to None. + downstream (bool, optional): If True, includes downstream models. Defaults to None. + test_name (str, optional): The name of the specific test to be executed. Defaults to None. + exclude (str, optional): Models to be excluded from the test execution. Defaults to None. + flags (str, optional): Additional flags for the `dbt test` command. Defaults to None. + _vars (Union[dict, List[Dict]], optional): Variables to pass to dbt. Defaults to None. + + Returns: + str: Logs resulting from the execution of the `dbt test` command. + """ + run_command = "dbt test" + + if not model: + model = dataset_id + if table_id: + model += f".{table_id}" + + if model: + run_command += " --select " + if upstream: + run_command += "+" + run_command += model + if downstream: + run_command += "+" + if test_name: + model += f",test_name:{test_name}" + + if exclude: + run_command += f" --exclude {exclude}" + + if _vars: + if isinstance(_vars, list): + vars_dict = {} + for elem in _vars: + vars_dict.update(elem) + vars_str = f'"{vars_dict}"' + run_command += f" --vars {vars_str}" + else: + vars_str = f'"{_vars}"' + run_command += f" --vars {vars_str}" + + if flags: + run_command += f" {flags}" + + root_path = get_root_path() + queries_dir = str(root_path / "queries") + + if flow_is_running_local(): + run_command += f' --profiles-dir "{queries_dir}/dev"' + + log(f"Running dbt with command: {run_command}") + dbt_task = DbtShellTask( + profiles_dir=queries_dir, + helper_script=f'cd "{queries_dir}"', + log_stderr=True, + return_all=True, + command=run_command, + ) + dbt_logs = dbt_task.run() + + log("\n".join(dbt_logs)) + dbt_logs = "\n".join(dbt_logs) + return dbt_logs + + +@task(trigger=all_finished) +def dbt_data_quality_checks( + dbt_logs: str, checks_list: dict, params: dict, webhook_key: str = "dataplex" +): + """ + Extracts the results of DBT tests and sends a message with the information to Discord. + + Args: + dbt_logs (str): Logs from DBT containing the test results. + checks_list (dict): Dictionary with the names of the tests and their descriptions. + date_range (dict): Dictionary representing a date range. + """ + if isinstance(dbt_logs, list): + dbt_logs = "\n".join(dbt_logs) + elif not isinstance(dbt_logs, str): + return + + checks_results = parse_dbt_test_output(dbt_logs) + + webhook_url = get_secret(secret_path=constants.WEBHOOKS_SECRET_PATH.value)[webhook_key] + + dados_tag = f" - <@&{constants.OWNERS_DISCORD_MENTIONS.value['dados_smtr']['user_id']}>\n" + + test_check = all(test["result"] == "PASS" for test in checks_results.values()) + + keys = [ + ("date_range_start", "date_range_end"), + ("start_date", "end_date"), + ("run_date", None), + ("data_versao_gtfs", None), + ] + + start_date = None + end_date = None + + for start_key, end_key in keys: + if start_key in params and "T" in params[start_key]: + start_date = params[start_key].split("T")[0] + + if end_key and end_key in params and "T" in params[end_key]: + end_date = params[end_key].split("T")[0] + + break + elif start_key in params: + start_date = params[start_key] + + if end_key and end_key in params: + end_date = params[end_key] + + date_range = ( + start_date + if not end_date + else (start_date if start_date == end_date else f"{start_date} a {end_date}") + ) + + if "(target='dev')" in dbt_logs or "(target='hmg')" in dbt_logs: + formatted_messages = [ + ":green_circle: " if test_check else ":red_circle: ", + f"**[DEV] Data Quality Checks - {prefect.context.get('flow_name')} - {date_range}**\n\n", # noqa + ] + else: + formatted_messages = [ + ":green_circle: " if test_check else ":red_circle: ", + f"**Data Quality Checks - {prefect.context.get('flow_name')} - {date_range}**\n\n", + ] + + for test_id, test_result in checks_results.items(): + parts = test_id.split("__") + + if len(parts) == 2: + table_name = parts[1] + else: + table_name = parts[2] + + matched_description = None + for existing_table_id, tests in checks_list.items(): + if table_name in existing_table_id: + for existing_test_id, test_info in tests.items(): + if test_id.endswith(existing_test_id): + matched_description = test_info.get("description", test_id) + break + if matched_description: + break + + test_id = test_id.replace("_", "\\_") + description = matched_description or f"Teste: {test_id}" + + test_message = ( + f'{":white_check_mark:" if test_result["result"] == "PASS" else ":x:"} ' + f"{description}" + ) + + table_exists = any(table_name in existing_table_id for existing_table_id in checks_list) + if table_exists or len(parts) >= 2: + table_header_exists = any(f"*{table_name}:*" in msg for msg in formatted_messages) + if not table_header_exists: + formatted_messages.append(f"*{table_name}:*\n") + + formatted_messages.append(test_message + "\n") + + formatted_messages.append("\n") + formatted_messages.append( + ":tada: **Status:** Sucesso" + if test_check + else ":warning: **Status:** Testes falharam. Necessidade de revisão dos dados finais!\n" + ) + + if not test_check: + formatted_messages.append(dados_tag) + + try: + format_send_discord_message(formatted_messages, webhook_url) + except Exception as e: + log(f"Falha ao enviar mensagem para o Discord: {e}", level="error") + raise + + if not test_check: + raise FAIL diff --git a/pipelines/treatment/templates/utils.py b/pipelines/treatment/templates/utils.py index 8058fcffd..4e130e5a4 100644 --- a/pipelines/treatment/templates/utils.py +++ b/pipelines/treatment/templates/utils.py @@ -1,7 +1,9 @@ # -*- coding: utf-8 -*- +import re from datetime import datetime, timedelta from google.cloud.dataplex_v1 import DataScanJob +from prefeitura_rio.pipelines_utils.io import get_root_path from prefeitura_rio.pipelines_utils.logging import log from prefeitura_rio.pipelines_utils.redis_pal import get_redis_client @@ -71,7 +73,7 @@ def get_last_materialized_datetime(self, env: str) -> datetime: ) ) - return last_datetime + return convert_timezone(timestamp=last_datetime) def get_datetime_end(self, timestamp: datetime) -> datetime: """ @@ -97,7 +99,9 @@ def is_up_to_date(self, env: str, timestamp: datetime) -> bool: bool: se está atualizado ou não """ last_materialization = self.get_last_materialized_datetime(env=env) + last_schedule = cron_get_last_date(cron_expr=self.schedule_cron, timestamp=timestamp) + return last_materialization >= last_schedule - timedelta(hours=self.incremental_delay_hours) def get_next_schedule_datetime(self, timestamp: datetime) -> datetime: @@ -205,3 +209,69 @@ def send_dataplex_discord_message( embed_messages=embed, timestamp=timestamp, ) + + +def parse_dbt_test_output(dbt_logs: str) -> dict: + """Parses DBT test output and returns a list of test results. + + Args: + dbt_logs: The DBT test output as a string. + + Returns: + A list of dictionaries, each representing a test result with the following keys: + - name: The test name. + - result: "PASS", "FAIL" or "ERROR". + - query: Query to see test failures. + - error: Message error. + """ + + # Remover sequências ANSI + dbt_logs = re.sub(r"\x1B[@-_][0-?]*[ -/]*[@-~]", "", dbt_logs) + + results = {} + result_pattern = r"\d+ of \d+ (PASS|FAIL|ERROR) (\d+ )?([\w._]+) .* \[(PASS|FAIL|ERROR) .*\]" + fail_pattern = r"Failure in test ([\w._]+) .*\n.*\n.*\n.* compiled Code at (.*)\n" + error_pattern = r"Error in test ([\w._]+) \(.*schema.yaml\)\n (.*)\n" + + root_path = get_root_path() + + for match in re.finditer(result_pattern, dbt_logs): + groups = match.groups() + test_name = groups[2] + results[test_name] = {"result": groups[3]} + + for match in re.finditer(fail_pattern, dbt_logs): + groups = match.groups() + test_name = groups[0] + file = groups[1] + filepath = f"{root_path}/queries/{file}" + + with open(filepath, "r") as arquivo: + query = arquivo.read() + + query = re.sub(r"\n+", "\n", query) + results[test_name]["query"] = query + + for match in re.finditer(error_pattern, dbt_logs): + groups = match.groups() + test_name = groups[0] + error = groups[1] + results[test_name]["error"] = error + + log_message = "" + for test, info in results.items(): + result = info["result"] + log_message += f"Test: {test} Status: {result}\n" + + if result == "FAIL": + log_message += "Query:\n" + log_message += f"{info['query']}\n" + + if result == "ERROR": + log_message += f"Error: {info['error']}\n" + + log_message += "\n" + + log(log_message) + + return results diff --git a/pipelines/utils/database.py b/pipelines/utils/database.py new file mode 100644 index 000000000..70da870e1 --- /dev/null +++ b/pipelines/utils/database.py @@ -0,0 +1,74 @@ +# -*- coding: utf-8 -*- +from prefeitura_rio.pipelines_utils.logging import log +from sqlalchemy import create_engine +from sqlalchemy.exc import OperationalError + +ENGINE_MAPPING = { + "mysql": {"driver": "pymysql", "port": "3306"}, + "postgresql": {"driver": "psycopg2", "port": "5432"}, +} + + +def create_database_url( + engine: str, + host: str, + user: str, + password: str, + database: str, +) -> str: + """ + Cria a URL para se conectar a um banco de dados + + Args: + engine (str): O banco de dados (postgresql ou mysql) + host (str): O host do banco de dados + user (str): O usuário para se conectar + password (str): A senha do usuário + database (str): O nome da base (schema) + + Returns: + str: a URL de conexão + """ + engine_info = ENGINE_MAPPING[engine] + driver = engine_info["driver"] + port = engine_info["port"] + return f"{engine}+{driver}://{user}:{password}@{host}:{port}/{database}" + + +def test_database_connection( + engine: str, + host: str, + user: str, + password: str, + database: str, +) -> tuple[bool, str]: + """ + Testa se é possível se conectar a um banco de dados + + Args: + engine (str): O banco de dados (postgresql ou mysql) + host (str): O host do banco de dados + user (str): O usuário para se conectar + password (str): A senha do usuário + database (str): O nome da base (schema) + + Returns: + bool: Se foi possível se conectar ou não + str: String do erro + """ + url = create_database_url( + engine=engine, + host=host, + user=user, + password=password, + database=database, + ) + connection = create_engine(url) + log(f"Tentando conexão com o banco de dados {database}") + try: + with connection.connect() as _: + log("Conexão bem-sucedida!") + return True, None + except OperationalError as e: + log("Conexão falhou", level="warning") + return False, str(e) diff --git a/pipelines/utils/discord.py b/pipelines/utils/discord.py index edbdf02c6..634db29e5 100644 --- a/pipelines/utils/discord.py +++ b/pipelines/utils/discord.py @@ -8,6 +8,19 @@ from pipelines.utils.secret import get_secret +def send_discord_message( + message: str, + webhook_url: str, +) -> None: + """ + Sends a message to a Discord channel. + """ + requests.post( + webhook_url, + data={"content": message}, + ) + + def send_discord_embed_message( webhook_secret_key: str, content: str, @@ -40,3 +53,41 @@ def send_discord_embed_message( log(response.text) log(response.status_code) + + +def format_send_discord_message(formatted_messages: list, webhook_url: str): + """ + Format and send a message to discord + + Args: + formatted_messages (list): The formatted messages + webhook_url (str): The webhook url + + Returns: + None + """ + formatted_message = "".join(formatted_messages) + log(formatted_message) + msg_ext = len(formatted_message) + if msg_ext > 2000: + log(f"** Message too long ({msg_ext} characters), will be split into multiple messages **") + # Split message into lines + lines = formatted_message.split("\n") + message_chunks = [] + chunk = "" + for line in lines: + if len(chunk) + len(line) + 1 > 2000: # +1 for the newline character + message_chunks.append(chunk) + chunk = "" + chunk += line + "\n" + message_chunks.append(chunk) # Append the last chunk + for chunk in message_chunks: + send_discord_message( + message=chunk, + webhook_url=webhook_url, + ) + else: + send_discord_message( + message=formatted_message, + webhook_url=webhook_url, + ) diff --git a/pipelines/utils/extractors/api.py b/pipelines/utils/extractors/api.py index 3b484129c..6738d87d8 100644 --- a/pipelines/utils/extractors/api.py +++ b/pipelines/utils/extractors/api.py @@ -21,7 +21,7 @@ def get_raw_api( Args: url (str): URL para fazer o request headers (Union[None, dict]): Headers para o request - params (Union[None, dict]): Paramêtros para o request + params (Union[None, dict]): Parâmetros para o request raw_filetype (str): Tipo de dado do arquivo (csv, json, ...) Returns: @@ -63,11 +63,12 @@ def get_raw_api_top_skip( max_page: int, ) -> list[dict]: """ - Função para extrair dados de API paginada do tipo top e skip. Deve + Função para extrair dados de API paginada do tipo top e skip. + Args: url (str): URL para fazer o request headers (Union[None, dict]): Headers para o request - params (Union[None, dict]): Paramêtros para o request + params (Union[None, dict]): Parâmetros para o request top_param_name (str): Nome do parâmetro que define a quantidade de registros em uma página skip_param_name (str): Nome do parâmetro que define quantos registros pular page_size (int): Número máximo de registros em uma página @@ -96,3 +97,27 @@ def get_raw_api_top_skip( params[skip_param_name] += page_size return data + + +def get_raw_api_params_list( + url: str, + params_list: list[dict], + headers: Union[None, dict] = None, +) -> list[dict]: + """ + Função para extrair dados de API agregando dados de multiplas + chamadas com parâmetros diferentes. + + Args: + url (str): URL para fazer os requests + params_list (list[dict): Lista de parâmetros para os requests + headers (Union[None, dict]): Headers para o request + + Returns: + list[dict]: Dados capturados da API + """ + data = [] + for params in params_list: + page_data = get_raw_api(url=url, headers=headers, params=params, raw_filetype="json") + data += page_data + return data diff --git a/pipelines/utils/extractors/db.py b/pipelines/utils/extractors/db.py index 10e92949e..055fe5f7d 100644 --- a/pipelines/utils/extractors/db.py +++ b/pipelines/utils/extractors/db.py @@ -4,6 +4,8 @@ from prefeitura_rio.pipelines_utils.logging import log from sqlalchemy import create_engine +from pipelines.utils.database import create_database_url + def get_raw_db( query: str, @@ -27,20 +29,21 @@ def get_raw_db( Returns: list[str]: Dados em formato JSON """ - engine_mapping = { - "mysql": {"driver": "pymysql", "port": "3306"}, - "postgresql": {"driver": "psycopg2", "port": "5432"}, - } - engine_details = engine_mapping[engine] - driver = engine_details["driver"] - port = engine_details["port"] - connection = create_engine(f"{engine}+{driver}://{user}:{password}@{host}:{port}/{database}") + url = create_database_url( + engine=engine, + host=host, + user=user, + password=password, + database=database, + ) + connection = create_engine(url) max_retries = 10 for retry in range(1, max_retries + 1): try: log(f"[ATTEMPT {retry}/{max_retries}]: {query}") - data = pd.read_sql(sql=query, con=connection).to_dict(orient="records") + data = pd.read_sql(sql=query, con=connection) + data = data.to_dict(orient="records") for d in data: for k, v in d.items(): if pd.isna(v): @@ -50,7 +53,7 @@ def get_raw_db( if retry == max_retries: raise err - return data + return data def get_raw_db_paginated( diff --git a/pipelines/utils/gcp/bigquery.py b/pipelines/utils/gcp/bigquery.py index 74bada52d..3d4b5b23e 100644 --- a/pipelines/utils/gcp/bigquery.py +++ b/pipelines/utils/gcp/bigquery.py @@ -1,6 +1,6 @@ # -*- coding: utf-8 -*- """Módulo para interagir com tabelas e datasets do BigQuery""" - +import csv from datetime import datetime, timedelta from typing import Callable @@ -14,7 +14,7 @@ from pipelines.constants import constants from pipelines.utils.gcp.base import GCPBase from pipelines.utils.gcp.storage import Storage -from pipelines.utils.utils import convert_timezone, cron_date_range +from pipelines.utils.utils import convert_timezone, cron_date_range, cron_get_last_date class Dataset(GCPBase): @@ -198,12 +198,12 @@ def __init__( # pylint: disable=R0913 self.pretreat_funcs = pretreat_funcs or [] self.schedule_cron = schedule_cron - def _create_table_schema(self) -> list[bigquery.SchemaField]: + def _create_table_schema(self, sample_filepath: str) -> list[bigquery.SchemaField]: """ Cria schema para os argumentos da criação de tabela externa no BQ """ log("Creating table schema...") - columns = self.primary_keys + ["content", "timestamp_captura"] + columns = next(csv.reader(open(sample_filepath, encoding="utf-8"))) log(f"Columns: {columns}") schema = [ @@ -212,7 +212,7 @@ def _create_table_schema(self) -> list[bigquery.SchemaField]: log("Schema created!") return schema - def _create_table_config(self) -> bigquery.ExternalConfig: + def _create_table_config(self, sample_filepath: str) -> bigquery.ExternalConfig: """ Cria as configurações da tabela externa no BQ """ @@ -220,19 +220,32 @@ def _create_table_config(self) -> bigquery.ExternalConfig: external_config.options.skip_leading_rows = 1 external_config.options.allow_quoted_newlines = True external_config.autodetect = False - external_config.schema = self._create_table_schema() + external_config.schema = self._create_table_schema(sample_filepath=sample_filepath) external_config.options.field_delimiter = "," external_config.options.allow_jagged_rows = False uri = f"gs://{self.bucket_name}/source/{self.dataset_id}/{self.table_id}/*" external_config.source_uris = uri hive_partitioning = HivePartitioningOptions() - hive_partitioning.mode = "STRINGS" + hive_partitioning.mode = "AUTO" hive_partitioning.source_uri_prefix = uri.replace("*", "") external_config.hive_partitioning = hive_partitioning return external_config + def get_last_scheduled_timestamp(self, timestamp: datetime) -> datetime: + """ + Retorna o último timestamp programado antes do timestamp fornecido + com base na expressão cron configurada + + Args: + timestamp (datetime): O timestamp de referência + + Returns: + datetime: O último timestamp programado com base na expressão cron + """ + return cron_get_last_date(cron_expr=self.schedule_cron, timestamp=timestamp) + def get_uncaptured_timestamps(self, timestamp: datetime, retroactive_days: int = 2) -> list: """ Retorna todas as timestamps não capturadas até um datetime @@ -263,7 +276,9 @@ def get_uncaptured_timestamps(self, timestamp: datetime, retroactive_days: int = files = files + [ convert_timezone(datetime.strptime(b.name.split("/")[-1], "%Y-%m-%d-%H-%M-%S.csv")) for b in st.bucket.list_blobs(prefix=prefix) + if ".csv" in b.name ] + return [d for d in full_range if d not in files][: self.max_recaptures] def upload_raw_file(self, raw_filepath: str, partition: str): @@ -281,12 +296,13 @@ def upload_raw_file(self, raw_filepath: str, partition: str): partition=partition, ) - def create(self, location: str = "US"): + def create(self, sample_filepath: str, location: str = "US"): """ Cria tabela externa do BQ Args: location (str): Localização do dataset + sample_filepath (str): Caminho com dados da tabela (usados para criar o schema) """ log(f"Creating External Table: {self.table_full_name}") dataset_obj = Dataset(dataset_id=self.dataset_id, env=self.env, location=location) @@ -296,7 +312,9 @@ def create(self, location: str = "US"): bq_table = bigquery.Table(self.table_full_name) bq_table.description = f"staging table for `{self.table_full_name}`" - bq_table.external_data_configuration = self._create_table_config() + bq_table.external_data_configuration = self._create_table_config( + sample_filepath=sample_filepath + ) client.create_table(bq_table) log("Table created!") diff --git a/pipelines/utils/pretreatment.py b/pipelines/utils/pretreatment.py index a404dac56..19f6f1bea 100644 --- a/pipelines/utils/pretreatment.py +++ b/pipelines/utils/pretreatment.py @@ -21,7 +21,10 @@ def transform_to_nested_structure(data: pd.DataFrame, primary_keys: list) -> pd. content_columns = [c for c in data.columns if c not in primary_keys] data["content"] = data.apply( lambda row: json.dumps( - row[content_columns].to_dict(), + { + key: value if not pd.isna(value) else None + for key, value in row[content_columns].to_dict().items() + } ), axis=1, ) diff --git a/poetry.lock b/poetry.lock index 9031d4c71..f62a65f5d 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 1.6.1 and should not be changed by hand. +# This file is automatically @generated by Poetry 1.7.1 and should not be changed by hand. [[package]] name = "agate" @@ -25,13 +25,13 @@ test = ["PyICU (>=2.4.2)", "coverage (>=3.7.1)", "cssselect (>=0.9.1)", "lxml (> [[package]] name = "async-timeout" -version = "4.0.3" +version = "5.0.1" description = "Timeout context manager for asyncio programs" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "async-timeout-4.0.3.tar.gz", hash = "sha256:4640d96be84d82d02ed59ea2b7105a0f7b33abe8703703cd0ab0bf87c427522f"}, - {file = "async_timeout-4.0.3-py3-none-any.whl", hash = "sha256:7405140ff1230c310e51dc27b3145b9092d659ce68ff733fb0cefe3ee42be028"}, + {file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"}, + {file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"}, ] [[package]] @@ -108,29 +108,33 @@ upload = ["gql (>=3.4,<4.0)", "requests-toolbelt (>=1,<2)"] [[package]] name = "black" -version = "23.11.0" +version = "23.12.1" description = "The uncompromising code formatter." optional = false python-versions = ">=3.8" files = [ - {file = "black-23.11.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:dbea0bb8575c6b6303cc65017b46351dc5953eea5c0a59d7b7e3a2d2f433a911"}, - {file = "black-23.11.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:412f56bab20ac85927f3a959230331de5614aecda1ede14b373083f62ec24e6f"}, - {file = "black-23.11.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d136ef5b418c81660ad847efe0e55c58c8208b77a57a28a503a5f345ccf01394"}, - {file = "black-23.11.0-cp310-cp310-win_amd64.whl", hash = "sha256:6c1cac07e64433f646a9a838cdc00c9768b3c362805afc3fce341af0e6a9ae9f"}, - {file = "black-23.11.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:cf57719e581cfd48c4efe28543fea3d139c6b6f1238b3f0102a9c73992cbb479"}, - {file = "black-23.11.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:698c1e0d5c43354ec5d6f4d914d0d553a9ada56c85415700b81dc90125aac244"}, - {file = "black-23.11.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:760415ccc20f9e8747084169110ef75d545f3b0932ee21368f63ac0fee86b221"}, - {file = "black-23.11.0-cp311-cp311-win_amd64.whl", hash = "sha256:58e5f4d08a205b11800332920e285bd25e1a75c54953e05502052738fe16b3b5"}, - {file = "black-23.11.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:45aa1d4675964946e53ab81aeec7a37613c1cb71647b5394779e6efb79d6d187"}, - {file = "black-23.11.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:4c44b7211a3a0570cc097e81135faa5f261264f4dfaa22bd5ee2875a4e773bd6"}, - {file = "black-23.11.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2a9acad1451632021ee0d146c8765782a0c3846e0e0ea46659d7c4f89d9b212b"}, - {file = "black-23.11.0-cp38-cp38-win_amd64.whl", hash = "sha256:fc7f6a44d52747e65a02558e1d807c82df1d66ffa80a601862040a43ec2e3142"}, - {file = "black-23.11.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:7f622b6822f02bfaf2a5cd31fdb7cd86fcf33dab6ced5185c35f5db98260b055"}, - {file = "black-23.11.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:250d7e60f323fcfc8ea6c800d5eba12f7967400eb6c2d21ae85ad31c204fb1f4"}, - {file = "black-23.11.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5133f5507007ba08d8b7b263c7aa0f931af5ba88a29beacc4b2dc23fcefe9c06"}, - {file = "black-23.11.0-cp39-cp39-win_amd64.whl", hash = "sha256:421f3e44aa67138ab1b9bfbc22ee3780b22fa5b291e4db8ab7eee95200726b07"}, - {file = "black-23.11.0-py3-none-any.whl", hash = "sha256:54caaa703227c6e0c87b76326d0862184729a69b73d3b7305b6288e1d830067e"}, - {file = "black-23.11.0.tar.gz", hash = "sha256:4c68855825ff432d197229846f971bc4d6666ce90492e5b02013bcaca4d9ab05"}, + {file = "black-23.12.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e0aaf6041986767a5e0ce663c7a2f0e9eaf21e6ff87a5f95cbf3675bfd4c41d2"}, + {file = "black-23.12.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c88b3711d12905b74206227109272673edce0cb29f27e1385f33b0163c414bba"}, + {file = "black-23.12.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a920b569dc6b3472513ba6ddea21f440d4b4c699494d2e972a1753cdc25df7b0"}, + {file = "black-23.12.1-cp310-cp310-win_amd64.whl", hash = "sha256:3fa4be75ef2a6b96ea8d92b1587dd8cb3a35c7e3d51f0738ced0781c3aa3a5a3"}, + {file = "black-23.12.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:8d4df77958a622f9b5a4c96edb4b8c0034f8434032ab11077ec6c56ae9f384ba"}, + {file = "black-23.12.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:602cfb1196dc692424c70b6507593a2b29aac0547c1be9a1d1365f0d964c353b"}, + {file = "black-23.12.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9c4352800f14be5b4864016882cdba10755bd50805c95f728011bcb47a4afd59"}, + {file = "black-23.12.1-cp311-cp311-win_amd64.whl", hash = "sha256:0808494f2b2df923ffc5723ed3c7b096bd76341f6213989759287611e9837d50"}, + {file = "black-23.12.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:25e57fd232a6d6ff3f4478a6fd0580838e47c93c83eaf1ccc92d4faf27112c4e"}, + {file = "black-23.12.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:2d9e13db441c509a3763a7a3d9a49ccc1b4e974a47be4e08ade2a228876500ec"}, + {file = "black-23.12.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6d1bd9c210f8b109b1762ec9fd36592fdd528485aadb3f5849b2740ef17e674e"}, + {file = "black-23.12.1-cp312-cp312-win_amd64.whl", hash = "sha256:ae76c22bde5cbb6bfd211ec343ded2163bba7883c7bc77f6b756a1049436fbb9"}, + {file = "black-23.12.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1fa88a0f74e50e4487477bc0bb900c6781dbddfdfa32691e780bf854c3b4a47f"}, + {file = "black-23.12.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:a4d6a9668e45ad99d2f8ec70d5c8c04ef4f32f648ef39048d010b0689832ec6d"}, + {file = "black-23.12.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b18fb2ae6c4bb63eebe5be6bd869ba2f14fd0259bda7d18a46b764d8fb86298a"}, + {file = "black-23.12.1-cp38-cp38-win_amd64.whl", hash = "sha256:c04b6d9d20e9c13f43eee8ea87d44156b8505ca8a3c878773f68b4e4812a421e"}, + {file = "black-23.12.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3e1b38b3135fd4c025c28c55ddfc236b05af657828a8a6abe5deec419a0b7055"}, + {file = "black-23.12.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:4f0031eaa7b921db76decd73636ef3a12c942ed367d8c3841a0739412b260a54"}, + {file = "black-23.12.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:97e56155c6b737854e60a9ab1c598ff2533d57e7506d97af5481141671abf3ea"}, + {file = "black-23.12.1-cp39-cp39-win_amd64.whl", hash = "sha256:dd15245c8b68fe2b6bd0f32c1556509d11bb33aec9b5d0866dd8e2ed3dba09c2"}, + {file = "black-23.12.1-py3-none-any.whl", hash = "sha256:78baad24af0f033958cad29731e27363183e140962595def56423e626f4bee3e"}, + {file = "black-23.12.1.tar.gz", hash = "sha256:4ce3ef14ebe8d9509188014d96af1c456a910d5b5cbf434a09fef7e024b3d0d5"}, ] [package.dependencies] @@ -144,7 +148,7 @@ typing-extensions = {version = ">=4.0.1", markers = "python_version < \"3.11\""} [package.extras] colorama = ["colorama (>=0.4.3)"] -d = ["aiohttp (>=3.7.4)"] +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)"] @@ -443,13 +447,13 @@ pyarrow = ">=3.0.0" [[package]] name = "dbt-bigquery" -version = "1.7.2" +version = "1.7.3" description = "The Bigquery adapter plugin for dbt" optional = false python-versions = ">=3.8" files = [ - {file = "dbt-bigquery-1.7.2.tar.gz", hash = "sha256:27c7f492f65ab5d1d43432a4467a436fc3637e3cb72c5b4ab07ddf7573c43596"}, - {file = "dbt_bigquery-1.7.2-py3-none-any.whl", hash = "sha256:75015755363d9e8b8cebe190d59a5e08375032b37bcfec41ec8753e7dea29f6e"}, + {file = "dbt-bigquery-1.7.3.tar.gz", hash = "sha256:8a132b614d548ea08edd0a7d23ba1c778ab77a8cbffbe177e50a14f1acfaae7f"}, + {file = "dbt_bigquery-1.7.3-py3-none-any.whl", hash = "sha256:c80665600bf2dc518b6d93f6e66e38f6642d7d794c7a19a290451f47a1dbd495"}, ] [package.dependencies] @@ -544,27 +548,28 @@ typing-extensions = ">=4.4,<5.0" [[package]] name = "dill" -version = "0.3.7" +version = "0.3.9" description = "serialize all of Python" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "dill-0.3.7-py3-none-any.whl", hash = "sha256:76b122c08ef4ce2eedcd4d1abd8e641114bfc6c2867f49f3c41facf65bf19f5e"}, - {file = "dill-0.3.7.tar.gz", hash = "sha256:cc1c8b182eb3013e24bd475ff2e9295af86c1a38eb1aff128dac8962a9ce3c03"}, + {file = "dill-0.3.9-py3-none-any.whl", hash = "sha256:468dff3b89520b474c0397703366b7b95eebe6303f108adf9b19da1f702be87a"}, + {file = "dill-0.3.9.tar.gz", hash = "sha256:81aa267dddf68cbfe8029c42ca9ec6a4ab3b22371d1c450abc54422577b4512c"}, ] [package.extras] graph = ["objgraph (>=1.7.2)"] +profile = ["gprof2dot (>=2022.7.29)"] [[package]] name = "distlib" -version = "0.3.7" +version = "0.3.9" description = "Distribution utilities" optional = false python-versions = "*" files = [ - {file = "distlib-0.3.7-py2.py3-none-any.whl", hash = "sha256:2e24928bc811348f0feb63014e97aaae3037f2cf48712d51ae61df7fd6075057"}, - {file = "distlib-0.3.7.tar.gz", hash = "sha256:9dafe54b34a028eafd95039d5e5d4851a13734540f1331060d31c9916e7147a8"}, + {file = "distlib-0.3.9-py2.py3-none-any.whl", hash = "sha256:47f8c22fd27c27e25a65601af709b38e4f0a45ea4fc2e710f65755fa8caaaf87"}, + {file = "distlib-0.3.9.tar.gz", hash = "sha256:a60f20dea646b8a33f3e7772f74dc0b2d0772d2837ee1342a00645c81edf9403"}, ] [[package]] @@ -618,30 +623,30 @@ websockets = ["websocket-client (>=1.3.0)"] [[package]] name = "et-xmlfile" -version = "1.1.0" +version = "2.0.0" description = "An implementation of lxml.xmlfile for the standard library" optional = false -python-versions = ">=3.6" +python-versions = ">=3.8" files = [ - {file = "et_xmlfile-1.1.0-py3-none-any.whl", hash = "sha256:a2ba85d1d6a74ef63837eed693bcb89c3f752169b0e3e7ae5b16ca5e1b3deada"}, - {file = "et_xmlfile-1.1.0.tar.gz", hash = "sha256:8eb9e2bc2f8c97e37a2dc85a09ecdcdec9d8a396530a6d5a33b30b9a92da0c5c"}, + {file = "et_xmlfile-2.0.0-py3-none-any.whl", hash = "sha256:7a91720bc756843502c3b7504c77b8fe44217c85c537d85037f0f536151b2caa"}, + {file = "et_xmlfile-2.0.0.tar.gz", hash = "sha256:dab3f4764309081ce75662649be815c4c9081e88f0837825f90fd28317d4da54"}, ] [[package]] name = "filelock" -version = "3.13.1" +version = "3.16.1" description = "A platform independent file lock." optional = false python-versions = ">=3.8" files = [ - {file = "filelock-3.13.1-py3-none-any.whl", hash = "sha256:57dbda9b35157b05fb3e58ee91448612eb674172fab98ee235ccb0b5bee19a1c"}, - {file = "filelock-3.13.1.tar.gz", hash = "sha256:521f5f56c50f8426f5e03ad3b281b490a87ef15bc6c526f168290f0c7148d44e"}, + {file = "filelock-3.16.1-py3-none-any.whl", hash = "sha256:2082e5703d51fbf98ea75855d9d5527e33d8ff23099bec374a134febee6946b0"}, + {file = "filelock-3.16.1.tar.gz", hash = "sha256:c249fbfcd5db47e5e2d6d62198e565475ee65e4831e2561c8e313fa7eb961435"}, ] [package.extras] -docs = ["furo (>=2023.9.10)", "sphinx (>=7.2.6)", "sphinx-autodoc-typehints (>=1.24)"] -testing = ["covdefaults (>=2.3)", "coverage (>=7.3.2)", "diff-cover (>=8)", "pytest (>=7.4.3)", "pytest-cov (>=4.1)", "pytest-mock (>=3.12)", "pytest-timeout (>=2.2)"] -typing = ["typing-extensions (>=4.8)"] +docs = ["furo (>=2024.8.6)", "sphinx (>=8.0.2)", "sphinx-autodoc-typehints (>=2.4.1)"] +testing = ["covdefaults (>=2.3)", "coverage (>=7.6.1)", "diff-cover (>=9.2)", "pytest (>=8.3.3)", "pytest-asyncio (>=0.24)", "pytest-cov (>=5)", "pytest-mock (>=3.14)", "pytest-timeout (>=2.3.1)", "virtualenv (>=20.26.4)"] +typing = ["typing-extensions (>=4.12.2)"] [[package]] name = "flake8" @@ -880,13 +885,13 @@ grpc = ["grpcio (>=1.38.0,<2.0dev)", "grpcio-status (>=1.38.0,<2.0.dev0)"] [[package]] name = "google-cloud-dataplex" -version = "1.12.3" +version = "1.13.0" description = "Google Cloud Dataplex API client library" optional = false python-versions = ">=3.7" files = [ - {file = "google-cloud-dataplex-1.12.3.tar.gz", hash = "sha256:c0692b6713fce6180df6d59bfb972d06f9fae264f33e7f2ba781d785f64ab8c3"}, - {file = "google_cloud_dataplex-1.12.3-py2.py3-none-any.whl", hash = "sha256:2ef7943b331ff254d2a8d2395c6a3b666837a73d32da1c4572a07453c0069dd3"}, + {file = "google-cloud-dataplex-1.13.0.tar.gz", hash = "sha256:afaa7163c06b33158be3286e9faf0c7334a1bf275dd19457f38ae7043499515f"}, + {file = "google_cloud_dataplex-1.13.0-py2.py3-none-any.whl", hash = "sha256:9c4bfadf01977aa93d904581587f57538788578b36969cc5d084efea15444e37"}, ] [package.dependencies] @@ -1090,69 +1095,84 @@ files = [ [[package]] name = "greenlet" -version = "3.0.3" +version = "3.1.1" description = "Lightweight in-process concurrent programming" optional = false python-versions = ">=3.7" files = [ - {file = "greenlet-3.0.3-cp310-cp310-macosx_11_0_universal2.whl", hash = "sha256:9da2bd29ed9e4f15955dd1595ad7bc9320308a3b766ef7f837e23ad4b4aac31a"}, - {file = "greenlet-3.0.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d353cadd6083fdb056bb46ed07e4340b0869c305c8ca54ef9da3421acbdf6881"}, - {file = "greenlet-3.0.3-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:dca1e2f3ca00b84a396bc1bce13dd21f680f035314d2379c4160c98153b2059b"}, - {file = "greenlet-3.0.3-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3ed7fb269f15dc662787f4119ec300ad0702fa1b19d2135a37c2c4de6fadfd4a"}, - {file = "greenlet-3.0.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd4f49ae60e10adbc94b45c0b5e6a179acc1736cf7a90160b404076ee283cf83"}, - {file = "greenlet-3.0.3-cp310-cp310-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:73a411ef564e0e097dbe7e866bb2dda0f027e072b04da387282b02c308807405"}, - {file = "greenlet-3.0.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:7f362975f2d179f9e26928c5b517524e89dd48530a0202570d55ad6ca5d8a56f"}, - {file = "greenlet-3.0.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:649dde7de1a5eceb258f9cb00bdf50e978c9db1b996964cd80703614c86495eb"}, - {file = "greenlet-3.0.3-cp310-cp310-win_amd64.whl", hash = "sha256:68834da854554926fbedd38c76e60c4a2e3198c6fbed520b106a8986445caaf9"}, - {file = "greenlet-3.0.3-cp311-cp311-macosx_11_0_universal2.whl", hash = "sha256:b1b5667cced97081bf57b8fa1d6bfca67814b0afd38208d52538316e9422fc61"}, - {file = "greenlet-3.0.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:52f59dd9c96ad2fc0d5724107444f76eb20aaccb675bf825df6435acb7703559"}, - {file = "greenlet-3.0.3-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:afaff6cf5200befd5cec055b07d1c0a5a06c040fe5ad148abcd11ba6ab9b114e"}, - {file = "greenlet-3.0.3-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:fe754d231288e1e64323cfad462fcee8f0288654c10bdf4f603a39ed923bef33"}, - {file = "greenlet-3.0.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2797aa5aedac23af156bbb5a6aa2cd3427ada2972c828244eb7d1b9255846379"}, - {file = "greenlet-3.0.3-cp311-cp311-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:b7f009caad047246ed379e1c4dbcb8b020f0a390667ea74d2387be2998f58a22"}, - {file = "greenlet-3.0.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:c5e1536de2aad7bf62e27baf79225d0d64360d4168cf2e6becb91baf1ed074f3"}, - {file = "greenlet-3.0.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:894393ce10ceac937e56ec00bb71c4c2f8209ad516e96033e4b3b1de270e200d"}, - {file = "greenlet-3.0.3-cp311-cp311-win_amd64.whl", hash = "sha256:1ea188d4f49089fc6fb283845ab18a2518d279c7cd9da1065d7a84e991748728"}, - {file = "greenlet-3.0.3-cp312-cp312-macosx_11_0_universal2.whl", hash = "sha256:70fb482fdf2c707765ab5f0b6655e9cfcf3780d8d87355a063547b41177599be"}, - {file = "greenlet-3.0.3-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d4d1ac74f5c0c0524e4a24335350edad7e5f03b9532da7ea4d3c54d527784f2e"}, - {file = "greenlet-3.0.3-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:149e94a2dd82d19838fe4b2259f1b6b9957d5ba1b25640d2380bea9c5df37676"}, - {file = "greenlet-3.0.3-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:15d79dd26056573940fcb8c7413d84118086f2ec1a8acdfa854631084393efcc"}, - {file = "greenlet-3.0.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:881b7db1ebff4ba09aaaeae6aa491daeb226c8150fc20e836ad00041bcb11230"}, - {file = "greenlet-3.0.3-cp312-cp312-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:fcd2469d6a2cf298f198f0487e0a5b1a47a42ca0fa4dfd1b6862c999f018ebbf"}, - {file = "greenlet-3.0.3-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:1f672519db1796ca0d8753f9e78ec02355e862d0998193038c7073045899f305"}, - {file = "greenlet-3.0.3-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:2516a9957eed41dd8f1ec0c604f1cdc86758b587d964668b5b196a9db5bfcde6"}, - {file = "greenlet-3.0.3-cp312-cp312-win_amd64.whl", hash = "sha256:bba5387a6975598857d86de9eac14210a49d554a77eb8261cc68b7d082f78ce2"}, - {file = "greenlet-3.0.3-cp37-cp37m-macosx_11_0_universal2.whl", hash = "sha256:5b51e85cb5ceda94e79d019ed36b35386e8c37d22f07d6a751cb659b180d5274"}, - {file = "greenlet-3.0.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:daf3cb43b7cf2ba96d614252ce1684c1bccee6b2183a01328c98d36fcd7d5cb0"}, - {file = "greenlet-3.0.3-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:99bf650dc5d69546e076f413a87481ee1d2d09aaaaaca058c9251b6d8c14783f"}, - {file = "greenlet-3.0.3-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2dd6e660effd852586b6a8478a1d244b8dc90ab5b1321751d2ea15deb49ed414"}, - {file = "greenlet-3.0.3-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e3391d1e16e2a5a1507d83e4a8b100f4ee626e8eca43cf2cadb543de69827c4c"}, - {file = "greenlet-3.0.3-cp37-cp37m-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:e1f145462f1fa6e4a4ae3c0f782e580ce44d57c8f2c7aae1b6fa88c0b2efdb41"}, - {file = "greenlet-3.0.3-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:1a7191e42732df52cb5f39d3527217e7ab73cae2cb3694d241e18f53d84ea9a7"}, - {file = "greenlet-3.0.3-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:0448abc479fab28b00cb472d278828b3ccca164531daab4e970a0458786055d6"}, - {file = "greenlet-3.0.3-cp37-cp37m-win32.whl", hash = "sha256:b542be2440edc2d48547b5923c408cbe0fc94afb9f18741faa6ae970dbcb9b6d"}, - {file = "greenlet-3.0.3-cp37-cp37m-win_amd64.whl", hash = "sha256:01bc7ea167cf943b4c802068e178bbf70ae2e8c080467070d01bfa02f337ee67"}, - {file = "greenlet-3.0.3-cp38-cp38-macosx_11_0_universal2.whl", hash = "sha256:1996cb9306c8595335bb157d133daf5cf9f693ef413e7673cb07e3e5871379ca"}, - {file = "greenlet-3.0.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3ddc0f794e6ad661e321caa8d2f0a55ce01213c74722587256fb6566049a8b04"}, - {file = "greenlet-3.0.3-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c9db1c18f0eaad2f804728c67d6c610778456e3e1cc4ab4bbd5eeb8e6053c6fc"}, - {file = "greenlet-3.0.3-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7170375bcc99f1a2fbd9c306f5be8764eaf3ac6b5cb968862cad4c7057756506"}, - {file = "greenlet-3.0.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6b66c9c1e7ccabad3a7d037b2bcb740122a7b17a53734b7d72a344ce39882a1b"}, - {file = "greenlet-3.0.3-cp38-cp38-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:098d86f528c855ead3479afe84b49242e174ed262456c342d70fc7f972bc13c4"}, - {file = "greenlet-3.0.3-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:81bb9c6d52e8321f09c3d165b2a78c680506d9af285bfccbad9fb7ad5a5da3e5"}, - {file = "greenlet-3.0.3-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:fd096eb7ffef17c456cfa587523c5f92321ae02427ff955bebe9e3c63bc9f0da"}, - {file = "greenlet-3.0.3-cp38-cp38-win32.whl", hash = "sha256:d46677c85c5ba00a9cb6f7a00b2bfa6f812192d2c9f7d9c4f6a55b60216712f3"}, - {file = "greenlet-3.0.3-cp38-cp38-win_amd64.whl", hash = "sha256:419b386f84949bf0e7c73e6032e3457b82a787c1ab4a0e43732898a761cc9dbf"}, - {file = "greenlet-3.0.3-cp39-cp39-macosx_11_0_universal2.whl", hash = "sha256:da70d4d51c8b306bb7a031d5cff6cc25ad253affe89b70352af5f1cb68e74b53"}, - {file = "greenlet-3.0.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:086152f8fbc5955df88382e8a75984e2bb1c892ad2e3c80a2508954e52295257"}, - {file = "greenlet-3.0.3-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d73a9fe764d77f87f8ec26a0c85144d6a951a6c438dfe50487df5595c6373eac"}, - {file = "greenlet-3.0.3-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b7dcbe92cc99f08c8dd11f930de4d99ef756c3591a5377d1d9cd7dd5e896da71"}, - {file = "greenlet-3.0.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1551a8195c0d4a68fac7a4325efac0d541b48def35feb49d803674ac32582f61"}, - {file = "greenlet-3.0.3-cp39-cp39-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:64d7675ad83578e3fc149b617a444fab8efdafc9385471f868eb5ff83e446b8b"}, - {file = "greenlet-3.0.3-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:b37eef18ea55f2ffd8f00ff8fe7c8d3818abd3e25fb73fae2ca3b672e333a7a6"}, - {file = "greenlet-3.0.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:77457465d89b8263bca14759d7c1684df840b6811b2499838cc5b040a8b5b113"}, - {file = "greenlet-3.0.3-cp39-cp39-win32.whl", hash = "sha256:57e8974f23e47dac22b83436bdcf23080ade568ce77df33159e019d161ce1d1e"}, - {file = "greenlet-3.0.3-cp39-cp39-win_amd64.whl", hash = "sha256:c5ee858cfe08f34712f548c3c363e807e7186f03ad7a5039ebadb29e8c6be067"}, - {file = "greenlet-3.0.3.tar.gz", hash = "sha256:43374442353259554ce33599da8b692d5aa96f8976d567d4badf263371fbe491"}, + {file = "greenlet-3.1.1-cp310-cp310-macosx_11_0_universal2.whl", hash = "sha256:0bbae94a29c9e5c7e4a2b7f0aae5c17e8e90acbfd3bf6270eeba60c39fce3563"}, + {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0fde093fb93f35ca72a556cf72c92ea3ebfda3d79fc35bb19fbe685853869a83"}, + {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:36b89d13c49216cadb828db8dfa6ce86bbbc476a82d3a6c397f0efae0525bdd0"}, + {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:94b6150a85e1b33b40b1464a3f9988dcc5251d6ed06842abff82e42632fac120"}, + {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:93147c513fac16385d1036b7e5b102c7fbbdb163d556b791f0f11eada7ba65dc"}, + {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:da7a9bff22ce038e19bf62c4dd1ec8391062878710ded0a845bcf47cc0200617"}, + {file = "greenlet-3.1.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:b2795058c23988728eec1f36a4e5e4ebad22f8320c85f3587b539b9ac84128d7"}, + {file = "greenlet-3.1.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:ed10eac5830befbdd0c32f83e8aa6288361597550ba669b04c48f0f9a2c843c6"}, + {file = "greenlet-3.1.1-cp310-cp310-win_amd64.whl", hash = "sha256:77c386de38a60d1dfb8e55b8c1101d68c79dfdd25c7095d51fec2dd800892b80"}, + {file = "greenlet-3.1.1-cp311-cp311-macosx_11_0_universal2.whl", hash = "sha256:e4d333e558953648ca09d64f13e6d8f0523fa705f51cae3f03b5983489958c70"}, + {file = "greenlet-3.1.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:09fc016b73c94e98e29af67ab7b9a879c307c6731a2c9da0db5a7d9b7edd1159"}, + {file = "greenlet-3.1.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d5e975ca70269d66d17dd995dafc06f1b06e8cb1ec1e9ed54c1d1e4a7c4cf26e"}, + {file = "greenlet-3.1.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3b2813dc3de8c1ee3f924e4d4227999285fd335d1bcc0d2be6dc3f1f6a318ec1"}, + {file = "greenlet-3.1.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e347b3bfcf985a05e8c0b7d462ba6f15b1ee1c909e2dcad795e49e91b152c383"}, + {file = "greenlet-3.1.1-cp311-cp311-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9e8f8c9cb53cdac7ba9793c276acd90168f416b9ce36799b9b885790f8ad6c0a"}, + {file = "greenlet-3.1.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:62ee94988d6b4722ce0028644418d93a52429e977d742ca2ccbe1c4f4a792511"}, + {file = "greenlet-3.1.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:1776fd7f989fc6b8d8c8cb8da1f6b82c5814957264d1f6cf818d475ec2bf6395"}, + {file = "greenlet-3.1.1-cp311-cp311-win_amd64.whl", hash = "sha256:48ca08c771c268a768087b408658e216133aecd835c0ded47ce955381105ba39"}, + {file = "greenlet-3.1.1-cp312-cp312-macosx_11_0_universal2.whl", hash = "sha256:4afe7ea89de619adc868e087b4d2359282058479d7cfb94970adf4b55284574d"}, + {file = "greenlet-3.1.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f406b22b7c9a9b4f8aa9d2ab13d6ae0ac3e85c9a809bd590ad53fed2bf70dc79"}, + {file = "greenlet-3.1.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c3a701fe5a9695b238503ce5bbe8218e03c3bcccf7e204e455e7462d770268aa"}, + {file = "greenlet-3.1.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2846930c65b47d70b9d178e89c7e1a69c95c1f68ea5aa0a58646b7a96df12441"}, + {file = "greenlet-3.1.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:99cfaa2110534e2cf3ba31a7abcac9d328d1d9f1b95beede58294a60348fba36"}, + {file = "greenlet-3.1.1-cp312-cp312-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:1443279c19fca463fc33e65ef2a935a5b09bb90f978beab37729e1c3c6c25fe9"}, + {file = "greenlet-3.1.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:b7cede291382a78f7bb5f04a529cb18e068dd29e0fb27376074b6d0317bf4dd0"}, + {file = "greenlet-3.1.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:23f20bb60ae298d7d8656c6ec6db134bca379ecefadb0b19ce6f19d1f232a942"}, + {file = "greenlet-3.1.1-cp312-cp312-win_amd64.whl", hash = "sha256:7124e16b4c55d417577c2077be379514321916d5790fa287c9ed6f23bd2ffd01"}, + {file = "greenlet-3.1.1-cp313-cp313-macosx_11_0_universal2.whl", hash = "sha256:05175c27cb459dcfc05d026c4232f9de8913ed006d42713cb8a5137bd49375f1"}, + {file = "greenlet-3.1.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:935e943ec47c4afab8965954bf49bfa639c05d4ccf9ef6e924188f762145c0ff"}, + {file = "greenlet-3.1.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:667a9706c970cb552ede35aee17339a18e8f2a87a51fba2ed39ceeeb1004798a"}, + {file = "greenlet-3.1.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b8a678974d1f3aa55f6cc34dc480169d58f2e6d8958895d68845fa4ab566509e"}, + {file = "greenlet-3.1.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:efc0f674aa41b92da8c49e0346318c6075d734994c3c4e4430b1c3f853e498e4"}, + {file = "greenlet-3.1.1-cp313-cp313-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:0153404a4bb921f0ff1abeb5ce8a5131da56b953eda6e14b88dc6bbc04d2049e"}, + {file = "greenlet-3.1.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:275f72decf9932639c1c6dd1013a1bc266438eb32710016a1c742df5da6e60a1"}, + {file = "greenlet-3.1.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:c4aab7f6381f38a4b42f269057aee279ab0fc7bf2e929e3d4abfae97b682a12c"}, + {file = "greenlet-3.1.1-cp313-cp313-win_amd64.whl", hash = "sha256:b42703b1cf69f2aa1df7d1030b9d77d3e584a70755674d60e710f0af570f3761"}, + {file = "greenlet-3.1.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f1695e76146579f8c06c1509c7ce4dfe0706f49c6831a817ac04eebb2fd02011"}, + {file = "greenlet-3.1.1-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7876452af029456b3f3549b696bb36a06db7c90747740c5302f74a9e9fa14b13"}, + {file = "greenlet-3.1.1-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4ead44c85f8ab905852d3de8d86f6f8baf77109f9da589cb4fa142bd3b57b475"}, + {file = "greenlet-3.1.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8320f64b777d00dd7ccdade271eaf0cad6636343293a25074cc5566160e4de7b"}, + {file = "greenlet-3.1.1-cp313-cp313t-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:6510bf84a6b643dabba74d3049ead221257603a253d0a9873f55f6a59a65f822"}, + {file = "greenlet-3.1.1-cp313-cp313t-musllinux_1_1_aarch64.whl", hash = "sha256:04b013dc07c96f83134b1e99888e7a79979f1a247e2a9f59697fa14b5862ed01"}, + {file = "greenlet-3.1.1-cp313-cp313t-musllinux_1_1_x86_64.whl", hash = "sha256:411f015496fec93c1c8cd4e5238da364e1da7a124bcb293f085bf2860c32c6f6"}, + {file = "greenlet-3.1.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:47da355d8687fd65240c364c90a31569a133b7b60de111c255ef5b606f2ae291"}, + {file = "greenlet-3.1.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:98884ecf2ffb7d7fe6bd517e8eb99d31ff7855a840fa6d0d63cd07c037f6a981"}, + {file = "greenlet-3.1.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f1d4aeb8891338e60d1ab6127af1fe45def5259def8094b9c7e34690c8858803"}, + {file = "greenlet-3.1.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:db32b5348615a04b82240cc67983cb315309e88d444a288934ee6ceaebcad6cc"}, + {file = "greenlet-3.1.1-cp37-cp37m-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:dcc62f31eae24de7f8dce72134c8651c58000d3b1868e01392baea7c32c247de"}, + {file = "greenlet-3.1.1-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:1d3755bcb2e02de341c55b4fca7a745a24a9e7212ac953f6b3a48d117d7257aa"}, + {file = "greenlet-3.1.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:b8da394b34370874b4572676f36acabac172602abf054cbc4ac910219f3340af"}, + {file = "greenlet-3.1.1-cp37-cp37m-win32.whl", hash = "sha256:a0dfc6c143b519113354e780a50381508139b07d2177cb6ad6a08278ec655798"}, + {file = "greenlet-3.1.1-cp37-cp37m-win_amd64.whl", hash = "sha256:54558ea205654b50c438029505def3834e80f0869a70fb15b871c29b4575ddef"}, + {file = "greenlet-3.1.1-cp38-cp38-macosx_11_0_universal2.whl", hash = "sha256:346bed03fe47414091be4ad44786d1bd8bef0c3fcad6ed3dee074a032ab408a9"}, + {file = "greenlet-3.1.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dfc59d69fc48664bc693842bd57acfdd490acafda1ab52c7836e3fc75c90a111"}, + {file = "greenlet-3.1.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d21e10da6ec19b457b82636209cbe2331ff4306b54d06fa04b7c138ba18c8a81"}, + {file = "greenlet-3.1.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:37b9de5a96111fc15418819ab4c4432e4f3c2ede61e660b1e33971eba26ef9ba"}, + {file = "greenlet-3.1.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6ef9ea3f137e5711f0dbe5f9263e8c009b7069d8a1acea822bd5e9dae0ae49c8"}, + {file = "greenlet-3.1.1-cp38-cp38-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:85f3ff71e2e60bd4b4932a043fbbe0f499e263c628390b285cb599154a3b03b1"}, + {file = "greenlet-3.1.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:95ffcf719966dd7c453f908e208e14cde192e09fde6c7186c8f1896ef778d8cd"}, + {file = "greenlet-3.1.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:03a088b9de532cbfe2ba2034b2b85e82df37874681e8c470d6fb2f8c04d7e4b7"}, + {file = "greenlet-3.1.1-cp38-cp38-win32.whl", hash = "sha256:8b8b36671f10ba80e159378df9c4f15c14098c4fd73a36b9ad715f057272fbef"}, + {file = "greenlet-3.1.1-cp38-cp38-win_amd64.whl", hash = "sha256:7017b2be767b9d43cc31416aba48aab0d2309ee31b4dbf10a1d38fb7972bdf9d"}, + {file = "greenlet-3.1.1-cp39-cp39-macosx_11_0_universal2.whl", hash = "sha256:396979749bd95f018296af156201d6211240e7a23090f50a8d5d18c370084dc3"}, + {file = "greenlet-3.1.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ca9d0ff5ad43e785350894d97e13633a66e2b50000e8a183a50a88d834752d42"}, + {file = "greenlet-3.1.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f6ff3b14f2df4c41660a7dec01045a045653998784bf8cfcb5a525bdffffbc8f"}, + {file = "greenlet-3.1.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:94ebba31df2aa506d7b14866fed00ac141a867e63143fe5bca82a8e503b36437"}, + {file = "greenlet-3.1.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:73aaad12ac0ff500f62cebed98d8789198ea0e6f233421059fa68a5aa7220145"}, + {file = "greenlet-3.1.1-cp39-cp39-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:63e4844797b975b9af3a3fb8f7866ff08775f5426925e1e0bbcfe7932059a12c"}, + {file = "greenlet-3.1.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:7939aa3ca7d2a1593596e7ac6d59391ff30281ef280d8632fa03d81f7c5f955e"}, + {file = "greenlet-3.1.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:d0028e725ee18175c6e422797c407874da24381ce0690d6b9396c204c7f7276e"}, + {file = "greenlet-3.1.1-cp39-cp39-win32.whl", hash = "sha256:5e06afd14cbaf9e00899fae69b24a32f2196c19de08fcb9f4779dd4f004e5e7c"}, + {file = "greenlet-3.1.1-cp39-cp39-win_amd64.whl", hash = "sha256:3319aa75e0e0639bc15ff54ca327e8dc7a6fe404003496e3c6925cd3142e0e22"}, + {file = "greenlet-3.1.1.tar.gz", hash = "sha256:4ce3ac6cdb6adf7946475d7ef31777c26d94bccc377e070a7986bd2d5c515467"}, ] [package.extras] @@ -1273,13 +1293,13 @@ pyparsing = {version = ">=2.4.2,<3.0.0 || >3.0.0,<3.0.1 || >3.0.1,<3.0.2 || >3.0 [[package]] name = "identify" -version = "2.5.32" +version = "2.6.2" description = "File identification library for Python" optional = false -python-versions = ">=3.8" +python-versions = ">=3.9" files = [ - {file = "identify-2.5.32-py2.py3-none-any.whl", hash = "sha256:0b7656ef6cba81664b783352c73f8c24b39cf82f926f78f4550eda928e5e0545"}, - {file = "identify-2.5.32.tar.gz", hash = "sha256:5d9979348ec1a21c768ae07e0a652924538e8bce67313a73cb0f681cf08ba407"}, + {file = "identify-2.6.2-py2.py3-none-any.whl", hash = "sha256:c097384259f49e372f4ea00a19719d95ae27dd5ff0fd77ad630aa891306b82f3"}, + {file = "identify-2.6.2.tar.gz", hash = "sha256:fab5c716c24d7a789775228823797296a2994b075fb6080ac83a102772a98cbd"}, ] [package.extras] @@ -1367,20 +1387,17 @@ six = "*" [[package]] name = "isort" -version = "5.12.0" +version = "5.13.2" description = "A Python utility / library to sort Python imports." optional = false python-versions = ">=3.8.0" files = [ - {file = "isort-5.12.0-py3-none-any.whl", hash = "sha256:f84c2818376e66cf843d497486ea8fed8700b340f308f076c6fb1229dff318b6"}, - {file = "isort-5.12.0.tar.gz", hash = "sha256:8bef7dde241278824a6d83f44a544709b065191b95b6e50894bdc722fcba0504"}, + {file = "isort-5.13.2-py3-none-any.whl", hash = "sha256:8ca5e72a8d85860d5a3fa69b8745237f2939afe12dbf656afbcb47fe72d947a6"}, + {file = "isort-5.13.2.tar.gz", hash = "sha256:48fdfcb9face5d58a4f6dde2e72a1fb8dcaf8ab26f95ab49fab84c2ddefb0109"}, ] [package.extras] -colors = ["colorama (>=0.4.3)"] -pipfile-deprecated-finder = ["pip-shims (>=0.5.2)", "pipreqs", "requirementslib"] -plugins = ["setuptools"] -requirements-deprecated-finder = ["pip-api", "pipreqs"] +colors = ["colorama (>=0.4.6)"] [[package]] name = "jaydebeapi" @@ -1587,16 +1604,6 @@ files = [ {file = "MarkupSafe-2.1.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:5bbe06f8eeafd38e5d0a4894ffec89378b6c6a625ff57e3028921f8ff59318ac"}, {file = "MarkupSafe-2.1.3-cp311-cp311-win32.whl", hash = "sha256:dd15ff04ffd7e05ffcb7fe79f1b98041b8ea30ae9234aed2a9168b5797c3effb"}, {file = "MarkupSafe-2.1.3-cp311-cp311-win_amd64.whl", hash = "sha256:134da1eca9ec0ae528110ccc9e48041e0828d79f24121a1a146161103c76e686"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:f698de3fd0c4e6972b92290a45bd9b1536bffe8c6759c62471efaa8acb4c37bc"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:aa57bd9cf8ae831a362185ee444e15a93ecb2e344c8e52e4d721ea3ab6ef1823"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ffcc3f7c66b5f5b7931a5aa68fc9cecc51e685ef90282f4a82f0f5e9b704ad11"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:47d4f1c5f80fc62fdd7777d0d40a2e9dda0a05883ab11374334f6c4de38adffd"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1f67c7038d560d92149c060157d623c542173016c4babc0c1913cca0564b9939"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:9aad3c1755095ce347e26488214ef77e0485a3c34a50c5a5e2471dff60b9dd9c"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:14ff806850827afd6b07a5f32bd917fb7f45b046ba40c57abdb636674a8b559c"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:8f9293864fe09b8149f0cc42ce56e3f0e54de883a9de90cd427f191c346eb2e1"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-win32.whl", hash = "sha256:715d3562f79d540f251b99ebd6d8baa547118974341db04f5ad06d5ea3eb8007"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-win_amd64.whl", hash = "sha256:1b8dd8c3fd14349433c79fa8abeb573a55fc0fdd769133baac1f5e07abf54aeb"}, {file = "MarkupSafe-2.1.3-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:8e254ae696c88d98da6555f5ace2279cf7cd5b3f52be2b5cf97feafe883b58d2"}, {file = "MarkupSafe-2.1.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cb0932dc158471523c9637e807d9bfb93e06a95cbf010f1a38b98623b929ef2b"}, {file = "MarkupSafe-2.1.3-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9402b03f1a1b4dc4c19845e5c749e3ab82d5078d16a2a4c2cd2df62d57bb0707"}, @@ -1792,13 +1799,13 @@ files = [ [[package]] name = "mslex" -version = "0.3.0" +version = "1.3.0" description = "shlex for windows" optional = false python-versions = ">=3.5" files = [ - {file = "mslex-0.3.0-py2.py3-none-any.whl", hash = "sha256:380cb14abf8fabf40e56df5c8b21a6d533dc5cbdcfe42406bbf08dda8f42e42a"}, - {file = "mslex-0.3.0.tar.gz", hash = "sha256:4a1ac3f25025cad78ad2fe499dd16d42759f7a3801645399cce5c404415daa97"}, + {file = "mslex-1.3.0-py3-none-any.whl", hash = "sha256:c7074b347201b3466fc077c5692fbce9b5f62a63a51f537a53fbbd02eff2eea4"}, + {file = "mslex-1.3.0.tar.gz", hash = "sha256:641c887d1d3db610eee2af37a8e5abda3f70b3006cdfd2d0d29dc0d1ae28a85d"}, ] [[package]] @@ -1915,18 +1922,15 @@ test = ["pytest (>=7.2)", "pytest-cov (>=4.0)"] [[package]] name = "nodeenv" -version = "1.8.0" +version = "1.9.1" description = "Node.js virtual environment builder" optional = false -python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,!=3.6.*" +python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,!=3.6.*,>=2.7" files = [ - {file = "nodeenv-1.8.0-py2.py3-none-any.whl", hash = "sha256:df865724bb3c3adc86b3876fa209771517b0cfe596beff01a92700e0e8be4cec"}, - {file = "nodeenv-1.8.0.tar.gz", hash = "sha256:d51e0c37e64fbf47d017feac3145cdbb58836d7eee8c6f6d3b6880c5456227d2"}, + {file = "nodeenv-1.9.1-py2.py3-none-any.whl", hash = "sha256:ba11c9782d29c27c70ffbdda2d7415098754709be8a7056d79a737cd901155c9"}, + {file = "nodeenv-1.9.1.tar.gz", hash = "sha256:6ec12890a2dab7946721edbfbcd91f3319c6ccc9aec47be7c7e6b7011ee6645f"}, ] -[package.dependencies] -setuptools = "*" - [[package]] name = "numpy" version = "1.26.2" @@ -1990,13 +1994,13 @@ signedtoken = ["cryptography (>=3.0.0)", "pyjwt (>=2.0.0,<3)"] [[package]] name = "openpyxl" -version = "3.1.4" +version = "3.1.5" description = "A Python library to read/write Excel 2010 xlsx/xlsm files" optional = false python-versions = ">=3.8" files = [ - {file = "openpyxl-3.1.4-py2.py3-none-any.whl", hash = "sha256:ec17f6483f2b8f7c88c57e5e5d3b0de0e3fb9ac70edc084d28e864f5b33bbefd"}, - {file = "openpyxl-3.1.4.tar.gz", hash = "sha256:8d2c8adf5d20d6ce8f9bca381df86b534835e974ed0156dacefa76f68c1d69fb"}, + {file = "openpyxl-3.1.5-py2.py3-none-any.whl", hash = "sha256:5282c12b107bffeef825f4617dc029afaf41d0ea60823bbb665ef3079dc79de2"}, + {file = "openpyxl-3.1.5.tar.gz", hash = "sha256:cf0e3cf56142039133628b5acffe8ef0c12bc902d2aadd3e0fe5878dc08d1050"}, ] [package.dependencies] @@ -2180,28 +2184,29 @@ pytzdata = ">=2020.1" [[package]] name = "platformdirs" -version = "4.0.0" -description = "A small Python package for determining appropriate platform-specific dirs, e.g. a \"user data dir\"." +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.7" +python-versions = ">=3.8" files = [ - {file = "platformdirs-4.0.0-py3-none-any.whl", hash = "sha256:118c954d7e949b35437270383a3f2531e99dd93cf7ce4dc8340d3356d30f173b"}, - {file = "platformdirs-4.0.0.tar.gz", hash = "sha256:cb633b2bcf10c51af60beb0ab06d2f1d69064b43abf4c185ca6b28865f3f9731"}, + {file = "platformdirs-4.3.6-py3-none-any.whl", hash = "sha256:73e575e1408ab8103900836b97580d5307456908a03e92031bab39e4554cc3fb"}, + {file = "platformdirs-4.3.6.tar.gz", hash = "sha256:357fb2acbc885b0419afd3ce3ed34564c13c9b95c89360cd9563f73aa5e2b907"}, ] [package.extras] -docs = ["furo (>=2023.7.26)", "proselint (>=0.13)", "sphinx (>=7.1.1)", "sphinx-autodoc-typehints (>=1.24)"] -test = ["appdirs (==1.4.4)", "covdefaults (>=2.3)", "pytest (>=7.4)", "pytest-cov (>=4.1)", "pytest-mock (>=3.11.1)"] +docs = ["furo (>=2024.8.6)", "proselint (>=0.14)", "sphinx (>=8.0.2)", "sphinx-autodoc-typehints (>=2.4)"] +test = ["appdirs (==1.4.4)", "covdefaults (>=2.3)", "pytest (>=8.3.2)", "pytest-cov (>=5)", "pytest-mock (>=3.14)"] +type = ["mypy (>=1.11.2)"] [[package]] name = "pre-commit" -version = "3.5.0" +version = "3.8.0" description = "A framework for managing and maintaining multi-language pre-commit hooks." optional = false -python-versions = ">=3.8" +python-versions = ">=3.9" files = [ - {file = "pre_commit-3.5.0-py2.py3-none-any.whl", hash = "sha256:841dc9aef25daba9a0238cd27984041fa0467b4199fc4852e27950664919f660"}, - {file = "pre_commit-3.5.0.tar.gz", hash = "sha256:5804465c675b659b0862f07907f96295d490822a450c4c40e747d0b1c6ebcb32"}, + {file = "pre_commit-3.8.0-py2.py3-none-any.whl", hash = "sha256:9a90a53bf82fdd8778d58085faf8d83df56e40dfe18f45b19446e26bf1b3a63f"}, + {file = "pre_commit-3.8.0.tar.gz", hash = "sha256:8bb6494d4a20423842e198980c9ecf9f96607a07ea29549e180eef9ae80fe7af"}, ] [package.dependencies] @@ -2386,83 +2391,78 @@ test = ["enum34", "ipaddress", "mock", "pywin32", "wmi"] [[package]] name = "psycopg2-binary" -version = "2.9.9" +version = "2.9.10" description = "psycopg2 - Python-PostgreSQL Database Adapter" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "psycopg2-binary-2.9.9.tar.gz", hash = "sha256:7f01846810177d829c7692f1f5ada8096762d9172af1b1a28d4ab5b77c923c1c"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c2470da5418b76232f02a2fcd2229537bb2d5a7096674ce61859c3229f2eb202"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c6af2a6d4b7ee9615cbb162b0738f6e1fd1f5c3eda7e5da17861eacf4c717ea7"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:75723c3c0fbbf34350b46a3199eb50638ab22a0228f93fb472ef4d9becc2382b"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:83791a65b51ad6ee6cf0845634859d69a038ea9b03d7b26e703f94c7e93dbcf9"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0ef4854e82c09e84cc63084a9e4ccd6d9b154f1dbdd283efb92ecd0b5e2b8c84"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ed1184ab8f113e8d660ce49a56390ca181f2981066acc27cf637d5c1e10ce46e"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:d2997c458c690ec2bc6b0b7ecbafd02b029b7b4283078d3b32a852a7ce3ddd98"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:b58b4710c7f4161b5e9dcbe73bb7c62d65670a87df7bcce9e1faaad43e715245"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:0c009475ee389757e6e34611d75f6e4f05f0cf5ebb76c6037508318e1a1e0d7e"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:8dbf6d1bc73f1d04ec1734bae3b4fb0ee3cb2a493d35ede9badbeb901fb40f6f"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-win32.whl", hash = "sha256:3f78fd71c4f43a13d342be74ebbc0666fe1f555b8837eb113cb7416856c79682"}, - {file = "psycopg2_binary-2.9.9-cp310-cp310-win_amd64.whl", hash = "sha256:876801744b0dee379e4e3c38b76fc89f88834bb15bf92ee07d94acd06ec890a0"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:ee825e70b1a209475622f7f7b776785bd68f34af6e7a46e2e42f27b659b5bc26"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:1ea665f8ce695bcc37a90ee52de7a7980be5161375d42a0b6c6abedbf0d81f0f"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:143072318f793f53819048fdfe30c321890af0c3ec7cb1dfc9cc87aa88241de2"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c332c8d69fb64979ebf76613c66b985414927a40f8defa16cf1bc028b7b0a7b0"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f7fc5a5acafb7d6ccca13bfa8c90f8c51f13d8fb87d95656d3950f0158d3ce53"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:977646e05232579d2e7b9c59e21dbe5261f403a88417f6a6512e70d3f8a046be"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:b6356793b84728d9d50ead16ab43c187673831e9d4019013f1402c41b1db9b27"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:bc7bb56d04601d443f24094e9e31ae6deec9ccb23581f75343feebaf30423359"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:77853062a2c45be16fd6b8d6de2a99278ee1d985a7bd8b103e97e41c034006d2"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:78151aa3ec21dccd5cdef6c74c3e73386dcdfaf19bced944169697d7ac7482fc"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-win32.whl", hash = "sha256:dc4926288b2a3e9fd7b50dc6a1909a13bbdadfc67d93f3374d984e56f885579d"}, - {file = "psycopg2_binary-2.9.9-cp311-cp311-win_amd64.whl", hash = "sha256:b76bedd166805480ab069612119ea636f5ab8f8771e640ae103e05a4aae3e417"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:8532fd6e6e2dc57bcb3bc90b079c60de896d2128c5d9d6f24a63875a95a088cf"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:b0605eaed3eb239e87df0d5e3c6489daae3f7388d455d0c0b4df899519c6a38d"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8f8544b092a29a6ddd72f3556a9fcf249ec412e10ad28be6a0c0d948924f2212"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2d423c8d8a3c82d08fe8af900ad5b613ce3632a1249fd6a223941d0735fce493"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2e5afae772c00980525f6d6ecf7cbca55676296b580c0e6abb407f15f3706996"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6e6f98446430fdf41bd36d4faa6cb409f5140c1c2cf58ce0bbdaf16af7d3f119"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:c77e3d1862452565875eb31bdb45ac62502feabbd53429fdc39a1cc341d681ba"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:cb16c65dcb648d0a43a2521f2f0a2300f40639f6f8c1ecbc662141e4e3e1ee07"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-musllinux_1_1_ppc64le.whl", hash = "sha256:911dda9c487075abd54e644ccdf5e5c16773470a6a5d3826fda76699410066fb"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:57fede879f08d23c85140a360c6a77709113efd1c993923c59fde17aa27599fe"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-win32.whl", hash = "sha256:64cf30263844fa208851ebb13b0732ce674d8ec6a0c86a4e160495d299ba3c93"}, - {file = "psycopg2_binary-2.9.9-cp312-cp312-win_amd64.whl", hash = "sha256:81ff62668af011f9a48787564ab7eded4e9fb17a4a6a74af5ffa6a457400d2ab"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:2293b001e319ab0d869d660a704942c9e2cce19745262a8aba2115ef41a0a42a"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:03ef7df18daf2c4c07e2695e8cfd5ee7f748a1d54d802330985a78d2a5a6dca9"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0a602ea5aff39bb9fac6308e9c9d82b9a35c2bf288e184a816002c9fae930b77"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8359bf4791968c5a78c56103702000105501adb557f3cf772b2c207284273984"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:275ff571376626195ab95a746e6a04c7df8ea34638b99fc11160de91f2fef503"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:f9b5571d33660d5009a8b3c25dc1db560206e2d2f89d3df1cb32d72c0d117d52"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:420f9bbf47a02616e8554e825208cb947969451978dceb77f95ad09c37791dae"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:4154ad09dac630a0f13f37b583eae260c6aa885d67dfbccb5b02c33f31a6d420"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:a148c5d507bb9b4f2030a2025c545fccb0e1ef317393eaba42e7eabd28eb6041"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-win32.whl", hash = "sha256:68fc1f1ba168724771e38bee37d940d2865cb0f562380a1fb1ffb428b75cb692"}, - {file = "psycopg2_binary-2.9.9-cp37-cp37m-win_amd64.whl", hash = "sha256:281309265596e388ef483250db3640e5f414168c5a67e9c665cafce9492eda2f"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:60989127da422b74a04345096c10d416c2b41bd7bf2a380eb541059e4e999980"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:246b123cc54bb5361588acc54218c8c9fb73068bf227a4a531d8ed56fa3ca7d6"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:34eccd14566f8fe14b2b95bb13b11572f7c7d5c36da61caf414d23b91fcc5d94"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:18d0ef97766055fec15b5de2c06dd8e7654705ce3e5e5eed3b6651a1d2a9a152"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d3f82c171b4ccd83bbaf35aa05e44e690113bd4f3b7b6cc54d2219b132f3ae55"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ead20f7913a9c1e894aebe47cccf9dc834e1618b7aa96155d2091a626e59c972"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:ca49a8119c6cbd77375ae303b0cfd8c11f011abbbd64601167ecca18a87e7cdd"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:323ba25b92454adb36fa425dc5cf6f8f19f78948cbad2e7bc6cdf7b0d7982e59"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:1236ed0952fbd919c100bc839eaa4a39ebc397ed1c08a97fc45fee2a595aa1b3"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:729177eaf0aefca0994ce4cffe96ad3c75e377c7b6f4efa59ebf003b6d398716"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-win32.whl", hash = "sha256:804d99b24ad523a1fe18cc707bf741670332f7c7412e9d49cb5eab67e886b9b5"}, - {file = "psycopg2_binary-2.9.9-cp38-cp38-win_amd64.whl", hash = "sha256:a6cdcc3ede532f4a4b96000b6362099591ab4a3e913d70bcbac2b56c872446f7"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:72dffbd8b4194858d0941062a9766f8297e8868e1dd07a7b36212aaa90f49472"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:30dcc86377618a4c8f3b72418df92e77be4254d8f89f14b8e8f57d6d43603c0f"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:31a34c508c003a4347d389a9e6fcc2307cc2150eb516462a7a17512130de109e"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:15208be1c50b99203fe88d15695f22a5bed95ab3f84354c494bcb1d08557df67"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1873aade94b74715be2246321c8650cabf5a0d098a95bab81145ffffa4c13876"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3a58c98a7e9c021f357348867f537017057c2ed7f77337fd914d0bedb35dace7"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:4686818798f9194d03c9129a4d9a702d9e113a89cb03bffe08c6cf799e053291"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:ebdc36bea43063116f0486869652cb2ed7032dbc59fbcb4445c4862b5c1ecf7f"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:ca08decd2697fdea0aea364b370b1249d47336aec935f87b8bbfd7da5b2ee9c1"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:ac05fb791acf5e1a3e39402641827780fe44d27e72567a000412c648a85ba860"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-win32.whl", hash = "sha256:9dba73be7305b399924709b91682299794887cbbd88e38226ed9f6712eabee90"}, - {file = "psycopg2_binary-2.9.9-cp39-cp39-win_amd64.whl", hash = "sha256:f7ae5d65ccfbebdfa761585228eb4d0df3a8b15cfb53bd953e713e09fbb12957"}, + {file = "psycopg2-binary-2.9.10.tar.gz", hash = "sha256:4b3df0e6990aa98acda57d983942eff13d824135fe2250e6522edaa782a06de2"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:0ea8e3d0ae83564f2fc554955d327fa081d065c8ca5cc6d2abb643e2c9c1200f"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-macosx_14_0_arm64.whl", hash = "sha256:3e9c76f0ac6f92ecfc79516a8034a544926430f7b080ec5a0537bca389ee0906"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2ad26b467a405c798aaa1458ba09d7e2b6e5f96b1ce0ac15d82fd9f95dc38a92"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:270934a475a0e4b6925b5f804e3809dd5f90f8613621d062848dd82f9cd62007"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:48b338f08d93e7be4ab2b5f1dbe69dc5e9ef07170fe1f86514422076d9c010d0"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7f4152f8f76d2023aac16285576a9ecd2b11a9895373a1f10fd9db54b3ff06b4"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:32581b3020c72d7a421009ee1c6bf4a131ef5f0a968fab2e2de0c9d2bb4577f1"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:2ce3e21dc3437b1d960521eca599d57408a695a0d3c26797ea0f72e834c7ffe5"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:e984839e75e0b60cfe75e351db53d6db750b00de45644c5d1f7ee5d1f34a1ce5"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:3c4745a90b78e51d9ba06e2088a2fe0c693ae19cc8cb051ccda44e8df8a6eb53"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-win32.whl", hash = "sha256:e5720a5d25e3b99cd0dc5c8a440570469ff82659bb09431c1439b92caf184d3b"}, + {file = "psycopg2_binary-2.9.10-cp310-cp310-win_amd64.whl", hash = "sha256:3c18f74eb4386bf35e92ab2354a12c17e5eb4d9798e4c0ad3a00783eae7cd9f1"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:04392983d0bb89a8717772a193cfaac58871321e3ec69514e1c4e0d4957b5aff"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-macosx_14_0_arm64.whl", hash = "sha256:1a6784f0ce3fec4edc64e985865c17778514325074adf5ad8f80636cd029ef7c"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b5f86c56eeb91dc3135b3fd8a95dc7ae14c538a2f3ad77a19645cf55bab1799c"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2b3d2491d4d78b6b14f76881905c7a8a8abcf974aad4a8a0b065273a0ed7a2cb"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2286791ececda3a723d1910441c793be44625d86d1a4e79942751197f4d30341"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:512d29bb12608891e349af6a0cccedce51677725a921c07dba6342beaf576f9a"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:5a507320c58903967ef7384355a4da7ff3f28132d679aeb23572753cbf2ec10b"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:6d4fa1079cab9018f4d0bd2db307beaa612b0d13ba73b5c6304b9fe2fb441ff7"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:851485a42dbb0bdc1edcdabdb8557c09c9655dfa2ca0460ff210522e073e319e"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:35958ec9e46432d9076286dda67942ed6d968b9c3a6a2fd62b48939d1d78bf68"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-win32.whl", hash = "sha256:ecced182e935529727401b24d76634a357c71c9275b356efafd8a2a91ec07392"}, + {file = "psycopg2_binary-2.9.10-cp311-cp311-win_amd64.whl", hash = "sha256:ee0e8c683a7ff25d23b55b11161c2663d4b099770f6085ff0a20d4505778d6b4"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:880845dfe1f85d9d5f7c412efea7a08946a46894537e4e5d091732eb1d34d9a0"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-macosx_14_0_arm64.whl", hash = "sha256:9440fa522a79356aaa482aa4ba500b65f28e5d0e63b801abf6aa152a29bd842a"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e3923c1d9870c49a2d44f795df0c889a22380d36ef92440ff618ec315757e539"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7b2c956c028ea5de47ff3a8d6b3cc3330ab45cf0b7c3da35a2d6ff8420896526"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f758ed67cab30b9a8d2833609513ce4d3bd027641673d4ebc9c067e4d208eec1"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8cd9b4f2cfab88ed4a9106192de509464b75a906462fb846b936eabe45c2063e"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6dc08420625b5a20b53551c50deae6e231e6371194fa0651dbe0fb206452ae1f"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:d7cd730dfa7c36dbe8724426bf5612798734bff2d3c3857f36f2733f5bfc7c00"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:155e69561d54d02b3c3209545fb08938e27889ff5a10c19de8d23eb5a41be8a5"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:c3cc28a6fd5a4a26224007712e79b81dbaee2ffb90ff406256158ec4d7b52b47"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-win32.whl", hash = "sha256:ec8a77f521a17506a24a5f626cb2aee7850f9b69a0afe704586f63a464f3cd64"}, + {file = "psycopg2_binary-2.9.10-cp312-cp312-win_amd64.whl", hash = "sha256:18c5ee682b9c6dd3696dad6e54cc7ff3a1a9020df6a5c0f861ef8bfd338c3ca0"}, + {file = "psycopg2_binary-2.9.10-cp313-cp313-macosx_12_0_x86_64.whl", hash = "sha256:26540d4a9a4e2b096f1ff9cce51253d0504dca5a85872c7f7be23be5a53eb18d"}, + {file = "psycopg2_binary-2.9.10-cp313-cp313-macosx_14_0_arm64.whl", hash = "sha256:e217ce4d37667df0bc1c397fdcd8de5e81018ef305aed9415c3b093faaeb10fb"}, + {file = "psycopg2_binary-2.9.10-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:245159e7ab20a71d989da00f280ca57da7641fa2cdcf71749c193cea540a74f7"}, + {file = "psycopg2_binary-2.9.10-cp313-cp313-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3c4ded1a24b20021ebe677b7b08ad10bf09aac197d6943bfe6fec70ac4e4690d"}, + {file = "psycopg2_binary-2.9.10-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3abb691ff9e57d4a93355f60d4f4c1dd2d68326c968e7db17ea96df3c023ef73"}, + {file = "psycopg2_binary-2.9.10-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8608c078134f0b3cbd9f89b34bd60a943b23fd33cc5f065e8d5f840061bd0673"}, + {file = "psycopg2_binary-2.9.10-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:230eeae2d71594103cd5b93fd29d1ace6420d0b86f4778739cb1a5a32f607d1f"}, + {file = "psycopg2_binary-2.9.10-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:bb89f0a835bcfc1d42ccd5f41f04870c1b936d8507c6df12b7737febc40f0909"}, + {file = "psycopg2_binary-2.9.10-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:f0c2d907a1e102526dd2986df638343388b94c33860ff3bbe1384130828714b1"}, + {file = "psycopg2_binary-2.9.10-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:f8157bed2f51db683f31306aa497311b560f2265998122abe1dce6428bd86567"}, + {file = "psycopg2_binary-2.9.10-cp38-cp38-macosx_12_0_x86_64.whl", hash = "sha256:eb09aa7f9cecb45027683bb55aebaaf45a0df8bf6de68801a6afdc7947bb09d4"}, + {file = "psycopg2_binary-2.9.10-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b73d6d7f0ccdad7bc43e6d34273f70d587ef62f824d7261c4ae9b8b1b6af90e8"}, + {file = "psycopg2_binary-2.9.10-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ce5ab4bf46a211a8e924d307c1b1fcda82368586a19d0a24f8ae166f5c784864"}, + {file = "psycopg2_binary-2.9.10-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:056470c3dc57904bbf63d6f534988bafc4e970ffd50f6271fc4ee7daad9498a5"}, + {file = "psycopg2_binary-2.9.10-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:73aa0e31fa4bb82578f3a6c74a73c273367727de397a7a0f07bd83cbea696baa"}, + {file = "psycopg2_binary-2.9.10-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:8de718c0e1c4b982a54b41779667242bc630b2197948405b7bd8ce16bcecac92"}, + {file = "psycopg2_binary-2.9.10-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:5c370b1e4975df846b0277b4deba86419ca77dbc25047f535b0bb03d1a544d44"}, + {file = "psycopg2_binary-2.9.10-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:ffe8ed017e4ed70f68b7b371d84b7d4a790368db9203dfc2d222febd3a9c8863"}, + {file = "psycopg2_binary-2.9.10-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:8aecc5e80c63f7459a1a2ab2c64df952051df196294d9f739933a9f6687e86b3"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:7a813c8bdbaaaab1f078014b9b0b13f5de757e2b5d9be6403639b298a04d218b"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d00924255d7fc916ef66e4bf22f354a940c67179ad3fd7067d7a0a9c84d2fbfc"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7559bce4b505762d737172556a4e6ea8a9998ecac1e39b5233465093e8cee697"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e8b58f0a96e7a1e341fc894f62c1177a7c83febebb5ff9123b579418fdc8a481"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6b269105e59ac96aba877c1707c600ae55711d9dcd3fc4b5012e4af68e30c648"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:79625966e176dc97ddabc142351e0409e28acf4660b88d1cf6adb876d20c490d"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:8aabf1c1a04584c168984ac678a668094d831f152859d06e055288fa515e4d30"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:19721ac03892001ee8fdd11507e6a2e01f4e37014def96379411ca99d78aeb2c"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:7f5d859928e635fa3ce3477704acee0f667b3a3d3e4bb109f2b18d4005f38287"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-win32.whl", hash = "sha256:3216ccf953b3f267691c90c6fe742e45d890d8272326b4a8b20850a03d05b7b8"}, + {file = "psycopg2_binary-2.9.10-cp39-cp39-win_amd64.whl", hash = "sha256:30e34c4e97964805f715206c7b789d54a78b70f3ff19fbe590104b71c45600e5"}, ] [[package]] @@ -2682,13 +2682,13 @@ files = [ [[package]] name = "pymysql" -version = "1.1.0" +version = "1.1.1" description = "Pure Python MySQL Driver" optional = false python-versions = ">=3.7" files = [ - {file = "PyMySQL-1.1.0-py3-none-any.whl", hash = "sha256:8969ec6d763c856f7073c4c64662882675702efcb114b4bcbb955aea3a069fa7"}, - {file = "PyMySQL-1.1.0.tar.gz", hash = "sha256:4f13a7df8bf36a51e81dd9f3605fede45a4878fe02f9236349fd82a3f0612f96"}, + {file = "PyMySQL-1.1.1-py3-none-any.whl", hash = "sha256:4de15da4c61dc132f4fb9ab763063e693d521a80fd0e87943b9a453dd4c19d6c"}, + {file = "pymysql-1.1.1.tar.gz", hash = "sha256:e127611aaf2b417403c60bf4dc570124aeb4a57f5f37b8e95ae399a42f904cd0"}, ] [package.extras] @@ -2833,25 +2833,29 @@ files = [ [[package]] name = "pywin32" -version = "306" +version = "308" description = "Python for Window Extensions" optional = false python-versions = "*" files = [ - {file = "pywin32-306-cp310-cp310-win32.whl", hash = "sha256:06d3420a5155ba65f0b72f2699b5bacf3109f36acbe8923765c22938a69dfc8d"}, - {file = "pywin32-306-cp310-cp310-win_amd64.whl", hash = "sha256:84f4471dbca1887ea3803d8848a1616429ac94a4a8d05f4bc9c5dcfd42ca99c8"}, - {file = "pywin32-306-cp311-cp311-win32.whl", hash = "sha256:e65028133d15b64d2ed8f06dd9fbc268352478d4f9289e69c190ecd6818b6407"}, - {file = "pywin32-306-cp311-cp311-win_amd64.whl", hash = "sha256:a7639f51c184c0272e93f244eb24dafca9b1855707d94c192d4a0b4c01e1100e"}, - {file = "pywin32-306-cp311-cp311-win_arm64.whl", hash = "sha256:70dba0c913d19f942a2db25217d9a1b726c278f483a919f1abfed79c9cf64d3a"}, - {file = "pywin32-306-cp312-cp312-win32.whl", hash = "sha256:383229d515657f4e3ed1343da8be101000562bf514591ff383ae940cad65458b"}, - {file = "pywin32-306-cp312-cp312-win_amd64.whl", hash = "sha256:37257794c1ad39ee9be652da0462dc2e394c8159dfd913a8a4e8eb6fd346da0e"}, - {file = "pywin32-306-cp312-cp312-win_arm64.whl", hash = "sha256:5821ec52f6d321aa59e2db7e0a35b997de60c201943557d108af9d4ae1ec7040"}, - {file = "pywin32-306-cp37-cp37m-win32.whl", hash = "sha256:1c73ea9a0d2283d889001998059f5eaaba3b6238f767c9cf2833b13e6a685f65"}, - {file = "pywin32-306-cp37-cp37m-win_amd64.whl", hash = "sha256:72c5f621542d7bdd4fdb716227be0dd3f8565c11b280be6315b06ace35487d36"}, - {file = "pywin32-306-cp38-cp38-win32.whl", hash = "sha256:e4c092e2589b5cf0d365849e73e02c391c1349958c5ac3e9d5ccb9a28e017b3a"}, - {file = "pywin32-306-cp38-cp38-win_amd64.whl", hash = "sha256:e8ac1ae3601bee6ca9f7cb4b5363bf1c0badb935ef243c4733ff9a393b1690c0"}, - {file = "pywin32-306-cp39-cp39-win32.whl", hash = "sha256:e25fd5b485b55ac9c057f67d94bc203f3f6595078d1fb3b458c9c28b7153a802"}, - {file = "pywin32-306-cp39-cp39-win_amd64.whl", hash = "sha256:39b61c15272833b5c329a2989999dcae836b1eed650252ab1b7bfbe1d59f30f4"}, + {file = "pywin32-308-cp310-cp310-win32.whl", hash = "sha256:796ff4426437896550d2981b9c2ac0ffd75238ad9ea2d3bfa67a1abd546d262e"}, + {file = "pywin32-308-cp310-cp310-win_amd64.whl", hash = "sha256:4fc888c59b3c0bef905ce7eb7e2106a07712015ea1c8234b703a088d46110e8e"}, + {file = "pywin32-308-cp310-cp310-win_arm64.whl", hash = "sha256:a5ab5381813b40f264fa3495b98af850098f814a25a63589a8e9eb12560f450c"}, + {file = "pywin32-308-cp311-cp311-win32.whl", hash = "sha256:5d8c8015b24a7d6855b1550d8e660d8daa09983c80e5daf89a273e5c6fb5095a"}, + {file = "pywin32-308-cp311-cp311-win_amd64.whl", hash = "sha256:575621b90f0dc2695fec346b2d6302faebd4f0f45c05ea29404cefe35d89442b"}, + {file = "pywin32-308-cp311-cp311-win_arm64.whl", hash = "sha256:100a5442b7332070983c4cd03f2e906a5648a5104b8a7f50175f7906efd16bb6"}, + {file = "pywin32-308-cp312-cp312-win32.whl", hash = "sha256:587f3e19696f4bf96fde9d8a57cec74a57021ad5f204c9e627e15c33ff568897"}, + {file = "pywin32-308-cp312-cp312-win_amd64.whl", hash = "sha256:00b3e11ef09ede56c6a43c71f2d31857cf7c54b0ab6e78ac659497abd2834f47"}, + {file = "pywin32-308-cp312-cp312-win_arm64.whl", hash = "sha256:9b4de86c8d909aed15b7011182c8cab38c8850de36e6afb1f0db22b8959e3091"}, + {file = "pywin32-308-cp313-cp313-win32.whl", hash = "sha256:1c44539a37a5b7b21d02ab34e6a4d314e0788f1690d65b48e9b0b89f31abbbed"}, + {file = "pywin32-308-cp313-cp313-win_amd64.whl", hash = "sha256:fd380990e792eaf6827fcb7e187b2b4b1cede0585e3d0c9e84201ec27b9905e4"}, + {file = "pywin32-308-cp313-cp313-win_arm64.whl", hash = "sha256:ef313c46d4c18dfb82a2431e3051ac8f112ccee1a34f29c263c583c568db63cd"}, + {file = "pywin32-308-cp37-cp37m-win32.whl", hash = "sha256:1f696ab352a2ddd63bd07430080dd598e6369152ea13a25ebcdd2f503a38f1ff"}, + {file = "pywin32-308-cp37-cp37m-win_amd64.whl", hash = "sha256:13dcb914ed4347019fbec6697a01a0aec61019c1046c2b905410d197856326a6"}, + {file = "pywin32-308-cp38-cp38-win32.whl", hash = "sha256:5794e764ebcabf4ff08c555b31bd348c9025929371763b2183172ff4708152f0"}, + {file = "pywin32-308-cp38-cp38-win_amd64.whl", hash = "sha256:3b92622e29d651c6b783e368ba7d6722b1634b8e70bd376fd7610fe1992e19de"}, + {file = "pywin32-308-cp39-cp39-win32.whl", hash = "sha256:7873ca4dc60ab3287919881a7d4f88baee4a6e639aa6962de25a98ba6b193341"}, + {file = "pywin32-308-cp39-cp39-win_amd64.whl", hash = "sha256:71b3322d949b4cc20776436a9c9ba0eeedcbc9c650daa536df63f0ff111bb920"}, ] [[package]] @@ -3183,19 +3187,23 @@ tornado = ["tornado (>=5)"] [[package]] name = "setuptools" -version = "69.0.2" +version = "75.3.0" description = "Easily download, build, install, upgrade, and uninstall Python packages" optional = false python-versions = ">=3.8" files = [ - {file = "setuptools-69.0.2-py3-none-any.whl", hash = "sha256:1e8fdff6797d3865f37397be788a4e3cba233608e9b509382a2777d25ebde7f2"}, - {file = "setuptools-69.0.2.tar.gz", hash = "sha256:735896e78a4742605974de002ac60562d286fa8051a7e2299445e8e8fbb01aa6"}, + {file = "setuptools-75.3.0-py3-none-any.whl", hash = "sha256:f2504966861356aa38616760c0f66568e535562374995367b4e69c7143cf6bcd"}, + {file = "setuptools-75.3.0.tar.gz", hash = "sha256:fba5dd4d766e97be1b1681d98712680ae8f2f26d7881245f2ce9e40714f1a686"}, ] [package.extras] -docs = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "rst.linker (>=1.9)", "sphinx (<7.2.5)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (>=1,<2)", "sphinx-reredirects", "sphinxcontrib-towncrier"] -testing = ["build[virtualenv]", "filelock (>=3.4.0)", "flake8-2020", "ini2toml[lite] (>=0.9)", "jaraco.develop (>=7.21)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pip (>=19.1)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-mypy (>=0.9.1)", "pytest-perf", "pytest-ruff", "pytest-timeout", "pytest-xdist", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel"] -testing-integration = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "packaging (>=23.1)", "pytest", "pytest-enabler", "pytest-xdist", "tomli", "virtualenv (>=13.0.0)", "wheel"] +check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1)", "ruff (>=0.5.2)"] +core = ["importlib-metadata (>=6)", "importlib-resources (>=5.10.2)", "jaraco.collections", "jaraco.functools", "jaraco.text (>=3.7)", "more-itertools", "more-itertools (>=8.8)", "packaging", "packaging (>=24)", "platformdirs (>=4.2.2)", "tomli (>=2.0.1)", "wheel (>=0.43.0)"] +cover = ["pytest-cov"] +doc = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "pyproject-hooks (!=1.1)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (>=1,<2)", "sphinx-reredirects", "sphinxcontrib-towncrier", "towncrier (<24.7)"] +enabler = ["pytest-enabler (>=2.2)"] +test = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "ini2toml[lite] (>=0.14)", "jaraco.develop (>=7.21)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "jaraco.test (>=5.5)", "packaging (>=23.2)", "pip (>=19.1)", "pyproject-hooks (!=1.1)", "pytest (>=6,!=8.1.*)", "pytest-home (>=0.5)", "pytest-perf", "pytest-subprocess", "pytest-timeout", "pytest-xdist (>=3)", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel (>=0.44.0)"] +type = ["importlib-metadata (>=7.0.2)", "jaraco.develop (>=7.21)", "mypy (==1.12.*)", "pytest-mypy"] [[package]] name = "six" @@ -3221,64 +3229,72 @@ files = [ [[package]] name = "sqlalchemy" -version = "2.0.25" +version = "2.0.36" description = "Database Abstraction Library" optional = false python-versions = ">=3.7" files = [ - {file = "SQLAlchemy-2.0.25-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:4344d059265cc8b1b1be351bfb88749294b87a8b2bbe21dfbe066c4199541ebd"}, - {file = "SQLAlchemy-2.0.25-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6f9e2e59cbcc6ba1488404aad43de005d05ca56e069477b33ff74e91b6319735"}, - {file = "SQLAlchemy-2.0.25-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:84daa0a2055df9ca0f148a64fdde12ac635e30edbca80e87df9b3aaf419e144a"}, - {file = "SQLAlchemy-2.0.25-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bc8b7dabe8e67c4832891a5d322cec6d44ef02f432b4588390017f5cec186a84"}, - {file = "SQLAlchemy-2.0.25-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:f5693145220517b5f42393e07a6898acdfe820e136c98663b971906120549da5"}, - {file = "SQLAlchemy-2.0.25-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:db854730a25db7c956423bb9fb4bdd1216c839a689bf9cc15fada0a7fb2f4570"}, - {file = "SQLAlchemy-2.0.25-cp310-cp310-win32.whl", hash = "sha256:14a6f68e8fc96e5e8f5647ef6cda6250c780612a573d99e4d881581432ef1669"}, - {file = "SQLAlchemy-2.0.25-cp310-cp310-win_amd64.whl", hash = "sha256:87f6e732bccd7dcf1741c00f1ecf33797383128bd1c90144ac8adc02cbb98643"}, - {file = "SQLAlchemy-2.0.25-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:342d365988ba88ada8af320d43df4e0b13a694dbd75951f537b2d5e4cb5cd002"}, - {file = "SQLAlchemy-2.0.25-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f37c0caf14b9e9b9e8f6dbc81bc56db06acb4363eba5a633167781a48ef036ed"}, - {file = "SQLAlchemy-2.0.25-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aa9373708763ef46782d10e950b49d0235bfe58facebd76917d3f5cbf5971aed"}, - {file = "SQLAlchemy-2.0.25-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d24f571990c05f6b36a396218f251f3e0dda916e0c687ef6fdca5072743208f5"}, - {file = "SQLAlchemy-2.0.25-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:75432b5b14dc2fff43c50435e248b45c7cdadef73388e5610852b95280ffd0e9"}, - {file = "SQLAlchemy-2.0.25-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:884272dcd3ad97f47702965a0e902b540541890f468d24bd1d98bcfe41c3f018"}, - {file = "SQLAlchemy-2.0.25-cp311-cp311-win32.whl", hash = "sha256:e607cdd99cbf9bb80391f54446b86e16eea6ad309361942bf88318bcd452363c"}, - {file = "SQLAlchemy-2.0.25-cp311-cp311-win_amd64.whl", hash = "sha256:7d505815ac340568fd03f719446a589162d55c52f08abd77ba8964fbb7eb5b5f"}, - {file = "SQLAlchemy-2.0.25-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:0dacf67aee53b16f365c589ce72e766efaabd2b145f9de7c917777b575e3659d"}, - {file = "SQLAlchemy-2.0.25-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:b801154027107461ee992ff4b5c09aa7cc6ec91ddfe50d02bca344918c3265c6"}, - {file = "SQLAlchemy-2.0.25-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:59a21853f5daeb50412d459cfb13cb82c089ad4c04ec208cd14dddd99fc23b39"}, - {file = "SQLAlchemy-2.0.25-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:29049e2c299b5ace92cbed0c1610a7a236f3baf4c6b66eb9547c01179f638ec5"}, - {file = "SQLAlchemy-2.0.25-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:b64b183d610b424a160b0d4d880995e935208fc043d0302dd29fee32d1ee3f95"}, - {file = "SQLAlchemy-2.0.25-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4f7a7d7fcc675d3d85fbf3b3828ecd5990b8d61bd6de3f1b260080b3beccf215"}, - {file = "SQLAlchemy-2.0.25-cp312-cp312-win32.whl", hash = "sha256:cf18ff7fc9941b8fc23437cc3e68ed4ebeff3599eec6ef5eebf305f3d2e9a7c2"}, - {file = "SQLAlchemy-2.0.25-cp312-cp312-win_amd64.whl", hash = "sha256:91f7d9d1c4dd1f4f6e092874c128c11165eafcf7c963128f79e28f8445de82d5"}, - {file = "SQLAlchemy-2.0.25-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:bb209a73b8307f8fe4fe46f6ad5979649be01607f11af1eb94aa9e8a3aaf77f0"}, - {file = "SQLAlchemy-2.0.25-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:798f717ae7c806d67145f6ae94dc7c342d3222d3b9a311a784f371a4333212c7"}, - {file = "SQLAlchemy-2.0.25-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5fdd402169aa00df3142149940b3bf9ce7dde075928c1886d9a1df63d4b8de62"}, - {file = "SQLAlchemy-2.0.25-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:0d3cab3076af2e4aa5693f89622bef7fa770c6fec967143e4da7508b3dceb9b9"}, - {file = "SQLAlchemy-2.0.25-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:74b080c897563f81062b74e44f5a72fa44c2b373741a9ade701d5f789a10ba23"}, - {file = "SQLAlchemy-2.0.25-cp37-cp37m-win32.whl", hash = "sha256:87d91043ea0dc65ee583026cb18e1b458d8ec5fc0a93637126b5fc0bc3ea68c4"}, - {file = "SQLAlchemy-2.0.25-cp37-cp37m-win_amd64.whl", hash = "sha256:75f99202324383d613ddd1f7455ac908dca9c2dd729ec8584c9541dd41822a2c"}, - {file = "SQLAlchemy-2.0.25-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:420362338681eec03f53467804541a854617faed7272fe71a1bfdb07336a381e"}, - {file = "SQLAlchemy-2.0.25-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:7c88f0c7dcc5f99bdb34b4fd9b69b93c89f893f454f40219fe923a3a2fd11625"}, - {file = "SQLAlchemy-2.0.25-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a3be4987e3ee9d9a380b66393b77a4cd6d742480c951a1c56a23c335caca4ce3"}, - {file = "SQLAlchemy-2.0.25-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f2a159111a0f58fb034c93eeba211b4141137ec4b0a6e75789ab7a3ef3c7e7e3"}, - {file = "SQLAlchemy-2.0.25-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:8b8cb63d3ea63b29074dcd29da4dc6a97ad1349151f2d2949495418fd6e48db9"}, - {file = "SQLAlchemy-2.0.25-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:736ea78cd06de6c21ecba7416499e7236a22374561493b456a1f7ffbe3f6cdb4"}, - {file = "SQLAlchemy-2.0.25-cp38-cp38-win32.whl", hash = "sha256:10331f129982a19df4284ceac6fe87353ca3ca6b4ca77ff7d697209ae0a5915e"}, - {file = "SQLAlchemy-2.0.25-cp38-cp38-win_amd64.whl", hash = "sha256:c55731c116806836a5d678a70c84cb13f2cedba920212ba7dcad53260997666d"}, - {file = "SQLAlchemy-2.0.25-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:605b6b059f4b57b277f75ace81cc5bc6335efcbcc4ccb9066695e515dbdb3900"}, - {file = "SQLAlchemy-2.0.25-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:665f0a3954635b5b777a55111ababf44b4fc12b1f3ba0a435b602b6387ffd7cf"}, - {file = "SQLAlchemy-2.0.25-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ecf6d4cda1f9f6cb0b45803a01ea7f034e2f1aed9475e883410812d9f9e3cfcf"}, - {file = "SQLAlchemy-2.0.25-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c51db269513917394faec5e5c00d6f83829742ba62e2ac4fa5c98d58be91662f"}, - {file = "SQLAlchemy-2.0.25-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:790f533fa5c8901a62b6fef5811d48980adeb2f51f1290ade8b5e7ba990ba3de"}, - {file = "SQLAlchemy-2.0.25-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:1b1180cda6df7af84fe72e4530f192231b1f29a7496951db4ff38dac1687202d"}, - {file = "SQLAlchemy-2.0.25-cp39-cp39-win32.whl", hash = "sha256:555651adbb503ac7f4cb35834c5e4ae0819aab2cd24857a123370764dc7d7e24"}, - {file = "SQLAlchemy-2.0.25-cp39-cp39-win_amd64.whl", hash = "sha256:dc55990143cbd853a5d038c05e79284baedf3e299661389654551bd02a6a68d7"}, - {file = "SQLAlchemy-2.0.25-py3-none-any.whl", hash = "sha256:a86b4240e67d4753dc3092d9511886795b3c2852abe599cffe108952f7af7ac3"}, - {file = "SQLAlchemy-2.0.25.tar.gz", hash = "sha256:a2c69a7664fb2d54b8682dd774c3b54f67f84fa123cf84dda2a5f40dcaa04e08"}, -] - -[package.dependencies] -greenlet = {version = "!=0.4.17", markers = "platform_machine == \"aarch64\" or platform_machine == \"ppc64le\" or platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"AMD64\" or platform_machine == \"win32\" or platform_machine == \"WIN32\""} + {file = "SQLAlchemy-2.0.36-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:59b8f3adb3971929a3e660337f5dacc5942c2cdb760afcabb2614ffbda9f9f72"}, + {file = "SQLAlchemy-2.0.36-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:37350015056a553e442ff672c2d20e6f4b6d0b2495691fa239d8aa18bb3bc908"}, + {file = "SQLAlchemy-2.0.36-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8318f4776c85abc3f40ab185e388bee7a6ea99e7fa3a30686580b209eaa35c08"}, + {file = "SQLAlchemy-2.0.36-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c245b1fbade9c35e5bd3b64270ab49ce990369018289ecfde3f9c318411aaa07"}, + {file = "SQLAlchemy-2.0.36-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:69f93723edbca7342624d09f6704e7126b152eaed3cdbb634cb657a54332a3c5"}, + {file = "SQLAlchemy-2.0.36-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:f9511d8dd4a6e9271d07d150fb2f81874a3c8c95e11ff9af3a2dfc35fe42ee44"}, + {file = "SQLAlchemy-2.0.36-cp310-cp310-win32.whl", hash = "sha256:c3f3631693003d8e585d4200730616b78fafd5a01ef8b698f6967da5c605b3fa"}, + {file = "SQLAlchemy-2.0.36-cp310-cp310-win_amd64.whl", hash = "sha256:a86bfab2ef46d63300c0f06936bd6e6c0105faa11d509083ba8f2f9d237fb5b5"}, + {file = "SQLAlchemy-2.0.36-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:fd3a55deef00f689ce931d4d1b23fa9f04c880a48ee97af488fd215cf24e2a6c"}, + {file = "SQLAlchemy-2.0.36-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:4f5e9cd989b45b73bd359f693b935364f7e1f79486e29015813c338450aa5a71"}, + {file = "SQLAlchemy-2.0.36-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d0ddd9db6e59c44875211bc4c7953a9f6638b937b0a88ae6d09eb46cced54eff"}, + {file = "SQLAlchemy-2.0.36-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2519f3a5d0517fc159afab1015e54bb81b4406c278749779be57a569d8d1bb0d"}, + {file = "SQLAlchemy-2.0.36-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:59b1ee96617135f6e1d6f275bbe988f419c5178016f3d41d3c0abb0c819f75bb"}, + {file = "SQLAlchemy-2.0.36-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:39769a115f730d683b0eb7b694db9789267bcd027326cccc3125e862eb03bfd8"}, + {file = "SQLAlchemy-2.0.36-cp311-cp311-win32.whl", hash = "sha256:66bffbad8d6271bb1cc2f9a4ea4f86f80fe5e2e3e501a5ae2a3dc6a76e604e6f"}, + {file = "SQLAlchemy-2.0.36-cp311-cp311-win_amd64.whl", hash = "sha256:23623166bfefe1487d81b698c423f8678e80df8b54614c2bf4b4cfcd7c711959"}, + {file = "SQLAlchemy-2.0.36-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:f7b64e6ec3f02c35647be6b4851008b26cff592a95ecb13b6788a54ef80bbdd4"}, + {file = "SQLAlchemy-2.0.36-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:46331b00096a6db1fdc052d55b101dbbfc99155a548e20a0e4a8e5e4d1362855"}, + {file = "SQLAlchemy-2.0.36-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fdf3386a801ea5aba17c6410dd1dc8d39cf454ca2565541b5ac42a84e1e28f53"}, + {file = "SQLAlchemy-2.0.36-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ac9dfa18ff2a67b09b372d5db8743c27966abf0e5344c555d86cc7199f7ad83a"}, + {file = "SQLAlchemy-2.0.36-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:90812a8933df713fdf748b355527e3af257a11e415b613dd794512461eb8a686"}, + {file = "SQLAlchemy-2.0.36-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:1bc330d9d29c7f06f003ab10e1eaced295e87940405afe1b110f2eb93a233588"}, + {file = "SQLAlchemy-2.0.36-cp312-cp312-win32.whl", hash = "sha256:79d2e78abc26d871875b419e1fd3c0bca31a1cb0043277d0d850014599626c2e"}, + {file = "SQLAlchemy-2.0.36-cp312-cp312-win_amd64.whl", hash = "sha256:b544ad1935a8541d177cb402948b94e871067656b3a0b9e91dbec136b06a2ff5"}, + {file = "SQLAlchemy-2.0.36-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:b5cc79df7f4bc3d11e4b542596c03826063092611e481fcf1c9dfee3c94355ef"}, + {file = "SQLAlchemy-2.0.36-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:3c01117dd36800f2ecaa238c65365b7b16497adc1522bf84906e5710ee9ba0e8"}, + {file = "SQLAlchemy-2.0.36-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9bc633f4ee4b4c46e7adcb3a9b5ec083bf1d9a97c1d3854b92749d935de40b9b"}, + {file = "SQLAlchemy-2.0.36-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9e46ed38affdfc95d2c958de328d037d87801cfcbea6d421000859e9789e61c2"}, + {file = "SQLAlchemy-2.0.36-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:b2985c0b06e989c043f1dc09d4fe89e1616aadd35392aea2844f0458a989eacf"}, + {file = "SQLAlchemy-2.0.36-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:4a121d62ebe7d26fec9155f83f8be5189ef1405f5973ea4874a26fab9f1e262c"}, + {file = "SQLAlchemy-2.0.36-cp313-cp313-win32.whl", hash = "sha256:0572f4bd6f94752167adfd7c1bed84f4b240ee6203a95e05d1e208d488d0d436"}, + {file = "SQLAlchemy-2.0.36-cp313-cp313-win_amd64.whl", hash = "sha256:8c78ac40bde930c60e0f78b3cd184c580f89456dd87fc08f9e3ee3ce8765ce88"}, + {file = "SQLAlchemy-2.0.36-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:be9812b766cad94a25bc63bec11f88c4ad3629a0cec1cd5d4ba48dc23860486b"}, + {file = "SQLAlchemy-2.0.36-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:50aae840ebbd6cdd41af1c14590e5741665e5272d2fee999306673a1bb1fdb4d"}, + {file = "SQLAlchemy-2.0.36-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4557e1f11c5f653ebfdd924f3f9d5ebfc718283b0b9beebaa5dd6b77ec290971"}, + {file = "SQLAlchemy-2.0.36-cp37-cp37m-musllinux_1_2_aarch64.whl", hash = "sha256:07b441f7d03b9a66299ce7ccf3ef2900abc81c0db434f42a5694a37bd73870f2"}, + {file = "SQLAlchemy-2.0.36-cp37-cp37m-musllinux_1_2_x86_64.whl", hash = "sha256:28120ef39c92c2dd60f2721af9328479516844c6b550b077ca450c7d7dc68575"}, + {file = "SQLAlchemy-2.0.36-cp37-cp37m-win32.whl", hash = "sha256:b81ee3d84803fd42d0b154cb6892ae57ea6b7c55d8359a02379965706c7efe6c"}, + {file = "SQLAlchemy-2.0.36-cp37-cp37m-win_amd64.whl", hash = "sha256:f942a799516184c855e1a32fbc7b29d7e571b52612647866d4ec1c3242578fcb"}, + {file = "SQLAlchemy-2.0.36-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:3d6718667da04294d7df1670d70eeddd414f313738d20a6f1d1f379e3139a545"}, + {file = "SQLAlchemy-2.0.36-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:72c28b84b174ce8af8504ca28ae9347d317f9dba3999e5981a3cd441f3712e24"}, + {file = "SQLAlchemy-2.0.36-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b11d0cfdd2b095e7b0686cf5fabeb9c67fae5b06d265d8180715b8cfa86522e3"}, + {file = "SQLAlchemy-2.0.36-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e32092c47011d113dc01ab3e1d3ce9f006a47223b18422c5c0d150af13a00687"}, + {file = "SQLAlchemy-2.0.36-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:6a440293d802d3011028e14e4226da1434b373cbaf4a4bbb63f845761a708346"}, + {file = "SQLAlchemy-2.0.36-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:c54a1e53a0c308a8e8a7dffb59097bff7facda27c70c286f005327f21b2bd6b1"}, + {file = "SQLAlchemy-2.0.36-cp38-cp38-win32.whl", hash = "sha256:1e0d612a17581b6616ff03c8e3d5eff7452f34655c901f75d62bd86449d9750e"}, + {file = "SQLAlchemy-2.0.36-cp38-cp38-win_amd64.whl", hash = "sha256:8958b10490125124463095bbdadda5aa22ec799f91958e410438ad6c97a7b793"}, + {file = "SQLAlchemy-2.0.36-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:dc022184d3e5cacc9579e41805a681187650e170eb2fd70e28b86192a479dcaa"}, + {file = "SQLAlchemy-2.0.36-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:b817d41d692bf286abc181f8af476c4fbef3fd05e798777492618378448ee689"}, + {file = "SQLAlchemy-2.0.36-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a4e46a888b54be23d03a89be510f24a7652fe6ff660787b96cd0e57a4ebcb46d"}, + {file = "SQLAlchemy-2.0.36-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c4ae3005ed83f5967f961fd091f2f8c5329161f69ce8480aa8168b2d7fe37f06"}, + {file = "SQLAlchemy-2.0.36-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:03e08af7a5f9386a43919eda9de33ffda16b44eb11f3b313e6822243770e9763"}, + {file = "SQLAlchemy-2.0.36-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:3dbb986bad3ed5ceaf090200eba750b5245150bd97d3e67343a3cfed06feecf7"}, + {file = "SQLAlchemy-2.0.36-cp39-cp39-win32.whl", hash = "sha256:9fe53b404f24789b5ea9003fc25b9a3988feddebd7e7b369c8fac27ad6f52f28"}, + {file = "SQLAlchemy-2.0.36-cp39-cp39-win_amd64.whl", hash = "sha256:af148a33ff0349f53512a049c6406923e4e02bf2f26c5fb285f143faf4f0e46a"}, + {file = "SQLAlchemy-2.0.36-py3-none-any.whl", hash = "sha256:fddbe92b4760c6f5d48162aef14824add991aeda8ddadb3c31d56eb15ca69f8e"}, + {file = "sqlalchemy-2.0.36.tar.gz", hash = "sha256:7f2767680b6d2398aea7082e45a774b2b0767b5c8d8ffb9c8b683088ea9b29c5"}, +] + +[package.dependencies] +greenlet = {version = "!=0.4.17", markers = "python_version < \"3.13\" and (platform_machine == \"aarch64\" or platform_machine == \"ppc64le\" or platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"AMD64\" or platform_machine == \"win32\" or platform_machine == \"WIN32\")"} typing-extensions = ">=4.6.0" [package.extras] @@ -3287,7 +3303,7 @@ aioodbc = ["aioodbc", "greenlet (!=0.4.17)"] aiosqlite = ["aiosqlite", "greenlet (!=0.4.17)", "typing_extensions (!=3.10.0.1)"] asyncio = ["greenlet (!=0.4.17)"] asyncmy = ["asyncmy (>=0.2.3,!=0.2.4,!=0.2.6)", "greenlet (!=0.4.17)"] -mariadb-connector = ["mariadb (>=1.0.1,!=1.1.2,!=1.1.5)"] +mariadb-connector = ["mariadb (>=1.0.1,!=1.1.2,!=1.1.5,!=1.1.10)"] mssql = ["pyodbc"] mssql-pymssql = ["pymssql"] mssql-pyodbc = ["pyodbc"] @@ -3338,18 +3354,18 @@ widechars = ["wcwidth"] [[package]] name = "taskipy" -version = "1.12.0" +version = "1.14.0" description = "tasks runner for python projects" optional = false -python-versions = ">=3.6,<4.0" +python-versions = "<4.0,>=3.6" files = [ - {file = "taskipy-1.12.0-py3-none-any.whl", hash = "sha256:38306fbc952a7ca314b8f842a74b2fc38535cdab21031fe89e714a83e6259a84"}, - {file = "taskipy-1.12.0.tar.gz", hash = "sha256:e3dd7c53f7c9c4fd17dc908b1037f545afc452907eb0953b84e91c0a9a9d809d"}, + {file = "taskipy-1.14.0-py3-none-any.whl", hash = "sha256:29040d9a8038170602feb71792bdef5203720ed30f595304aee843625892452b"}, + {file = "taskipy-1.14.0.tar.gz", hash = "sha256:5d9631c29980481d59858f0a100ed3200cf7468ca8c0540ef19388586485532d"}, ] [package.dependencies] colorama = ">=0.4.4,<0.5.0" -mslex = {version = ">=0.3.0,<0.4.0", markers = "sys_platform == \"win32\""} +mslex = {version = ">=1.1.0,<2.0.0", markers = "sys_platform == \"win32\""} psutil = ">=5.7.2,<6.0.0" tomli = {version = ">=2.0.1,<3.0.0", markers = "python_version >= \"3.7\" and python_version < \"4.0\""} @@ -3388,13 +3404,13 @@ files = [ [[package]] name = "tomli" -version = "2.0.1" +version = "2.1.0" description = "A lil' TOML parser" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "tomli-2.0.1-py3-none-any.whl", hash = "sha256:939de3e7a6161af0c887ef91b7d41a53e7c5a1ca976325f429cb46ea9bc30ecc"}, - {file = "tomli-2.0.1.tar.gz", hash = "sha256:de526c12914f0c550d15924c62d72abc48d6fe7364aa87328337a31007fe8a4f"}, + {file = "tomli-2.1.0-py3-none-any.whl", hash = "sha256:a5c57c3d1c56f5ccdf89f6523458f60ef716e210fc47c4cfb188c5ba473e0391"}, + {file = "tomli-2.1.0.tar.gz", hash = "sha256:3f646cae2aec94e17d04973e4249548320197cfabdf130015d023de4b74d8ab8"}, ] [[package]] @@ -3461,13 +3477,13 @@ telegram = ["requests"] [[package]] name = "typer" -version = "0.9.0" +version = "0.9.4" description = "Typer, build great CLIs. Easy to code. Based on Python type hints." optional = false python-versions = ">=3.6" files = [ - {file = "typer-0.9.0-py3-none-any.whl", hash = "sha256:5d96d986a21493606a358cae4461bd8cdf83cbf33a5aa950ae629ca3b51467ee"}, - {file = "typer-0.9.0.tar.gz", hash = "sha256:50922fd79aea2f4751a8e0408ff10d2662bd0c8bbfa84755a699f3bada2978b2"}, + {file = "typer-0.9.4-py3-none-any.whl", hash = "sha256:aa6c4a4e2329d868b80ecbaf16f807f2b54e192209d7ac9dd42691d63f7a54eb"}, + {file = "typer-0.9.4.tar.gz", hash = "sha256:f714c2d90afae3a7929fcd72a3abb08df305e1ff61719381384211c4070af57f"}, ] [package.dependencies] @@ -3478,7 +3494,7 @@ typing-extensions = ">=3.7.4.3" all = ["colorama (>=0.4.3,<0.5.0)", "rich (>=10.11.0,<14.0.0)", "shellingham (>=1.3.0,<2.0.0)"] dev = ["autoflake (>=1.3.1,<2.0.0)", "flake8 (>=3.8.3,<4.0.0)", "pre-commit (>=2.17.0,<3.0.0)"] doc = ["cairosvg (>=2.5.2,<3.0.0)", "mdx-include (>=1.4.1,<2.0.0)", "mkdocs (>=1.1.2,<2.0.0)", "mkdocs-material (>=8.1.4,<9.0.0)", "pillow (>=9.3.0,<10.0.0)"] -test = ["black (>=22.3.0,<23.0.0)", "coverage (>=6.2,<7.0)", "isort (>=5.0.6,<6.0.0)", "mypy (==0.910)", "pytest (>=4.4.0,<8.0.0)", "pytest-cov (>=2.10.0,<5.0.0)", "pytest-sugar (>=0.9.4,<0.10.0)", "pytest-xdist (>=1.32.0,<4.0.0)", "rich (>=10.11.0,<14.0.0)", "shellingham (>=1.3.0,<2.0.0)"] +test = ["black (>=22.3.0,<23.0.0)", "coverage (>=6.2,<7.0)", "isort (>=5.0.6,<6.0.0)", "mypy (==0.971)", "pytest (>=4.4.0,<8.0.0)", "pytest-cov (>=2.10.0,<5.0.0)", "pytest-sugar (>=0.9.4,<0.10.0)", "pytest-xdist (>=1.32.0,<4.0.0)", "rich (>=10.11.0,<14.0.0)", "shellingham (>=1.3.0,<2.0.0)"] [[package]] name = "typing-extensions" @@ -3502,6 +3518,17 @@ files = [ {file = "tzdata-2023.3.tar.gz", hash = "sha256:11ef1e08e54acb0d4f95bdb1be05da659673de4acbd21bf9c69e94cc5e907a3a"}, ] +[[package]] +name = "unidecode" +version = "1.3.8" +description = "ASCII transliterations of Unicode text" +optional = false +python-versions = ">=3.5" +files = [ + {file = "Unidecode-1.3.8-py3-none-any.whl", hash = "sha256:d130a61ce6696f8148a3bd8fe779c99adeb4b870584eeb9526584e9aa091fd39"}, + {file = "Unidecode-1.3.8.tar.gz", hash = "sha256:cfdb349d46ed3873ece4586b96aa75258726e2fa8ec21d6f00a591d98806c2f4"}, +] + [[package]] name = "uritemplate" version = "4.1.1" @@ -3531,13 +3558,13 @@ socks = ["PySocks (>=1.5.6,!=1.5.7,<2.0)"] [[package]] name = "virtualenv" -version = "20.24.7" +version = "20.27.1" description = "Virtual Python Environment builder" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "virtualenv-20.24.7-py3-none-any.whl", hash = "sha256:a18b3fd0314ca59a2e9f4b556819ed07183b3e9a3702ecfe213f593d44f7b3fd"}, - {file = "virtualenv-20.24.7.tar.gz", hash = "sha256:69050ffb42419c91f6c1284a7b24e0475d793447e35929b488bf6a0aade39353"}, + {file = "virtualenv-20.27.1-py3-none-any.whl", hash = "sha256:f11f1b8a29525562925f745563bfd48b189450f61fb34c4f9cc79dd5aa32a1f4"}, + {file = "virtualenv-20.27.1.tar.gz", hash = "sha256:142c6be10212543b32c6c45d3d3893dff89112cc588b7d0879ae5a1ec03a47ba"}, ] [package.dependencies] @@ -3546,7 +3573,7 @@ filelock = ">=3.12.2,<4" platformdirs = ">=3.9.1,<5" [package.extras] -docs = ["furo (>=2023.7.26)", "proselint (>=0.13)", "sphinx (>=7.1.2)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier (>=23.6)"] +docs = ["furo (>=2023.7.26)", "proselint (>=0.13)", "sphinx (>=7.1.2,!=7.3)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier (>=23.6)"] test = ["covdefaults (>=2.3)", "coverage (>=7.2.7)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23.1)", "pytest (>=7.4)", "pytest-env (>=0.8.2)", "pytest-freezer (>=0.4.8)", "pytest-mock (>=3.11.1)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)", "setuptools (>=68)", "time-machine (>=2.10)"] [[package]] @@ -3695,4 +3722,4 @@ testing = ["big-O", "jaraco.functools", "jaraco.itertools", "more-itertools", "p [metadata] lock-version = "2.0" python-versions = ">=3.10,<3.11" -content-hash = "5d5ac7e559d00768e8d9a1e0e867be0ff587de81f8f2268697c35ccadcc310e0" +content-hash = "6511da8862567ed4397518e74ec8f84cfcdc02294a33e7b4b558a9da727e127a" diff --git a/pyproject.toml b/pyproject.toml index 27011dc3a..734baf5d3 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -6,7 +6,7 @@ authors = ["Gabriel Gazola Milan "] [tool.poetry.dependencies] python = ">=3.10,<3.11" -dbt-bigquery = "1.7.2" +dbt-bigquery = "1.7.3" google-cloud-storage = "2.13.0" prefect = "1.4.1" prefeitura-rio = "1.1.3a1" @@ -131,6 +131,7 @@ redis-pal = "^1.0.0" sqlalchemy = "^2.0.25" google-cloud-dataplex = "^1.12.3" openpyxl = "^3.1.4" +unidecode = "^1.3.8" [tool.poetry.group.dev] diff --git a/queries/.kubernetes/docs/deployment.yaml b/queries/.kubernetes/docs/deployment.yaml index e14e4d5ac..d7b14dadb 100644 --- a/queries/.kubernetes/docs/deployment.yaml +++ b/queries/.kubernetes/docs/deployment.yaml @@ -53,7 +53,7 @@ spec: tls: - hosts: - docs.mobilidade.rio - secretName: dbt-docs-tls + secretName: p-dbt-docs-tls rules: - host: docs.mobilidade.rio http: @@ -73,10 +73,11 @@ metadata: spec: acme: server: https://acme-v02.api.letsencrypt.org/directory - email: gsperim@gmail.com + email: caiorogerio@prefeitura.rio privateKeySecretRef: name: dbt-docs-tls solvers: - http01: ingress: class: nginx +# trigger cd diff --git a/queries/.kubernetes/docs/generate-docs.sh b/queries/.kubernetes/docs/generate-docs.sh index 4c3a25bfd..a8b4437ae 100644 --- a/queries/.kubernetes/docs/generate-docs.sh +++ b/queries/.kubernetes/docs/generate-docs.sh @@ -1,40 +1,42 @@ echo "Current working directory is:" && pwd -echo "Creating directories..." +# echo "Creating directories..." -mkdir ./credentials-dev +# mkdir ./credentials-dev -mkdir ./credentials-prod +# mkdir ./credentials-prod -mkdir ./profiles +# mkdir ./profiles echo "Mounting files from env..." -bash -c "echo $1 | base64 --decode > ./credentials-dev/dev.json" - -bash -c "echo $1 | base64 --decode > ./credentials-prod/prod.json" - -echo """ -queries: - target: dev - outputs: - dev: - type: bigquery - method: service-account - project: rj-smtr - dataset: dbt - location: US - threads: 2 - keyfile: $PWD/credentials-dev/dev.json - prod: - type: bigquery - method: service-account - project: rj-smtr - dataset: dbt - location: US - threads: 2 - keyfile: $PWD/credentials-prod/prod.json""" > profiles/profiles.yml - -ls ./profiles - -dbt docs generate --profiles-dir ./profiles \ No newline at end of file +# bash -c "echo $1 | base64 --decode > ./credentials-dev/dev.json" + +bash -c "echo $1 | base64 --decode > /tmp/credentials.json" + +# echo """ +# queries: +# target: dev +# outputs: +# dev: +# type: bigquery +# method: service-account +# project: rj-smtr +# dataset: dbt +# location: US +# threads: 2 +# keyfile: $PWD/credentials-dev/dev.json +# prod: +# type: bigquery +# method: service-account +# project: rj-smtr +# dataset: dbt +# location: US +# threads: 2 +# keyfile: $PWD/credentials-prod/prod.json""" > profiles/profiles.yml + +# ls ./profiles + +dbt deps + +dbt docs generate \ No newline at end of file diff --git a/queries/CHANGELOG.md b/queries/CHANGELOG.md new file mode 100644 index 000000000..6bbcbf006 --- /dev/null +++ b/queries/CHANGELOG.md @@ -0,0 +1,7 @@ +# Changelog - queries + +## [1.0.0] - 2024-10-29 + +### Adicionado + +- Adiciona package: `dbt-labs/dbt_utils` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/288) \ No newline at end of file diff --git a/queries/dbt_project.yml b/queries/dbt_project.yml index bd594b102..6d03be30d 100644 --- a/queries/dbt_project.yml +++ b/queries/dbt_project.yml @@ -95,10 +95,12 @@ vars: ### Subsídio SPPO (Ônibus) ### buffer: 500 # distância em metros para buffer + conformidade_velocidade_min: 110 perc_conformidade_distancia_min: 0 perc_conformidade_shape_min: 80 perc_conformidade_registros_min: 50 perc_distancia_total_subsidio: 80 + distancia_inicio_fim_conformidade_velocidade_min: 2000 quadro_horario: "`rj-smtr-staging.projeto_subsidio_sppo_staging.quadro_horario`" subsidio_shapes: "`rj-smtr-staging.projeto_subsidio_sppo_staging.shapes`" subsidio_trips: "`rj-smtr-staging.projeto_subsidio_sppo_staging.trips`" @@ -123,7 +125,12 @@ vars: DATA_SUBSIDIO_V8_INICIO: "2024-07-20" # Feature Apuração por faixa horária DATA_SUBSIDIO_V9_INICIO: "2024-08-16" - # valor_subsidio: "`rj-smtr-dev.projeto_subsidio_sppo.valor_subsidio`" + # Feature Apuração por novas faixas horárias (RESOLUÇÃO SMTR 3777/2024) + DATA_SUBSIDIO_V10_INICIO: "2024-11-01" + # Feature Apuração por novas faixas horárias (RESOLUÇÃO SMTR 3777/2024) - Feed (GTFS) + DATA_SUBSIDIO_V11_INICIO: "2024-11-06" + # Parâmetro 110 km/h + alterações de regras do modelo `viagem_transacao.sql` + DATA_SUBSIDIO_V12_INICIO: "2024-11-16" # Recursos # recurso_staging: "rj-smtr-staging.projeto_subsidio_sppo_staging.recurso" @@ -186,6 +193,21 @@ vars: ### Encontro de Contas ### encontro_contas_modo: "" + ### Viagens 2.0 ### + + limite_reducao_area_buffer: 0.5 + comprimento_minimo_segmento_shape: 990 + buffer_segmento_metros: 20 + buffer_tunel_metros: 50 + parametro_validacao: 0.9 + comprimento_shape: 1000 + projecao_sirgas_2000: "EPSG:31983" + projecao_wgs_84: "EPSG:4326" + data_inicial_gps_validacao_viagem: "2024-10-11" + + ### Planejamento ### + feed_inicial_viagem_planejada: "2024-11-16" + tests: rj_smtr: where: "DATA BETWEEN DATE('__date_range_start__') AND DATE('__date_range_end__')" @@ -325,10 +347,27 @@ models: staging: +materialized: view +schema: transito_staging + planejamento: + +materialized: incremental + +incremental_strategy: insert_overwrite + +schema: planejamento + staging: + +materialized: view + +schema: planejamento_staging monitoramento: +materialized: incremental +incremental_strategy: insert_overwrite +schema: monitoramento staging: +materialized: view - +schema: monitoramento_staging \ No newline at end of file + +schema: monitoramento_staging + bilhetagem: + +materialized: incremental + +incremental_strategy: insert_overwrite + +schema: bilhetagem + staging: + +materialized: view + +schema: bilhetagem_staging + datario: + +materialized: view + +schema: datario diff --git a/queries/dev/run.py b/queries/dev/run.py index f6fa99f3e..c205ea309 100644 --- a/queries/dev/run.py +++ b/queries/dev/run.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # import os -from utils import run_dbt_model +from queries.dev.utils import run_dbt_model # Veja os parâmetros disponíveis da função run_dbt_model em util.py diff --git a/queries/dev/utils.py b/queries/dev/utils.py index 38dac7824..8f227e542 100644 --- a/queries/dev/utils.py +++ b/queries/dev/utils.py @@ -1,13 +1,24 @@ # -*- coding: utf-8 -*- +# flake8: noqa +import json import os - -# from datetime import datetime as dt -# from datetime import timedelta +import re +import subprocess +from pathlib import Path from typing import Dict, List, Union +import basedosdados as bd import requests +from prefect import context + +# from datetime import datetime as dt +# from datetime import timedelta +from pipelines.constants import constants as smtr_constants +from pipelines.utils.discord import format_send_discord_message +from pipelines.utils.secret import get_secret # import pandas as pd +bd.config.from_file = True def run_dbt_model( @@ -62,6 +73,41 @@ def run_dbt_model( if flags: run_command += f" {flags}" + print(f"\n>>> RUNNING: {run_command}\n") + os.chdir(os.path.dirname(os.path.dirname(os.path.realpath(__file__)))) + return os.system(run_command) + + +def run_dbt_selector( + selector_name: str, + flags: str = None, + _vars: dict | list[dict] = None, +): + """ + Runs a DBT selector. + + Args: + selector_name (str): The name of the DBT selector to run. + flags (str, optional): Flags to pass to the dbt run command. + _vars (Union[dict, list[dict]], optional): Variables to pass to dbt. Defaults to None. + """ + # Build the dbt command + run_command = f"dbt run --selector {selector_name}" + + if _vars: + if isinstance(_vars, list): + vars_dict = {} + for elem in _vars: + vars_dict.update(elem) + vars_str = f'"{vars_dict}"' + run_command += f" --vars {vars_str}" + else: + vars_str = f'"{_vars}"' + run_command += f" --vars {vars_str}" + + if flags: + run_command += f" {flags}" + print(f"\n>>> RUNNING: {run_command}\n") os.chdir(os.path.dirname(os.path.dirname(os.path.realpath(__file__)))) os.system(run_command) @@ -78,3 +124,277 @@ def fetch_dataset_sha(dataset_id: str): dataset_version = response.json()[0]["sha"] return {"version": dataset_version} + + +def run_dbt_tests( + dataset_id: str = None, + table_id: str = None, + model: str = None, + upstream: bool = None, + downstream: bool = None, + test_name: str = None, + exclude: str = None, + flags: str = None, + _vars: Union[dict, List[Dict]] = None, +): + """ + Run DBT test + """ + run_command = "dbt test" + + common_flags = "--profiles-dir ./dev" + + if flags: + flags = f"{common_flags} {flags}" + else: + flags = common_flags + + if not model: + model = dataset_id + if table_id: + model += f".{table_id}" + + if model: + run_command += " --select " + if upstream: + run_command += "+" + run_command += model + if downstream: + run_command += "+" + if test_name: + model += f",test_name:{test_name}" + + if exclude: + run_command += f" --exclude {exclude}" + + if _vars: + if isinstance(_vars, list): + vars_dict = {} + for elem in _vars: + vars_dict.update(elem) + vars_str = f'"{vars_dict}"' + run_command += f" --vars {vars_str}" + else: + vars_str = f'"{_vars}"' + run_command += f" --vars {vars_str}" + + if flags: + run_command += f" {flags}" + + print(f"\n>>> RUNNING: {run_command}\n") + + project_dir = os.path.dirname(os.path.dirname(os.path.realpath(__file__))) + os.chdir(project_dir) + dbt_logs = subprocess.run(run_command, shell=True, capture_output=True, text=True) + + print(dbt_logs.stdout) + return dbt_logs.stdout + + +def parse_dbt_test_output(dbt_logs: str) -> dict: + # Remover sequências ANSI + dbt_logs = re.sub(r"\x1B[@-_][0-?]*[ -/]*[@-~]", "", dbt_logs) + + results = {} + result_pattern = r"\d+ of \d+ (PASS|FAIL|ERROR) (\d+ )?([\w_]+) .* \[(PASS|FAIL|ERROR) .*\]" + fail_pattern = r"Failure in test ([\w_]+) .*\n.*\n.*\n.* compiled Code at (.*)\n" + error_pattern = r"Error in test ([\w_]+) \(.*schema.yaml\)\n (.*)\n" + + for match in re.finditer(result_pattern, dbt_logs): + groups = match.groups() + test_name = groups[2] + results[test_name] = {"result": groups[3]} + + for match in re.finditer(fail_pattern, dbt_logs): + groups = match.groups() + test_name = groups[0] + file = groups[1] + + with open(file, "r") as arquivo: + query = arquivo.read() + + query = re.sub(r"\n+", "\n", query) + results[test_name]["query"] = query + + for match in re.finditer(error_pattern, dbt_logs): + groups = match.groups() + test_name = groups[0] + error = groups[1] + results[test_name]["error"] = error + + log_message = "" + for test, info in results.items(): + result = info["result"] + log_message += f"Test: {test} Status: {result}\n" + + if result == "FAIL": + log_message += "Query:\n" + log_message += f"{info['query']}\n" + + if result == "ERROR": + log_message += f"Error: {info['error']}\n" + + log_message += "\n" + + print(log_message) + + return results + + +def dbt_data_quality_checks( + checks_list: dict, checks_results: dict, params: dict, webhook_url: str = None +) -> bool: + + if webhook_url is None: + webhook_url = get_secret(secret_path=smtr_constants.WEBHOOKS_SECRET_PATH.value)["dataplex"] + + dados_tag = f" - <@&{smtr_constants.OWNERS_DISCORD_MENTIONS.value['dados_smtr']['user_id']}>\n" + + test_check = all(test["result"] == "PASS" for test in checks_results.values()) + + date_range = ( + params["date_range_start"] + if params["date_range_start"] == params["date_range_end"] + else f'{params["date_range_start"]} a {params["date_range_end"]}' + ) + + formatted_messages = [ + ":green_circle: " if test_check else ":red_circle: ", + f"**[DEV]Data Quality Checks - {context.get('flow_name')} - {date_range}**\n\n", + ] + + for table_id, tests in checks_list.items(): + formatted_messages.append( + f"*{table_id}:*\n" + + "\n".join( + f'{":white_check_mark:" if checks_results[test_id]["result"] == "PASS" else ":x:"} ' + f'{test["description"]}' + for test_id, test in tests.items() + ) + ) + + formatted_messages.append("\n\n") + formatted_messages.append( + ":tada: **Status:** Sucesso" + if test_check + else ":warning: **Status:** Testes falharam. Necessidade de revisão dos dados finais!\n" + ) + + formatted_messages.append(dados_tag) + format_send_discord_message(formatted_messages, webhook_url) + + +def get_model_table_info(model_name): + os.chdir(os.path.dirname(os.path.dirname(os.path.realpath(__file__)))) + target_path = Path("target/manifest.json") + + with target_path.open() as file: + manifest = json.load(file) + for _, node in manifest["nodes"].items(): + if node["resource_type"] == "model" and node["name"] == model_name: + return { + "project": node["database"], + "dataset_id": node["schema"], + "table_id": node["alias"], + } + + raise ValueError("modelo não encontrado no arquivo manifest.json") + + +def test_incremental_model( + model: str, + _vars: dict, + partition_column_name: str, + expected_changed_partitions: list[str], + unique_key: Union[str, list[str]] = None, +): + model_info = get_model_table_info(model_name=model) + changed_partitions_sql = f""" + select + cast(parse_date('%Y%m%d', partition_id) as string) as particao, + last_modified_time + from {model_info["project"]}.{model_info["dataset_id"]}.INFORMATION_SCHEMA.PARTITIONS + where table_name = '{model_info["table_id"]}' and partition_id != "__NULL__" + """ + changed_partitions_before = bd.read_sql(changed_partitions_sql).rename( + columns={"last_modified_time": "last_modified_time_before"} + ) + partition_filter = ",".join([f"'{p}'" for p in expected_changed_partitions]) + partition_count_sql = f""" + select + {partition_column_name} as particao, + count(*) as qt_registros + from {model_info["project"]}.{model_info["dataset_id"]}.{model_info["table_id"]} + where {partition_column_name} in ({partition_filter}) + group by 1 + """ + + partition_count_before = bd.read_sql(partition_count_sql).rename( + columns={"qt_registros": "qt_registros_before"} + ) + + assert run_dbt_model(model=model, _vars=_vars) == 0, "DBT Falhou!" + + changed_partitions_after = bd.read_sql(changed_partitions_sql).rename( + columns={"last_modified_time": "last_modified_time_after"} + ) + + partition_count_after = bd.read_sql(partition_count_sql).rename( + columns={"qt_registros": "qt_registros_after"} + ) + + changed_partitions = changed_partitions_before.merge(changed_partitions_after, on="particao") + changed_partitions = changed_partitions.loc[ + changed_partitions["last_modified_time_before"] + != changed_partitions["last_modified_time_after"] + ] + changed_partitions = changed_partitions["particao"].to_list() + not_expected_changed = [p for p in changed_partitions if p not in expected_changed_partitions] + expected_not_changed = [p for p in expected_changed_partitions if p not in changed_partitions] + + if len(not_expected_changed) > 0: + print(f"Foram alteradas partições não esperadas: {', '.join(not_expected_changed)}") + if len(expected_not_changed) > 0: + print(f"Não foram alteradas partições esperadas: {', '.join(expected_not_changed)}") + + changed_record_count = partition_count_before.merge(partition_count_after, on="particao") + changed_record_count = changed_record_count.loc[ + changed_record_count["qt_registros_before"] != changed_record_count["qt_registros_after"] + ] + + changed_record_count = changed_record_count.to_dict("records") + + if len(changed_record_count) > 0: + print("Contagem de registros modificada:") + print( + "\n".join( + [ + f"{a['particao']}: {a['qt_registros_before']} > {a['qt_registros_after']}" + for a in changed_record_count + ] + ) + ) + + if unique_key: + join_pattern = ", '_', " + unique_key = ( + unique_key + if isinstance(unique_key, str) + else f"concat({join_pattern.join(unique_key)})" + ) + partition_filter = ",".join( + [f"'{p}'" for p in changed_partitions_after["particao"].to_list()] + ) + unique_count_sql = f""" + select + {unique_key} as unique_key, + count(*) as qtd + from {model_info["project"]}.{model_info["dataset_id"]}.{model_info["table_id"]} + where {partition_column_name} in ({partition_filter}) + group by 1 + having count(*) > 1 + """ + unique_count = bd.read_sql(unique_count_sql) + if len(unique_count) > 0: + print("Linhas duplicadas:") + print(unique_count) diff --git a/queries/macros/custom_get_where_subquery.sql b/queries/macros/custom_get_where_subquery.sql index 03a368448..95099f9ee 100644 --- a/queries/macros/custom_get_where_subquery.sql +++ b/queries/macros/custom_get_where_subquery.sql @@ -12,7 +12,7 @@ {% set where = where | replace("__date_range_end__", date_range_end) %} {% endif %} {%- set filtered -%} - (select * from {{ relation }} where {{ where }}) dbt_subquery + (select * from {{ relation }} where {{ where }}) {%- endset -%} {% do return(filtered) %} {%- else -%} diff --git a/queries/macros/list_columns.sql b/queries/macros/list_columns.sql new file mode 100644 index 000000000..cfe94143c --- /dev/null +++ b/queries/macros/list_columns.sql @@ -0,0 +1,8 @@ +{% macro list_columns() %} + {% set relation = adapter.get_relation( + database=this.database, schema=this.schema, identifier=this.identifier + ) %} + {% set columns = adapter.get_columns_in_relation(relation) %} + {% set column_names = columns | map(attribute="name") | list %} + {{ return(column_names) }} +{% endmacro %} diff --git a/queries/macros/test_check_gps_capture.sql b/queries/macros/test_check_gps_capture.sql new file mode 100644 index 000000000..848512292 --- /dev/null +++ b/queries/macros/test_check_gps_capture.sql @@ -0,0 +1,42 @@ +{% test check_gps_capture(model, table_id, interval) -%} + +WITH + t AS ( + SELECT + DATETIME(timestamp_array) AS timestamp_array + FROM + UNNEST( GENERATE_TIMESTAMP_ARRAY( TIMESTAMP("{{ var('date_range_start') }}"), TIMESTAMP("{{ var('date_range_end') }}"), INTERVAL {{ interval }} minute) ) AS timestamp_array + WHERE + timestamp_array < TIMESTAMP("{{ var('date_range_end') }}") ), + logs_table AS ( + SELECT + SAFE_CAST(DATETIME(TIMESTAMP(timestamp_captura), "America/Sao_Paulo") AS DATETIME) timestamp_captura, + SAFE_CAST(sucesso AS BOOLEAN) sucesso, + SAFE_CAST(erro AS STRING) erro, + SAFE_CAST(DATA AS DATE) DATA + FROM + rj-smtr-staging.br_rj_riodejaneiro_onibus_gps_staging.{{ table_id }}_logs AS t ), + logs AS ( + SELECT + *, + TIMESTAMP_TRUNC(timestamp_captura, minute) AS timestamp_array + FROM + logs_table + WHERE + DATA BETWEEN DATE(TIMESTAMP("{{ var('date_range_start') }}")) + AND DATE(TIMESTAMP("{{ var('date_range_end') }}")) + AND timestamp_captura BETWEEN "{{ var('date_range_start') }}" + AND "{{ var('date_range_end') }}" ) + SELECT + COALESCE(logs.timestamp_captura, t.timestamp_array) AS timestamp_captura, + logs.erro + FROM + t + LEFT JOIN + logs + ON + logs.timestamp_array = t.timestamp_array + WHERE + logs.sucesso IS NOT TRUE + +{%- endtest %} diff --git a/queries/macros/test_check_gps_treatment.sql b/queries/macros/test_check_gps_treatment.sql new file mode 100644 index 000000000..91502ce17 --- /dev/null +++ b/queries/macros/test_check_gps_treatment.sql @@ -0,0 +1,104 @@ +{% test check_gps_treatment(model) -%} +WITH + data_hora AS ( + SELECT + EXTRACT(date + FROM + timestamp_array) AS DATA, + EXTRACT(hour + FROM + timestamp_array) AS hora, + FROM + UNNEST(GENERATE_TIMESTAMP_ARRAY("{{ var('date_range_start') }}", "{{ var('date_range_end') }}", INTERVAL 1 hour)) AS timestamp_array ), + gps_raw AS ( + SELECT + EXTRACT(date + FROM + timestamp_gps) AS DATA, + EXTRACT(hour + FROM + timestamp_gps) AS hora, + COUNT(*) AS q_gps_raw + FROM + -- `rj-smtr.br_rj_riodejaneiro_onibus_gps.sppo_registros` + {{ ref('sppo_registros') }} + WHERE + DATA BETWEEN DATE("{{ var('date_range_start') }}") + AND DATE("{{ var('date_range_end') }}") + GROUP BY + 1, + 2 ), + gps_filtrada AS ( + SELECT + EXTRACT(date + FROM + timestamp_gps) AS DATA, + EXTRACT(hour + FROM + timestamp_gps) AS hora, + COUNT(*) AS q_gps_filtrada + FROM + -- `rj-smtr.br_rj_riodejaneiro_onibus_gps.sppo_aux_registros_filtrada` + {{ ref('sppo_aux_registros_filtrada') }} + WHERE + DATA BETWEEN DATE("{{ var('date_range_start') }}") + AND DATE("{{ var('date_range_end') }}") + GROUP BY + 1, + 2 ), + gps_sppo AS ( + SELECT + DATA, + EXTRACT(hour + FROM + timestamp_gps) AS hora, + COUNT(*) AS q_gps_treated + FROM + -- `rj-smtr.br_rj_riodejaneiro_veiculos.gps_sppo` + {{ ref('gps_sppo') }} + WHERE + DATA BETWEEN DATE("{{ var('date_range_start') }}") + AND DATE("{{ var('date_range_end') }}") + GROUP BY + 1, + 2), + gps_join AS ( + SELECT + *, + SAFE_DIVIDE(q_gps_filtrada, q_gps_raw) as indice_tratamento_raw, + SAFE_DIVIDE(q_gps_treated, q_gps_filtrada) as indice_tratamento_filtrada, + CASE + WHEN q_gps_raw = 0 OR q_gps_filtrada = 0 OR q_gps_treated = 0 -- Hipótese de perda de dados no tratamento + OR q_gps_raw IS NULL OR q_gps_filtrada IS NULL OR q_gps_treated IS NULL -- Hipótese de perda de dados no tratamento + OR (q_gps_raw <= q_gps_filtrada) OR (q_gps_filtrada < q_gps_treated) -- Hipótese de duplicação de dados + OR (COALESCE(SAFE_DIVIDE(q_gps_filtrada, q_gps_raw), 0) < 0.96) -- Hipótese de perda de dados no tratamento (superior a 3%) + OR (COALESCE(SAFE_DIVIDE(q_gps_treated, q_gps_filtrada), 0) < 0.96) -- Hipótese de perda de dados no tratamento (superior a 3%) + THEN FALSE + ELSE + TRUE + END + AS status + FROM + data_hora + LEFT JOIN + gps_raw + USING + (DATA, + hora) + LEFT JOIN + gps_filtrada + USING + (DATA, + hora) + LEFT JOIN + gps_sppo + USING + (DATA, + hora)) + SELECT + * + FROM + gps_join + WHERE + status IS FALSE +{%- endtest %} \ No newline at end of file diff --git a/queries/macros/test_check_trips_processing.sql b/queries/macros/test_check_trips_processing.sql new file mode 100644 index 000000000..8afadecfa --- /dev/null +++ b/queries/macros/test_check_trips_processing.sql @@ -0,0 +1,31 @@ +{% test check_trips_processing(model) -%} +SELECT + s.data, + s.tipo_dia, + s.subtipo_dia, + s.tipo_os, + s.feed_version, + s.feed_start_date AS feed_start_date_invalido, + i.feed_start_date AS feed_start_date_valido, +FROM ( + SELECT + * + FROM + {{ ref('subsidio_data_versao_efetiva') }} + -- rj-smtr.projeto_subsidio_sppo.subsidio_data_versao_efetiva + WHERE + DATA >= "{{ var('DATA_SUBSIDIO_V6_INICIO') }}" + AND DATA BETWEEN DATE("{{ var('start_date') }}") + AND DATE("{{ var('end_date') }}") + ) AS s +LEFT JOIN + -- rj-smtr.gtfs.feed_info AS i + {{ ref('feed_info') }} AS i +ON +(DATA BETWEEN i.feed_start_date + AND i.feed_end_date + OR (DATA >= i.feed_start_date + AND i.feed_end_date IS NULL)) +WHERE + i.feed_start_date != s.feed_start_date +{%- endtest %} \ No newline at end of file diff --git a/queries/macros/test_check_viagem_completa.sql b/queries/macros/test_check_viagem_completa.sql new file mode 100644 index 000000000..41bd10e63 --- /dev/null +++ b/queries/macros/test_check_viagem_completa.sql @@ -0,0 +1,53 @@ +{% test check_viagem_completa(model) -%} +WITH + data_versao_efetiva AS ( + SELECT + * + FROM + -- rj-smtr.projeto_subsidio_sppo.subsidio_data_versao_efetiva + {{ ref('subsidio_data_versao_efetiva') }} + WHERE + DATA >= "{{ var('DATA_SUBSIDIO_V6_INICIO') }}" + AND DATA BETWEEN DATE("{{ var('start_date') }}") + AND DATE("{{ var('end_date') }}")), + viagem_completa AS ( + SELECT + * + FROM + -- rj-smtr.projeto_subsidio_sppo.viagem_completa + {{ ref('viagem_completa') }} + WHERE + DATA >= "{{ var('DATA_SUBSIDIO_V6_INICIO') }}" + AND DATA BETWEEN DATE("{{ var('start_date') }}") + AND DATE("{{ var('end_date') }}")), + feed_info AS ( + SELECT + * + FROM + -- rj-smtr.gtfs.feed_info + {{ ref('feed_info_gtfs') }} + WHERE + feed_version IN ( + SELECT + feed_version + FROM + data_versao_efetiva) ) + SELECT + DISTINCT DATA + FROM + viagem_completa + LEFT JOIN + data_versao_efetiva AS d + USING + (DATA) + LEFT JOIN + feed_info AS i + ON + (DATA BETWEEN i.feed_start_date + AND i.feed_end_date + OR (DATA >= i.feed_start_date + AND i.feed_end_date IS NULL)) + WHERE + i.feed_start_date != d.feed_start_date + OR datetime_ultima_atualizacao < feed_update_datetime +{%- endtest %} \ No newline at end of file diff --git a/queries/macros/test_subsido_viagens_atualizadas.sql b/queries/macros/test_subsido_viagens_atualizadas.sql new file mode 100644 index 000000000..d8cc52991 --- /dev/null +++ b/queries/macros/test_subsido_viagens_atualizadas.sql @@ -0,0 +1,33 @@ +{% test subsidio_viagens_atualizadas(model) -%} +WITH + viagem_completa AS ( + SELECT + data, + datetime_ultima_atualizacao + FROM + -- rj-smtr.projeto_subsidio_sppo.viagem_completa + {{ ref('viagem_completa') }} + WHERE + DATA >= "{{ var('DATA_SUBSIDIO_V6_INICIO') }}" + AND DATA BETWEEN DATE("{{ var('start_date') }}") + AND DATE("{{ var('end_date') }}")), + sumario_servico_dia_historico AS ( + SELECT + data, + datetime_ultima_atualizacao + FROM + {{ model }} + WHERE + DATA BETWEEN DATE("{{ var('start_date') }}") + AND DATE("{{ var('end_date') }}")) + SELECT + DISTINCT DATA + FROM + viagem_completa as c + LEFT JOIN + sumario_servico_dia_historico AS h + USING + (DATA) + WHERE + c.datetime_ultima_atualizacao > h.datetime_ultima_atualizacao +{%- endtest %} \ No newline at end of file diff --git a/queries/macros/test_sumario_servico_dia_tipo_soma_km.sql b/queries/macros/test_sumario_servico_dia_tipo_soma_km.sql new file mode 100644 index 000000000..8ee20c965 --- /dev/null +++ b/queries/macros/test_sumario_servico_dia_tipo_soma_km.sql @@ -0,0 +1,20 @@ +{% test sumario_servico_dia_tipo_soma_km(model, column_name) -%} +WITH + kms AS ( + SELECT + * EXCEPT({{ column_name }}), + {{ column_name }}, + ROUND(COALESCE(km_apurada_registrado_com_ar_inoperante,0) + COALESCE(km_apurada_n_licenciado,0) + COALESCE(km_apurada_autuado_ar_inoperante,0) + COALESCE(km_apurada_autuado_seguranca,0) + COALESCE(km_apurada_autuado_limpezaequipamento,0) + COALESCE(km_apurada_licenciado_sem_ar_n_autuado,0) + COALESCE(km_apurada_licenciado_com_ar_n_autuado,0) + COALESCE(km_apurada_n_vistoriado, 0) + COALESCE(km_apurada_sem_transacao, 0),2) AS km_apurada2 + FROM + {{ model }} + WHERE + DATA BETWEEN DATE("{{ var('start_date') }}") + AND DATE("{{ var('end_date') }}")) +SELECT + *, + ABS(km_apurada2-{{ column_name }}) AS dif +FROM + kms +WHERE + ABS(km_apurada2-{{ column_name }}) > 0.02 +{%- endtest %} \ No newline at end of file diff --git a/queries/macros/test_teto_pagamento_valor_subsidio_pago.sql b/queries/macros/test_teto_pagamento_valor_subsidio_pago.sql new file mode 100644 index 000000000..79c5155af --- /dev/null +++ b/queries/macros/test_teto_pagamento_valor_subsidio_pago.sql @@ -0,0 +1,35 @@ +{% test teto_pagamento_valor_subsidio_pago(model, table_id, schema, expression) -%} +WITH +{{ table_id }} AS ( + SELECT + *, + FROM + {{ model }} + WHERE + DATA BETWEEN DATE("{{ var('date_range_start') }}") + AND DATE("{{ var('date_range_end') }}")), +subsidio_valor_km_tipo_viagem AS ( + SELECT + data_inicio, + data_fim, + MAX(subsidio_km) AS subsidio_km_teto + FROM + -- `rj-smtr`.`dashboard_subsidio_sppo_staging`.`subsidio_valor_km_tipo_viagem` + {{ ref('subsidio_valor_km_tipo_viagem') }} + WHERE + subsidio_km > 0 + GROUP BY + 1, + 2) +SELECT + * +FROM + {{ table_id }} AS s +LEFT JOIN + subsidio_valor_km_tipo_viagem AS p +ON + s.data BETWEEN p.data_inicio + AND p.data_fim +WHERE + NOT({{ expression }}) +{%- endtest %} \ No newline at end of file diff --git a/queries/models/bilhetagem/CHANGELOG.md b/queries/models/bilhetagem/CHANGELOG.md new file mode 100644 index 000000000..e17c3ac73 --- /dev/null +++ b/queries/models/bilhetagem/CHANGELOG.md @@ -0,0 +1,192 @@ +# Changelog - bilhetagem + +## [3.0.0] - 2024-11-25 + +### Adicionado +- Cria os modelos `aux_transacao_id_ordem_pagamento.sql` e `staging_transacao_ordem.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/333) + +### Alterado +- Adiciona colunas de ordem de pagamento e `datetime_ultima_atualizacao` no modelo `transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/333) +- Adiciona colunas de id único nos modelos `ordem_pagamento_consorcio_dia.sql` e `ordem_pagamento_servico_operador_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/333) + +## [2.5.2] - 2024-11-13 + +### Alterado +- Cria coluna `datetime_ultima_atualizacao` no modelo `ordem_pagamento_consorcio_operador_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/321) +- Converte `timestamp_captura` para datetime no modelo `staging_ordem_pagamento_consorcio_operadora` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/321) + +### Corrigido +- Ajusta lógica de preenchimento de valores pagos no modelo `ordem_pagamento_consorcio_operador_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/321) + +## [2.5.1] - 2024-10-07 + +### Alterado +- Retira tratamento de valores nulos da coluna `tipo_gratuidade` no modelo `transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/261) +- Adiciona tratamento de valores nulos na coluna `tipo_transacao_detalhe_smtr` no modelo `aux_passageiros_hora.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/261) + +## [2.5.0] - 2024-09-10 + +### Alterado +- Adiciona coluna `id_ordem_pagamento_consorcio_operador_dia` no modelo `ordem_pagamento_consorcio_operador_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/191) +- Cria tratamento da coluna `timestamp_captura` no modelo `staging_linha_consorcio_operadora_transporte.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/191) + + +## [2.4.0] - 2024-09-05 + +### Alterado +- Adiciona coluna `hash_cliente` no modelo `transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/189) + +## [2.3.0] - 2024-08-26 + +### Adicionado +- Cria modelos `staging_linha_consorcio_operadora_transporte.sql` e `staging_endereco.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/164) + +### Alterado +- Utiliza data e hora no filtro incremental dos modelos (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/164): + - `transacao.sql` + - `transacao_riocard.sql` + - `passageiros_hora.sql` + - `passageiros_tile_hora.sql` +- Adiciona coluna geo_point_transacao nos modelos `transacao.sql` e `transacao_riocard.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/164) +- Altera materialized do modelo `aux_passageiros_hora.sql` de `table` para `ephemeral` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/164) +- Adiciona transações do RioCard no modelo `aux_passageiros_hora.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/164) + + +## [2.2.1] - 2024-08-09 + +### Adicionado +- Adiciona descrição da coluna `id_validador` em `transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/139) +- Adiciona descrição da coluna `quantidade_total_transacao` em `ordem_pagamento_consorcio_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/139) +- Adiciona descrição das colunas `quantidade_total_transacao` e `valor_pago` em `ordem_pagamento_consorcio_operador_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/139) +- Adiciona descrição da coluna `quantidade_total_transacao` em `ordem_pagamento_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/139) +- Adiciona descrição das colunas `tipo_gratuidade` e `tipo_pagamento` em `passageiros_hora.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/139) +- Adiciona descrição das colunas `tipo_gratuidade` e `tipo_pagamento` em `passageiros_tile_hora.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/139) + +## [2.2.0] - 2024-08-05 + +### Adicionado +- Cria modelo `aux_h3_res9.sql` para tratar os dados geográficos de tile (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/130) +- Cria modelo `staging_linha_consorcio.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/130) + +### Alterado +- `aux_passageiros_hora.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/130) + - Faz conversão dos dados geograficos no modelo + - Altera materialized de `ephemeral` para `table` + - Adiciona transações do tipo RioCard +- Muda tipo das colunas de valor do modelo `staging_transacao_riocard.sql` de `float` para `numeric` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/130) +- Altera lógica incremental do modelo `transacao_riocard.sql` para rodar de hora em hora (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/130) + +## [2.1.4] - 2024-08-02 + +### Alterado +- Adiciona tag `geolocalizacao` aos modelos `gps_validador_van.sql` e `gps_validador.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/127) +- Adiciona tag `identificacao` ao modelo `staging_cliente.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/127) + +## [2.1.3] - 2024-07-18 + +### Adicionado +- Adiciona transações de Van no modelo `passageiros_hora.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/110) + +### Alterado +- Define o tipo_gratuidade de transações do tipo `Gratuidade` que o cliente não foi encontrado na tabela `br_rj_riodejaneiro_bilhetagem_staging.aux_gratuidade` como `Não Identificado` no modelo `transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/110) + +## [2.1.2] - 2024-07-05 + +### Adicionado +- Adiciona coluna `versao_app` nos modelos `gps_validador_aux.sql`, `gps_validador.sql` e `gps_validador_van.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/96) + +## [2.1.1] - 2024-06-19 + +### Corrigido +- Remove filtro de partições de gratuidade no modelo `transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/69) + +## [2.1.0] - 2024-06-13 + +### Alterado +- Adiciona colunas `data_pagamento` e `valor_pago` no modelo `ordem_pagamento_consorcio_operador_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/60) +- Subtrai valor pago a mais ao VLT (data_ordem = 2024-05-31) da ordem do dia 2024-06-07 (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/60) +- Adiciona prioridade dos dados novos em relação aos antigos no modelo `transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/60) + +## [2.0.0] - 2024-06-11 + +### Adicionado +- Cria modelos `passageiros_hora_aux.sql` e `passageiros_tile_hora.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/56) + +### Alterado +- Adiciona colunas tipo_transacao_smtr e valor_pagamento no modelo `transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/56) +- Altera lógica da execução incremental no modelo `transacao.sql` para atualizar valores de rateio e o tipo de transação da primeira perna das integrações (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/56) + +### Corrigido +- Corrige filtro no modelo `passageiros_hora_aux.sql` para adicionar transações de Ônibus (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/56) + +## [1.2.1] - 2024-05-20 + +### Corrigido +- Altera alias da tabela `linha_sem_ressarcimento` no modelo `transacao.sql` (https://github.com/prefeitura-rio/queries-rj-smtr/pull/317) +- Corrige select servico no modelo `ordem_pagamento_servico_operador_dia.sql` (https://github.com/prefeitura-rio/queries-rj-smtr/pull/317) + +## [1.2.0] - 2024-05-20 + +### Alterado +- Adiciona colunas `servico_jae` e `descricao_servico_jae` nos modelos (https://github.com/prefeitura-rio/queries-rj-smtr/pull/311): + - `transacao.sql` + - `integracao.sql` + - `ordem_pagamento_servico_operador_dia.sql` + - `passageiros_hora.sql` + - `gps_validador.sql` + - `gps_validador_van.sql` + - `staging/gps_validador_aux.sql` +- Adiciona coluna id_servico_jae nos modelos (https://github.com/prefeitura-rio/queries-rj-smtr/pull/311): + - `gps_validador.sql` + - `gps_validador_van.sql` + - `staging/gps_validador_aux.sql` + +- Remove coluna `servico` no modelo de `staging/gps_validador_aux.sql` para pegar o dado da tabela de cadastro + +## [1.1.0] - 2024-05-16 + +### Alterado +- Adiciona tratamento da coluna id_veiculo nos modelos ` transacao.sql` e `gps_validador.sql` (https://github.com/prefeitura-rio/queries-rj-smtr/pull/297) +- Adiciona coluna `quantidade_total_transacao` nos modelos `ordem_pagamento_consorcio_dia.sql`, `ordem_pagamento_consorcio_operador_dia.sql` e `ordem_pagamento_dia.sql` (https://github.com/prefeitura-rio/queries-rj-smtr/pull/297) +- Remove validação do modelo `ordem_pagamento_servico_operador_dia.sql` (https://github.com/prefeitura-rio/queries-rj-smtr/pull/297) + +## [1.0.3] - 2024-05-03 + +### Corrigido +- Removido tratamento de arredondamento nos valores totais (https://github.com/prefeitura-rio/queries-rj-smtr/pull/294): + - `bilhetagem.ordem_pagamento_dia` + - `bilhetagem.ordem_pagamento_consorcio_operador_dia` + - `bilhetagem.ordem_pagamento_consorcio_dia` + +### Alterado +- Alterado cast de float para numeric (https://github.com/prefeitura-rio/queries-rj-smtr/pull/294): + - `bilhetagem_staging.staging_ordem_pagamento` + - `bilhetagem_staging.staging_ordem_pagamento_consorcio` + - `bilhetagem_staging.staging_ordem_pagamento_consorcio_operadora` + - `bilhetagem_staging.staging_ordem_rateio` + - `bilhetagem_staging.staging_ordem_ressarcimento` + +## [1.0.2] - 2024-04-18 + +### Alterado +- Filtra transações inválidas ou de teste no modelo `transacao.sql` (https://github.com/prefeitura-rio/queries-rj-smtr/pull/275) + - Transações inválidas: + - datas anteriores a 2023-07-17 + - Transações teste: + - linhas sem ressarcimento +- Limita quantidade de ids listados no filtro da tabela de gratuidades no modelo `transacao.sql` (https://github.com/prefeitura-rio/queries-rj-smtr/pull/275) + +## [1.0.1] - 2024-04-16 + +### Adicionado +- Adicionada descrições das colunas das tabelas: `ordem_pagamento_consorcio_dia`, `ordem_pagamento_consorcio_operador_dia`, `ordem_pagamento_dia`,`transacao_riocard` +- Adicionada a descrição da coluna intervalo_integracao na tabela `integracao` + +### Corrigido +- deletada a tabela ordem_pagamento do schema + +## [1.0.0] - 2024-04-05 + +### Adicionado +- Nova view para consultar os dados staging de transações do RioCard capturados pela Jaé: `br_rj_riodejaneiro_bilhetagem_staging/staging_transacao_riocard.sql` (https://github.com/prefeitura-rio/queries-rj-smtr/pull/254) +- Tabela tratada de transações do RioCard capturados pela Jaé: `transacao_riocard.sql` (https://github.com/prefeitura-rio/queries-rj-smtr/pull/254) diff --git a/queries/models/bilhetagem/staging/aux_transacao_id_ordem_pagamento.sql b/queries/models/bilhetagem/staging/aux_transacao_id_ordem_pagamento.sql new file mode 100644 index 000000000..f7821bebc --- /dev/null +++ b/queries/models/bilhetagem/staging/aux_transacao_id_ordem_pagamento.sql @@ -0,0 +1,83 @@ +{{ + config( + materialized="incremental", + partition_by={"field": "data", "data_type": "date", "granularity": "day"}, + incremental_strategy="insert_overwrite", + ) +}} + +{% set incremental_filter %} + DATE(data) BETWEEN DATE("{{var('date_range_start')}}") AND DATE("{{var('date_range_end')}}") + AND timestamp_captura BETWEEN DATETIME("{{var('date_range_start')}}") AND DATETIME("{{var('date_range_end')}}") +{% endset %} + +{% set transacao_ordem = ref("staging_transacao_ordem") %} +{% if execute %} + {% if is_incremental() %} + {% set partitions_query %} + + SELECT DISTINCT + CONCAT("'", DATE(data_transacao), "'") AS data_transacao + FROM + {{ transacao_ordem }} + WHERE + {{ incremental_filter }} + + {% endset %} + + {% set partitions = run_query(partitions_query).columns[0].values() %} + + {% endif %} +{% endif %} + +with + staging as ( + select + date(data_transacao) as data, + id as id_transacao, + id_ordem_ressarcimento as id_ordem_pagamento_servico_operador_dia, + data_transacao as datetime_transacao, + data_processamento as datetime_processamento, + timestamp_captura as datetime_captura + from {{ transacao_ordem }} + {% if is_incremental() %} where {{ incremental_filter }} {% endif %} + ), + new_data as ( + select + s.data, + s.id_transacao, + s.datetime_transacao, + s.datetime_processamento, + o.data_ordem, + id_ordem_pagamento_servico_operador_dia, + o.id_ordem_pagamento_consorcio_operador_dia, + o.id_ordem_pagamento_consorcio_dia, + o.id_ordem_pagamento, + s.datetime_captura + from staging s + join + {{ ref("ordem_pagamento_servico_operador_dia") }} o using ( + id_ordem_pagamento_servico_operador_dia + ) + ), + complete_partitions as ( + select *, 0 as priority + from new_data + {% if is_incremental() and partitions | length > 0 %} + union all + select *, 1 as priority + from {{ this }} + where data in ({{ partitions | join(", ") }}) + {% endif %} + ) +select * except (rn, priority) +from + ( + select + *, + row_number() over ( + partition by id_transacao order by datetime_captura desc, priority + ) as rn + from complete_partitions + ) +where rn = 1 diff --git a/queries/models/bilhetagem/staging/staging_transacao_ordem.sql b/queries/models/bilhetagem/staging/staging_transacao_ordem.sql new file mode 100644 index 000000000..1541a994b --- /dev/null +++ b/queries/models/bilhetagem/staging/staging_transacao_ordem.sql @@ -0,0 +1,23 @@ +{{ + config( + alias="transacao_ordem", + ) +}} + +select + data, + id, + datetime( + parse_timestamp('%Y-%m-%d %H:%M:%S%Ez', timestamp_captura), "America/Sao_Paulo" + ) as timestamp_captura, + cast( + cast(cast(id_ordem_ressarcimento as float64) as integer) as string + ) as id_ordem_ressarcimento, + datetime( + parse_timestamp('%Y-%m-%dT%H:%M:%E6S%Ez', data_processamento), + "America/Sao_Paulo" + ) as data_processamento, + datetime( + parse_timestamp('%Y-%m-%dT%H:%M:%E6S%Ez', data_transacao), "America/Sao_Paulo" + ) as data_transacao +from {{ source("source_jae", "transacao_ordem") }} diff --git a/queries/models/br_rj_riodejaneiro_bilhetagem/CHANGELOG.md b/queries/models/br_rj_riodejaneiro_bilhetagem/CHANGELOG.md index 8f945772d..64ffecb31 100644 --- a/queries/models/br_rj_riodejaneiro_bilhetagem/CHANGELOG.md +++ b/queries/models/br_rj_riodejaneiro_bilhetagem/CHANGELOG.md @@ -1,5 +1,28 @@ # Changelog - bilhetagem +## [2.0.1] - 2024-12-12 + +### Alterado +- Adiciona coluna `datetime_captura` no modelo `ordem_pagamento_consorcio_operador_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/360) + +## [2.0.0] - 2024-11-25 + +### Adicionado +- Cria os modelos `aux_transacao_id_ordem_pagamento.sql` e `staging_transacao_ordem.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/333) + +### Alterado +- Adiciona colunas de ordem de pagamento e `datetime_ultima_atualizacao` no modelo `transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/333) +- Adiciona colunas de id único nos modelos `ordem_pagamento_consorcio_dia.sql` e `ordem_pagamento_servico_operador_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/333) + +## [2.5.2] - 2024-11-13 + +### Alterado +- Cria coluna `datetime_ultima_atualizacao` no modelo `ordem_pagamento_consorcio_operador_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/321) +- Converte `timestamp_captura` para datetime no modelo `staging_ordem_pagamento_consorcio_operadora` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/321) + +### Corrigido +- Ajusta lógica de preenchimento de valores pagos no modelo `ordem_pagamento_consorcio_operador_dia.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/321) + ## [2.5.1] - 2024-10-07 ### Alterado diff --git a/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_consorcio_dia.sql b/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_consorcio_dia.sql index fc3d7dca6..5bb4dd81d 100644 --- a/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_consorcio_dia.sql +++ b/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_consorcio_dia.sql @@ -13,6 +13,7 @@ -- depends_on: {{ ref("ordem_pagamento_consorcio_operador_dia") }} SELECT o.data_ordem, + o.id_ordem_pagamento_consorcio as id_ordem_pagamento_consorcio_dia, dc.id_consorcio, dc.consorcio, o.id_ordem_pagamento AS id_ordem_pagamento, diff --git a/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_consorcio_operador_dia.sql b/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_consorcio_operador_dia.sql index 4dd3be947..2b7002675 100644 --- a/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_consorcio_operador_dia.sql +++ b/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_consorcio_operador_dia.sql @@ -1,218 +1,226 @@ {{ - config( - materialized="incremental", - partition_by={ - "field":"data_ordem", - "data_type":"date", - "granularity": "day" - }, - incremental_strategy="insert_overwrite" - ) + config( + materialized="incremental", + partition_by={ + "field": "data_ordem", + "data_type": "date", + "granularity": "day", + }, + incremental_strategy="insert_overwrite", + ) }} -- depends_on: {{ ref("ordem_pagamento_servico_operador_dia") }} - -{% set ordem_pagamento_consorcio_operadora_staging = ref('staging_ordem_pagamento_consorcio_operadora') %} +{% set ordem_pagamento_consorcio_operadora_staging = ref( + "staging_ordem_pagamento_consorcio_operadora" +) %} +{% set aux_retorno_ordem_pagamento = ref("aux_retorno_ordem_pagamento") %} {% if execute %} - {% if is_incremental() %} - -- Verifica as ordens de pagamento capturadas - {% set partitions_query %} - SELECT DISTINCT - CONCAT("'", DATE(data_ordem), "'") AS data_ordem - FROM - {{ ordem_pagamento_consorcio_operadora_staging }} - WHERE - DATE(data) BETWEEN DATE("{{var('date_range_start')}}") AND DATE("{{var('date_range_end')}}") - {% endset %} + {% if is_incremental() %} + -- Verifica as ordens de pagamento capturadas + {% set partitions_query %} + SELECT DISTINCT + CONCAT("'", DATE(data_ordem), "'") AS data_ordem + FROM + {{ ordem_pagamento_consorcio_operadora_staging }} + WHERE + DATE(data) BETWEEN DATE("{{var('date_range_start')}}") AND DATE("{{var('date_range_end')}}") + AND timestamp_captura > DATETIME("{{var('date_range_start')}}") + AND timestamp_captura <= DATETIME("{{var('date_range_end')}}") + {% endset %} - {% set partitions = run_query(partitions_query).columns[0].values() %} - {% endif %} -{% endif %} + {% set partitions = run_query(partitions_query).columns[0].values() %} -WITH pagamento AS ( - SELECT - data_pagamento, - data_ordem, - id_consorcio, - id_operadora, - valor_pago - FROM - {{ ref("aux_retorno_ordem_pagamento") }} - -- `rj-smtr.controle_financeiro_staging.aux_retorno_ordem_pagamento` - {% if is_incremental() %} - WHERE - {% if partitions|length > 0 %} - data_ordem IN ({{ partitions|join(', ') }}) - {% else %} - data_ordem = '2000-01-01' + {% set paid_orders_query %} + SELECT + CONCAT("'", PARSE_DATE("%Y%m%d", partition_id), "'") AS data + FROM + `{{ aux_retorno_ordem_pagamento.database }}.{{ aux_retorno_ordem_pagamento.schema }}.INFORMATION_SCHEMA.PARTITIONS` + {# `rj-smtr.controle_financeiro_staging.INFORMATION_SCHEMA.PARTITIONS` #} + WHERE + table_name = "{{ aux_retorno_ordem_pagamento.identifier }}" + AND partition_id != "__NULL__" + AND DATETIME(last_modified_time, "America/Sao_Paulo") BETWEEN DATETIME("{{var('date_range_start')}}") AND (DATETIME("{{var('date_range_end')}}")) + {% endset %} + + {% set paid_orders = run_query(paid_orders_query).columns[0].values() %} {% endif %} - {% endif %} -), -ordem_pagamento AS ( - SELECT - o.data_ordem, - o.id_ordem_pagamento_consorcio_operadora AS id_ordem_pagamento_consorcio_operador_dia, - dc.id_consorcio, - dc.consorcio, - do.id_operadora, - do.operadora, - op.id_ordem_pagamento AS id_ordem_pagamento, - o.qtd_debito AS quantidade_transacao_debito, - o.valor_debito, - o.qtd_vendaabordo AS quantidade_transacao_especie, - o.valor_vendaabordo AS valor_especie, - o.qtd_gratuidade AS quantidade_transacao_gratuidade, - o.valor_gratuidade, - o.qtd_integracao AS quantidade_transacao_integracao, - o.valor_integracao, - o.qtd_rateio_credito AS quantidade_transacao_rateio_credito, - o.valor_rateio_credito AS valor_rateio_credito, - o.qtd_rateio_debito AS quantidade_transacao_rateio_debito, - o.valor_rateio_debito AS valor_rateio_debito, - ( - o.qtd_debito - + o.qtd_vendaabordo - + o.qtd_gratuidade - + o.qtd_integracao - ) AS quantidade_total_transacao, - o.valor_bruto AS valor_total_transacao_bruto, - o.valor_taxa AS valor_desconto_taxa, - o.valor_liquido AS valor_total_transacao_liquido_ordem - FROM - {{ ordem_pagamento_consorcio_operadora_staging }} o - -- `rj-smtr.br_rj_riodejaneiro_bilhetagem_staging.ordem_pagamento_consorcio_operadora` o - JOIN - {{ ref("staging_ordem_pagamento") }} op - -- `rj-smtr.br_rj_riodejaneiro_bilhetagem_staging.ordem_pagamento` op - ON - o.data_ordem = op.data_ordem - LEFT JOIN - {{ ref("operadoras") }} do - -- `rj-smtr.cadastro.operadoras` do - ON - o.id_operadora = do.id_operadora_jae - LEFT JOIN - {{ ref("consorcios") }} dc - -- `rj-smtr.cadastro.consorcios` dc - ON - o.id_consorcio = dc.id_consorcio_jae - {% if is_incremental() %} - WHERE - DATE(o.data) BETWEEN DATE("{{var('date_range_start')}}") AND DATE("{{var('date_range_end')}}") - {% endif %} -), -ordem_pagamento_completa AS ( - SELECT - *, - 0 AS priority - FROM - ordem_pagamento +{% endif %} + - {% if is_incremental() and paid_orders|length > 0 %} - UNION ALL +with + pagamento as ( + select data_pagamento, data_ordem, id_consorcio, id_operadora, valor_pago + from {{ aux_retorno_ordem_pagamento }} + -- `rj-smtr.controle_financeiro_staging.aux_retorno_ordem_pagamento` + {% if is_incremental() %} + where + {% if partitions | length > 0 %} + data_ordem in ({{ partitions | join(", ") }}) + {% else %} data_ordem = '2000-01-01' + {% endif %} + {% if paid_orders | length > 0 %} + or data_ordem in ({{ paid_orders | join(", ") }}) + {% endif %} - SELECT - data_ordem, - id_ordem_pagamento_consorcio_operador_dia, - id_consorcio, - consorcio, - id_operadora, - operadora, - id_ordem_pagamento, - quantidade_transacao_debito, - valor_debito, - quantidade_transacao_especie, - valor_especie, - quantidade_transacao_gratuidade, - valor_gratuidade, - quantidade_transacao_integracao, - valor_integracao, - quantidade_transacao_rateio_credito, - valor_rateio_credito, - quantidade_transacao_rateio_debito, - valor_rateio_debito, - quantidade_total_transacao, - valor_total_transacao_bruto, - valor_desconto_taxa, - valor_total_transacao_liquido_ordem, - 1 AS priority - FROM - {{ this }} - WHERE - data_ordem IN ({{ paid_orders|join(', ') }}) - {% endif %} -), -ordem_valor_pagamento AS ( - SELECT + {% endif %} + ), + ordem_pagamento as ( + select + o.data_ordem, + o.id_ordem_pagamento_consorcio_operadora + as id_ordem_pagamento_consorcio_operador_dia, + dc.id_consorcio, + dc.consorcio, + do.id_operadora, + do.operadora, + op.id_ordem_pagamento as id_ordem_pagamento, + o.qtd_debito as quantidade_transacao_debito, + o.valor_debito, + o.qtd_vendaabordo as quantidade_transacao_especie, + o.valor_vendaabordo as valor_especie, + o.qtd_gratuidade as quantidade_transacao_gratuidade, + o.valor_gratuidade, + o.qtd_integracao as quantidade_transacao_integracao, + o.valor_integracao, + o.qtd_rateio_credito as quantidade_transacao_rateio_credito, + o.valor_rateio_credito as valor_rateio_credito, + o.qtd_rateio_debito as quantidade_transacao_rateio_debito, + o.valor_rateio_debito as valor_rateio_debito, + ( + o.qtd_debito + o.qtd_vendaabordo + o.qtd_gratuidade + o.qtd_integracao + ) as quantidade_total_transacao, + o.valor_bruto as valor_total_transacao_bruto, + o.valor_taxa as valor_desconto_taxa, + o.valor_liquido as valor_total_transacao_liquido_ordem, + o.timestamp_captura as datetime_captura, + current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao + from {{ ordem_pagamento_consorcio_operadora_staging }} o + -- `rj-smtr.br_rj_riodejaneiro_bilhetagem_staging.ordem_pagamento_consorcio_operadora` o + join + {{ ref("staging_ordem_pagamento") }} op + {# `rj-smtr.br_rj_riodejaneiro_bilhetagem_staging.ordem_pagamento` op #} + on o.data_ordem = op.data_ordem + left join {{ ref("operadoras") }} do on o.id_operadora = do.id_operadora_jae + {# `rj-smtr.cadastro.operadoras` do on o.id_operadora = do.id_operadora_jae #} + left join {{ ref("consorcios") }} dc on o.id_consorcio = dc.id_consorcio_jae + {# `rj-smtr.cadastro.consorcios` dc on o.id_consorcio = dc.id_consorcio_jae #} + {% if is_incremental() %} + where + date(o.data) between date("{{var('date_range_start')}}") and date( + "{{var('date_range_end')}}" + ) + and o.timestamp_captura > datetime("{{var('date_range_start')}}") + and o.timestamp_captura <= datetime("{{var('date_range_end')}}") + {% else %} where date(o.data) < date("2024-11-13") + {% endif %} + ), + ordem_pagamento_completa as ( + select *, 0 as priority + from ordem_pagamento + + {% if is_incremental() and paid_orders | length > 0 %} + union all + + select + data_ordem, + id_ordem_pagamento_consorcio_operador_dia, + id_consorcio, + consorcio, + id_operadora, + operadora, + id_ordem_pagamento, + quantidade_transacao_debito, + valor_debito, + quantidade_transacao_especie, + valor_especie, + quantidade_transacao_gratuidade, + valor_gratuidade, + quantidade_transacao_integracao, + valor_integracao, + quantidade_transacao_rateio_credito, + valor_rateio_credito, + quantidade_transacao_rateio_debito, + valor_rateio_debito, + quantidade_total_transacao, + valor_total_transacao_bruto, + valor_desconto_taxa, + valor_total_transacao_liquido_ordem, + datetime_captura, + datetime_ultima_atualizacao, + 1 as priority + from {{ this }} + where data_ordem in ({{ paid_orders | join(", ") }}) + {% endif %} + ), + ordem_valor_pagamento as ( + select + data_ordem, + id_ordem_pagamento_consorcio_operador_dia, + id_consorcio, + o.consorcio, + id_operadora, + o.operadora, + o.id_ordem_pagamento, + o.quantidade_transacao_debito, + o.valor_debito, + o.quantidade_transacao_especie, + o.valor_especie, + o.quantidade_transacao_gratuidade, + o.valor_gratuidade, + o.quantidade_transacao_integracao, + o.valor_integracao, + o.quantidade_transacao_rateio_credito, + o.valor_rateio_credito, + o.quantidade_transacao_rateio_debito, + o.valor_rateio_debito, + o.quantidade_total_transacao, + o.valor_total_transacao_bruto, + o.valor_desconto_taxa, + o.valor_total_transacao_liquido_ordem, + p.data_pagamento, + p.valor_pago, + o.datetime_captura, + o.datetime_ultima_atualizacao, + row_number() over ( + partition by data_ordem, id_consorcio, id_operadora order by priority + ) as rn + from ordem_pagamento_completa o + left join pagamento p using (data_ordem, id_consorcio, id_operadora) + ) +select data_ordem, id_ordem_pagamento_consorcio_operador_dia, id_consorcio, - o.consorcio, + consorcio, id_operadora, - o.operadora, - o.id_ordem_pagamento, - o.quantidade_transacao_debito, - o.valor_debito, - o.quantidade_transacao_especie, - o.valor_especie, - o.quantidade_transacao_gratuidade, - o.valor_gratuidade, - o.quantidade_transacao_integracao, - o.valor_integracao, - o.quantidade_transacao_rateio_credito, - o.valor_rateio_credito, - o.quantidade_transacao_rateio_debito, - o.valor_rateio_debito, - o.quantidade_total_transacao, - o.valor_total_transacao_bruto, - o.valor_desconto_taxa, - o.valor_total_transacao_liquido_ordem, - p.data_pagamento, - p.valor_pago, - ROW_NUMBER() OVER (PARTITION BY data_ordem, id_consorcio, id_operadora ORDER BY priority) AS rn - FROM - ordem_pagamento_completa o - LEFT JOIN - pagamento p - USING(data_ordem, id_consorcio, id_operadora) -) -SELECT - data_ordem, - id_ordem_pagamento_consorcio_operador_dia, - id_consorcio, - consorcio, - id_operadora, - operadora, - id_ordem_pagamento, - quantidade_transacao_debito, - valor_debito, - quantidade_transacao_especie, - valor_especie, - quantidade_transacao_gratuidade, - valor_gratuidade, - quantidade_transacao_integracao, - valor_integracao, - quantidade_transacao_rateio_credito, - valor_rateio_credito, - quantidade_transacao_rateio_debito, - valor_rateio_debito, - quantidade_total_transacao, - valor_total_transacao_bruto, - valor_desconto_taxa, - valor_total_transacao_liquido_ordem, - CASE - WHEN - data_ordem = '2024-06-07' - AND id_consorcio = '2' - AND id_operadora = '8' - THEN - valor_total_transacao_liquido_ordem - 1403.4532 -- Corrigir valor pago incorretamente ao VLT na ordem do dia 2024-05-31 - ELSE valor_total_transacao_liquido_ordem - END AS valor_total_transacao_liquido, - data_pagamento, - valor_pago, - '{{ var("version") }}' AS versao -FROM - ordem_valor_pagamento -WHERE - rn = 1 \ No newline at end of file + operadora, + id_ordem_pagamento, + quantidade_transacao_debito, + valor_debito, + quantidade_transacao_especie, + valor_especie, + quantidade_transacao_gratuidade, + valor_gratuidade, + quantidade_transacao_integracao, + valor_integracao, + quantidade_transacao_rateio_credito, + valor_rateio_credito, + quantidade_transacao_rateio_debito, + valor_rateio_debito, + quantidade_total_transacao, + valor_total_transacao_bruto, + valor_desconto_taxa, + valor_total_transacao_liquido_ordem, + case + when data_ordem = '2024-06-07' and id_consorcio = '2' and id_operadora = '8' + then valor_total_transacao_liquido_ordem - 1403.4532 -- Corrigir valor pago incorretamente ao VLT na ordem do dia 2024-05-31 + else valor_total_transacao_liquido_ordem + end as valor_total_transacao_liquido, + data_pagamento, + valor_pago, + datetime_captura, + '{{ var("version") }}' as versao, + datetime_ultima_atualizacao +from ordem_valor_pagamento +where rn = 1 diff --git a/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_servico_operador_dia.sql b/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_servico_operador_dia.sql index 5d15dcaab..c0adf5360 100644 --- a/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_servico_operador_dia.sql +++ b/queries/models/br_rj_riodejaneiro_bilhetagem/ordem_pagamento_servico_operador_dia.sql @@ -1,102 +1,106 @@ {{ - config( - materialized="incremental", - partition_by={ - "field":"data_ordem", - "data_type":"date", - "granularity": "day" - }, - incremental_strategy="insert_overwrite" - ) + config( + materialized="incremental", + partition_by={ + "field": "data_ordem", + "data_type": "date", + "granularity": "day", + }, + incremental_strategy="insert_overwrite", + ) }} -WITH ordem_pagamento AS ( - SELECT - r.data_ordem, - dc.id_consorcio, - dc.consorcio, - do.id_operadora, - do.operadora, - r.id_linha AS id_servico_jae, - -- s.servico, - l.nr_linha AS servico_jae, - l.nm_linha AS descricao_servico_jae, - r.id_ordem_pagamento AS id_ordem_pagamento, - r.id_ordem_ressarcimento AS id_ordem_ressarcimento, - r.qtd_debito AS quantidade_transacao_debito, - r.valor_debito, - r.qtd_vendaabordo AS quantidade_transacao_especie, - r.valor_vendaabordo AS valor_especie, - r.qtd_gratuidade AS quantidade_transacao_gratuidade, - r.valor_gratuidade, - r.qtd_integracao AS quantidade_transacao_integracao, - r.valor_integracao, - COALESCE(rat.qtd_rateio_compensacao_credito_total, r.qtd_rateio_credito) AS quantidade_transacao_rateio_credito, - COALESCE(rat.valor_rateio_compensacao_credito_total, r.valor_rateio_credito) AS valor_rateio_credito, - COALESCE(rat.qtd_rateio_compensacao_debito_total, r.qtd_rateio_debito) AS quantidade_transacao_rateio_debito, - COALESCE(rat.valor_rateio_compensacao_debito_total, r.valor_rateio_debito) AS valor_rateio_debito, - ( - r.qtd_debito - + r.qtd_vendaabordo - + r.qtd_gratuidade - + r.qtd_integracao - ) AS quantidade_total_transacao, - r.valor_bruto AS valor_total_transacao_bruto, - r.valor_taxa AS valor_desconto_taxa, - r.valor_liquido AS valor_total_transacao_liquido - FROM - {{ ref("staging_ordem_ressarcimento") }} r - LEFT JOIN - {{ ref("staging_ordem_rateio") }} rat - USING(data_ordem, id_consorcio, id_operadora, id_linha) - LEFT JOIN - {{ ref("operadoras") }} AS do - ON - r.id_operadora = do.id_operadora_jae - LEFT JOIN - {{ ref("consorcios") }} AS dc - ON - r.id_consorcio = dc.id_consorcio_jae - LEFT JOIN - {{ ref("staging_linha") }} AS l - ON - r.id_linha = l.cd_linha - -- LEFT JOIN - -- {{ ref("servicos") }} AS s - -- ON - -- r.id_linha = s.id_servico_jae - {% if is_incremental() %} - WHERE - DATE(r.data) BETWEEN DATE("{{var('date_range_start')}}") AND DATE("{{var('date_range_end')}}") - {% endif %} -) -SELECT - o.data_ordem, - o.id_consorcio, - o.consorcio, - o.id_operadora, - o.operadora, - o.id_servico_jae, - o.servico_jae, - o.descricao_servico_jae, - o.id_ordem_pagamento, - o.id_ordem_ressarcimento, - o.quantidade_transacao_debito, - o.valor_debito, - o.quantidade_transacao_especie, - o.valor_especie, - o.quantidade_transacao_gratuidade, - o.valor_gratuidade, - o.quantidade_transacao_integracao, - o.valor_integracao, - o.quantidade_transacao_rateio_credito, - o.valor_rateio_credito, - o.quantidade_transacao_rateio_debito, - o.valor_rateio_debito, - o.quantidade_total_transacao, - o.valor_total_transacao_bruto + o.valor_rateio_debito + o.valor_rateio_credito AS valor_total_transacao_bruto, - o.valor_desconto_taxa, - o.valor_total_transacao_liquido + o.valor_rateio_debito + o.valor_rateio_credito AS valor_total_transacao_liquido, - '{{ var("version") }}' AS versao -FROM - ordem_pagamento o +with + ordem_pagamento as ( + select + r.data_ordem, + r.id_ordem_ressarcimento as id_ordem_pagamento_servico_operador_dia, + dc.id_consorcio, + dc.consorcio, + do.id_operadora, + do.operadora, + r.id_linha as id_servico_jae, + l.nr_linha as servico_jae, + l.nm_linha as descricao_servico_jae, + r.id_ordem_pagamento_consorcio_operadora + as id_ordem_pagamento_consorcio_operador_dia, + r.id_ordem_pagamento_consorcio as id_ordem_pagamento_consorcio_dia, + r.id_ordem_pagamento as id_ordem_pagamento, + r.id_ordem_ressarcimento as id_ordem_ressarcimento, + r.qtd_debito as quantidade_transacao_debito, + r.valor_debito, + r.qtd_vendaabordo as quantidade_transacao_especie, + r.valor_vendaabordo as valor_especie, + r.qtd_gratuidade as quantidade_transacao_gratuidade, + r.valor_gratuidade, + r.qtd_integracao as quantidade_transacao_integracao, + r.valor_integracao, + coalesce( + rat.qtd_rateio_compensacao_credito_total, r.qtd_rateio_credito + ) as quantidade_transacao_rateio_credito, + coalesce( + rat.valor_rateio_compensacao_credito_total, r.valor_rateio_credito + ) as valor_rateio_credito, + coalesce( + rat.qtd_rateio_compensacao_debito_total, r.qtd_rateio_debito + ) as quantidade_transacao_rateio_debito, + coalesce( + rat.valor_rateio_compensacao_debito_total, r.valor_rateio_debito + ) as valor_rateio_debito, + ( + r.qtd_debito + r.qtd_vendaabordo + r.qtd_gratuidade + r.qtd_integracao + ) as quantidade_total_transacao, + r.valor_bruto as valor_total_transacao_bruto, + r.valor_taxa as valor_desconto_taxa, + r.valor_liquido as valor_total_transacao_liquido + from {{ ref("staging_ordem_ressarcimento") }} r + left join + {{ ref("staging_ordem_rateio") }} rat using ( + data_ordem, id_consorcio, id_operadora, id_linha + ) + left join {{ ref("operadoras") }} as do on r.id_operadora = do.id_operadora_jae + left join {{ ref("consorcios") }} as dc on r.id_consorcio = dc.id_consorcio_jae + left join {{ ref("staging_linha") }} as l on r.id_linha = l.cd_linha + {% if is_incremental() %} + where + date(r.data) between date("{{var('date_range_start')}}") and date( + "{{var('date_range_end')}}" + ) + {% endif %} + ) +select + data_ordem, + id_ordem_pagamento_servico_operador_dia, + id_consorcio, + consorcio, + id_operadora, + operadora, + id_servico_jae, + servico_jae, + descricao_servico_jae, + id_ordem_pagamento_consorcio_operador_dia, + id_ordem_pagamento_consorcio_dia, + id_ordem_pagamento, + id_ordem_ressarcimento, + quantidade_transacao_debito, + valor_debito, + quantidade_transacao_especie, + valor_especie, + quantidade_transacao_gratuidade, + valor_gratuidade, + quantidade_transacao_integracao, + valor_integracao, + quantidade_transacao_rateio_credito, + valor_rateio_credito, + quantidade_transacao_rateio_debito, + valor_rateio_debito, + quantidade_total_transacao, + valor_total_transacao_bruto + + valor_rateio_debito + + valor_rateio_credito as valor_total_transacao_bruto, + valor_desconto_taxa, + valor_total_transacao_liquido + + valor_rateio_debito + + valor_rateio_credito as valor_total_transacao_liquido, + '{{ var("version") }}' as versao +from ordem_pagamento o diff --git a/queries/models/br_rj_riodejaneiro_bilhetagem/schema.yml b/queries/models/br_rj_riodejaneiro_bilhetagem/schema.yml index 63c3aca43..5ae100993 100644 --- a/queries/models/br_rj_riodejaneiro_bilhetagem/schema.yml +++ b/queries/models/br_rj_riodejaneiro_bilhetagem/schema.yml @@ -76,6 +76,16 @@ models: description: "Valor debitado na transação atual (R$)" - name: valor_pagamento description: "Valor valor a ser pago pela transação (R$). Valores nulos indicam que o valor de pagamento para a transação ainda não foi calculado." + - name: data_ordem + description: "Data da ordem de pagamento da transação" + - name: id_ordem_pagamento_servico_operador_dia + description: "Identificador único da tabela rj-smtr.br_rj_riodejaneiro_bilhetagem.ordem_pagamento_servico_operador_dia" + - name: id_ordem_pagamento_consorcio_operador_dia + description: "Identificador único da tabela rj-smtr.br_rj_riodejaneiro_bilhetagem.ordem_pagamento_consorcio_operador_dia" + - name: id_ordem_pagamento_consorcio_dia + description: "Identificador único da tabela rj-smtr.br_rj_riodejaneiro_bilhetagem.ordem_pagamento_consorcio_dia" + - name: id_ordem_pagamento + description: "Identificador único da tabela rj-smtr.br_rj_riodejaneiro_bilhetagem.ordem_pagamento_dia" - name: versao description: "Código de controle de versão do dado (SHA Github)" - name: integracao @@ -427,8 +437,14 @@ models: description: "Data em que foi realizado o pagamento ao operador ou consórcio" - name: valor_pago description: "Valor pago ao operador ou consórcio (R$)" + - name: datetime_captura + description: Data e hora da captura da ordem em GMT-3 - name: versao description: "Código de controle de versão do dado (SHA Github)" + - name: datetime_ultima_atualizacao + data_type: datetime + quote: true + description: "{{ doc('datetime_ultima_atualizacao') }}" - name: ordem_pagamento_dia description: "Tabela com os valores totais de pagamento da Jaé agregados por dia" columns: diff --git a/queries/models/br_rj_riodejaneiro_bilhetagem/transacao.sql b/queries/models/br_rj_riodejaneiro_bilhetagem/transacao.sql index 7bb29d7d4..1c6573290 100644 --- a/queries/models/br_rj_riodejaneiro_bilhetagem/transacao.sql +++ b/queries/models/br_rj_riodejaneiro_bilhetagem/transacao.sql @@ -9,8 +9,6 @@ ) }} --- TODO: Usar variável de run_date_hour para otimizar o numero de partições lidas em --- staging {% set incremental_filter %} DATE(data) BETWEEN DATE("{{var('date_range_start')}}") AND DATE("{{var('date_range_end')}}") AND timestamp_captura BETWEEN DATETIME("{{var('date_range_start')}}") AND DATETIME("{{var('date_range_end')}}") @@ -18,46 +16,70 @@ {% set transacao_staging = ref("staging_transacao") %} {% set integracao_staging = ref("staging_integracao_transacao") %} +{% set transacao_ordem = ref("aux_transacao_id_ordem_pagamento") %} {% if execute %} {% if is_incremental() %} {% set transacao_partitions_query %} - WITH particoes_integracao AS ( - SELECT DISTINCT - CONCAT("'", DATE(data_transacao), "'") AS data_transacao - FROM - {{ integracao_staging }}, - UNNEST([ - data_transacao_t0, - data_transacao_t1, - data_transacao_t2, - data_transacao_t3, - data_transacao_t4 - ]) AS data_transacao - WHERE - {{ incremental_filter }} - ), - particoes_transacao AS ( - SELECT DISTINCT - CONCAT("'", DATE(data_transacao), "'") AS data_transacao - FROM - {{ transacao_staging }} - WHERE - {{ incremental_filter }} - ) - SELECT - COALESCE(t.data_transacao, i.data_transacao) AS data_transacao - FROM - particoes_transacao t - FULL OUTER JOIN - particoes_integracao i - USING(data_transacao) - WHERE - COALESCE(t.data_transacao, i.data_transacao) IS NOT NULL + WITH particoes_integracao AS ( + SELECT DISTINCT + CONCAT("'", DATE(data_transacao), "'") AS data_transacao + FROM + {{ integracao_staging }}, + UNNEST([ + data_transacao_t0, + data_transacao_t1, + data_transacao_t2, + data_transacao_t3, + data_transacao_t4 + ]) AS data_transacao + WHERE + {{ incremental_filter }} + ), + particoes_transacao AS ( + SELECT DISTINCT + CONCAT("'", DATE(data_transacao), "'") AS data_transacao + FROM + {{ transacao_staging }} + WHERE + {{ incremental_filter }} + ), + particoes_transacao_ordem AS ( + SELECT + CONCAT("'", PARSE_DATE("%Y%m%d", partition_id), "'") AS data_transacao + FROM + `rj-smtr.{{ transacao_ordem.schema }}.INFORMATION_SCHEMA.PARTITIONS` + WHERE + table_name = "{{ transacao_ordem.identifier }}" + AND partition_id != "__NULL__" + AND DATETIME(last_modified_time, "America/Sao_Paulo") BETWEEN DATETIME("{{var('date_range_start')}}") AND (DATETIME("{{var('date_range_end')}}")) + ) + SELECT + data_transacao + FROM + particoes_transacao + WHERE + data_transacao IS NOT NULL + UNION DISTINCT + SELECT + data_transacao + FROM + particoes_integracao + WHERE + data_transacao IS NOT NULL + UNION DISTINCT + SELECT + data_transacao + FROM + particoes_transacao_ordem + WHERE + data_transacao IS NOT NULL + {% endset %} {% set transacao_partitions = run_query(transacao_partitions_query) %} {% set transacao_partition_list = transacao_partitions.columns[0].values() %} + {% endif %} {% endif %} @@ -65,7 +87,6 @@ with transacao as ( select * from {{ transacao_staging }} - -- `rj-smtr.br_rj_riodejaneiro_bilhetagem_staging.transacao` {% if is_incremental() %} where {{ incremental_filter }} {% endif %} ), tipo_transacao as ( @@ -100,6 +121,17 @@ with {% endif %} {% endif %} ), + transacao_ordem as ( + select * + from {{ ref("aux_transacao_id_ordem_pagamento") }} + {% if is_incremental() %} + where + {% if transacao_partition_list | length > 0 %} + data in ({{ transacao_partition_list | join(", ") }}) + {% else %} data = "2000-01-01" + {% endif %} + {% endif %} + ), new_data as ( select extract(date from data_transacao) as data, @@ -153,10 +185,7 @@ with {{ ref("consorcios") }} dc -- `rj-smtr.cadastro.consorcios` dc on t.cd_consorcio = dc.id_consorcio_jae - left join - {{ ref("staging_linha") }} l - -- `rj-smtr.br_rj_riodejaneiro_bilhetagem_staging.linha` l - on t.cd_linha = l.cd_linha + left join {{ ref("staging_linha") }} l on t.cd_linha = l.cd_linha -- LEFT JOIN -- {{ ref("servicos") }} AS s -- ON @@ -171,11 +200,20 @@ with and (t.data_transacao < g.data_fim_validade or g.data_fim_validade is null) left join {{ ref("staging_linha_sem_ressarcimento") }} lsr - -- `rj-smtr.br_rj_riodejaneiro_bilhetagem_staging.linha_sem_ressarcimento` - -- lsr on t.cd_linha = lsr.id_linha where lsr.id_linha is null and date(data_transacao) >= "2023-07-17" ), + {% if is_incremental() %} + transacao_atual as ( + select * + from {{ this }} + where + {% if transacao_partition_list | length > 0 %} + data in ({{ transacao_partition_list | join(", ") }}) + {% else %} data = "2000-01-01" + {% endif %} + ), + {% endif %} complete_partitions as ( select data, @@ -246,12 +284,7 @@ with stop_lon, valor_transacao, 1 as priority - from {{ this }} - where - {% if transacao_partition_list | length > 0 %} - data in ({{ transacao_partition_list | join(", ") }}) - {% else %} data = "2000-01-01" - {% endif %} + from transacao_atual {% endif %} ), transacao_deduplicada as ( @@ -267,53 +300,109 @@ with from complete_partitions ) where rn = 1 + ), + transacao_final as ( + select + t.data, + t.hora, + t.datetime_transacao, + t.datetime_processamento, + t.datetime_captura, + t.modo, + t.id_consorcio, + t.consorcio, + t.id_operadora, + t.operadora, + t.id_servico_jae, + t.servico_jae, + t.descricao_servico_jae, + t.sentido, + t.id_veiculo, + t.id_validador, + t.id_cliente, + sha256(t.id_cliente) as hash_cliente, + t.id_transacao, + t.tipo_pagamento, + t.tipo_transacao, + case + when t.tipo_transacao = "Integração" or i.id_transacao is not null + then "Integração" + when t.tipo_transacao in ("Débito", "Botoeira") + then "Integral" + else t.tipo_transacao + end as tipo_transacao_smtr, + t.tipo_gratuidade, + t.id_tipo_integracao, + t.id_integracao, + t.latitude, + t.longitude, + t.geo_point_transacao, + t.stop_id, + t.stop_lat, + t.stop_lon, + t.valor_transacao, + case + when + i.id_transacao is not null + or o.id_transacao is not null + or date(t.datetime_processamento) + < (select max(data_ordem) from {{ ref("ordem_pagamento_dia") }}) + then coalesce(i.valor_rateio, t.valor_transacao) * 0.96 + end as valor_pagamento, + o.data_ordem, + o.id_ordem_pagamento_servico_operador_dia, + o.id_ordem_pagamento_consorcio_operador_dia, + o.id_ordem_pagamento_consorcio_dia, + o.id_ordem_pagamento + from transacao_deduplicada t + left join integracao i using (id_transacao) + left join transacao_ordem o using (id_transacao) ) + {% set columns = ( + list_columns() + | reject("in", ["versao", "datetime_ultima_atualizacao"]) + | list + ) %} select - t.data, - t.hora, - t.datetime_transacao, - t.datetime_processamento, - t.datetime_captura, - t.modo, - t.id_consorcio, - t.consorcio, - t.id_operadora, - t.operadora, - t.id_servico_jae, - t.servico_jae, - t.descricao_servico_jae, - t.sentido, - t.id_veiculo, - t.id_validador, - t.id_cliente, - sha256(t.id_cliente) as hash_cliente, - t.id_transacao, - t.tipo_pagamento, - t.tipo_transacao, - case - when t.tipo_transacao = "Integração" or i.id_transacao is not null - then "Integração" - when t.tipo_transacao in ("Débito", "Botoeira") - then "Integral" - else t.tipo_transacao - end as tipo_transacao_smtr, - t.tipo_gratuidade, - t.id_tipo_integracao, - t.id_integracao, - t.latitude, - t.longitude, - t.geo_point_transacao, - t.stop_id, - t.stop_lat, - t.stop_lon, - t.valor_transacao, - case - when - i.id_transacao is not null - or date(t.datetime_processamento) - < (select max(data_ordem) from {{ ref("ordem_pagamento_dia") }}) - then coalesce(i.valor_rateio, t.valor_transacao) * 0.96 - end as valor_pagamento, - '{{ var("version") }}' as versao -from transacao_deduplicada t -left join integracao i using (id_transacao) + f.*, + '{{ var("version") }}' as versao, + {% if is_incremental() %} + case + when + a.id_transacao is null + or sha256( + concat( + {% for c in columns %} + {% if c == "geo_point_transacao" %} + ifnull(st_astext(f.geo_point_transacao), 'n/a') + {% elif c == "hash_cliente" %} + ifnull(to_base64(f.hash_cliente), 'n/a') + {% else %}ifnull(cast(f.{{ c }} as string), 'n/a') + {% endif %} + + {% if not loop.last %}, {% endif %} + + {% endfor %} + ) + ) != sha256( + concat( + {% for c in columns %} + {% if c == "geo_point_transacao" %} + ifnull(st_astext(a.geo_point_transacao), 'n/a') + {% elif c == "hash_cliente" %} + ifnull(to_base64(f.hash_cliente), 'n/a') + {% else %}ifnull(cast(a.{{ c }} as string), 'n/a') + {% endif %} + + {% if not loop.last %}, {% endif %} + + {% endfor %} + ) + ) + then current_datetime("America/Sao_Paulo") + else a.datetime_ultima_atualizacao + end + {% else %} current_datetime("America/Sao_Paulo") + {% endif %} as datetime_ultima_atualizacao +from transacao_final f +{% if is_incremental() %} left join transacao_atual a using (id_transacao) {% endif %} diff --git a/queries/models/br_rj_riodejaneiro_bilhetagem_staging/staging_ordem_pagamento_consorcio_operadora.sql b/queries/models/br_rj_riodejaneiro_bilhetagem_staging/staging_ordem_pagamento_consorcio_operadora.sql index f3283a809..65202cafb 100644 --- a/queries/models/br_rj_riodejaneiro_bilhetagem_staging/staging_ordem_pagamento_consorcio_operadora.sql +++ b/queries/models/br_rj_riodejaneiro_bilhetagem_staging/staging_ordem_pagamento_consorcio_operadora.sql @@ -1,47 +1,87 @@ {{ - config( - alias='ordem_pagamento_consorcio_operadora', - ) + config( + alias="ordem_pagamento_consorcio_operadora", + ) }} -WITH ordem_pagamento_consorcio_operadora AS ( - SELECT - data, - SAFE_CAST(id AS STRING) AS id_ordem_pagamento_consorcio_operadora, - timestamp_captura, - DATETIME(PARSE_TIMESTAMP('%Y-%m-%dT%H:%M:%E*S%Ez', SAFE_CAST(JSON_VALUE(content, '$.data_inclusao') AS STRING)), "America/Sao_Paulo") AS datetime_inclusao, - PARSE_DATE('%Y-%m-%d', SAFE_CAST(JSON_VALUE(content, '$.data_ordem') AS STRING)) AS data_ordem, - SAFE_CAST(JSON_VALUE(content, '$.id_consorcio') AS STRING) AS id_consorcio, - SAFE_CAST(JSON_VALUE(content, '$.id_operadora') AS STRING) AS id_operadora, - SAFE_CAST(JSON_VALUE(content, '$.id_ordem_pagamento_consorcio') AS STRING) AS id_ordem_pagamento_consorcio, - SAFE_CAST(JSON_VALUE(content, '$.qtd_debito') AS INTEGER) AS qtd_debito, - SAFE_CAST(JSON_VALUE(content, '$.qtd_gratuidade') AS INTEGER) AS qtd_gratuidade, - SAFE_CAST(JSON_VALUE(content, '$.qtd_integracao') AS INTEGER) AS qtd_integracao, - SAFE_CAST(JSON_VALUE(content, '$.qtd_rateio_credito') AS INTEGER) AS qtd_rateio_credito, - SAFE_CAST(JSON_VALUE(content, '$.qtd_rateio_debito') AS INTEGER) AS qtd_rateio_debito, - SAFE_CAST(JSON_VALUE(content, '$.qtd_vendaabordo') AS INTEGER) AS qtd_vendaabordo, - SAFE_CAST(JSON_VALUE(content, '$.valor_bruto') AS NUMERIC) AS valor_bruto, - SAFE_CAST(JSON_VALUE(content, '$.valor_debito') AS NUMERIC) AS valor_debito, - SAFE_CAST(JSON_VALUE(content, '$.valor_gratuidade') AS NUMERIC) AS valor_gratuidade, - SAFE_CAST(JSON_VALUE(content, '$.valor_integracao') AS NUMERIC) AS valor_integracao, - SAFE_CAST(JSON_VALUE(content, '$.valor_liquido') AS NUMERIC) AS valor_liquido, - SAFE_CAST(JSON_VALUE(content, '$.valor_rateio_credito') AS NUMERIC) AS valor_rateio_credito, - SAFE_CAST(JSON_VALUE(content, '$.valor_rateio_debito') AS NUMERIC) AS valor_rateio_debito, - SAFE_CAST(JSON_VALUE(content, '$.valor_taxa') AS NUMERIC) AS valor_taxa, - SAFE_CAST(JSON_VALUE(content, '$.valor_vendaabordo') AS NUMERIC) AS valor_vendaabordo - FROM - {{ source("br_rj_riodejaneiro_bilhetagem_staging", "ordem_pagamento_consorcio_operadora") }} -), -ordem_pagamento_consorcio_operadora_rn AS ( - SELECT - *, - ROW_NUMBER() OVER (PARTITION BY id_ordem_pagamento_consorcio_operadora ORDER BY timestamp_captura DESC) AS rn - FROM - ordem_pagamento_consorcio_operadora -) -SELECT - * EXCEPT(rn) -FROM - ordem_pagamento_consorcio_operadora_rn -WHERE - rn = 1 \ No newline at end of file +with + ordem_pagamento_consorcio_operadora as ( + select + data, + safe_cast(id as string) as id_ordem_pagamento_consorcio_operadora, + datetime( + parse_timestamp('%Y-%m-%d %H:%M:%S%Ez', timestamp_captura), + "America/Sao_Paulo" + ) as timestamp_captura, + datetime( + parse_timestamp( + '%Y-%m-%dT%H:%M:%E*S%Ez', + safe_cast(json_value(content, '$.data_inclusao') as string) + ), + "America/Sao_Paulo" + ) as datetime_inclusao, + parse_date( + '%Y-%m-%d', safe_cast(json_value(content, '$.data_ordem') as string) + ) as data_ordem, + safe_cast(json_value(content, '$.id_consorcio') as string) as id_consorcio, + safe_cast(json_value(content, '$.id_operadora') as string) as id_operadora, + safe_cast( + json_value(content, '$.id_ordem_pagamento_consorcio') as string + ) as id_ordem_pagamento_consorcio, + safe_cast(json_value(content, '$.qtd_debito') as integer) as qtd_debito, + safe_cast( + json_value(content, '$.qtd_gratuidade') as integer + ) as qtd_gratuidade, + safe_cast( + json_value(content, '$.qtd_integracao') as integer + ) as qtd_integracao, + safe_cast( + json_value(content, '$.qtd_rateio_credito') as integer + ) as qtd_rateio_credito, + safe_cast( + json_value(content, '$.qtd_rateio_debito') as integer + ) as qtd_rateio_debito, + safe_cast( + json_value(content, '$.qtd_vendaabordo') as integer + ) as qtd_vendaabordo, + safe_cast(json_value(content, '$.valor_bruto') as numeric) as valor_bruto, + safe_cast(json_value(content, '$.valor_debito') as numeric) as valor_debito, + safe_cast( + json_value(content, '$.valor_gratuidade') as numeric + ) as valor_gratuidade, + safe_cast( + json_value(content, '$.valor_integracao') as numeric + ) as valor_integracao, + safe_cast( + json_value(content, '$.valor_liquido') as numeric + ) as valor_liquido, + safe_cast( + json_value(content, '$.valor_rateio_credito') as numeric + ) as valor_rateio_credito, + safe_cast( + json_value(content, '$.valor_rateio_debito') as numeric + ) as valor_rateio_debito, + safe_cast(json_value(content, '$.valor_taxa') as numeric) as valor_taxa, + safe_cast( + json_value(content, '$.valor_vendaabordo') as numeric + ) as valor_vendaabordo + from + {{ + source( + "br_rj_riodejaneiro_bilhetagem_staging", + "ordem_pagamento_consorcio_operadora", + ) + }} + ), + ordem_pagamento_consorcio_operadora_rn as ( + select + *, + row_number() over ( + partition by id_ordem_pagamento_consorcio_operadora + order by timestamp_captura desc + ) as rn + from ordem_pagamento_consorcio_operadora + ) +select * except (rn) +from ordem_pagamento_consorcio_operadora_rn +where rn = 1 diff --git a/queries/models/br_rj_riodejaneiro_onibus_gps/CHANGELOG.md b/queries/models/br_rj_riodejaneiro_onibus_gps/CHANGELOG.md new file mode 100644 index 000000000..0c39d2a43 --- /dev/null +++ b/queries/models/br_rj_riodejaneiro_onibus_gps/CHANGELOG.md @@ -0,0 +1,14 @@ +# Changelog - br_rj_riodejaneiro_onibus_gps + + +## [1.0.2] - 2024-11-13 + +### Adicionado +- Adiciona testes do subsidio para `sppo_realocacao` e `sppo_registros` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/256) + +## [1.0.1] - 2024-10-25 + +#### Alterado + +- Altera lógica do filtro do modelo `sppo_aux_registros_realocacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/287) + diff --git a/queries/models/br_rj_riodejaneiro_onibus_gps/schema.yaml b/queries/models/br_rj_riodejaneiro_onibus_gps/schema.yaml index 908fafc29..6e21d741a 100644 --- a/queries/models/br_rj_riodejaneiro_onibus_gps/schema.yaml +++ b/queries/models/br_rj_riodejaneiro_onibus_gps/schema.yaml @@ -237,6 +237,11 @@ models: - name: sppo_realocacao description: "Tabela com todas as realocações de linha informadas." + tests: + - check_gps_capture: + table_id: realocacao + interval: 10 + name: check_gps_capture__sppo_realocacao columns: - name: id_veiculo description: "Código identificador do veículo (número de ordem)" @@ -284,6 +289,11 @@ models: - name: sppo_registros description: "Tabela com todos os registros de GPS." + tests: + - check_gps_capture: + table_id: registros + interval: 1 + name: check_gps_capture__sppo_registros columns: - name: ordem description: "Código identificador do veículo (número de ordem)" diff --git a/queries/models/br_rj_riodejaneiro_sigmob/schema.yaml b/queries/models/br_rj_riodejaneiro_sigmob/schema.yaml index 64f730a57..bc2dc8b39 100644 --- a/queries/models/br_rj_riodejaneiro_sigmob/schema.yaml +++ b/queries/models/br_rj_riodejaneiro_sigmob/schema.yaml @@ -70,12 +70,12 @@ models: description: "View para os dados capturados em staging com os valores da coluna content desaninhados do cadastro dos serviços (routes) (Descontinuada a partir de 2022-09-13)." columns: - name: route_id - tests: - - not_null: - partition_column: data_versao - - unique_key: - partition_column: data_versao - combined_keys: "" + # tests: + # - not_null: + # partition_column: data_versao + # - unique_key: + # partition_column: data_versao + # combined_keys: "" - name: trips description: "View para os dados capturados em staging do cadastro dos itinerários (Descontinuada a partir de 2022-09-13)." - name: trips_desaninhada @@ -83,39 +83,39 @@ models: columns: - name: trip_id description: "Identificador único para um itinerário" - tests: - - not_null: - partition_column: data_versao - - unique_key: - partition_column: data_versao - combined_keys: "" - - one_to_one: - partition_column: data_versao - to_table: ref('shapes_geom') + # tests: + # - not_null: + # partition_column: data_versao + # - unique_key: + # partition_column: data_versao + # combined_keys: "" + # - one_to_one: + # partition_column: data_versao + # to_table: ref('shapes_geom') - name: route_id description: "Identificador única para a route (serviço)" - tests: - - many_to_one: - foreign_key: route_id - partition_column: data_versao - to_table: ref('routes_desaninhada') + # tests: + # - many_to_one: + # foreign_key: route_id + # partition_column: data_versao + # to_table: ref('routes_desaninhada') - name: shapes_geom description: "Tabela com as linestrings de cada itinarário consolidadas a partir da captura ponto a ponto (Descontinuada a partir de 2022-09-13)." columns: - - name: shape_id - tests: - - not_null: - partition_column: data_versao + # - name: shape_id + # tests: + # - not_null: + # partition_column: data_versao - name: trip_id - tests: - - not_null: - partition_column: data_versao - - unique_key: - partition_column: data_versao - combined_keys: "" - - one_to_one: - partition_column: data_versao - to_table: ref('trips_desaninhada') + # tests: + # - not_null: + # partition_column: data_versao + # - unique_key: + # partition_column: data_versao + # combined_keys: "" + # - one_to_one: + # partition_column: data_versao + # to_table: ref('trips_desaninhada') - name: agency description: "View para os dados capturados em staging do cadastro das agências de transporte público (Descontinuada a partir de 2022-09-13)." columns: diff --git a/queries/models/br_rj_riodejaneiro_veiculos/CHANGELOG.md b/queries/models/br_rj_riodejaneiro_veiculos/CHANGELOG.md index 46f965c70..a7362c9a1 100644 --- a/queries/models/br_rj_riodejaneiro_veiculos/CHANGELOG.md +++ b/queries/models/br_rj_riodejaneiro_veiculos/CHANGELOG.md @@ -1,5 +1,19 @@ # Changelog - br_rj_riodejaneiro_veiculos + +## [1.0.5] - 2024-11-13 + +### Adicionado + +- Adiciona testes do subsidio para `gps_sppo` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/256) + +## [1.0.4] - 2024-10-25 + +#### Adicionado + +- Adiciona testes do DBT no schema (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/287) + + ## [1.0.3] - 2024-08-09 ### Adicionado diff --git a/queries/models/br_rj_riodejaneiro_veiculos/schema.yaml b/queries/models/br_rj_riodejaneiro_veiculos/schema.yaml index aa8eb4a7a..054628718 100644 --- a/queries/models/br_rj_riodejaneiro_veiculos/schema.yaml +++ b/queries/models/br_rj_riodejaneiro_veiculos/schema.yaml @@ -45,8 +45,10 @@ models: - name: gps_sppo description: "Tabela com os dados tratados de registros de GPS do SPPO, incluindo velocidade estimada, estado de movimento, parada em terminal ou garagem e interseção com o traçado da linha informada." tests: + - check_gps_treatment: + name: check_gps_treatment__gps_sppo - dbt_utils.unique_combination_of_columns: - name: unique_columns__gps_sppo + name: dbt_utils.unique_combination_of_columns__gps_sppo combination_of_columns: - timestamp_gps - id_veiculo diff --git a/queries/models/config.yml b/queries/models/config.yml new file mode 100644 index 000000000..182839e3e --- /dev/null +++ b/queries/models/config.yml @@ -0,0 +1,9 @@ +version: 2 + +models: + - name: aux_segmento_shape + config: + buffer_segmento_metros: "{{ var('buffer_segmento_metros') }}" + comprimento_shape: "{{ var('comprimento_shape') }}" + projecao_sirgas_2000: "{{ var('projecao_sirgas_2000') }}" + projecao_wgs_84: "{{ var('projecao_wgs_84') }}" \ No newline at end of file diff --git a/queries/models/controle_financeiro/CHANGELOG.md b/queries/models/controle_financeiro/CHANGELOG.md index e4d1bc4fd..e8c2e9b42 100644 --- a/queries/models/controle_financeiro/CHANGELOG.md +++ b/queries/models/controle_financeiro/CHANGELOG.md @@ -1,5 +1,10 @@ # Changelog - controle_financeiro +## [1.0.2] - 2024-11-13 + +### Alterado +- Adiciona deduplicação no modelo `aux_retorno_ordem_pagamento.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/321) + ## [1.0.1] - 2024-06-13 ### Corrigido diff --git a/queries/models/controle_financeiro/staging/aux_retorno_ordem_pagamento.sql b/queries/models/controle_financeiro/staging/aux_retorno_ordem_pagamento.sql index ef459e56d..1366b4324 100644 --- a/queries/models/controle_financeiro/staging/aux_retorno_ordem_pagamento.sql +++ b/queries/models/controle_financeiro/staging/aux_retorno_ordem_pagamento.sql @@ -1,26 +1,42 @@ {{ - config( - materialized="incremental", - partition_by={ - "field":"data_ordem", - "data_type":"date", - "granularity": "day" - }, - unique_key="unique_id" - ) + config( + materialized="incremental", + partition_by={ + "field": "data_ordem", + "data_type": "date", + "granularity": "day", + }, + unique_key="unique_id", + ) }} -SELECT DISTINCT - dataOrdem AS data_ordem, - DATE(dataVencimento) AS data_pagamento, - idConsorcio AS id_consorcio, - idOperadora AS id_operadora, - CONCAT(dataOrdem, idConsorcio, idOperadora) AS unique_id, - valorRealEfetivado AS valor_pago -FROM - {{ ref("staging_arquivo_retorno") }} -WHERE - isPago = TRUE -{% if is_incremental() %} - AND DATE(data) BETWEEN DATE("{{var('date_range_start')}}") AND DATE("{{var('date_range_end')}}") -{% endif %} \ No newline at end of file +with + arquivo_retorno as ( + select * except (rn) + from + ( + select + *, + row_number() over ( + partition by dataordem, idconsorcio, idoperadora + order by timestamp_captura desc + ) as rn + from {{ ref("staging_arquivo_retorno") }} + {% if is_incremental() %} + where + date(data) between date("{{var('date_range_start')}}") and date( + "{{var('date_range_end')}}" + ) + {% endif %} + ) + where rn = 1 + ) +select distinct + dataordem as data_ordem, + date(datavencimento) as data_pagamento, + idconsorcio as id_consorcio, + idoperadora as id_operadora, + concat(dataordem, idconsorcio, idoperadora) as unique_id, + valorrealefetivado as valor_pago +from arquivo_retorno +where ispago = true diff --git a/queries/models/dashboard_subsidio_sppo/CHANGELOG.md b/queries/models/dashboard_subsidio_sppo/CHANGELOG.md index 8ed584abb..ab6f21648 100644 --- a/queries/models/dashboard_subsidio_sppo/CHANGELOG.md +++ b/queries/models/dashboard_subsidio_sppo/CHANGELOG.md @@ -1,5 +1,17 @@ # Changelog - dashboard_subsidio_sppo +## [7.0.6] - 2024-10-30 + +### Adicionado + +- Adiciona testes do subsidio para `sumario_servico_dia`, `sumario_servico_dia_historico`, `sumario_servico_dia_tipo`, `sumario_servico_dia_tipo_sem_glosa` e `viagens_remuneradas` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/256) + +## [7.0.6] - 2024-11-05 + +### Alterado + +- Altera regras do modelo `viagens_remuneradas` conforme Resolução SMTR N° 3777/2024 e MTR-MEM-2024/02465 (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/300) + ## [7.0.5] - 2024-10-16 ### Corrigido diff --git a/queries/models/dashboard_subsidio_sppo/schema.yml b/queries/models/dashboard_subsidio_sppo/schema.yml index 14d14e11f..e571caa67 100644 --- a/queries/models/dashboard_subsidio_sppo/schema.yml +++ b/queries/models/dashboard_subsidio_sppo/schema.yml @@ -3,33 +3,104 @@ version: 2 models: - name: sumario_servico_dia description: "Sumário do subsídio dos serviços de ônibus (SPPO) por dia" + tests: + - dbt_expectations.expect_row_values_to_have_data_for_every_n_datepart: + date_col: data + row_condition: "data is not null" + test_start_date: "{{ var('date_range_start') }}" + test_end_date: "{{ modules.datetime.datetime.fromisoformat(var('date_range_end')) + modules.datetime.timedelta(1) }}" + name: dbt_expectations.expect_row_values_to_have_data_for_every_n_datepart__sumario_servico_dia + - teto_pagamento_valor_subsidio_pago: + table_id: sumario_servico_dia + expression: ROUND(valor_subsidio_pago/subsidio_km_teto,2) <= ROUND(km_apurada+0.01,2) + name: teto_pagamento_valor_subsidio_pago__sumario_servico_dia + - dbt_utils.unique_combination_of_columns: + combination_of_columns: + - data + - servico + name: dbt_utils.unique_combination_of_columns__sumario_servico_dia + - dbt_expectations.expect_table_aggregation_to_equal_other_table: + name: dbt_expectations.expect_table_aggregation_to_equal_other_table__sumario_servico_dia + expression: count(distinct valor_penalidade) + compare_model: ref("valor_tipo_penalidade") + compare_expression: count(distinct valor) + group_by: [valor_penalidade] + compare_group_by: [-valor] + row_condition: "DATA BETWEEN DATE('{{ var('date_range_start') }}') AND DATE('{{ var('date_range_end') }}') AND valor_penalidade IS NOT null AND valor_penalidade != 0" + compare_row_condition: "valor IS NOT null AND valor != 0" + where: "1=1" columns: - name: data description: "Data de emissão do sinal de GPS" + tests: + - not_null: + name: not_null__data__sumario_servico_dia - name: tipo_dia description: "Dia da semana considerado para o cálculo da distância planejada - categorias: Dia Útil, Sabado, Domingo" + tests: + - not_null: + name: not_null__tipo_dia__sumario_servico_dia - name: consorcio description: "Consórcio que opera o serviço" + tests: + - not_null: + name: not_null__consorcio__sumario_servico_dia - name: servico description: "Serviço realizado pelo veículo" + tests: + - not_null: + name: not_null__servico__sumario_servico_dia - name: viagens description: "Quantidade de viagens apuradas" + tests: + - not_null: + name: not_null__viagens__sumario_servico_dia + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens__sumario_servico_dia - name: km_apurada description: "Distância apurada do serviço (km)\n Distância planejada da viagem multiplicada pela quantidade de viagens" + tests: + - not_null: + name: not_null__km_apurada__sumario_servico_dia + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_apurada__sumario_servico_dia - name: km_planejada description: "Distância planejada para o serviço (km)" + tests: + - not_null: + name: not_null__km_planejada__sumario_servico_dia + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_planejada__sumario_servico_dia - name: perc_km_planejada description: "Indicador percentual de quilometragem apurada em relação à planejada da linha" + tests: + - not_null: + name: not_null__perc_km_planejada__sumario_servico_dia + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__perc_km_planejada__sumario_servico_dia - name: valor_subsidio_pago description: "Valor de subsídio apurado (R$)\n Distância apurada do serviço multiplicada pelos respectivos valores pela classificação do veículo" + tests: + - not_null: + name: not_null__valor_subsidio_pago__sumario_servico_dia + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__valor_subsidio_pago__sumario_servico_dia - name: valor_penalidade description: "Valor de penalidade apurado (R$)\n Linha com operação entre 40% e 60% da quilometragem estipulada - penalidade equivalente a uma infração média prevista no Código Disciplinar do Serviço Público de Transporte de Passageiros por Meio de Ônibus do Município do Rio de Janeiro - SPPO.\n Linha com operação inferior a 40% da quilometragem estipulada - penalidade equivalente a uma infração grave prevista no Código Disciplinar do Serviço Público de Transporte de Passageiros por Meio de Ônibus do Município do Rio de Janeiro - SPPO." - name: sumario_servico_dia_historico description: "Sumário histórico do subsídio dos serviços de ônibus (SPPO) por dia" + tests: + - subsidio_viagens_atualizadas: + name: subsidio_viagens_atualizadas__sumario_servico_dia_historico columns: - name: data description: "Data de emissão do sinal de GPS" @@ -124,6 +195,9 @@ models: - name: km_apurada description: "Distância apurada do serviço (km) Distância planejada da viagem multiplicada pela quantidade de viagens." + tests: + - sumario_servico_dia_tipo_soma_km: + name: sumario_servico_dia_tipo_soma_km__km_apurada__sumario_servico_dia_tipo - name: km_planejada description: "Distância planejada para o serviço (km)." - name: perc_km_planejada @@ -272,7 +346,7 @@ models: description: "Distância do shape (trajeto) planejado (km)" - name: distancia_aferida description: "Distância aferida da viagem (distância euclidiana entre posições consecutivas do sinal de GPS)" - - name: n_registros_start ## nao tem + - name: n_registros_start description: "Contagem de sinais de GPS emitidos no meio do trajeto." - name: n_registros_shape description: "Contagem de sinais de GPS emitidos dentro do trajeto." @@ -294,75 +368,269 @@ models: description: "{{ doc('datetime_ultima_atualizacao') }}" - name: sumario_servico_dia_tipo_sem_glosa description: "Tabela temporária sem as glosas e com os detalhes de viagens subsidiadas por serviço-dia." + tests: + - dbt_expectations.expect_row_values_to_have_data_for_every_n_datepart: + date_col: data + row_condition: "data is not null" + test_start_date: "{{ var('date_range_start') }}" + test_end_date: "{{ modules.datetime.datetime.fromisoformat(var('date_range_end')) + modules.datetime.timedelta(1) }}" + name: dbt_expectations.expect_row_values_to_have_data_for_every_n_datepart__sumario_servico_dia_tipo_sem_glosa + - teto_pagamento_valor_subsidio_pago: + table_id: sumario_servico_dia_tipo_sem_glosa + expression: ROUND(valor_total_subsidio/subsidio_km_teto,2) <= ROUND(distancia_total_subsidio+0.01,2) + name: teto_pagamento_valor_subsidio_pago__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.unique_combination_of_columns: + combination_of_columns: + - data + - servico + name: dbt_utils.unique_combination_of_columns__sumario_servico_dia_tipo_sem_glosa columns: - - name: consorcio - description: "Consórcio que opera o serviço." - name: data description: "Data de emissão do sinal de GPS." + tests: + - not_null: + name: not_null__data__sumario_servico_dia_tipo_sem_glosa + - name: consorcio + description: "Consórcio que opera o serviço." + tests: + - not_null: + name: not_null__consorcio__sumario_servico_dia_tipo_sem_glosa - name: tipo_dia description: "Dia da semana considerado para o cálculo da distância planejada - categorias: Dia Útil, Sabado, Domingo." + tests: + - not_null: + name: not_null__tipo_dia__sumario_servico_dia_tipo_sem_glosa - name: servico description: "Serviço realizado pelo veículo." + tests: + - not_null: + name: not_null__servico__sumario_servico_dia_tipo_sem_glosa - name: viagens_subsidio description: "Quantidade de viagens apuradas." + tests: + - not_null: + name: not_null__viagens_subsidio__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens_subsidio__sumario_servico_dia_tipo_sem_glosa - name: distancia_total_planejada description: "Distância planejada para o serviço (km)." + tests: + - not_null: + name: not_null__distancia_total_planejada__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__distancia_total_planejada__sumario_servico_dia_tipo_sem_glosa - name: distancia_total_subsidio description: "Distância apurada para o serviço (km)." + tests: + - not_null: + name: not_null__distancia_total_subsidio__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__distancia_total_planejada__sumario_servico_dia_tipo_sem_glosa - name: valor_total_aferido - description: "." + description: "Valor de subsídio aferido (R$)\n + Distância apurada do serviço multiplicada pelo valor de subsídio (R$/km)" - name: perc_distancia_total_subsidio description: "Indicador percentual de quilometragem apurada em relação à planejada da linha." + tests: + - not_null: + name: not_null__perc_distancia_total_subsidio__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__perc_distancia_total_subsidio__sumario_servico_dia_tipo_sem_glosa - name: valor_total_subsidio description: "Valor da distância apurada multiplicada pelo subsídio por quilômetro (sem glosa). É zerado quando POD < 80%." + tests: + - not_null: + name: not_null__valor_total_subsidio__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__valor_total_subsidio__sumario_servico_dia_tipo_sem_glosa - name: viagens_n_licenciado description: "Quantidade de viagens de veículos não licenciados." - name: km_apurada_n_licenciado description: "Quilometragem apurada de viagens de veículos não licenciados." + tests: + - not_null: + name: not_null__km_apurada_n_licenciado__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_apurada_n_licenciado__sumario_servico_dia_tipo_sem_glosa - name: viagens_autuado_ar_inoperante description: "Quantidade de viagens de veículos autuados por ar inoperante." + tests: + - not_null: + name: not_null__viagens_autuado_ar_inoperante__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens_autuado_ar_inoperante__sumario_servico_dia_tipo_sem_glosa - name: km_apurada_autuado_ar_inoperante description: "Quilometragem apurada de viagens de veículos autuados por ar inoperante." + tests: + - not_null: + name: not_null__km_apurada_autuado_ar_inoperante__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_apurada_autuado_ar_inoperante__sumario_servico_dia_tipo_sem_glosa - name: viagens_autuado_seguranca description: "Quantidade de viagens de veículos autuados por segurança." + tests: + - not_null: + name: not_null__viagens_autuado_seguranca__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens_autuado_seguranca__sumario_servico_dia_tipo_sem_glosa - name: km_apurada_autuado_seguranca description: "Quilometragem apurada de viagens de veículos autuados por segurança." + tests: + - not_null: + name: not_null__km_apurada_autuado_seguranca__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_apurada_autuado_seguranca__sumario_servico_dia_tipo_sem_glosa - name: viagens_autuado_limpezaequipamento description: "Quantidade de viagens de veículos autuados por limpeza ou equipamento." + tests: + - not_null: + name: not_null__viagens_autuado_limpezaequipamento__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens_autuado_limpezaequipamento__sumario_servico_dia_tipo_sem_glosa - name: km_apurada_autuado_limpezaequipamento description: "Quilometragem apurada de viagens de veículos autuados por limpeza ou equipamento." + tests: + - not_null: + name: not_null__km_apurada_autuado_limpezaequipamento__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_apurada_autuado_limpezaequipamento__sumario_servico_dia_tipo_sem_glosa - name: viagens_licenciado_sem_ar_n_autuado description: "Quantidade de viagens de veículos sem ar e não autuados." + tests: + - not_null: + name: not_null__viagens_licenciado_sem_ar_n_autuado__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens_licenciado_sem_ar_n_autuado__sumario_servico_dia_tipo_sem_glosa - name: km_apurada_licenciado_sem_ar_n_autuado description: "Quilometragem apurada de viagens de veículos sem ar e não autuados." + tests: + - not_null: + name: not_null__km_apurada_licenciado_sem_ar_n_autuado__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_apurada_licenciado_sem_ar_n_autuado__sumario_servico_dia_tipo_sem_glosa - name: viagens_licenciado_com_ar_n_autuado description: "Quantidade de viagens de veículos com ar e não autuados." + tests: + - not_null: + name: not_null__viagens_licenciado_com_ar_n_autuado__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens_licenciado_com_ar_n_autuado__sumario_servico_dia_tipo_sem_glosa - name: km_apurada_licenciado_com_ar_n_autuado description: "Quilometragem apurada de viagens de veículos com ar e não autuados." + tests: + - not_null: + name: not_null__km_apurada_licenciado_com_ar_n_autuado__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_apurada_licenciado_com_ar_n_autuado__sumario_servico_dia_tipo_sem_glosa - name: viagens_n_vistoriado description: "Quantidade de viagens apuradas realizadas por veículo não vistoriado tempestivamente conforme calendário de vistoria." + tests: + - not_null: + name: not_null__viagens_n_vistoriado__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens_n_vistoriado__sumario_servico_dia_tipo_sem_glosa - name: km_apurada_n_vistoriado description: "Distância apurada de viagens realizadas por veículo não vistoriado tempestivamente conforme calendário de vistoria (km)." + tests: + - not_null: + name: not_null__km_apurada_n_vistoriado__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_apurada_n_vistoriado__sumario_servico_dia_tipo_sem_glosa - name: viagens_registrado_com_ar_inoperante description: "Quantidade de viagens apuradas realizadas por veículo licenciado com ar condicionado e registrado por agente de verão (RESOLUÇÃO SMTR Nº 3.682/2024) em razão de inoperância ou mau funcionamento deste." + tests: + - not_null: + name: not_null__viagens_registrado_com_ar_inoperante__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens_registrado_com_ar_inoperante__sumario_servico_dia_tipo_sem_glosa - name: km_apurada_registrado_com_ar_inoperante description: "Distância apurada de viagens realizadas por veículo licenciado com ar condicionado e registrado por agente de verão (RESOLUÇÃO SMTR Nº 3.682/2024) em razão de inoperância ou mau funcionamento deste (km)." + tests: + - not_null: + name: not_null__km_apurada_registrado_com_ar_inoperante__sumario_servico_dia_tipo_sem_glosa + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_apurada_registrado_com_ar_inoperante__sumario_servico_dia_tipo_sem_glosa - name: viagens_remuneradas description: "Tabela com indicação de remuneração de viagens para fins de verificação de teto de pagamento conforme regra de 120%/200% (RESOLUÇÃO SMTR Nº 3.645/2023)" + tests: + - dbt_utils.unique_combination_of_columns: + combination_of_columns: + - id_viagem + name: dbt_utils.unique_combination_of_columns__viagens_remuneradas + - dbt_expectations.expect_row_values_to_have_data_for_every_n_datepart: + date_col: data + row_condition: "data is not null" + test_start_date: "{{ var('date_range_start') }}" + test_end_date: "{{ modules.datetime.datetime.fromisoformat(var('date_range_end')) + modules.datetime.timedelta(1) }}" + name: dbt_expectations.expect_row_values_to_have_data_for_every_n_datepart__viagens_remuneradas + - check_viagem_completa: + name: check_viagem_completa__viagens_remuneradas columns: - name: data description: "Data da viagem." + tests: + - not_null: + name: not_null__data__viagens_remuneradas - name: servico description: "Serviço realizado pelo veículo." + tests: + - not_null: + name: not_null__servico__viagens_remuneradas - name: tipo_viagem description: "Classificação do tipo de viagem." + tests: + - not_null: + name: not_null__data__tipo_viagem - name: id_viagem description: "Código único identificador da viagem." + tests: + - not_null: + name: not_null__id_viagem__viagens_remuneradas - name: distancia_planejada description: "Distância planejada da viagem (km)." + tests: + - not_null: + name: not_null__distancia_planejada__viagens_remuneradas + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__distancia_planejada__viagens_remuneradas - name: subsidio_km description: "Valor de subsídio, conforme tipo de viagem (R$/km)." + tests: + - not_null: + name: not_null__subsidio_km__viagens_remuneradas + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__subsidio_km__viagens_remuneradas - name: subsidio_km_teto description: "Valor máximo de subsídio, conforme tipo de viagem (R$/km)." + tests: + - not_null: + name: not_null__distancia_planejada__subsidio_km_teto - name: indicador_viagem_dentro_limite - description: "Indica se a viagem foi remunerada por estar abaixo do teto de 120%/200%." \ No newline at end of file + description: "Indica se a viagem foi remunerada por estar abaixo do teto de 120%/200%." + tests: + - not_null: + name: not_null__indicador_viagem_dentro_limite__viagens_remuneradas + + diff --git a/queries/models/dashboard_subsidio_sppo/viagens_remuneradas.sql b/queries/models/dashboard_subsidio_sppo/viagens_remuneradas.sql index ac6ec8d2c..5e7c55970 100644 --- a/queries/models/dashboard_subsidio_sppo/viagens_remuneradas.sql +++ b/queries/models/dashboard_subsidio_sppo/viagens_remuneradas.sql @@ -26,6 +26,7 @@ WITH faixa_horaria_fim, partidas_total_planejada, distancia_total_planejada AS km_planejada, + IF(sentido = "C", TRUE, FALSE) AS indicador_circular FROM {{ ref("viagem_planejada") }} -- rj-smtr.projeto_subsidio_sppo.viagem_planejada @@ -75,7 +76,8 @@ WITH p.faixa_horaria_fim, v.viagens_planejadas, p.km_planejada, - IF(p.data >= DATE("{{ var("DATA_SUBSIDIO_V9_INICIO") }}"), p.partidas_total_planejada, v.partidas_ida + v.partidas_volta) AS viagens_planejadas_ida_volta + IF(p.data >= DATE("{{ var("DATA_SUBSIDIO_V9_INICIO") }}"), p.partidas_total_planejada, v.partidas_ida + v.partidas_volta) AS viagens_planejadas_ida_volta, + p.indicador_circular FROM planejado AS p LEFT JOIN @@ -160,8 +162,39 @@ WITH ) -- 6. Flag de viagens que serão consideradas ou não para fins de remuneração (apuração de valor de subsídio) - RESOLUÇÃO SMTR Nº 3645/2023 SELECT - v.* EXCEPT(rn, datetime_partida, viagens_planejadas, viagens_planejadas_ida_volta, km_planejada, tipo_dia, consorcio, faixa_horaria_inicio, faixa_horaria_fim), + v.* EXCEPT(rn, datetime_partida, viagens_planejadas, viagens_planejadas_ida_volta, km_planejada, tipo_dia, consorcio, faixa_horaria_inicio, faixa_horaria_fim, indicador_circular), CASE + WHEN v.data >= DATE("{{ var("DATA_SUBSIDIO_V10_INICIO") }}") + AND v.tipo_dia = "Dia Útil" + AND viagens_planejadas < 10 + AND viagens_planejadas > 5 + AND pof > 100 + AND rn > (viagens_planejadas_ida_volta + IF(indicador_circular, 1, 2)) + THEN FALSE + WHEN v.data >= DATE("{{ var("DATA_SUBSIDIO_V10_INICIO") }}") + AND v.tipo_dia = "Dia Útil" + AND viagens_planejadas >= 10 + AND pof > 110 + AND rn > viagens_planejadas_ida_volta*1.1 + THEN FALSE + WHEN v.data >= DATE("{{ var("DATA_SUBSIDIO_V10_INICIO") }}") + AND v.tipo_dia = "Dia Útil" + AND viagens_planejadas <= 5 + AND pof > 200 + AND rn > viagens_planejadas_ida_volta*2 + THEN FALSE + WHEN v.data >= DATE("{{ var("DATA_SUBSIDIO_V10_INICIO") }}") + AND v.tipo_dia != "Dia Útil" + AND viagens_planejadas < 5 + AND pof > 100 + AND rn > (viagens_planejadas_ida_volta + IF(indicador_circular, 1, 2)) + THEN FALSE + WHEN v.data >= DATE("{{ var("DATA_SUBSIDIO_V10_INICIO") }}") + AND v.tipo_dia != "Dia Útil" + AND viagens_planejadas >= 5 + AND pof > 120 + AND rn > viagens_planejadas_ida_volta*1.2 + THEN FALSE WHEN v.data >= DATE("{{ var("DATA_SUBSIDIO_V3A_INICIO") }}") AND v.tipo_dia = "Dia Útil" AND viagens_planejadas > 10 @@ -184,7 +217,9 @@ SELECT THEN NULL ELSE TRUE - END AS indicador_viagem_dentro_limite + END AS indicador_viagem_dentro_limite, + '{{ var("version") }}' AS versao, + CURRENT_DATETIME("America/Sao_Paulo") as datetime_ultima_atualizacao FROM ( SELECT v.*, diff --git a/queries/models/dashboard_subsidio_sppo_v2/CHANGELOG.md b/queries/models/dashboard_subsidio_sppo_v2/CHANGELOG.md index d397b9ede..8ec688189 100644 --- a/queries/models/dashboard_subsidio_sppo_v2/CHANGELOG.md +++ b/queries/models/dashboard_subsidio_sppo_v2/CHANGELOG.md @@ -1,5 +1,11 @@ # Changelog - dashboard_subsidio_sppo_v2 +## [1.0.2] - 2024-10-02 + +### Adicionado + +- Adiciona testes do subsidio para `sumario_servico_dia_pagamento` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/256) + ## [1.0.1] - 2024-09-20 ### Alterado diff --git a/queries/models/dashboard_subsidio_sppo_v2/schema.yml b/queries/models/dashboard_subsidio_sppo_v2/schema.yml index 5bfc35034..029fe0cec 100644 --- a/queries/models/dashboard_subsidio_sppo_v2/schema.yml +++ b/queries/models/dashboard_subsidio_sppo_v2/schema.yml @@ -58,25 +58,98 @@ models: description: "{{ doc('datetime_ultima_atualizacao') }}" - name: sumario_servico_dia_pagamento description: "Sumário do subsídio dos serviços de ônibus (SPPO) por dia." + tests: + - dbt_expectations.expect_row_values_to_have_data_for_every_n_datepart: + date_col: data + row_condition: "data is not null" + test_start_date: "{{ var('date_range_start') }}" + test_end_date: "{{ modules.datetime.datetime.fromisoformat(var('date_range_end')) + modules.datetime.timedelta(1) }}" + name: dbt_expectations.expect_row_values_to_have_data_for_every_n_datepart__sumario_servico_dia_pagamento + - teto_pagamento_valor_subsidio_pago: + table_id: sumario_servico_dia_pagamento + expression: ROUND((valor_a_pagar - valor_penalidade)/subsidio_km_teto,2) <= ROUND(km_apurada_dia+0.01,2) + name: teto_pagamento_valor_subsidio_pago__sumario_servico_dia_pagamento + - dbt_utils.unique_combination_of_columns: + combination_of_columns: + - data + - servico + name: unique_combination_of_columns__sumario_servico_dia_pagamento + - dbt_utils.expression_is_true: + expression: (valor_a_pagar - valor_penalidade) IS NOT NULL AND (valor_a_pagar - valor_penalidade) >= 0 + name: expression_is_true__sumario_servico_dia_pagamento + - dbt_expectations.expect_table_aggregation_to_equal_other_table: + name: dbt_expectations.expect_table_aggregation_to_equal_other_table__sumario_servico_dia_pagamento + expression: count(distinct valor_penalidade) + compare_model: ref("valor_tipo_penalidade") + compare_expression: count(distinct valor) + group_by: [valor_penalidade] + compare_group_by: [-valor] + row_condition: "DATA BETWEEN DATE('{{ var('date_range_start') }}') AND DATE('{{ var('date_range_end') }}') AND valor_penalidade IS NOT null AND valor_penalidade != 0" + compare_row_condition: "valor IS NOT null AND valor != 0" + where: "1=1" columns: - name: data description: "Data de emissão do sinal de GPS." + tests: + - not_null: + name: not_null__data__sumario_servico_dia_pagamento - name: tipo_dia description: "Dia da semana considerado para o cálculo da distância planejada - categorias: Dia Útil, Sabado, Domingo." + tests: + - not_null: + name: not_null__tipo_dia__sumario_servico_dia_pagamento - name: consorcio description: "Consórcio que opera o serviço." + tests: + - not_null: + name: not_null__consorcio__sumario_servico_dia_pagamento - name: servico description: "Serviço realizado pelo veículo." + tests: + - not_null: + name: not_null__servico__sumario_servico_dia_pagamento - name: viagens_dia description: "Quantidade de viagens apuradas por dia." + tests: + - not_null: + name: not_null__viagens_dia__sumario_servico_dia_pagamento + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens_dia__sumario_servico_dia_pagamento - name: km_apurada_dia description: "Distância apurada para o serviço por dia (km)." + tests: + - not_null: + name: not_null__viagens_dia__sumario_servico_dia_pagamento + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__viagens_dia__sumario_servico_dia_pagamento + - sumario_servico_dia_tipo_soma_km: + name: sumario_servico_dia_tipo_soma_km__km_apurada_dia__sumario_servico_dia_pagamento - name: km_subsidiada_dia description: "Distância subsidiada para o serviço por dia (km)." + tests: + - not_null: + name: not_null__km_subsidiada_dia__sumario_servico_dia_pagamento + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_subsidiada_dia__sumario_servico_dia_pagamento - name: km_planejada_dia description: "Distância planejada para o serviço por dia (km)." + tests: + - not_null: + name: not_null__km_planejada_dia__sumario_servico_dia_pagamento + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__km_planejada_dia__sumario_servico_dia_pagamento - name: media_pof description: "Média do indicador percentual de operação por faixa horária." + tests: + - not_null: + name: not_null__media_pof__sumario_servico_dia_pagamento + - dbt_utils.accepted_range: + min_value: 0 + name: dbt_utils.accepted_range__media_pof__sumario_servico_dia_pagamento - name: desvp_pof description: "Desvio padrão do indicador percentual de operação por faixa horária." - name: km_apurada_registrado_com_ar_inoperante diff --git a/queries/models/datario/CHANGELOG.md b/queries/models/datario/CHANGELOG.md new file mode 100644 index 000000000..edadb74a8 --- /dev/null +++ b/queries/models/datario/CHANGELOG.md @@ -0,0 +1,9 @@ +# Changelog - datario + +## [1.0.0] - 2024-12-16 + +### Adicionado +- Cria modelos para views do datario (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/361): + - `gps_brt_datario.sql` + - `gps_onibus_datario.sql` + - `viagem_onibus_datario.sql` \ No newline at end of file diff --git a/queries/models/datario/gps_brt_datario.sql b/queries/models/datario/gps_brt_datario.sql new file mode 100644 index 000000000..720ed015b --- /dev/null +++ b/queries/models/datario/gps_brt_datario.sql @@ -0,0 +1,19 @@ +{{ config(alias="gps_brt") }} + +select + modo, + timestamp_gps, + data, + hora, + id_veiculo, + servico, + latitude, + longitude, + flag_em_movimento, + tipo_parada, + flag_linha_existe_sigmob, + velocidade_instantanea, + velocidade_estimada_10_min, + distancia, + versao +from {{ ref("gps_brt") }} diff --git a/queries/models/datario/gps_onibus_datario.sql b/queries/models/datario/gps_onibus_datario.sql new file mode 100644 index 000000000..35cd10589 --- /dev/null +++ b/queries/models/datario/gps_onibus_datario.sql @@ -0,0 +1,40 @@ +{{ config(alias="gps_onibus") }} +select + modo, + timestamp_gps, + data, + hora, + id_veiculo, + servico, + latitude, + longitude, + flag_em_movimento, + tipo_parada, + flag_linha_existe_sigmob, + velocidade_instantanea, + velocidade_estimada_10_min, + distancia, + 'conecta' as fonte_gps, + versao +from {{ ref("gps_sppo") }} + +union all + +select + modo, + timestamp_gps, + data, + hora, + id_veiculo, + servico, + latitude, + longitude, + flag_em_movimento, + tipo_parada, + flag_linha_existe_sigmob, + velocidade_instantanea, + velocidade_estimada_10_min, + distancia, + 'zirix' as fonte_gps, + versao +from {{ ref("gps_sppo_zirix") }} diff --git a/queries/models/datario/schema.yml b/queries/models/datario/schema.yml new file mode 100644 index 000000000..168fc690e --- /dev/null +++ b/queries/models/datario/schema.yml @@ -0,0 +1,107 @@ +version: 2 + +models: + - name: gps_brt_datario + description: "Dados tratados de registros de GPS do BRT." + columns: + - name: modo + description: "BRT – nesse arquivo só consta esse modo" + - name: timestamp_gps + description: "{{ doc('timestamp_gps') }}" + - name: data + description: "{{ doc('data_gps') }}" + - name: hora + description: "{{ doc('hora_gps') }}" + - name: id_veiculo + description: "{{ doc('id_veiculo') }}" + - name: servico + description: "{{ doc('servico') }}" + - name: latitude + description: "{{ doc('latitude_gps') }}" + - name: longitude + description: "{{ doc('longitude_gps') }}" + - name: flag_em_movimento + description: "{{ doc('flag_em_movimento') }}" + - name: tipo_parada + description: "{{ doc('tipo_parada') }}" + - name: flag_linha_existe_sigmob + description: "{{ doc('flag_linha_existe_sigmob') }}" + - name: velocidade_instantanea + description: "{{ doc('velocidade_instantanea') }}" + - name: velocidade_estimada_10_min + description: "{{ doc('velocidade_estimada_10_min') }}" + - name: distancia + description: "{{ doc('distancia_gps') }}" + - name: versao + description: "{{ doc('versao') }}" + + - name: gps_onibus_datario + description: "Dados tratados de registros de GPS dos Ônibus." + columns: + - name: modo + description: "BRT – nesse arquivo só consta esse modo" + - name: timestamp_gps + description: "{{ doc('timestamp_gps') }}" + - name: data + description: "{{ doc('data_gps') }}" + - name: hora + description: "{{ doc('hora_gps') }}" + - name: id_veiculo + description: "{{ doc('id_veiculo') }}" + - name: servico + description: "{{ doc('servico') }}" + - name: latitude + description: "{{ doc('latitude_gps') }}" + - name: longitude + description: "{{ doc('longitude_gps') }}" + - name: flag_em_movimento + description: "{{ doc('flag_em_movimento') }}" + - name: tipo_parada + description: "{{ doc('tipo_parada') }}" + - name: flag_linha_existe_sigmob + description: "{{ doc('flag_linha_existe_sigmob') }}" + - name: velocidade_instantanea + description: "{{ doc('velocidade_instantanea') }}" + - name: velocidade_estimada_10_min + description: "{{ doc('velocidade_estimada_10_min') }}" + - name: distancia + description: "{{ doc('distancia_gps') }}" + - name: fonte_gps + description: "Fornecedor dos dados de GPS (zirix ou conecta)" + - name: versao + description: "{{ doc('versao') }}" + - name: viagem_onibus_datario + description: "Detalhes de todas as viagens completas identificadas." + columns: + - name: data + description: "Data da viagem" + - name: consorcio + description: "Consórcio ao qual o serviço pertence" + - name: tipo_dia + description: "Dia da semana considerado para o cálculo da distância planejada - categorias: Dia Útil, Sábado, Domingo" + - name: id_empresa + description: "Código identificador da empresa que opera o veículo" + - name: id_veiculo + description: "Código identificador do veículo (número de ordem)" + - name: id_viagem + description: "Código identificador da viagem (ex: id_veiculo + servico + sentido + shape_id + datetime_partida)" + - name: servico + description: "Serviço realizado pelo veículo (com base na identificação do trajeto)" + - name: shape_id + description: "Código identificador do trajeto (shape) operado" + - name: sentido + description: "Sentido do trajeto identificado - categorias: I (ida), V (volta), C (circular)" + - name: datetime_partida + description: "Horário de início da viagem" + - name: datetime_chegada + description: "Horário de fim da viagem" + - name: tempo_viagem + description: "Tempo aferido da viagem (em minutos)" + - name: distancia_planejada + description: "Distância do shape (trajeto) planejado" + - name: perc_conformidade_shape + description: "Percentual de sinais emitidos dentro do shape (trajeto) ao longo da viagem" + - name: perc_conformidade_registros + description: "Percentual de minutos da viagem com registro de sinal de GPS" + - name: versao_modelo + description: "Versão da metodologia de cálculo da respectiva linha na tabela" diff --git a/queries/models/datario/viagem_onibus_datario.sql b/queries/models/datario/viagem_onibus_datario.sql new file mode 100644 index 000000000..cd798b409 --- /dev/null +++ b/queries/models/datario/viagem_onibus_datario.sql @@ -0,0 +1,19 @@ +{{ config(alias="viagem_onibus") }} +select + data, + consorcio, + tipo_dia, + id_empresa, + id_veiculo, + id_viagem, + servico_realizado as servico, + shape_id, + sentido, + datetime_partida, + datetime_chegada, + tempo_viagem, + distancia_planejada, + perc_conformidade_shape, + perc_conformidade_registros, + versao_modelo +from {{ ref("viagem_completa") }} diff --git a/queries/models/docs.md b/queries/models/docs.md index 724f6ebce..b471afcc8 100644 --- a/queries/models/docs.md +++ b/queries/models/docs.md @@ -200,6 +200,10 @@ Fim do período de operação planejado (Partição) Data inicial do feed (versão). {% enddocs %} +{% docs feed_version %} +String que indica a versão atual do conjunto de dados GTFS. +{% enddocs %} + {% docs linha %} Número da Linha {% enddocs %} diff --git a/queries/models/gtfs/CHANGELOG.md b/queries/models/gtfs/CHANGELOG.md index 7f32babb4..f54e43191 100644 --- a/queries/models/gtfs/CHANGELOG.md +++ b/queries/models/gtfs/CHANGELOG.md @@ -1,5 +1,11 @@ # Changelog - gtfs +## [1.2.0] - 2024-12-04 + +### Alterado + +- Inserido ajuste para o tipo_os `Enem` com feed_start_date `2024-09-29` e `2024-11-06` para considerar o planejamento do GTFS de sábado no domingo. Afetado o modelo `ordem_servico_trips_shapes_gtfs.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/354) + ## [1.1.9] - 2024-09-10 ### Alterado diff --git a/queries/models/gtfs/ordem_servico_trips_shapes_gtfs.sql b/queries/models/gtfs/ordem_servico_trips_shapes_gtfs.sql index 6e3d70adc..8a179965f 100644 --- a/queries/models/gtfs/ordem_servico_trips_shapes_gtfs.sql +++ b/queries/models/gtfs/ordem_servico_trips_shapes_gtfs.sql @@ -53,9 +53,10 @@ WITH t.feed_version = o.feed_version AND o.servico = t.trip_short_name AND - ((o.tipo_dia = t.tipo_dia AND o.tipo_os != "CNU") + ((o.tipo_dia = t.tipo_dia AND o.tipo_os NOT IN ("CNU", "Enem")) OR (o.tipo_dia = "Ponto Facultativo" AND t.tipo_dia = "Dia Útil" AND o.tipo_os != "CNU") - OR (o.feed_start_date = "2024-08-16" AND o.tipo_os = "CNU" AND o.tipo_dia = "Domingo" AND t.tipo_dia = "Sabado")) -- Domingo CNU + OR (o.feed_start_date = "2024-08-16" AND o.tipo_os = "CNU" AND o.tipo_dia = "Domingo" AND t.tipo_dia = "Sabado") -- Domingo CNU + OR (o.feed_start_date IN ("2024-09-29", "2024-11-06") AND o.tipo_os = "Enem" AND o.tipo_dia = "Domingo" AND t.tipo_dia = "Sabado")) -- Domingo Enem AND ((o.sentido IN ("I", "C") AND t.direction_id = "0") OR (o.sentido = "V" AND t.direction_id = "1")) diff --git a/queries/models/gtfs/schema.yml b/queries/models/gtfs/schema.yml index fd0a88532..7156f99e1 100644 --- a/queries/models/gtfs/schema.yml +++ b/queries/models/gtfs/schema.yml @@ -30,6 +30,11 @@ models: columns: - name: service_id description: "Identifica um conjunto de datas em que o serviço está disponível para uma ou mais rotas." + tests: + - dbt_expectations.expect_column_values_to_match_regex: + name: dbt_expectations.expect_column_values_to_match_regex__service_id__calendar_gtfs + regex: "^([USD]_|EXCEP)" + where: "feed_start_date = '{{ var('data_versao_gtfs') }}'" - name: monday description: "Indica se o serviço funciona todas as segundas-feiras no intervalo de datas especificado pelos campos start_date e end_date" - name: tuesday @@ -450,6 +455,17 @@ models: - name: ordem_servico_trajeto_alternativo_gtfs description: "Anexo da Ordem de Serviço (OS) com trajetos alternativos de serviços" + tests: + - dbt_expectations.expect_table_aggregation_to_equal_other_table: + name: dbt_expectations.expect_table_aggregation_to_equal_other_table__ordem_servico_trajeto_alternativo_gtfs + expression: count(distinct tipo_os) + compare_model: ref("ordem_servico_gtfs") + compare_expression: count(distinct tipo_os) + group_by: [feed_start_date, tipo_os] + compare_group_by: [feed_start_date, tipo_os] + row_condition: "feed_start_date = '{{ var('data_versao_gtfs') }}'" + compare_row_condition: "feed_start_date = '{{ var('data_versao_gtfs') }}'" + where: "1=1" columns: - name: feed_start_date description: "(Partição) Data inicial do feed (versão)." @@ -483,6 +499,36 @@ models: description: "Tipo de Ordem de Serviço (ex: 'Regular', 'Extraordinária - Verão')" - name: ordem_servico_trips_shapes_gtfs description: "Junção da Ordem de Serviço (OS) com trips e shapes dos serviços" + tests: + - dbt_expectations.expect_table_aggregation_to_equal_other_table: + name: dbt_expectations.expect_table_aggregation_to_equal_other_table__ordem_servico_trips_shapes_gtfs + expression: count(distinct feed_start_date) + compare_model: ref("ordem_servico_faixa_horaria") + compare_expression: count(distinct feed_start_date) + group_by: [feed_start_date, tipo_os, tipo_dia, servico, faixa_horaria_inicio] + compare_group_by: [feed_start_date, tipo_os, tipo_dia, servico, faixa_horaria_inicio] + row_condition: "feed_start_date = '{{ var('data_versao_gtfs') }}'" + compare_row_condition: "feed_start_date = '{{ var('data_versao_gtfs') }}' AND (quilometragem != 0 AND (partidas != 0 OR partidas IS NULL))" + where: "1=1" + - dbt_utils.unique_combination_of_columns: + name: dbt_utils.unique_combination_of_columns__ordem_servico_trips_shapes_gtfs + combination_of_columns: + - feed_start_date + - tipo_dia + - tipo_os + - servico + - sentido + - faixa_horaria_inicio + - shape_id + where: "feed_start_date = '{{ var('data_versao_gtfs') }}'" + - dbt_expectations.expect_table_row_count_to_be_between: + name: dbt_expectations.expect_table_row_count_to_be_between__ordem_servico_trips_shapes_gtfs + min_value: 1 + max_value: 2 + group_by: [feed_start_date, tipo_dia, tipo_os, servico, faixa_horaria_inicio, shape_id] + row_condition: "feed_start_date = '{{ var('data_versao_gtfs') }}'" + strictly: false + where: "1=1" columns: - name: feed_start_date description: "(Partição) Data inicial do feed (versão)." diff --git a/queries/models/monitoramento/CHANGELOG.md b/queries/models/monitoramento/CHANGELOG.md index 04bf81226..8df2f9f02 100644 --- a/queries/models/monitoramento/CHANGELOG.md +++ b/queries/models/monitoramento/CHANGELOG.md @@ -1,5 +1,23 @@ # Changelog - monitoramento +## [1.2.0] - 2024-11-28 + +### Adicionado +- Cria modelo `staging_viagem_informada_brt.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/337) + +### Alterado +- Adiciona viagens BRT no modelo: `gps_viagem.sql` e `viagem_informada_monitoramento.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/337) +- Altera data hardcoded por variável no modelo `gps_viagem.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/337) +- Cria corte de viagens na execução full nos modelos `gps_viagem.sql` e `gps_segmento_viagem.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/337) + +## [1.1.0] - 2024-11-08 + +### Adicionado +- Cria modelos de validação de viagens: `gps_viagem.sql`, `gps_segmento_viagem.sql` e `viagem_validacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/237) + +### Alterado +- Adiciona coluna `modo` no modelo `viagem_informada_monitoramento.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/237) + ## [1.0.1] - 2024-10-23 ### Corrigido diff --git a/queries/models/monitoramento/schema.yml b/queries/models/monitoramento/schema.yml index f70306d6f..6bcb0a811 100644 --- a/queries/models/monitoramento/schema.yml +++ b/queries/models/monitoramento/schema.yml @@ -7,19 +7,23 @@ models: data_type: date quote: true - name: id_viagem - description: Identificador único da viagem + description: "{{ doc('id_viagem') }}" data_type: string quote: true - name: datetime_partida - description: Data e hora da partida da viagem em GMT-3 + description: "{{ doc('datetime_partida') }}" data_type: datetime quote: true - name: datetime_chegada - description: Data e hora da chegada da viagem em GMT-3 + description: "{{ doc('datetime_chegada') }}" data_type: datetime quote: true + - name: modo + description: "{{ doc('modo') }}" + data_type: string + quote: true - name: id_veiculo - description: Código identificador do veículo (número de ordem) + description: "{{ doc('id_veiculo') }}" data_type: string quote: true - name: trip_id @@ -35,11 +39,11 @@ models: data_type: string quote: true - name: servico - description: "Nome curto da linha operada pelo veículo com variação de serviço (ex: 010, 011SN, ...) ou código da estação" + description: "{{ doc('servico') }}" data_type: string quote: true - name: sentido - description: Sentido de operação do serviço + description: "{{ doc('sentido') }}" data_type: string quote: true - name: fonte_gps @@ -62,3 +66,94 @@ models: description: "{{ doc('datetime_ultima_atualizacao') }}" data_type: datetime quote: true + - name: viagem_validacao + description: Tabela que informa se a viagem informada está ou não válida de acordo com os parâmetros definidos. + columns: + - name: data + description: Data da viagem (partição). + data_type: date + quote: true + - name: id_viagem + description: "{{ doc('id_viagem') }}" + data_type: string + quote: true + - name: datetime_partida + description: "{{ doc('datetime_partida') }}" + data_type: datetime + quote: true + - name: datetime_chegada + description: "{{ doc('datetime_chegada') }}" + data_type: datetime + quote: true + - name: modo + description: "{{ doc('modo') }}" + data_type: string + quote: true + - name: id_veiculo + description: "{{ doc('id_veiculo') }}" + data_type: string + quote: true + - name: trip_id + description: Identificador de uma viagem da tabela trips do GTFS. + data_type: int64 + quote: true + - name: route_id + description: Identificador de uma rota da tabela routes do GTFS. + data_type: string + quote: true + - name: shape_id + description: Identificador de um shape da tabela shapes do GTFS. + data_type: string + quote: true + - name: servico + description: "{{ doc('servico') }}" + data_type: string + quote: true + - name: sentido + description: "{{ doc('sentido') }}" + data_type: string + quote: true + - name: quantidade_segmentos_verificados + description: Quantidade de segmentos considerados no validação. + data_type: int64 + quote: true + - name: quantidade_segmentos_validos + description: Quantidade de segmentos com pelo menos um sinal de gps. + data_type: int64 + quote: true + - name: indice_validacao + description: quantidade_segmentos_validos dividido por quantidade_segmentos_verificados. + data_type: float64 + quote: true + - name: indicador_trajeto_valido + description: Indica se o veículo cumpriu o trajeto (indice_validacao >= parametro_validacao). + data_type: boolean + quote: true + - name: indicador_servico_planejado + description: Indica se o serviço estava planejado para o tipo dia. + data_type: boolean + quote: true + - name: indicador_viagem_valida + description: Indica se a viagem está válida ou não. + data_type: boolean + quote: true + - name: parametro_validacao + description: Valor mínimo do indice_validacao para que a viagem seja validada. + data_type: float64 + quote: true + - name: feed_version + description: "{{ doc('feed_version') }}" + data_type: string + quote: true + - name: feed_start_date + description: "Data inicial do feed (versão)." + data_type: date + quote: true + - name: versao + description: "{{ doc('versao') }}" + data_type: string + quote: true + - name: datetime_ultima_atualizacao + description: "{{ doc('datetime_ultima_atualizacao') }}" + data_type: datetime + quote: true diff --git a/queries/models/monitoramento/staging/gps_segmento_viagem.sql b/queries/models/monitoramento/staging/gps_segmento_viagem.sql new file mode 100644 index 000000000..8fa8857ed --- /dev/null +++ b/queries/models/monitoramento/staging/gps_segmento_viagem.sql @@ -0,0 +1,149 @@ +{{ + config( + materialized="incremental", + partition_by={ + "field": "data", + "data_type": "date", + "granularity": "day", + }, + incremental_strategy="insert_overwrite", + ) +}} + +{% set incremental_filter %} + data between + date('{{ var("date_range_start") }}') + and date('{{ var("date_range_end") }}') +{% endset %} + +{% set calendario = ref("calendario") %} +{# {% set calendario = "rj-smtr.planejamento.calendario" %} #} +{% if execute %} + {% if is_incremental() %} + {% set gtfs_feeds_query %} + select distinct concat("'", feed_start_date, "'") as feed_start_date + from {{ calendario }} + where {{ incremental_filter }} + {% endset %} + + {% set gtfs_feeds = run_query(gtfs_feeds_query).columns[0].values() %} + {% endif %} +{% endif %} + +with + calendario as ( + select * + from {{ calendario }} + {% if is_incremental() %} + where + data between date("{{ var('date_range_start') }}") and date( + "{{ var('date_range_end') }}" + ) + {% endif %} + ), + gps_viagem as ( + select + data, + gv.id_viagem, + gv.shape_id, + gv.geo_point_gps, + c.feed_version, + c.feed_start_date + from {{ ref("gps_viagem") }} gv + join calendario c using (data) + {% if is_incremental() %} where {{ incremental_filter }} {% endif %} + ), + segmento as ( + select + feed_version, + feed_start_date, + feed_end_date, + shape_id, + id_segmento, + buffer, + indicador_segmento_desconsiderado + from {{ ref("segmento_shape") }} + {# from `rj-smtr.planejamento.segmento_shape` #} + {% if is_incremental() %} + where feed_start_date in ({{ gtfs_feeds | join(", ") }}) + {% endif %} + ), + gps_segmento as ( + select g.id_viagem, g.shape_id, s.id_segmento, count(*) as quantidade_gps + from gps_viagem g + join + segmento s + on g.feed_version = s.feed_version + and g.shape_id = s.shape_id + and st_intersects(s.buffer, g.geo_point_gps) + group by 1, 2, 3 + ), + viagem as ( + select + data, + v.id_viagem, + v.datetime_partida, + v.datetime_chegada, + v.modo, + v.id_veiculo, + v.trip_id, + v.route_id, + v.shape_id, + v.servico, + v.sentido, + c.service_ids, + c.tipo_dia, + c.feed_start_date, + c.feed_version + from {{ ref("viagem_informada_monitoramento") }} v + join calendario c using (data) + {% if is_incremental() %} where {{ incremental_filter }} {% endif %} + ), + viagem_segmento as ( + select + v.data, + v.id_viagem, + v.datetime_partida, + v.datetime_chegada, + v.modo, + v.id_veiculo, + v.trip_id, + v.route_id, + shape_id, + s.id_segmento, + s.indicador_segmento_desconsiderado, + v.servico, + v.sentido, + v.service_ids, + v.tipo_dia, + feed_version, + feed_start_date + from viagem v + join segmento s using (shape_id, feed_version, feed_start_date) + ) +select + v.data, + id_viagem, + v.datetime_partida, + v.datetime_chegada, + v.modo, + v.id_veiculo, + v.trip_id, + v.route_id, + shape_id, + id_segmento, + v.indicador_segmento_desconsiderado, + v.servico, + v.sentido, + ifnull(g.quantidade_gps, 0) as quantidade_gps, + v.feed_version, + v.feed_start_date, + v.service_ids, + v.tipo_dia, + '{{ var("version") }}' as versao, + current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao +from viagem_segmento v +left join gps_segmento g using (id_viagem, shape_id, id_segmento) +{% if not is_incremental() %} + where v.data <= date_sub(current_date("America/Sao_Paulo"), interval 2 day) +{% endif %} diff --git a/queries/models/monitoramento/staging/gps_viagem.sql b/queries/models/monitoramento/staging/gps_viagem.sql new file mode 100644 index 000000000..338257805 --- /dev/null +++ b/queries/models/monitoramento/staging/gps_viagem.sql @@ -0,0 +1,116 @@ +{{ + config( + materialized="incremental", + partition_by={ + "field": "data", + "data_type": "date", + "granularity": "day", + }, + incremental_strategy="insert_overwrite", + ) +}} + +with + viagem as ( + select + data, + id_viagem, + datetime_partida, + datetime_chegada, + modo, + id_veiculo, + trip_id, + route_id, + shape_id, + servico, + sentido, + fonte_gps + from {{ ref("viagem_informada_monitoramento") }} + {# from `rj-smtr.monitoramento.viagem_informada` #} + {% if is_incremental() %} + where + data between date('{{ var("date_range_start") }}') and date( + '{{ var("date_range_end") }}' + ) + {% endif %} + ), + gps_conecta as ( + select data, timestamp_gps, servico, id_veiculo, latitude, longitude + {# from `rj-smtr.br_rj_riodejaneiro_veiculos.gps_sppo` #} + from {{ ref("gps_sppo") }} + where + {% if is_incremental() %} + data between date_sub( + date('{{ var("date_range_start") }}'), interval 1 day + ) and date_add(date('{{ var("date_range_end") }}'), interval 1 day) + {% else %} data >= date('{{ var("data_inicial_gps_validacao_viagem") }}') + {% endif %} + + ), + gps_zirix as ( + select data, timestamp_gps, servico, id_veiculo, latitude, longitude + {# from `rj-smtr.br_rj_riodejaneiro_onibus_gps_zirix.gps_sppo` #} + from {{ ref("gps_sppo_zirix") }} + where + {% if is_incremental() %} + data between date_sub( + date('{{ var("date_range_start") }}'), interval 1 day + ) and date_add(date('{{ var("date_range_end") }}'), interval 1 day) + {% else %} data >= date('{{ var("data_inicial_gps_validacao_viagem") }}') + {% endif %} + ), + gps_brt as ( + select data, timestamp_gps, servico, id_veiculo, latitude, longitude + {# from `rj-smtr.br_rj_riodejaneiro_veiculos.gps_brt` #} + from {{ ref("gps_brt") }} + where + {% if is_incremental() %} + data between date_sub( + date('{{ var("date_range_start") }}'), interval 1 day + ) and date_add(date('{{ var("date_range_end") }}'), interval 1 day) + {% else %} data >= date('{{ var("data_inicial_gps_validacao_viagem") }}') + {% endif %} + ), + gps_union as ( + select *, 'conecta' as fornecedor + from gps_conecta + + union all + + select *, 'zirix' as fornecedor + from gps_zirix + + union all + + select *, 'brt' as fornecedor + from gps_brt + ) +select + v.data, + g.timestamp_gps, + v.modo, + g.id_veiculo, + g.servico, + v.sentido, + g.latitude, + g.longitude, + st_geogpoint(g.longitude, g.latitude) as geo_point_gps, + v.id_viagem, + v.datetime_partida, + v.datetime_chegada, + v.trip_id, + v.route_id, + v.shape_id, + v.fonte_gps, + '{{ var("version") }}' as versao, + current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao +from gps_union g +join + viagem v + on g.timestamp_gps between v.datetime_partida and v.datetime_chegada + and g.id_veiculo = v.id_veiculo + and g.servico = v.servico + and g.fornecedor = v.fonte_gps +{% if not is_incremental() %} + where v.data <= date_sub(current_date("America/Sao_Paulo"), interval 2 day) +{% endif %} diff --git a/queries/models/monitoramento/staging/staging_viagem_informada_brt.sql b/queries/models/monitoramento/staging/staging_viagem_informada_brt.sql new file mode 100644 index 000000000..3674a82ac --- /dev/null +++ b/queries/models/monitoramento/staging/staging_viagem_informada_brt.sql @@ -0,0 +1,43 @@ +{{ + config( + alias="viagem_informada_brt", + ) +}} + +select + data, + safe_cast(id_viagem as string) as id_viagem, + timestamp_captura, + date( + parse_timestamp( + '%Y-%m-%d', safe_cast(json_value(content, '$.data_viagem') as string) + ) + ) as data_viagem, + datetime( + parse_timestamp( + '%Y-%m-%d %H:%M:%S', + safe_cast(json_value(content, '$.datetime_chegada') as string) + ), + 'America/Sao_Paulo' + ) as datetime_chegada, + datetime( + parse_timestamp( + '%Y-%m-%d %H:%M:%S', + safe_cast(json_value(content, '$.datetime_partida') as string) + ), + 'America/Sao_Paulo' + ) as datetime_partida, + datetime( + parse_timestamp( + '%Y-%m-%d %H:%M:%S', + safe_cast(json_value(content, '$.datetime_processamento') as string) + ), + 'America/Sao_Paulo' + ) as datetime_processamento, + safe_cast(json_value(content, '$.id_veiculo') as string) as id_veiculo, + safe_cast(json_value(content, '$.route_id') as string) as route_id, + safe_cast(json_value(content, '$.sentido') as string) as sentido, + safe_cast(json_value(content, '$.servico') as string) as servico, + safe_cast(json_value(content, '$.shape_id') as string) as shape_id, + safe_cast(json_value(content, '$.trip_id') as integer) as trip_id +from {{ source("source_sonda", "viagem_informada") }} diff --git a/queries/models/monitoramento/viagem_informada_monitoramento.sql b/queries/models/monitoramento/viagem_informada_monitoramento.sql index a87f3be38..b90bf13e1 100644 --- a/queries/models/monitoramento/viagem_informada_monitoramento.sql +++ b/queries/models/monitoramento/viagem_informada_monitoramento.sql @@ -13,19 +13,44 @@ {% endset %} {% set staging_viagem_informada_rioonibus = ref("staging_viagem_informada_rioonibus") %} +{% set staging_viagem_informada_brt = ref("staging_viagem_informada_brt") %} +{% set calendario = ref("calendario") %} +{# {% set calendario = "rj-smtr.planejamento.calendario" %} #} {% if execute %} {% if is_incremental() %} {% set partitions_query %} - SELECT DISTINCT - CONCAT("'", DATE(data_viagem), "'") AS data_viagem - FROM - {{ staging_viagem_informada_rioonibus }} - WHERE - {{ incremental_filter }} + SELECT DISTINCT + CONCAT("'", DATE(data_viagem), "'") AS data_viagem + FROM + {{ staging_viagem_informada_rioonibus }} + WHERE + {{ incremental_filter }} + + UNION DISTINCT + + SELECT DISTINCT + CONCAT("'", DATE(data_viagem), "'") AS data_viagem + FROM + {{ staging_viagem_informada_brt }} + WHERE + {{ incremental_filter }} + {% endset %} {% set partitions = run_query(partitions_query).columns[0].values() %} + + {% if partitions | length > 0 %} + {% set gtfs_feeds_query %} + select distinct concat("'", feed_start_date, "'") as feed_start_date + from {{ calendario }} + where data in ({{ partitions | join(", ") }}) + {% endset %} + + {% set gtfs_feeds = run_query(gtfs_feeds_query).columns[0].values() %} + {% else %} {% set gtfs_feeds = [] %} + {% endif %} {% endif %} + {% endif %} with @@ -40,6 +65,53 @@ with route_id, shape_id, servico, + sentido, + fornecedor as fonte_gps, + datetime_processamento, + timestamp_captura as datetime_captura + from {{ staging_viagem_informada_rioonibus }} + {% if is_incremental() %} where {{ incremental_filter }} {% endif %} + ), + staging_brt as ( + select + data_viagem as data, + id_viagem, + datetime_partida, + datetime_chegada, + id_veiculo, + trip_id, + route_id, + shape_id, + servico, + sentido, + "brt" as fonte_gps, + datetime_processamento, + timestamp_captura as datetime_captura + from {{ staging_viagem_informada_brt }} + where + {% if is_incremental() %} {{ incremental_filter }} and {% endif %} + datetime_processamento >= "2024-09-10 13:00:00" + ), + staging_union as ( + select * + from staging_rioonibus + + union all + + select * + from staging_brt + ), + staging as ( + select + data, + id_viagem, + datetime_partida, + datetime_chegada, + id_veiculo, + trip_id, + route_id, + shape_id, + servico, case when sentido = 'I' then 'Ida' @@ -49,20 +121,19 @@ with then 'Circular' else sentido end as sentido, - fornecedor as fonte_gps, + fonte_gps, datetime_processamento, - timestamp_captura as datetime_captura, - from {{ staging_viagem_informada_rioonibus }} - {% if is_incremental() %} where {{ incremental_filter }} {% endif %} + datetime_captura + from staging_union ), complete_partitions as ( select *, 0 as priority - from staging_rioonibus + from staging {% if is_incremental() and partitions | length > 0 %} union all - select * except (versao, datetime_ultima_atualizacao), 1 as priority + select * except (modo, versao, datetime_ultima_atualizacao), 1 as priority from {{ this }} where data in ({{ partitions | join(", ") }}) {% endif %} @@ -79,9 +150,49 @@ with from complete_partitions ) where rn = 1 + ), + calendario as ( + select * + from {{ calendario }} + {% if is_incremental() %} where {{ incremental_filter }} {% endif %} + ), + routes as ( + select * + from {{ ref("routes_gtfs") }} + {# from `rj-smtr.gtfs.routes` #} + {% if is_incremental() %} + where feed_start_date in ({{ gtfs_feeds | join(", ") }}) + {% endif %} + ), + viagem_modo as ( + select + data, + v.id_viagem, + v.datetime_partida, + v.datetime_chegada, + case + when v.fonte_gps = 'brt' + then 'BRT' + when r.route_type = '200' + then 'Ônibus Executivo' + when r.route_type = '700' + then 'Ônibus SPPO' + end as modo, + v.id_veiculo, + v.trip_id, + v.route_id, + v.shape_id, + v.servico, + v.sentido, + v.fonte_gps, + v.datetime_processamento, + v.datetime_captura + from deduplicado v + join calendario c using (data) + left join routes r using (route_id, feed_start_date, feed_version) ) select *, '{{ var("version") }}' as versao, current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao -from deduplicado +from viagem_modo diff --git a/queries/models/monitoramento/viagem_validacao.sql b/queries/models/monitoramento/viagem_validacao.sql new file mode 100644 index 000000000..2b30b219d --- /dev/null +++ b/queries/models/monitoramento/viagem_validacao.sql @@ -0,0 +1,144 @@ +{{ + config( + partition_by={ + "field": "data", + "data_type": "date", + "granularity": "day", + }, + ) +}} + +{% set incremental_filter %} + data between + date('{{ var("date_range_start") }}') + and date('{{ var("date_range_end") }}') +{% endset %} + +{% set calendario = ref("calendario") %} +{# {% set calendario = "rj-smtr.planejamento.calendario" %} #} +{% if execute %} + {% if is_incremental() %} + {% set gtfs_feeds_query %} + select distinct concat("'", feed_start_date, "'") as feed_start_date + from {{ calendario }} + where {{ incremental_filter }} + {% endset %} + {% set gtfs_feeds = run_query(gtfs_feeds_query).columns[0].values() %} + {% endif %} +{% endif %} + +with + contagem as ( + select + data, + id_viagem, + datetime_partida, + datetime_chegada, + modo, + id_veiculo, + trip_id, + route_id, + shape_id, + servico, + sentido, + count(*) as quantidade_segmentos_verificados, + countif(quantidade_gps > 0) as quantidade_segmentos_validos, + service_ids, + tipo_dia, + feed_version, + feed_start_date + from {{ ref("gps_segmento_viagem") }} + where + not indicador_segmento_desconsiderado + {% if is_incremental() %} and {{ incremental_filter }} {% endif %} + group by + data, + id_viagem, + datetime_partida, + datetime_chegada, + modo, + id_veiculo, + trip_id, + route_id, + shape_id, + servico, + sentido, + service_ids, + tipo_dia, + feed_version, + feed_start_date + ), + indice as ( + select + data, + id_viagem, + datetime_partida, + datetime_chegada, + modo, + id_veiculo, + trip_id, + route_id, + shape_id, + servico, + sentido, + quantidade_segmentos_verificados, + quantidade_segmentos_validos, + quantidade_segmentos_validos + / quantidade_segmentos_verificados as indice_validacao, + service_ids, + tipo_dia, + feed_version, + feed_start_date + from contagem + ), + trips as ( + select distinct + feed_start_date, + feed_version, + route_id, + array_agg(service_id) as service_ids, + from {{ ref("trips_gtfs") }} + {# from `rj-smtr.gtfs.trips` #} + {% if is_incremental() %} + where feed_start_date in ({{ gtfs_feeds | join(", ") }}) + {% endif %} + group by 1, 2, 3 + ), + servicos_planejados as ( + select + i.*, + ( + select count(*) + from unnest(i.service_ids) as service_id + join unnest(t.service_ids) as service_id using (service_id) + ) + > 0 as indicador_servico_planejado + from indice i + left join trips t using (feed_start_date, feed_version, route_id) + ) +select + data, + id_viagem, + datetime_partida, + datetime_chegada, + modo, + id_veiculo, + trip_id, + route_id, + shape_id, + servico, + sentido, + quantidade_segmentos_verificados, + quantidade_segmentos_validos, + indice_validacao, + indice_validacao >= {{ var("parametro_validacao") }} as indicador_trajeto_valido, + indicador_servico_planejado, + indice_validacao >= {{ var("parametro_validacao") }} + and indicador_servico_planejado as indicador_viagem_valida, + {{ var("parametro_validacao") }} as parametro_validacao, + tipo_dia, + feed_version, + feed_start_date, + '{{ var("version") }}' as versao, + current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao +from servicos_planejados diff --git a/queries/models/planejamento/CHANGELOG.md b/queries/models/planejamento/CHANGELOG.md index e8a1a3a9e..efa480149 100644 --- a/queries/models/planejamento/CHANGELOG.md +++ b/queries/models/planejamento/CHANGELOG.md @@ -1,5 +1,37 @@ # Changelog - planejamento +## [1.2.0] - 2024-12-04 + +### Adicionado + +- Cria modelos para tabela de viagens planejadas de acordo com o GTFS (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/352): + - `aux_frequencies_horario_tratado.sql` + - `aux_ordem_servico_horario_tratado.sql` + - `aux_trips_dia.sql` + - `viagem_planejada_planejamento.sql` + +### Alterado +- Adiciona colunas start_pt e end_pt no modelo `shapes_geom_planejamento.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/352) + +## [1.1.0] - 2024-11-08 + +### Adicionado + +- Cria modelos para tabela de calendario: `aux_calendario_manual.sql` e `calendario.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/237) +- Cria modelos para divisão de shapes: `aux_segmento_shape.py`, `aux_shapes_geom_filtrada.sql`, `shapes_geom_planejamento.sql` e `segmento_shape.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/237) + +## [1.0.2] - 2024-11-07 + +### Alterado + +- Refatora modelo `ordem_servico_faixa_horaria` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/306) + +## [1.0.1] - 2024-10-30 + +### Alterado + +- Alterado modelo `ordem_servico_faixa_horaria` em razão das novas faixas horárias (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/295) + ## [1.0.0] - 2024-08-29 ### Adicionado diff --git a/queries/models/planejamento/calendario.sql b/queries/models/planejamento/calendario.sql new file mode 100644 index 000000000..c1871cca5 --- /dev/null +++ b/queries/models/planejamento/calendario.sql @@ -0,0 +1,204 @@ +{{ + config( + partition_by={ + "field": "data", + "data_type": "date", + "granularity": "day", + }, + materialized="incremental", + incremental_strategy="insert_overwrite", + ) +}} + +{% set gtfs_feed_info = ref("feed_info_gtfs") %} +{# {% set gtfs_feed_info = "rj-smtr.gtfs.feed_info" %} #} +{% set calendario_manual = ref("aux_calendario_manual") %} + +{% if execute %} + {% if is_incremental() %} + {% set gtfs_feeds_query %} + select concat("'", feed_start_date, "'") as feed_start_date + from {{ gtfs_feed_info }} + where + feed_start_date <= date("{{ var('date_range_end') }}") + and (feed_end_date IS NULL OR feed_end_date >= date("{{ var('date_range_end') }}")) + + union distinct + + select distinct concat("'", feed_start_date, "'") as feed_start_date + from {{ calendario_manual }} + where + data between date("{{ var('date_range_start') }}") + and date("{{ var('date_range_end') }}") + and feed_start_date is not null + {% endset %} + + {% set gtfs_feeds = run_query(gtfs_feeds_query).columns[0].values() %} + {% endif %} +{% endif %} + +with + calendar as ( + select * + {# from `rj-smtr.gtfs.calendar` #} + from {{ ref("calendar_gtfs") }} + {% if is_incremental() %} + where feed_start_date in ({{ gtfs_feeds | join(", ") }}) + {% endif %} + ), + datas as ( + select data, extract(dayofweek from data) as dia_semana, feed_start_date + from + {{ gtfs_feed_info }}, + unnest( + generate_date_array( + {% if is_incremental() %} + date("{{ var('date_range_start') }}"), + date("{{ var('date_range_end') }}") + {% else %}date("2024-10-12"), current_date("America/Sao_Paulo") + {% endif %} + ) + ) as data + where + data >= feed_start_date and (data <= feed_end_date or feed_end_date is null) + ), + modificacao_manual as ( + select + d.data, + d.dia_semana, + coalesce(m.feed_start_date, d.feed_start_date) as feed_start_date, + m.tipo_dia, + ifnull(m.tipo_os, "Regular") as tipo_os + from datas d + left join {{ calendario_manual }} m using (data) + ), + calendar_dates as ( + select + cd.date as data, + m.tipo_dia, + m.tipo_os, + case + when cd.exception_type = '2' + then + regexp_replace( + cd.service_id, + "^[U|S|D]_", + case + when extract(dayofweek from cd.date) = 7 + then "S_" + when extract(dayofweek from cd.date) = 1 + then "D_" + else "U_" + end + ) + else cd.service_id + end as service_id, + cd.exception_type, + cd.feed_start_date, + {# from `rj-smtr.gtfs.calendar_dates` cd #} + from {{ ref("calendar_dates_gtfs") }} cd + join + modificacao_manual m + on cd.date = m.data + and cd.feed_start_date = m.feed_start_date + where + {% if is_incremental() %} + cd.feed_start_date in ({{ gtfs_feeds | join(", ") }}) + and cd.date between date("{{ var('date_range_start') }}") and date( + "{{ var('date_range_end') }}" + ) + {% else %} date <= current_date("America/Sao_Paulo") + {% endif %} + ), + datas_service_id as ( + select d.data, d.tipo_dia, d.tipo_os, c.service_id, d.feed_start_date + from modificacao_manual d + join + calendar c + on d.feed_start_date = c.feed_start_date + and ( + (d.dia_semana = 1 and c.sunday = '1') + or (d.dia_semana = 2 and c.monday = '1') + or (d.dia_semana = 3 and c.tuesday = '1') + or (d.dia_semana = 4 and c.wednesday = '1') + or (d.dia_semana = 5 and c.thursday = '1') + or (d.dia_semana = 6 and c.friday = '1') + or (d.dia_semana = 7 and c.saturday = '1') + ) + ), + service_ids_retirados as ( + select d.* + from datas_service_id d + left join + (select * from calendar_dates where exception_type = '2') c using ( + data, service_id + ) + where c.service_id is null + + ), + service_ids_adicionados as ( + select data, tipo_dia, tipo_os, service_id, feed_start_date + from service_ids_retirados + + union distinct + + select data, tipo_dia, tipo_os, service_id, feed_start_date + from calendar_dates + where exception_type = '1' + ), + service_id_corrigido as ( + select + data, + tipo_dia, + tipo_os, + case + when tipo_dia = "Domingo" + then regexp_replace(service_id, "^[U|S]_", "D_") + when tipo_dia = "Sabado" + then regexp_replace(service_id, "^[U|D]_", "S_") + when tipo_dia = "Dia Útil" + then regexp_replace(service_id, "^[S|D]_", "U_") + else service_id + end as service_id, + feed_start_date + from service_ids_adicionados + ), + service_id_agg as ( + select + data, + tipo_dia, + tipo_os, + feed_start_date, + array_agg(service_id) as service_ids + from service_id_corrigido + group by 1, 2, 3, 4 + ) +select + data, + case + when c.tipo_dia is not null + then c.tipo_dia + when "D_REG" in unnest(c.service_ids) + then "Domingo" + when "S_REG" in unnest(c.service_ids) + then "Sabado" + when "U_REG" in unnest(c.service_ids) + then "Dia Útil" + end as tipo_dia, + case + when c.tipo_os = "Extraordinária - Verão" + then "Verão" + when c.tipo_os like "%Madonna%" + then "Madonna" + when c.tipo_os = "Regular" + then null + else c.tipo_os + end as subtipo_dia, + c.tipo_os, + c.service_ids, + i.feed_version, + c.feed_start_date, + '{{ var("version") }}' as versao, + current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao +from service_id_agg c +join {{ gtfs_feed_info }} i using (feed_start_date) diff --git a/queries/models/planejamento/ordem_servico_faixa_horaria.sql b/queries/models/planejamento/ordem_servico_faixa_horaria.sql index 7148c20d9..e1bd2cfd0 100644 --- a/queries/models/planejamento/ordem_servico_faixa_horaria.sql +++ b/queries/models/planejamento/ordem_servico_faixa_horaria.sql @@ -8,6 +8,32 @@ ) }} +{% if execute %} + {% if var("data_versao_gtfs") < var("DATA_SUBSIDIO_V11_INICIO") %} + {% set intervalos = [ + {'inicio': '00', 'fim': '03'}, + {'inicio': '03', 'fim': '12'}, + {'inicio': '12', 'fim': '21'}, + {'inicio': '21', 'fim': '24'}, + {'inicio': '24', 'fim': '03'} + ] %} + {% else %} + {% set intervalos = [ + {'inicio': '00', 'fim': '03'}, + {'inicio': '03', 'fim': '06'}, + {'inicio': '06', 'fim': '09'}, + {'inicio': '09', 'fim': '12'}, + {'inicio': '12', 'fim': '15'}, + {'inicio': '15', 'fim': '18'}, + {'inicio': '18', 'fim': '21'}, + {'inicio': '21', 'fim': '24'}, + {'inicio': '24', 'fim': '03'} + ] %} + {% endif %} + {% set dias = ['dias_uteis', 'sabado', 'domingo', 'ponto_facultativo'] %} +{% endif %} + + WITH dados AS ( SELECT @@ -15,46 +41,17 @@ WITH SAFE_CAST(tipo_os AS STRING) AS tipo_os, SAFE_CAST(servico AS STRING) AS servico, SAFE_CAST(JSON_VALUE(content, "$.consorcio") AS STRING) AS consorcio, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_00h_e_03h_dias_uteis') AS STRING) AS partidas_entre_00h_e_03h_dias_uteis, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_00h_e_03h_dias_uteis') AS STRING) AS quilometragem_entre_00h_e_03h_dias_uteis, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_03h_e_12h_dias_uteis') AS STRING) AS partidas_entre_03h_e_12h_dias_uteis, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_03h_e_12h_dias_uteis') AS STRING) AS quilometragem_entre_03h_e_12h_dias_uteis, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_12h_e_21h_dias_uteis') AS STRING) AS partidas_entre_12h_e_21h_dias_uteis, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_12h_e_21h_dias_uteis') AS STRING) AS quilometragem_entre_12h_e_21h_dias_uteis, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_21h_e_24h_dias_uteis') AS STRING) AS partidas_entre_21h_e_24h_dias_uteis, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_21h_e_24h_dias_uteis') AS STRING) AS quilometragem_entre_21h_e_24h_dias_uteis, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_24h_e_03h_diaseguinte_dias_uteis') AS STRING) AS partidas_entre_24h_e_03h_diaseguinte_dias_uteis, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_24h_e_03h_diaseguinte_dias_uteis') AS STRING) AS quilometragem_entre_24h_e_03h_diaseguinte_dias_uteis, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_00h_e_03h_sabado') AS STRING) AS partidas_entre_00h_e_03h_sabado, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_00h_e_03h_sabado') AS STRING) AS quilometragem_entre_00h_e_03h_sabado, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_03h_e_12h_sabado') AS STRING) AS partidas_entre_03h_e_12h_sabado, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_03h_e_12h_sabado') AS STRING) AS quilometragem_entre_03h_e_12h_sabado, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_12h_e_21h_sabado') AS STRING) AS partidas_entre_12h_e_21h_sabado, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_12h_e_21h_sabado') AS STRING) AS quilometragem_entre_12h_e_21h_sabado, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_21h_e_24h_sabado') AS STRING) AS partidas_entre_21h_e_24h_sabado, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_21h_e_24h_sabado') AS STRING) AS quilometragem_entre_21h_e_24h_sabado, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_24h_e_03h_diaseguinte_sabado') AS STRING) AS partidas_entre_24h_e_03h_diaseguinte_sabado, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_24h_e_03h_diaseguinte_sabado') AS STRING) AS quilometragem_entre_24h_e_03h_diaseguinte_sabado, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_00h_e_03h_domingo') AS STRING) AS partidas_entre_00h_e_03h_domingo, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_00h_e_03h_domingo') AS STRING) AS quilometragem_entre_00h_e_03h_domingo, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_03h_e_12h_domingo') AS STRING) AS partidas_entre_03h_e_12h_domingo, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_03h_e_12h_domingo') AS STRING) AS quilometragem_entre_03h_e_12h_domingo, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_12h_e_21h_domingo') AS STRING) AS partidas_entre_12h_e_21h_domingo, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_12h_e_21h_domingo') AS STRING) AS quilometragem_entre_12h_e_21h_domingo, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_21h_e_24h_domingo') AS STRING) AS partidas_entre_21h_e_24h_domingo, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_21h_e_24h_domingo') AS STRING) AS quilometragem_entre_21h_e_24h_domingo, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_24h_e_03h_diaseguinte_domingo') AS STRING) AS partidas_entre_24h_e_03h_diaseguinte_domingo, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_24h_e_03h_diaseguinte_domingo') AS STRING) AS quilometragem_entre_24h_e_03h_diaseguinte_domingo, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_00h_e_03h_ponto_facultativo') AS STRING) AS partidas_entre_00h_e_03h_ponto_facultativo, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_00h_e_03h_ponto_facultativo') AS STRING) AS quilometragem_entre_00h_e_03h_ponto_facultativo, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_03h_e_12h_ponto_facultativo') AS STRING) AS partidas_entre_03h_e_12h_ponto_facultativo, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_03h_e_12h_ponto_facultativo') AS STRING) AS quilometragem_entre_03h_e_12h_ponto_facultativo, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_12h_e_21h_ponto_facultativo') AS STRING) AS partidas_entre_12h_e_21h_ponto_facultativo, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_12h_e_21h_ponto_facultativo') AS STRING) AS quilometragem_entre_12h_e_21h_ponto_facultativo, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_21h_e_24h_ponto_facultativo') AS STRING) AS partidas_entre_21h_e_24h_ponto_facultativo, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_21h_e_24h_ponto_facultativo') AS STRING) AS quilometragem_entre_21h_e_24h_ponto_facultativo, - SAFE_CAST(JSON_VALUE(content, '$.partidas_entre_24h_e_03h_diaseguinte_ponto_facultativo') AS STRING) AS partidas_entre_24h_e_03h_diaseguinte_ponto_facultativo, - SAFE_CAST(JSON_VALUE(content, '$.quilometragem_entre_24h_e_03h_diaseguinte_ponto_facultativo') AS STRING) AS quilometragem_entre_24h_e_03h_diaseguinte_ponto_facultativo + {% for dia in dias %} + {% for intervalo in intervalos %} + {% if intervalo.inicio != '24' %} + SAFE_CAST(JSON_VALUE(content, "$.partidas_entre_{{ intervalo.inicio }}h_e_{{ intervalo.fim }}h_{{ dia|lower }}") AS STRING) AS {{ 'partidas_entre_' ~ intervalo.inicio ~ 'h_e_' ~ intervalo.fim ~ 'h_' ~ dia|lower }}, + SAFE_CAST(JSON_VALUE(content, "$.quilometragem_entre_{{ intervalo.inicio }}h_e_{{ intervalo.fim }}h_{{ dia|lower }}") AS STRING) AS {{ 'quilometragem_entre_' ~ intervalo.inicio ~ 'h_e_' ~ intervalo.fim ~ 'h_' ~ dia|lower }}, + {% else %} + SAFE_CAST(JSON_VALUE(content, "$.partidas_entre_{{ intervalo.inicio }}h_e_{{ intervalo.fim }}h_dia_seguinte_{{ dia|lower }}") AS STRING) AS {{ 'partidas_entre_' ~ intervalo.inicio ~ 'h_e_' ~ intervalo.fim ~ 'h_dia_seguinte_' ~ dia|lower }}, + SAFE_CAST(JSON_VALUE(content, "$.quilometragem_entre_{{ intervalo.inicio }}h_e_{{ intervalo.fim }}h_dia_seguinte_{{ dia|lower }}") AS STRING) AS {{ 'quilometragem_entre_' ~ intervalo.inicio ~ 'h_e_' ~ intervalo.fim ~ 'h_dia_seguinte_' ~ dia|lower }}, + {% endif %} + {% endfor %} + {% endfor %} FROM {{ source("br_rj_riodejaneiro_gtfs_staging", "ordem_servico_faixa_horaria") }} {% if is_incremental() -%} @@ -75,28 +72,22 @@ WITH WHEN column_name LIKE '%ponto_facultativo%' THEN 'Ponto Facultativo' END AS tipo_dia, CASE - WHEN column_name LIKE '%00h_e_03h%' THEN - '00:00:00' - WHEN column_name LIKE '%03h_e_12h%' THEN - '03:00:00' - WHEN column_name LIKE '%12h_e_21h%' THEN - '12:00:00' - WHEN column_name LIKE '%21h_e_24h%' THEN - '21:00:00' - WHEN column_name LIKE '%24h_e_03h_diaseguinte%' THEN - '24:00:00' + {% for intervalo in intervalos %} + {% if intervalo.inicio != '24' %} + WHEN column_name LIKE '%{{ intervalo.inicio }}h_e_{{ intervalo.fim }}h%' THEN '{{ intervalo.inicio }}:00:00' + {% else %} + WHEN column_name LIKE '%{{ intervalo.inicio }}h_e_{{ intervalo.fim }}h_dia_seguinte%' THEN '{{ intervalo.inicio }}:00:00' + {% endif %} + {% endfor %} END AS faixa_horaria_inicio, CASE - WHEN column_name LIKE '%00h_e_03h%' THEN - '02:59:59' - WHEN column_name LIKE '%03h_e_12h%' THEN - '11:59:59' - WHEN column_name LIKE '%12h_e_21h%' THEN - '20:59:59' - WHEN column_name LIKE '%21h_e_24h%' THEN - '23:59:59' - WHEN column_name LIKE '%24h_e_03h_diaseguinte%' THEN - '26:59:59' + {% for intervalo in intervalos %} + {% if intervalo.inicio != '24' %} + WHEN column_name LIKE '%{{ intervalo.inicio }}h_e_{{ intervalo.fim }}h%' THEN '{{ '%02d'|format(intervalo.fim|int - 1) }}:59:59' + {% else %} + WHEN column_name LIKE '%{{ intervalo.inicio }}h_e_{{ intervalo.fim }}h_dia_seguinte%' THEN '26:59:59' + {% endif %} + {% endfor %} END AS faixa_horaria_fim, SUM(CASE WHEN column_name LIKE '%partidas%' THEN SAFE_CAST(value AS INT64) @@ -109,46 +100,18 @@ WITH FROM dados UNPIVOT ( value FOR column_name IN ( - partidas_entre_00h_e_03h_dias_uteis, - quilometragem_entre_00h_e_03h_dias_uteis, - partidas_entre_03h_e_12h_dias_uteis, - quilometragem_entre_03h_e_12h_dias_uteis, - partidas_entre_12h_e_21h_dias_uteis, - quilometragem_entre_12h_e_21h_dias_uteis, - partidas_entre_21h_e_24h_dias_uteis, - quilometragem_entre_21h_e_24h_dias_uteis, - partidas_entre_24h_e_03h_diaseguinte_dias_uteis, - quilometragem_entre_24h_e_03h_diaseguinte_dias_uteis, - partidas_entre_00h_e_03h_sabado, - quilometragem_entre_00h_e_03h_sabado, - partidas_entre_03h_e_12h_sabado, - quilometragem_entre_03h_e_12h_sabado, - partidas_entre_12h_e_21h_sabado, - quilometragem_entre_12h_e_21h_sabado, - partidas_entre_21h_e_24h_sabado, - quilometragem_entre_21h_e_24h_sabado, - partidas_entre_24h_e_03h_diaseguinte_sabado, - quilometragem_entre_24h_e_03h_diaseguinte_sabado, - partidas_entre_00h_e_03h_domingo, - quilometragem_entre_00h_e_03h_domingo, - partidas_entre_03h_e_12h_domingo, - quilometragem_entre_03h_e_12h_domingo, - partidas_entre_12h_e_21h_domingo, - quilometragem_entre_12h_e_21h_domingo, - partidas_entre_21h_e_24h_domingo, - quilometragem_entre_21h_e_24h_domingo, - partidas_entre_24h_e_03h_diaseguinte_domingo, - quilometragem_entre_24h_e_03h_diaseguinte_domingo, - partidas_entre_00h_e_03h_ponto_facultativo, - quilometragem_entre_00h_e_03h_ponto_facultativo, - partidas_entre_03h_e_12h_ponto_facultativo, - quilometragem_entre_03h_e_12h_ponto_facultativo, - partidas_entre_12h_e_21h_ponto_facultativo, - quilometragem_entre_12h_e_21h_ponto_facultativo, - partidas_entre_21h_e_24h_ponto_facultativo, - quilometragem_entre_21h_e_24h_ponto_facultativo, - partidas_entre_24h_e_03h_diaseguinte_ponto_facultativo, - quilometragem_entre_24h_e_03h_diaseguinte_ponto_facultativo + {% for dia in dias %} + {% for intervalo in intervalos %} + {% if intervalo.inicio != '24' %} + {{ 'partidas_entre_' ~ intervalo.inicio ~ 'h_e_' ~ intervalo.fim ~ 'h_' ~ dia|lower }}, + {{ 'quilometragem_entre_' ~ intervalo.inicio ~ 'h_e_' ~ intervalo.fim ~ 'h_' ~ dia|lower }}, + {% else %} + {{ 'partidas_entre_' ~ intervalo.inicio ~ 'h_e_' ~ intervalo.fim ~ 'h_dia_seguinte_' ~ dia|lower }}, + {{ 'quilometragem_entre_' ~ intervalo.inicio ~ 'h_e_' ~ intervalo.fim ~ 'h_dia_seguinte_' ~ dia|lower }} + {% endif %} + {% endfor %} + {% if not loop.last %},{% endif %} + {% endfor %} ) ) GROUP BY 1, 2, 3, 4, 5, 6, 7 diff --git a/queries/models/planejamento/schema.yml b/queries/models/planejamento/schema.yml index a352dc462..9a9c203b9 100644 --- a/queries/models/planejamento/schema.yml +++ b/queries/models/planejamento/schema.yml @@ -3,6 +3,17 @@ version: 2 models: - name: ordem_servico_faixa_horaria description: "Quantidade de viagens planejadas por faixa horária." + tests: + - dbt_expectations.expect_table_aggregation_to_equal_other_table: + name: dbt_expectations.expect_table_aggregation_to_equal_other_table__ordem_servico_faixa_horaria + expression: count(distinct tipo_os) + compare_model: ref("ordem_servico_gtfs") + compare_expression: count(distinct tipo_os) + group_by: [ feed_start_date, tipo_os ] + compare_group_by: [ feed_start_date, tipo_os ] + row_condition: "feed_start_date = '{{ var('data_versao_gtfs') }}'" + compare_row_condition: "feed_start_date = '{{ var('data_versao_gtfs') }}'" + where: "1=1" columns: - name: servico description: "{{ doc('servico') }}" @@ -14,5 +25,247 @@ models: description: "{{ doc('faixa_horaria_fim') }}" - name: partidas description: "{{ doc('partidas') }}" + tests: + - dbt_expectations.expect_column_sum_to_be_between: + name: dbt_expectations.expect_column_sum_to_be_between__partidas__ordem_servico_faixa_horaria + min_value: 0 + group_by: [ feed_start_date, tipo_os, tipo_dia, servico ] + strictly: true + where: "feed_start_date = '{{ var('data_versao_gtfs') }}'" - name: quilometragem - description: "{{ doc('quilometragem') }}" \ No newline at end of file + description: "{{ doc('quilometragem') }}" + tests: + - dbt_expectations.expect_column_sum_to_be_between: + name: dbt_expectations.expect_column_sum_to_be_between__quilometragem__ordem_servico_faixa_horaria + min_value: 0 + group_by: [ feed_start_date, tipo_os, tipo_dia, servico ] + strictly: true + where: "feed_start_date = '{{ var('data_versao_gtfs') }}'" + - name: segmento_shape + description: Tabela contendo os shapes segmentados usados na validação de viagens. + columns: + - name: feed_version + description: "{{ doc('feed_version') }}" + data_type: string + quote: true + - name: feed_start_date + description: "{{ doc('feed_start_date') }}" + data_type: date + quote: true + - name: feed_end_date + description: Data final do feed (versão). + data_type: date + quote: true + - name: shape_id + description: Identificador de shape. + data_type: string + quote: true + - name: id_segmento + description: Identificador do segmento. + data_type: string + quote: true + - name: segmento + description: Segmento em formato geográfico. + data_type: geography + quote: true + - name: wkt_segmento + description: Segmento em formato string WKT + data_type: string + quote: true + - name: comprimento_segmento + description: Comprimento do segmento emm metros. + data_type: float64 + quote: true + - name: buffer_completo + description: Área de 20m ao redor do segmento. + data_type: geography + quote: true + - name: buffer + description: Área de 20m ao redor do segmento com tratamento para não haver interseções entre ele e os segmentos posteriores. + data_type: geography + quote: true + - name: indicador_tunel + description: Indica se o segmento passa por um túnel. + data_type: boolean + quote: true + - name: indicador_area_prejudicada + description: Indica se o buffer teve a área reduzida em mais de 50% após o tratamento. + data_type: boolean + quote: true + - name: indicador_segmento_pequeno + description: Indica se o segmento é menor que 990 metros. + data_type: boolean + quote: true + - name: indicador_segmento_desconsiderado + description: Indica se o segmento deve ser desconsiderado na validação de viagens + data_type: boolean + quote: true + - name: versao + description: "{{ doc('versao') }}" + data_type: string + quote: true + - name: shapes_geom_planejamento + description: Tabela contendo shapes em formatos geográficos + columns: + - name: feed_version + description: "{{ doc('feed_version') }}" + data_type: string + quote: true + - name: feed_start_date + description: "{{ doc('feed_start_date') }}" + data_type: date + quote: true + - name: feed_end_date + description: Data final do feed (versão). + data_type: date + quote: true + - name: shape_id + description: Identificador de shape. + data_type: string + quote: true + - name: shape + description: Shape em formato geográfico (usualmente LineString). + data_type: geography + quote: true + - name: wkt_shape + description: Shape em formato string WKT + data_type: string + quote: true + - name: versao + description: "{{ doc('versao') }}" + data_type: string + quote: true + - name: start_pt + description: Primeiro ponto do shape + data_type: geography + quote: true + - name: end_pt + description: Último ponto do shape + data_type: geography + quote: true + - name: viagem_planejada_planejamento + description: Tabela contendo as viagens planejadas para o dia com base no GTFS + columns: + - name: data + description: Data da viagem (partição) + data_type: date + quote: true + - name: id_viagem + description: "{{ doc('id_viagem') }}" + data_type: string + quote: true + - name: datetime_partida + description: "{{ doc('datetime_partida') }}" + data_type: datetime + quote: true + - name: modo + description: "{{ doc('modo') }}" + data_type: string + quote: true + - name: service_id + description: Identifica um conjunto de datas em que o serviço está disponível para uma ou mais rotas + data_type: string + quote: true + - name: trip_id + description: Identificador de uma viagem da tabela trips do GTFS + data_type: string + quote: true + - name: route_id + description: Identificador de uma rota da tabela routes do GTFS + data_type: string + quote: true + - name: shape_id + description: Identificador de um shape da tabela shapes do GTFS + data_type: string + quote: true + - name: servico + description: "{{ doc('servico') }}" + data_type: string + quote: true + - name: sentido + description: "{{ doc('sentido') }}" + data_type: string + quote: true + - name: evento + description: "Evento relacionado ao trajeto (nulo se o trajeto for regular)" + data_type: string + quote: true + - name: extensao + description: Distância planejada da viagem, conforme Ordem de Serviço (OS) + data_type: float64 + quote: true + - name: trajetos_alternativos + description: Outros trajetos que podem, alternativamente, serem utilizados na viagem + data_type: record + quote: true + - name: data_referencia + description: Data de referência que foi utilizada de base para gerar a viagem + data_type: date + quote: true + - name: tipo_dia + description: Tipo de dia, se dia útil, sábado, domingo ou ponto facultativo + data_type: string + quote: true + - name: subtipo_dia + description: "Subtipo de dia (ex: 'Verão')" + data_type: string + quote: true + - name: tipo_os + description: "Tipo de Ordem de Serviço (ex: 'Regular', 'Extraordinária - Verão')" + data_type: string + quote: true + - name: feed_version + description: "{{ doc('feed_version') }}" + data_type: string + quote: true + - name: feed_start_date + description: Data inicial do feed do GTFS (versão). + data_type: date + quote: true + - name: versao + description: "{{ doc('versao') }}" + data_type: string + quote: true + - name: datetime_ultima_atualizacao + description: "{{ doc('datetime_ultima_atualizacao') }}" + data_type: datetime + quote: true + - name: calendario + description: Tabela calendário + columns: + - name: data + description: Data + data_type: date + quote: true + - name: tipo_dia + description: "{{ doc('tipo_dia') }}" + data_type: string + quote: true + - name: subtipo_dia + description: "Subtipo de dia (ex: 'Verão')" + data_type: string + quote: true + - name: tipo_os + description: "Tipo de Ordem de Serviço (ex: 'Regular', 'Extraordinária - Verão')" + data_type: string + quote: true + - name: service_ids + description: Lista de service_ids válidos para o dia + data_type: string + quote: true + - name: feed_version + description: "{{ doc('feed_version') }}" + data_type: string + quote: true + - name: feed_start_date + description: "{{ doc('feed_start_date') }}" + data_type: date + quote: true + - name: versao + description: "{{ doc('versao') }}" + data_type: string + quote: true + - name: datetime_ultima_atualizacao + description: "{{ doc('datetime_ultima_atualizacao') }}" + data_type: datetime + quote: true diff --git a/queries/models/planejamento/segmento_shape.sql b/queries/models/planejamento/segmento_shape.sql new file mode 100644 index 000000000..90c214704 --- /dev/null +++ b/queries/models/planejamento/segmento_shape.sql @@ -0,0 +1,120 @@ +{{ + config( + partition_by={ + "field": "feed_start_date", + "data_type": "date", + "granularity": "day", + }, + tags=["geolocalizacao"], + ) +}} + +-- depends_on: {{ ref('feed_info_gtfs') }} +{% if execute and is_incremental() %} + {% set last_feed_version = get_last_feed_start_date(var("data_versao_gtfs")) %} +{% endif %} + +with + aux_segmento as ( + select + feed_start_date, + feed_end_date, + feed_version, + shape_id, + id_segmento, + st_geogfromtext(wkt_segmento) as segmento, + wkt_segmento, + round(cast(comprimento_segmento as float64), 2) as comprimento_segmento, + st_geogfromtext(buffer_completo) as buffer_completo + from {{ ref("aux_segmento_shape") }} + ), + tunel as ( + select + st_union_agg( + st_buffer(geometry, {{ var("buffer_tunel_metros") }}) + ) as buffer_tunel + from {{ source("dados_mestres", "logradouro") }} + where tipo = "Túnel" + + ), + intercessao_segmento as ( + select + s1.shape_id, + s1.id_segmento, + st_union(array_agg(s2.buffer_completo)) as buffer_segmento_posterior + from aux_segmento s1 + join + aux_segmento s2 + on s1.shape_id = s2.shape_id + and s1.id_segmento < s2.id_segmento + and st_intersects(s1.buffer_completo, s2.buffer_completo) + group by 1, 2 + ), + buffer_segmento_recortado as ( + select + s.*, + coalesce( + st_difference(buffer_completo, i.buffer_segmento_posterior), + buffer_completo + ) as buffer + from aux_segmento s + left join intercessao_segmento i using (shape_id, id_segmento) + ), + indicador_validacao_shape as ( + select + s.*, + st_intersects(s.segmento, t.buffer_tunel) as indicador_tunel, + st_area(s.buffer) / st_area(s.buffer_completo) + < {{ var("limite_reducao_area_buffer") }} as indicador_area_prejudicada, + s.comprimento_segmento + < {{ var("comprimento_minimo_segmento_shape") }} + as indicador_segmento_pequeno, + cast(id_segmento as integer) as id_segmento_int + from buffer_segmento_recortado s + cross join tunel t + ) +select + * except (id_segmento_int), + ( + ( + indicador_tunel + and ( + (id_segmento_int > 1) + and ( + id_segmento_int + < max(id_segmento_int) over (partition by feed_start_date, shape_id) + ) + ) + ) + or indicador_area_prejudicada + or indicador_segmento_pequeno + ) as indicador_segmento_desconsiderado, + '{{ var("version") }}' as versao +from indicador_validacao_shape + +{% if is_incremental() %} + + union all + + select + s.feed_start_date, + fi.feed_end_date, + s.feed_version, + s.shape_id, + s.id_segmento, + s.segmento, + s.wkt_segmento, + s.comprimento_segmento, + s.buffer_completo, + s.buffer, + s.indicador_tunel, + s.indicador_area_prejudicada, + s.indicador_segmento_pequeno, + s.indicador_segmento_desconsiderado, + s.versao + from {{ this }} s + join {{ ref("feed_info_gtfs") }} fi using (feed_start_date) + {# join `rj-smtr.gtfs.feed_info` fi using (feed_start_date) #} + where feed_start_date = '{{ last_feed_version }}' + +{% endif %} diff --git a/queries/models/planejamento/shapes_geom_planejamento.sql b/queries/models/planejamento/shapes_geom_planejamento.sql new file mode 100644 index 000000000..fb073def0 --- /dev/null +++ b/queries/models/planejamento/shapes_geom_planejamento.sql @@ -0,0 +1,60 @@ +{{ + config( + partition_by={ + "field": "feed_start_date", + "data_type": "date", + "granularity": "day", + }, + alias="shapes_geom", + tags=["geolocalizacao"], + ) +}} + +-- depends_on: {{ ref('feed_info_gtfs') }} +{% if execute and is_incremental() %} + {% set last_feed_version = get_last_feed_start_date(var("data_versao_gtfs")) %} +{% endif %} + +with + shapes as ( + select + feed_version, + feed_start_date, + feed_end_date, + shape_id, + shape_pt_sequence, + st_geogpoint(shape_pt_lon, shape_pt_lat) as ponto_shape, + concat(shape_pt_lon, " ", shape_pt_lat) as lon_lat, + from {{ ref("shapes_gtfs") }} + {# from `rj-smtr.gtfs.shapes` #} + {% if is_incremental() %} + where + feed_start_date + in ('{{ last_feed_version }}', '{{ var("data_versao_gtfs") }}') + {% endif %} + ), + shapes_agg as ( + select + feed_start_date, + feed_end_date, + feed_version, + shape_id, + array_agg(ponto_shape order by shape_pt_sequence) as array_shape, + concat( + "LINESTRING(", string_agg(lon_lat, ", " order by shape_pt_sequence), ")" + ) as wkt_shape + + from shapes + group by 1, 2, 3, 4 + ) +select + feed_start_date, + feed_end_date, + feed_version, + shape_id, + st_makeline(array_shape) as shape, + wkt_shape, + array_shape[ordinal(1)] as start_pt, + array_shape[ordinal(array_length(array_shape))] as end_pt, + '{{ var("version") }}' as versao +from shapes_agg diff --git a/queries/models/planejamento/staging/aux_calendario_manual.sql b/queries/models/planejamento/staging/aux_calendario_manual.sql new file mode 100644 index 000000000..8f0c07648 --- /dev/null +++ b/queries/models/planejamento/staging/aux_calendario_manual.sql @@ -0,0 +1,53 @@ +{{ + config( + partition_by={ + "field": "data", + "data_type": "date", + "granularity": "day", + }, + materialized="incremental", + incremental_strategy="insert_overwrite", + ) +}} + +with + datas as ( + select + data, + date(null) as feed_start_date, + case + when data = "2024-10-21" + then "Ponto Facultativo" -- Ponto Facultativo - Dia do Comerciário - (Processo.Rio MTR-DES-2024/64171) + when data = "2024-10-28" + then "Ponto Facultativo" -- Ponto Facultativo - Dia do Servidor Público - (Processo.Rio MTR-DES-2024/64417) + when data between date(2024, 11, 18) and date(2024, 11, 19) + then "Ponto Facultativo" -- Ponto Facultativo - G20 - (Processo.Rio MTR-DES-2024/67477) + end as tipo_dia, + case + when data between date(2024, 09, 14) and date(2024, 09, 15) + then "Verão + Rock in Rio" + when data between date(2024, 09, 19) and date(2024, 09, 22) + then "Rock in Rio" + when data = date(2024, 10, 06) + then "Eleição" + when data = date(2024, 11, 03) + then "Enem" + when data = date(2024, 11, 10) + then "Enem" + when data = date(2024, 11, 24) + then "Parada LGBTQI+" -- Processo.Rio MTR-DES-2024/70057 + end as tipo_os + from + unnest( + generate_date_array( + {% if is_incremental() %} + date("{{ var('date_range_start') }}"), + date("{{ var('date_range_end') }}") + {% else %}date("2024-09-01"), current_date("America/Sao_Paulo") + {% endif %} + ) + ) as data + ) +select * +from datas +where feed_start_date is not null or tipo_dia is not null or tipo_os is not null diff --git a/queries/models/planejamento/staging/aux_frequencies_horario_tratado.sql b/queries/models/planejamento/staging/aux_frequencies_horario_tratado.sql new file mode 100644 index 000000000..68dae9be2 --- /dev/null +++ b/queries/models/planejamento/staging/aux_frequencies_horario_tratado.sql @@ -0,0 +1,51 @@ +{{ config(materialized="ephemeral") }} + +with + frequencies as ( + select + *, + split(start_time, ":") as start_time_parts, + split(end_time, ":") as end_time_parts, + {# from `rj-smtr.gtfs.frequencies` #} + from {{ ref("frequencies_gtfs") }} + ) + +select + * except (start_time_parts, end_time_parts, start_time, end_time), + div(cast(start_time_parts[0] as integer), 24) days_to_add_start, + div(cast(end_time_parts[0] as integer), 24) days_to_add_end, + concat( + lpad( + cast( + if( + cast(start_time_parts[0] as integer) >= 24, + cast(start_time_parts[0] as integer) - 24, + cast(start_time_parts[0] as integer) + ) as string + ), + 2, + '0' + ), + ":", + start_time_parts[1], + ":", + start_time_parts[2] + ) as start_time, + concat( + lpad( + cast( + if( + cast(end_time_parts[0] as integer) >= 24, + cast(end_time_parts[0] as integer) - 24, + cast(end_time_parts[0] as integer) + ) as string + ), + 2, + '0' + ), + ":", + end_time_parts[1], + ":", + end_time_parts[2] + ) as end_time +from frequencies diff --git a/queries/models/planejamento/staging/aux_ordem_servico_horario_tratado.sql b/queries/models/planejamento/staging/aux_ordem_servico_horario_tratado.sql new file mode 100644 index 000000000..5f48956b8 --- /dev/null +++ b/queries/models/planejamento/staging/aux_ordem_servico_horario_tratado.sql @@ -0,0 +1,35 @@ +{{ config(materialized="ephemeral") }} + +with + ordem_servico as ( + select + * except (horario_inicio, horario_fim), + parse_time("%H:%M:%S", lpad(horario_inicio, 8, '0')) as horario_inicio, + split(horario_fim, ":") horario_fim_parts + {# from `rj-smtr.gtfs.ordem_servico` #} + from {{ ref("ordem_servico_gtfs") }} + ) +select + * except (horario_fim_parts), + div(cast(horario_fim_parts[0] as integer), 24) as dias_horario_fim, + parse_time( + "%H:%M:%S", + concat( + lpad( + cast( + if( + cast(horario_fim_parts[0] as integer) >= 24, + cast(horario_fim_parts[0] as integer) - 24, + cast(horario_fim_parts[0] as integer) + ) as string + ), + 2, + '0' + ), + ":", + horario_fim_parts[1], + ":", + horario_fim_parts[2] + ) + ) as horario_fim, +from ordem_servico diff --git a/queries/models/planejamento/staging/aux_segmento_shape.py b/queries/models/planejamento/staging/aux_segmento_shape.py new file mode 100644 index 000000000..69745d7c0 --- /dev/null +++ b/queries/models/planejamento/staging/aux_segmento_shape.py @@ -0,0 +1,89 @@ +# -*- coding: utf-8 -*- +import numpy as np +import pyproj +from pyspark.sql.functions import col, explode, lit, udf +from pyspark.sql.types import ArrayType, StringType +from shapely import wkt + +# from shapely.geometry import LineString, Point +from shapely.ops import substring, transform + + +def model(dbt, session): + dbt.config( + materialized="table", + ) + df = dbt.ref("aux_shapes_geom_filtrada") + bq_projection = pyproj.CRS(dbt.config.get("projecao_wgs_84")) + shapely_projection = pyproj.CRS(dbt.config.get("projecao_sirgas_2000")) + + def transform_projection(shape, from_shapely=False): + if from_shapely: + project = pyproj.Transformer.from_crs( + shapely_projection, bq_projection, always_xy=True + ).transform + else: + project = pyproj.Transformer.from_crs( + bq_projection, shapely_projection, always_xy=True + ).transform + + return transform(project, shape) + + def cut(line, distance, buffer_size): + line_len = line.length + dist_mod = line_len % distance + dist_range = list(np.arange(0, line_len, distance)) + middle_index = (len(dist_range) // 2) + 1 + + last_final_dist = 0 + lines = [] + + for i, _ in enumerate(dist_range, start=1): + if i == middle_index: + cut_distance = dist_mod + else: + cut_distance = distance + final_dist = last_final_dist + cut_distance + segment = substring(line, last_final_dist, final_dist) + lines.append( + [ + str(i), + transform_projection(segment, True).wkt, + segment.length, + transform_projection(segment.buffer(distance=buffer_size), True).wkt, + ] + ) + last_final_dist = final_dist + + return lines + + def cut_udf(wkt_string, distance, buffer_size): + line = transform_projection(wkt.loads(wkt_string)) + return cut(line, distance, buffer_size=buffer_size) + + cut_udf = udf(cut_udf, ArrayType(ArrayType(StringType()))) + df_segments = df.withColumn( + "shape_lists", + cut_udf( + col("wkt_shape"), + lit(dbt.config.get("comprimento_shape")), + lit(dbt.config.get("buffer_segmento_metros")), + ), + ) + + df_exploded = ( + df_segments.select( + "feed_version", + "feed_start_date", + "feed_end_date", + "shape_id", + explode(col("shape_lists")).alias("shape_list"), + ) + .withColumn("id_segmento", col("shape_list").getItem(0)) + .withColumn("wkt_segmento", col("shape_list").getItem(1)) + .withColumn("comprimento_segmento", col("shape_list").getItem(2)) + .withColumn("buffer_completo", col("shape_list").getItem(3)) + .drop("shape_list") + ) + + return df_exploded diff --git a/queries/models/planejamento/staging/aux_shapes_geom_filtrada.sql b/queries/models/planejamento/staging/aux_shapes_geom_filtrada.sql new file mode 100644 index 000000000..285e19979 --- /dev/null +++ b/queries/models/planejamento/staging/aux_shapes_geom_filtrada.sql @@ -0,0 +1,6 @@ +SELECT + * +FROM + {{ ref("shapes_geom_planejamento") }} +WHERE + feed_start_date = '{{ var("data_versao_gtfs") }}' \ No newline at end of file diff --git a/queries/models/planejamento/staging/aux_trips_dia.sql b/queries/models/planejamento/staging/aux_trips_dia.sql new file mode 100644 index 000000000..7b8ba1fba --- /dev/null +++ b/queries/models/planejamento/staging/aux_trips_dia.sql @@ -0,0 +1,65 @@ +{{ config(materialized="ephemeral") }} + + +with + routes as ( + select + *, + case + when agency_id in ("22005", "22002", "22004", "22003") + then "Ônibus" + when agency_id = "20001" + then "BRT" + end as modo + {# from `rj-smtr.gtfs.routes` #} + from {{ ref("routes_gtfs") }} + ), + trips_dia as ( + select + c.data, + t.trip_id, + r.modo, + t.route_id, + t.service_id, + r.route_short_name as servico, + t.direction_id, + t.shape_id, + c.tipo_dia, + c.subtipo_dia, + c.tipo_os, + t.feed_version, + t.feed_start_date, + regexp_extract(t.trip_headsign, r'\[.*?\]') as evento + {# from `rj-smtr.planejamento.calendario` c #} + from {{ ref("calendario") }} c + {# join `rj-smtr.gtfs.trips` t using (feed_start_date, feed_version) #} + join {{ ref("trips_gtfs") }} t using (feed_start_date, feed_version) + join routes r using (feed_start_date, feed_version, route_id) + where t.service_id in unnest(c.service_ids) + ) +select + td.* except (evento), + osa.evento, + case + when td.direction_id = '0' + then ifnull(osa.extensao_ida, os.extensao_ida) + when td.direction_id = '1' + then ifnull(osa.extensao_volta, os.extensao_volta) + end as extensao, + os.distancia_total_planejada, + os.feed_start_date is not null as indicador_possui_os, + os.horario_inicio, + os.horario_fim, + os.dias_horario_fim +from trips_dia td +left join + {{ ref("ordem_servico_trajeto_alternativo_gtfs") }} osa using ( + feed_start_date, feed_version, tipo_os, servico, evento + ) +{# `rj-smtr.gtfs.ordem_servico_trajeto_alternativo` osa using ( + feed_start_date, feed_version, tipo_os, servico, evento + ) #} +left join + {{ ref("aux_ordem_servico_horario_tratado") }} os using ( + feed_start_date, feed_version, tipo_os, tipo_dia, servico + ) diff --git a/queries/models/planejamento/viagem_planejada_planejamento.sql b/queries/models/planejamento/viagem_planejada_planejamento.sql new file mode 100644 index 000000000..54ae5f70d --- /dev/null +++ b/queries/models/planejamento/viagem_planejada_planejamento.sql @@ -0,0 +1,194 @@ +{{ + config( + partition_by={ + "field": "data", + "data_type": "date", + "granularity": "day", + }, + alias="viagem_planejada", + incremental_strategy="merge", + unique_key="id_viagem", + incremental_predicates=[ + "DBT_INTERNAL_DEST.data between date('" + + var("date_range_start") + + "') and date_add(date('" + + var("date_range_end") + + "'), interval 1 day)" + ], + ) +}} + + +{% set calendario = ref("calendario") %} + +{% if execute %} + {% if is_incremental() %} + {% set gtfs_feeds_query %} + select distinct concat("'", feed_start_date, "'") as feed_start_date + from {{ calendario }} + where + data between date("{{ var('date_range_start') }}") + and date("{{ var('date_range_end') }}") + {% endset %} + + {% set gtfs_feeds = run_query(gtfs_feeds_query).columns[0].values() %} + {% endif %} +{% endif %} + +with + trips_dia as ( + select * + from {{ ref("aux_trips_dia") }} + where + {% if is_incremental() %} + feed_start_date in ({{ gtfs_feeds | join(", ") }}) + and data between date("{{ var('date_range_start') }}") and date( + "{{ var('date_range_end') }}" + ) + {% else %} feed_start_date >= '{{ var("feed_inicial_viagem_planejada") }}' + {% endif %} + ), + frequencies_tratada as ( + select * + from {{ ref("aux_frequencies_horario_tratado") }} + where + {% if is_incremental() %} feed_start_date in ({{ gtfs_feeds | join(", ") }}) + {% else %} feed_start_date >= '{{ var("feed_inicial_viagem_planejada") }}' + {% endif %} + ), + trips_frequences_dia as ( + select + td.*, + timestamp( + concat( + cast(date_add(data, interval f.days_to_add_start day) as string), + ' ', + f.start_time + ), + "America/Sao_Paulo" + ) as start_timestamp, + timestamp( + concat( + cast(date_add(data, interval f.days_to_add_end day) as string), + ' ', + f.end_time + ), + "America/Sao_Paulo" + ) as end_timestamp, + f.headway_secs + from trips_dia td + join frequencies_tratada f using (feed_start_date, feed_version, trip_id) + ), + trips_alternativas as ( + select + data, + servico, + direction_id, + array_agg( + struct( + trip_id as trip_id, + shape_id as shape_id, + evento as evento, + extensao as extensao + ) + ) as trajetos_alternativos + from trips_dia td + where td.trip_id not in (select trip_id from frequencies_tratada) + group by 1, 2, 3 + ), + viagens as ( + select + tfd.*, + datetime(partida, "America/Sao_Paulo") as datetime_partida, + ta.trajetos_alternativos + from + trips_frequences_dia tfd, + unnest( + generate_timestamp_array( + start_timestamp, + timestamp_sub(end_timestamp, interval 1 second), + interval headway_secs second + ) + ) as partida + left join trips_alternativas ta using (data, servico, direction_id) + ), + viagem_filtrada as ( + -- filtra viagens fora do horario de inicio e fim e em dias não previstos na OS + select * + from viagens + where + (distancia_total_planejada is null or distancia_total_planejada > 0) + and ( + not indicador_possui_os + or datetime_partida between datetime(data, horario_inicio) and datetime( + date_add(data, interval dias_horario_fim day), horario_fim + ) + ) + ), + servico_circular as ( + select feed_start_date, feed_version, shape_id + {# from `rj-smtr.planejamento.shapes_geom` #} + from {{ ref("shapes_geom_planejamento") }} + where + {% if is_incremental() %} feed_start_date in ({{ gtfs_feeds | join(", ") }}) + {% else %} feed_start_date >= '{{ var("feed_inicial_viagem_planejada") }}' + {% endif %} + and round(st_y(start_pt), 4) = round(st_y(end_pt), 4) + and round(st_x(start_pt), 4) = round(st_x(end_pt), 4) + ), + viagem_planejada as ( + select + date(datetime_partida) as data, + datetime_partida, + modo, + service_id, + trip_id, + route_id, + shape_id, + servico, + case + when c.shape_id is not null + then "C" + when direction_id = '0' + then "I" + else "V" + end as sentido, + evento, + extensao, + trajetos_alternativos, + data as data_referencia, + tipo_dia, + subtipo_dia, + tipo_os, + feed_version, + feed_start_date, + '{{ var("version") }}' as versao, + current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao + from viagem_filtrada v + left join servico_circular c using (shape_id, feed_version, feed_start_date) + ), + viagem_planejada_id as ( + select + *, + concat( + servico, + "_", + sentido, + "_", + shape_id, + "_", + format_datetime("%Y%m%d%H%M%S", datetime_partida) + ) as id_viagem + from viagem_planejada + ) +select data, id_viagem, * except (data, id_viagem, rn) +from + ( + select + *, + row_number() over ( + partition by id_viagem order by data_referencia desc + ) as rn + from viagem_planejada_id + ) +where rn = 1 diff --git a/queries/models/projeto_subsidio_sppo/CHANGELOG.md b/queries/models/projeto_subsidio_sppo/CHANGELOG.md index 984d82d9c..677ca808b 100644 --- a/queries/models/projeto_subsidio_sppo/CHANGELOG.md +++ b/queries/models/projeto_subsidio_sppo/CHANGELOG.md @@ -1,9 +1,49 @@ # Changelog - projeto_subsidio_sppo -## [9.0.6] - 2024-10-22 +## [9.1.2] - 2024-12-18 + +### Alterado + +- Alterado o tipo os de `2024-12-07`, `2024-12-08`, `2024-12-14` e `2024-12-15` para `Extraordinária - Verão` no modelo `subsidio_data_versao_efetiva.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/365) + +## [9.1.1] - 2024-12-13 + +### Adicionado + +- Adiciona coluna `velocidade_media` e `datetime_ultima_atualizacao` no modelo `viagem_conformidade.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/340) +- Adiciona coluna `velocidade_media` no modelo `viagem_completa.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/340) +- Adiciona filtro para remover as viagens com velocidade média superior a 110 km/h, exceto os serviços com itinerários complexos - não circulares com distância linear entre início e fim inferior a 2 km e mais de uma interseção entre o buffer de início/fim e o itinerário - no modelo `viagem_completa.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/340) + +## [9.1.0] - 2024-12-02 ### Corrigido +- Corrigido o cálculo de quilometragem e viagens nas faixas horárias de 24h às 27h e 00h às 03h na mudança de feed do GTFS no modelo `viagem_planejada.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/349) + +## [9.0.9] - 2024-11-29 + +### Corrigido + +- Corrigida a origem da coluna `distancia_total_planejada` na faixa horária de 24h às 27h no modelo `viagem_planejada.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/346) + +## [9.0.8] - 2024-11-28 + +### Alterado + +- Alterado o tipo_os de `2024-11-24` para `Parada LGBTQI+` no modelo `subsidio_data_versao_efetiva` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/345) + +## [9.0.7] - 2024-11-11 + +### Alterado + +- Alterado o tipo_os de `2024-11-03` e `2024-11-10` e para `Enem` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/311) + +- Alterado o tipo_dia de `2024-10-21`, `2024-10-28`, `2024-11-18`, `2024-11-19` e para `Ponto Facultativo` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/311) + +## [9.0.6] - 2024-10-22 + +### Alterado + - Alterado o tipo_os de 2024-10-06 para `Eleição` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/281) ## [9.0.5] - 2024-10-08 diff --git a/queries/models/projeto_subsidio_sppo/schema.yml b/queries/models/projeto_subsidio_sppo/schema.yml index 44169ba13..d264404d7 100644 --- a/queries/models/projeto_subsidio_sppo/schema.yml +++ b/queries/models/projeto_subsidio_sppo/schema.yml @@ -620,6 +620,8 @@ models: tests: - not_null - greater_than_zero + - name: velocidade_media + description: "Velocidade média da viagem [km/h]" - name: perc_conformidade_shape description: "Percentual de sinais emitidos dentro do shape (trajeto) ao longo da viagem" @@ -759,6 +761,8 @@ models: tests: - not_null - greater_than_zero + - name: velocidade_media + description: "Velocidade média da viagem [km/h]" - name: perc_conformidade_shape description: "Percentual de sinais emitidos dentro do shape (trajeto) ao longo da viagem" diff --git a/queries/models/projeto_subsidio_sppo/subsidio_data_versao_efetiva.sql b/queries/models/projeto_subsidio_sppo/subsidio_data_versao_efetiva.sql index a47f99d74..17b758137 100644 --- a/queries/models/projeto_subsidio_sppo/subsidio_data_versao_efetiva.sql +++ b/queries/models/projeto_subsidio_sppo/subsidio_data_versao_efetiva.sql @@ -322,6 +322,9 @@ WITH WHEN data = "2024-04-22" THEN "Ponto Facultativo" -- Ponto Facultativo - DECRETO RIO Nº 54267/2024 WHEN data = "2024-05-30" THEN "Domingo" -- Feriado de Corpus Christi - (Decreto Rio Nº 54525/2024) WHEN data = "2024-05-31" THEN "Ponto Facultativo" -- Ponto Facultativo - (Decreto Rio Nº 54525/2024) + WHEN data = "2024-10-21" THEN "Ponto Facultativo" -- Ponto Facultativo - Dia do Comérciario - (Processo.Rio MTR-DES-2024/64171) + WHEN data = "2024-10-28" THEN "Ponto Facultativo" -- Ponto Facultativo - Dia do Servidor Público - (Processo.Rio MTR-DES-2024/64417) + WHEN data BETWEEN DATE(2024,11,18) AND DATE(2024,11,19) THEN "Ponto Facultativo" -- Ponto Facultativo - G20 - (Processo.Rio MTR-DES-2024/67477) WHEN EXTRACT(DAY FROM data) = 20 AND EXTRACT(MONTH FROM data) = 1 THEN "Domingo" -- Dia de São Sebastião -- Art. 8°, I - Lei Municipal nº 5146/2010 WHEN EXTRACT(DAY FROM data) = 23 AND EXTRACT(MONTH FROM data) = 4 THEN "Domingo" -- Dia de São Jorge -- Art. 8°, II - Lei Municipal nº 5146/2010 / Lei Estadual Nº 5198/2008 / Lei Estadual Nº 5645/2010 WHEN EXTRACT(DAY FROM data) = 20 AND EXTRACT(MONTH FROM data) = 11 THEN "Domingo" -- Aniversário de morte de Zumbi dos Palmares / Dia da Consciência Negra -- Art. 8°, IV - Lei Municipal nº 5146/2010 / Lei Estadual nº 526/1982 / Lei Estadual nº 1929/1991 / Lei Estadual nº 4007/2002 / Lei Estadual Nº 5645/2010 @@ -351,6 +354,11 @@ WITH WHEN data BETWEEN DATE(2024,09,14) AND DATE(2024,09,15) THEN "Verão + Rock in Rio" WHEN data BETWEEN DATE(2024,09,19) AND DATE(2024,09,22) THEN "Rock in Rio" WHEN data = DATE(2024,10,06) THEN "Eleição" + WHEN data = DATE(2024,11,03) THEN "Enem" + WHEN data = DATE(2024,11,10) THEN "Enem" + WHEN data = DATE(2024,11,24) THEN "Parada LGBTQI+" -- Processo.Rio MTR-DES-2024/70057 + WHEN data BETWEEN DATE(2024,12,07) AND DATE(2024,12,08) THEN "Extraordinária - Verão" -- Processo.Rio MTR-DES-2024/72800 + WHEN data BETWEEN DATE(2024,12,14) AND DATE(2024,12,15) THEN "Extraordinária - Verão" -- Processo.Rio MTR-DES-2024/74396 ELSE "Regular" END AS tipo_os, FROM UNNEST(GENERATE_DATE_ARRAY("{{var('DATA_SUBSIDIO_V6_INICIO')}}", "2024-12-31")) AS data), diff --git a/queries/models/projeto_subsidio_sppo/viagem_completa.sql b/queries/models/projeto_subsidio_sppo/viagem_completa.sql index 2f0c77ffa..345935b4e 100644 --- a/queries/models/projeto_subsidio_sppo/viagem_completa.sql +++ b/queries/models/projeto_subsidio_sppo/viagem_completa.sql @@ -1,3 +1,4 @@ +-- depends_on: {{ ref('subsidio_data_versao_efetiva') }} {{ config( materialized='incremental', @@ -10,6 +11,11 @@ config( incremental_strategy='insert_overwrite' ) }} + +{% if execute %} + {% set result = run_query("SELECT feed_start_date FROM " ~ ref('subsidio_data_versao_efetiva') ~ " WHERE data = DATE_SUB(DATE('" ~ var("run_date") ~ "'), INTERVAL 1 DAY)") %} + {% set feed_start_date = result.columns[0].values()[0] %} +{% endif %} -- 1. Identifica viagens que estão dentro do quadro planejado (por -- enquanto, consideramos o dia todo). with viagem_periodo as ( @@ -51,6 +57,14 @@ with viagem_periodo as ( v.trip_id = p.trip_id and v.data = p.data ), +shapes AS ( + SELECT + * + FROM + {{ ref("shapes_geom_gtfs") }} + WHERE + feed_start_date = "{{ feed_start_date }}" +), -- 2. Seleciona viagens completas de acordo com a conformidade viagem_comp_conf as ( select distinct @@ -82,6 +96,7 @@ select distinct n_registros_shape, n_registros_total, n_registros_minuto, + velocidade_media, perc_conformidade_shape, perc_conformidade_distancia, perc_conformidade_registros, @@ -92,7 +107,17 @@ select distinct CURRENT_DATETIME("America/Sao_Paulo") as datetime_ultima_atualizacao from viagem_periodo v +left join + shapes AS s +using + (shape_id) where ( +{% if var("run_date") > var("DATA_SUBSIDIO_V12_INICIO") %} + velocidade_media <= {{ var("conformidade_velocidade_min") }} or (((ST_NUMGEOMETRIES(ST_INTERSECTION(ST_BUFFER(start_pt, {{ var("buffer") }}), shape)) > 1 or ST_NUMGEOMETRIES(ST_INTERSECTION(ST_BUFFER(end_pt, {{ var("buffer") }}), shape)) > 1) + and ST_DISTANCE(start_pt, end_pt) < {{ var("distancia_inicio_fim_conformidade_velocidade_min") }}) and sentido != "C") +) +and ( +{% endif %} perc_conformidade_shape >= {{ var("perc_conformidade_shape_min") }} ) and ( diff --git a/queries/models/projeto_subsidio_sppo/viagem_conformidade.sql b/queries/models/projeto_subsidio_sppo/viagem_conformidade.sql index 43731ddc4..205ed67e4 100644 --- a/queries/models/projeto_subsidio_sppo/viagem_conformidade.sql +++ b/queries/models/projeto_subsidio_sppo/viagem_conformidade.sql @@ -62,10 +62,12 @@ with viagem as ( select distinct v.* except(versao_modelo), d.* except(id_viagem, versao_modelo), + IF(data >= DATE("{{ var("DATA_SUBSIDIO_V12_INICIO") }}"), v.distancia_planejada*60/tempo_viagem, NULL) as velocidade_media, round(100 * n_registros_shape/n_registros_total, 2) as perc_conformidade_shape, round(100 * d.distancia_aferida/v.distancia_planejada, 2) as perc_conformidade_distancia, round(100 * n_registros_minuto/tempo_viagem, 2) as perc_conformidade_registros, - '{{ var("version") }}' as versao_modelo + '{{ var("version") }}' as versao_modelo, + current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao from viagem v inner join diff --git a/queries/models/projeto_subsidio_sppo/viagem_planejada.sql b/queries/models/projeto_subsidio_sppo/viagem_planejada.sql index 4160b8284..14e596f82 100644 --- a/queries/models/projeto_subsidio_sppo/viagem_planejada.sql +++ b/queries/models/projeto_subsidio_sppo/viagem_planejada.sql @@ -1,547 +1,738 @@ -{{ config( - materialized='incremental', - partition_by={ - "field":"data", - "data_type": "date", - "granularity":"day" - }, - unique_key=['data', 'trip_id'], - incremental_strategy='insert_overwrite' -) -}} - -{% if var("run_date") <= var("DATA_SUBSIDIO_V6_INICIO") %} - --- 1. Define datas do período planejado -with data_efetiva as ( - select - data, - tipo_dia, - data_versao_shapes, - data_versao_trips, - data_versao_frequencies - from {{ ref("subsidio_data_versao_efetiva") }} - where data between date_sub("{{ var("run_date") }}", interval 1 day) and date("{{ var("run_date") }}") -), --- 2. Puxa dados de distancia quadro no quadro horário -quadro as ( - select - e.data, - e.tipo_dia, - p.* except(tipo_dia, data_versao, horario_inicio, horario_fim), - IF(horario_inicio IS NOT NULL AND ARRAY_LENGTH(SPLIT(horario_inicio, ":")) = 3, - DATETIME_ADD( - DATETIME( - e.data, - PARSE_TIME("%T", - CONCAT( - SAFE_CAST(MOD(SAFE_CAST(SPLIT(horario_inicio, ":")[OFFSET(0)] AS INT64), 24) AS INT64), - ":", - SAFE_CAST(SPLIT(horario_inicio, ":")[OFFSET(1)] AS INT64), - ":", - SAFE_CAST(SPLIT(horario_inicio, ":")[OFFSET(2)] AS INT64) - ) - ) - ), - INTERVAL DIV(SAFE_CAST(SPLIT(horario_inicio, ":")[OFFSET(0)] AS INT64), 24) DAY - ), - NULL - ) AS inicio_periodo, - IF(horario_fim IS NOT NULL AND ARRAY_LENGTH(SPLIT(horario_fim, ":")) = 3, - DATETIME_ADD( - DATETIME( - e.data, - PARSE_TIME("%T", - CONCAT( - SAFE_CAST(MOD(SAFE_CAST(SPLIT(horario_fim, ":")[OFFSET(0)] AS INT64), 24) AS INT64), - ":", - SAFE_CAST(SPLIT(horario_fim, ":")[OFFSET(1)] AS INT64), - ":", - SAFE_CAST(SPLIT(horario_fim, ":")[OFFSET(2)] AS INT64) - ) - ) - ), - INTERVAL DIV(SAFE_CAST(SPLIT(horario_fim, ":")[OFFSET(0)] AS INT64), 24) DAY - ), - NULL - ) AS fim_periodo - from - data_efetiva e - inner join ( - select * - from {{ ref("subsidio_quadro_horario") }} - {% if is_incremental() %} - where - data_versao in (select data_versao_frequencies from data_efetiva) - {% endif %} - ) p - on - e.data_versao_frequencies = p.data_versao - and - e.tipo_dia = p.tipo_dia -), --- 3. Trata informação de trips: adiciona ao sentido da trip o sentido --- planejado (os shapes/trips circulares são separados em --- ida/volta no sigmob) -trips as ( - select - e.data, - t.* - from ( - select * - from {{ ref('subsidio_trips_desaninhada') }} - {% if is_incremental() %} - where - data_versao in (select data_versao_trips from data_efetiva) - {% endif %} - ) t - inner join - data_efetiva e - on - t.data_versao = e.data_versao_trips -), -quadro_trips as ( - select - * - from ( - select distinct - * except(trip_id), - trip_id as trip_id_planejado, - trip_id - from - quadro - where sentido = "I" or sentido = "V" - ) - union all ( - select - * except(trip_id), - trip_id as trip_id_planejado, - concat(trip_id, "_0") as trip_id, - from - quadro - where sentido = "C" - ) - union all ( - select - * except(trip_id), - trip_id as trip_id_planejado, - concat(trip_id, "_1") as trip_id, - from - quadro - where sentido = "C" - ) -), -quadro_tratada as ( - select - q.*, - t.shape_id as shape_id_planejado, - case - when sentido = "C" - then shape_id || "_" || split(q.trip_id, "_")[offset(1)] - else shape_id - end as shape_id, -- TODO: adicionar no sigmob - from - quadro_trips q - left join - trips t - on - t.data = q.data - and - t.trip_id = q.trip_id_planejado -), --- 4. Trata informações de shapes: junta trips e shapes para resgatar o sentido --- planejado (os shapes/trips circulares são separados em --- ida/volta no sigmob) -shapes as ( - select - e.data, - data_versao as data_shape, - shape_id, - shape, - start_pt, - end_pt - from - data_efetiva e - inner join ( - select * - from {{ ref('subsidio_shapes_geom') }} - {% if is_incremental() %} - where - data_versao in (select data_versao_shapes from data_efetiva) - {% endif %} - ) s - on - s.data_versao = e.data_versao_shapes -) --- 5. Junta shapes e trips aos servicos planejados no quadro horário -select - p.*, - s.data_shape, - s.shape, - case - when p.sentido = "C" and split(p.shape_id, "_")[offset(1)] = "0" then "I" - when p.sentido = "C" and split(p.shape_id, "_")[offset(1)] = "1" then "V" - when p.sentido = "I" or p.sentido = "V" then p.sentido - end as sentido_shape, - s.start_pt, - s.end_pt, - SAFE_CAST(NULL AS INT64) AS id_tipo_trajeto, -- Adaptação para formato da SUBSIDIO_V6 - SAFE_CAST(NULL AS STRING) AS feed_version, -- Adaptação para formato da SUBSIDIO_V6 - CURRENT_DATETIME("America/Sao_Paulo") AS datetime_ultima_atualizacao -- Adaptação para formato da SUBSIDIO_V7 -from - quadro_tratada p -inner join - shapes s -on - p.shape_id = s.shape_id -and - p.data = s.data - -{% else %} -{% if execute %} - {% set result = run_query("SELECT tipo_os, feed_version, feed_start_date, tipo_dia FROM " ~ ref('subsidio_data_versao_efetiva') ~ " WHERE data BETWEEN DATE_SUB(DATE('" ~ var("run_date") ~ "'), INTERVAL 2 DAY) AND DATE_SUB(DATE('" ~ var("run_date") ~ "'), INTERVAL 1 DAY) ORDER BY data") %} - {% set tipo_oss = result.columns[0].values() %} - {% set feed_versions = result.columns[1].values() %} - {% set feed_start_dates = result.columns[2].values() %} - {% set tipo_dias = result.columns[3].values() %} -{% endif %} - -WITH --- 1. Define datas do período planejado - data_versao_efetiva AS ( - SELECT - DATA, - tipo_dia, - subtipo_dia, - feed_version, - feed_start_date, - tipo_os, - FROM - {{ ref("subsidio_data_versao_efetiva") }} - -- rj-smtr.projeto_subsidio_sppo.subsidio_data_versao_efetiva - WHERE - data BETWEEN DATE_SUB("{{ var('run_date') }}", INTERVAL 2 DAY) AND DATE_SUB("{{ var('run_date') }}", INTERVAL 1 DAY) - ), - -- 2. Busca partidas e quilometragem da faixa horaria (dia anterior) - dia_anterior AS ( - SELECT - "{{ feed_versions[1] }}" AS feed_version, - DATE("{{ feed_start_dates[1] }}") AS feed_start_date, - feed_end_date, - "{{ tipo_oss[1] }}" AS tipo_os, - "{{ tipo_dias[1] }}" AS tipo_dia, - servico, - vista, - consorcio, - sentido, - partidas_total_planejada, - distancia_planejada, - distancia_total_planejada, - inicio_periodo, - fim_periodo, - "00:00:00" AS faixa_horaria_inicio, - "02:59:59" AS faixa_horaria_fim, - trip_id_planejado, - trip_id, - shape_id, - shape_id_planejado, - shape, - sentido_shape, - start_pt, - end_pt, - id_tipo_trajeto, - FROM - {{ ref("ordem_servico_trips_shapes_gtfs") }} - -- rj-smtr.gtfs.ordem_servico_trips_shapes - WHERE - faixa_horaria_inicio = "24:00:00" - AND tipo_os = "{{ tipo_oss[0] }}" - AND feed_version = "{{ feed_versions[0] }}" - AND feed_start_date = DATE("{{ feed_start_dates[0] }}") - AND tipo_dia = "{{ tipo_dias[0] }}" - ), - trips AS ( - SELECT DISTINCT - feed_version, - feed_start_date, - tipo_os, - tipo_dia, - servico, - sentido, - trip_id_planejado, - trip_id, - shape_id, - FROM - {{ ref("ordem_servico_trips_shapes_gtfs") }} - -- rj-smtr.gtfs.ordem_servico_trips_shapes - WHERE - tipo_os = "{{ tipo_oss[1] }}" - AND feed_version = "{{ feed_versions[1] }}" - AND feed_start_date = DATE("{{ feed_start_dates[1] }}") - AND tipo_dia = "{{ tipo_dias[1] }}" - ), -combina_trips_shapes AS ( - SELECT - feed_version, - feed_start_date, - feed_end_date, - tipo_os, - tipo_dia, - servico, - vista, - consorcio, - sentido, - partidas_total_planejada, - distancia_planejada, - distancia_total_planejada, - inicio_periodo, - fim_periodo, - faixa_horaria_inicio, - faixa_horaria_fim, - shape_id, - shape_id_planejado, - sentido_shape, - id_tipo_trajeto, - FROM - {{ ref("ordem_servico_trips_shapes_gtfs") }} - -- rj-smtr.gtfs.ordem_servico_trips_shapes - WHERE - tipo_os = "{{ tipo_oss[1] }}" - AND feed_version = "{{ feed_versions[1] }}" - AND feed_start_date = DATE("{{ feed_start_dates[1] }}") - AND tipo_dia = "{{ tipo_dias[1] }}" - UNION ALL - SELECT - feed_version, - feed_start_date, - feed_end_date, - tipo_os, - tipo_dia, - servico, - vista, - consorcio, - sentido, - partidas_total_planejada, - distancia_planejada, - distancia_total_planejada, - inicio_periodo, - fim_periodo, - faixa_horaria_inicio, - faixa_horaria_fim, - shape_id, - shape_id_planejado, - sentido_shape, - id_tipo_trajeto, - FROM dia_anterior -), -data_trips_shapes AS (SELECT - d.data, - CASE - WHEN subtipo_dia IS NOT NULL THEN CONCAT(o.tipo_dia, " - ", subtipo_dia) - ELSE o.tipo_dia - END AS tipo_dia, - servico, - vista, - consorcio, - sentido, - partidas_total_planejada, - distancia_planejada, - distancia_total_planejada, - IF(inicio_periodo IS NOT NULL AND ARRAY_LENGTH(SPLIT(inicio_periodo, ":")) = 3, - DATETIME_ADD( - DATETIME( - d.data, - PARSE_TIME("%T", - CONCAT( - SAFE_CAST(MOD(SAFE_CAST(SPLIT(inicio_periodo, ":")[OFFSET(0)] AS INT64), 24) AS INT64), - ":", - SAFE_CAST(SPLIT(inicio_periodo, ":")[OFFSET(1)] AS INT64), - ":", - SAFE_CAST(SPLIT(inicio_periodo, ":")[OFFSET(2)] AS INT64) - ) - ) - ), - INTERVAL DIV(SAFE_CAST(SPLIT(inicio_periodo, ":")[OFFSET(0)] AS INT64), 24) DAY - ), - NULL - ) AS inicio_periodo, - IF(fim_periodo IS NOT NULL AND ARRAY_LENGTH(SPLIT(fim_periodo, ":")) = 3, - DATETIME_ADD( - DATETIME( - d.data, - PARSE_TIME("%T", - CONCAT( - SAFE_CAST(MOD(SAFE_CAST(SPLIT(fim_periodo, ":")[OFFSET(0)] AS INT64), 24) AS INT64), - ":", - SAFE_CAST(SPLIT(fim_periodo, ":")[OFFSET(1)] AS INT64), - ":", - SAFE_CAST(SPLIT(fim_periodo, ":")[OFFSET(2)] AS INT64) - ) - ) - ), - INTERVAL DIV(SAFE_CAST(SPLIT(fim_periodo, ":")[OFFSET(0)] AS INT64), 24) DAY - ), - NULL - ) AS fim_periodo, - IF(d.data >= DATE("{{ var("DATA_SUBSIDIO_V9_INICIO") }}"), - DATETIME_ADD( - DATETIME( - d.data, - PARSE_TIME("%T", - CONCAT( - SAFE_CAST(MOD(SAFE_CAST(SPLIT(o.faixa_horaria_inicio, ":")[OFFSET(0)] AS INT64), 24) AS INT64), - ":", - SAFE_CAST(SPLIT(o.faixa_horaria_inicio, ":")[OFFSET(1)] AS INT64), - ":", - SAFE_CAST(SPLIT(o.faixa_horaria_inicio, ":")[OFFSET(2)] AS INT64) - ) - ) - ), - INTERVAL DIV(SAFE_CAST(SPLIT(o.faixa_horaria_inicio, ":")[OFFSET(0)] AS INT64), 24) DAY - ), - DATETIME_ADD( - DATETIME( - d.data, - PARSE_TIME("%T", - CONCAT( - SAFE_CAST(MOD(SAFE_CAST(SPLIT("00:00:00", ":")[OFFSET(0)] AS INT64), 24) AS INT64), - ":", - SAFE_CAST(SPLIT("00:00:00", ":")[OFFSET(1)] AS INT64), - ":", - SAFE_CAST(SPLIT("00:00:00", ":")[OFFSET(2)] AS INT64) - ) - ) - ), - INTERVAL DIV(SAFE_CAST(SPLIT("00:00:00", ":")[OFFSET(0)] AS INT64), 24) DAY - ) - ) AS faixa_horaria_inicio, - IF(d.data >= DATE("{{ var("DATA_SUBSIDIO_V9_INICIO") }}"), - DATETIME_ADD( - DATETIME( - d.data, - PARSE_TIME("%T", - CONCAT( - SAFE_CAST(MOD(SAFE_CAST(SPLIT(o.faixa_horaria_fim, ":")[OFFSET(0)] AS INT64), 24) AS INT64), - ":", - SAFE_CAST(SPLIT(o.faixa_horaria_fim, ":")[OFFSET(1)] AS INT64), - ":", - SAFE_CAST(SPLIT(o.faixa_horaria_fim, ":")[OFFSET(2)] AS INT64) - ) - ) - ), - INTERVAL DIV(SAFE_CAST(SPLIT(o.faixa_horaria_fim, ":")[OFFSET(0)] AS INT64), 24) DAY - ), - DATETIME_ADD( - DATETIME( - d.data, - PARSE_TIME("%T", - CONCAT( - SAFE_CAST(MOD(SAFE_CAST(SPLIT("23:59:59", ":")[OFFSET(0)] AS INT64), 24) AS INT64), - ":", - SAFE_CAST(SPLIT("23:59:59", ":")[OFFSET(1)] AS INT64), - ":", - SAFE_CAST(SPLIT("23:59:59", ":")[OFFSET(2)] AS INT64) - ) - ) - ), - INTERVAL DIV(SAFE_CAST(SPLIT("23:59:59", ":")[OFFSET(0)] AS INT64), 24) DAY - ) - ) AS faixa_horaria_fim, - t.trip_id_planejado, - t.trip_id, - shape_id, - shape_id_planejado, - SAFE_CAST(NULL AS DATE) AS data_shape, - sentido_shape, - id_tipo_trajeto, - feed_version, - feed_start_date -FROM - data_versao_efetiva AS d -LEFT JOIN - combina_trips_shapes AS o -USING (feed_start_date, feed_version, tipo_dia, tipo_os) -LEFT JOIN - trips AS t -USING (feed_start_date, feed_version, tipo_dia, tipo_os, servico, sentido, shape_id) -WHERE - data = DATE_SUB("{{ var('run_date') }}", INTERVAL 1 DAY) - AND faixa_horaria_inicio != "24:00:00" -), -shapes AS ( - SELECT - * - FROM - {{ ref("shapes_geom_gtfs") }} - -- rj-smtr.gtfs.shapes_geom - WHERE - feed_start_date IN (SELECT feed_start_date FROM data_versao_efetiva WHERE data BETWEEN DATE_SUB("{{ var('run_date') }}", INTERVAL 2 DAY) AND DATE_SUB("{{ var('run_date') }}", INTERVAL 1 DAY)) -), -dados_agregados AS ( -SELECT - data, - tipo_dia, - servico, - vista, - consorcio, - sentido, - SUM(COALESCE(partidas_total_planejada, 0)) AS partidas_total_planejada, - distancia_planejada, - SUM(distancia_total_planejada) AS distancia_total_planejada, - inicio_periodo, - fim_periodo, - faixa_horaria_inicio, - faixa_horaria_fim, - trip_id_planejado, - trip_id, - shape_id, - shape_id_planejado, - data_shape, - sentido_shape, - id_tipo_trajeto, - feed_version, - feed_start_date -FROM - data_trips_shapes -GROUP BY - data, tipo_dia, servico, vista, consorcio, sentido, distancia_planejada, inicio_periodo, fim_periodo, faixa_horaria_inicio, faixa_horaria_fim, trip_id_planejado, trip_id, shape_id, shape_id_planejado, data_shape, sentido_shape, id_tipo_trajeto, feed_version, feed_start_date -) -SELECT - data, - tipo_dia, - servico, - vista, - consorcio, - sentido, - partidas_total_planejada, - distancia_planejada, - distancia_total_planejada, - inicio_periodo, - fim_periodo, - faixa_horaria_inicio, - faixa_horaria_fim, - trip_id_planejado, - trip_id, - shape_id, - shape_id_planejado, - data_shape, - s.shape, - sentido_shape, - s.start_pt, - s.end_pt, - id_tipo_trajeto, - feed_version, - feed_start_date, - CURRENT_DATETIME("America/Sao_Paulo") AS datetime_ultima_atualizacao -FROM - dados_agregados -LEFT JOIN - shapes AS s -USING - (feed_version, feed_start_date, shape_id) -{% if var("run_date") == "2024-05-05" %} - -- Apuração "Madonna · The Celebration Tour in Rio" -WHERE - AND servico != "SE001" - {% endif %} -{% endif %} \ No newline at end of file +{{ + config( + materialized="incremental", + partition_by={"field": "data", "data_type": "date", "granularity": "day"}, + incremental_strategy="insert_overwrite", + ) +}} + +{% if var("run_date") <= var("DATA_SUBSIDIO_V6_INICIO") %} + + -- 1. Define datas do período planejado + with + data_efetiva as ( + select + data, + tipo_dia, + data_versao_shapes, + data_versao_trips, + data_versao_frequencies + from {{ ref("subsidio_data_versao_efetiva") }} + where + data between date_sub("{{ var('run_date') }}", interval 1 day) and date("{{ var('run_date') }}") -- fmt: off + ), + -- 2. Puxa dados de distancia quadro no quadro horário + quadro as ( + select + e.data, + e.tipo_dia, + p.* except (tipo_dia, data_versao, horario_inicio, horario_fim), + if( + horario_inicio is not null + and array_length(split(horario_inicio, ":")) = 3, + datetime_add( + datetime( + e.data, + parse_time( + "%T", + concat( + safe_cast( + mod( + safe_cast( + split(horario_inicio, ":")[ + offset(0) + ] as int64 + ), + 24 + ) as int64 + ), + ":", + safe_cast( + split(horario_inicio, ":")[offset(1)] as int64 + ), + ":", + safe_cast( + split(horario_inicio, ":")[offset(2)] as int64 + ) + ) + ) + ), + interval div( + safe_cast(split(horario_inicio, ":")[offset(0)] as int64), + 24 + ) day + ), + null + ) as inicio_periodo, + if( + horario_fim is not null + and array_length(split(horario_fim, ":")) = 3, + datetime_add( + datetime( + e.data, + parse_time( + "%T", + concat( + safe_cast( + mod( + safe_cast( + split(horario_fim, ":")[ + offset(0) + ] as int64 + ), + 24 + ) as int64 + ), + ":", + safe_cast( + split(horario_fim, ":")[offset(1)] as int64 + ), + ":", + safe_cast( + split(horario_fim, ":")[offset(2)] as int64 + ) + ) + ) + ), + interval div( + safe_cast(split(horario_fim, ":")[offset(0)] as int64), 24 + ) day + ), + null + ) as fim_periodo + from data_efetiva e + inner join + ( + select * + from {{ ref("subsidio_quadro_horario") }} + {% if is_incremental() %} + where + data_versao + in (select data_versao_frequencies from data_efetiva) + {% endif %} + ) p + on e.data_versao_frequencies = p.data_versao + and e.tipo_dia = p.tipo_dia + ), + -- 3. Trata informação de trips: adiciona ao sentido da trip o sentido + -- planejado (os shapes/trips circulares são separados em + -- ida/volta no sigmob) + trips as ( + select e.data, t.* + from + ( + select * + from {{ ref("subsidio_trips_desaninhada") }} + {% if is_incremental() %} + where + data_versao in (select data_versao_trips from data_efetiva) + {% endif %} + ) t + inner join data_efetiva e on t.data_versao = e.data_versao_trips + ), + quadro_trips as ( + select * + from + ( + select distinct + * except (trip_id), trip_id as trip_id_planejado, trip_id + from quadro + where sentido = "I" or sentido = "V" + ) + union all + ( + select + * except (trip_id), + trip_id as trip_id_planejado, + concat(trip_id, "_0") as trip_id, + from quadro + where sentido = "C" + ) + union all + ( + select + * except (trip_id), + trip_id as trip_id_planejado, + concat(trip_id, "_1") as trip_id, + from quadro + where sentido = "C" + ) + ), + quadro_tratada as ( + select + q.*, + t.shape_id as shape_id_planejado, + case + when sentido = "C" + then shape_id || "_" || split(q.trip_id, "_")[offset(1)] + else shape_id + end as shape_id, -- TODO: adicionar no sigmob + from quadro_trips q + left join trips t on t.data = q.data and t.trip_id = q.trip_id_planejado + ), + -- 4. Trata informações de shapes: junta trips e shapes para resgatar o sentido + -- planejado (os shapes/trips circulares são separados em + -- ida/volta no sigmob) + shapes as ( + select e.data, data_versao as data_shape, shape_id, shape, start_pt, end_pt + from data_efetiva e + inner join + ( + select * + from {{ ref("subsidio_shapes_geom") }} + {% if is_incremental() %} + where + data_versao in (select data_versao_shapes from data_efetiva) + {% endif %} + ) s + on s.data_versao = e.data_versao_shapes + ) + -- 5. Junta shapes e trips aos servicos planejados no quadro horário + select + p.*, + s.data_shape, + s.shape, + case + when p.sentido = "C" and split(p.shape_id, "_")[offset(1)] = "0" + then "I" + when p.sentido = "C" and split(p.shape_id, "_")[offset(1)] = "1" + then "V" + when p.sentido = "I" or p.sentido = "V" + then p.sentido + end as sentido_shape, + s.start_pt, + s.end_pt, + safe_cast(null as int64) as id_tipo_trajeto, -- Adaptação para formato da SUBSIDIO_V6 + safe_cast(null as string) as feed_version, -- Adaptação para formato da SUBSIDIO_V6 + current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao -- Adaptação para formato da SUBSIDIO_V7 + from quadro_tratada p + inner join shapes s on p.shape_id = s.shape_id and p.data = s.data + +{% else %} + {% if execute %} + {% set result = run_query( + "SELECT tipo_os, feed_version, feed_start_date, tipo_dia FROM " + ~ ref("subsidio_data_versao_efetiva") + ~ " WHERE data BETWEEN DATE_SUB(DATE('" + ~ var("run_date") + ~ "'), INTERVAL 2 DAY) AND DATE_SUB(DATE('" + ~ var("run_date") + ~ "'), INTERVAL 1 DAY) ORDER BY data" + ) %} + {% set tipo_oss = result.columns[0].values() %} + {% set feed_versions = result.columns[1].values() %} + {% set feed_start_dates = result.columns[2].values() %} + {% set tipo_dias = result.columns[3].values() %} + {% endif %} + + with + -- 1. Define datas do período planejado + data_versao_efetiva as ( + select data, tipo_dia, subtipo_dia, feed_version, feed_start_date, tipo_os, + from + {{ ref("subsidio_data_versao_efetiva") }} + -- `rj-smtr.projeto_subsidio_sppo.subsidio_data_versao_efetiva` + where + data between date_sub("{{ var('run_date') }}", interval 2 day) and date_sub("{{ var('run_date') }}", interval 1 day) -- fmt: off + ), + dia_atual as ( + select + feed_version, + feed_start_date, + feed_end_date, + tipo_os, + tipo_dia, + servico, + vista, + consorcio, + sentido, + partidas_total_planejada, + distancia_planejada, + distancia_total_planejada, + inicio_periodo, + fim_periodo, + faixa_horaria_inicio, + faixa_horaria_fim, + shape_id, + shape_id_planejado, + sentido_shape, + id_tipo_trajeto, + from + {{ ref("ordem_servico_trips_shapes_gtfs") }} + -- `rj-smtr.gtfs.ordem_servico_trips_shapes` + where + tipo_os = "{{ tipo_oss[1] }}" + and feed_version = "{{ feed_versions[1] }}" + and feed_start_date = date("{{ feed_start_dates[1] }}") + and tipo_dia = "{{ tipo_dias[1] }}" + ), + -- 2. Busca partidas e quilometragem da faixa horaria (dia anterior) + dia_anterior as ( + select + "{{ feed_versions[1] }}" as feed_version, + date("{{ feed_start_dates[1] }}") as feed_start_date, + ts.feed_end_date, + "{{ tipo_oss[1] }}" as tipo_os, + "{{ tipo_dias[1] }}" as tipo_dia, + ts.servico, + ts.vista, + ts.consorcio, + ts.sentido, + ts.partidas_total_planejada, + da.distancia_planejada, + ts.distancia_total_planejada, + da.inicio_periodo, + da.fim_periodo, + "00:00:00" as faixa_horaria_inicio, + "02:59:59" as faixa_horaria_fim, + ts.trip_id_planejado, + ts.trip_id, + ts.shape_id, + ts.shape_id_planejado, + ts.shape, + ts.sentido_shape, + ts.start_pt, + ts.end_pt, + ts.id_tipo_trajeto, + from + {{ ref("ordem_servico_trips_shapes_gtfs") }} as ts + -- `rj-smtr.gtfs.ordem_servico_trips_shapes` as ts + left join + ( + select distinct + servico, + sentido, + shape_id, + distancia_planejada, + inicio_periodo, + fim_periodo + from dia_atual + ) as da using (servico, sentido, shape_id) + where + ts.faixa_horaria_inicio = "24:00:00" + and ts.tipo_os = "{{ tipo_oss[0] }}" + and ts.feed_version = "{{ feed_versions[0] }}" + and ts.feed_start_date = date("{{ feed_start_dates[0] }}") + and ts.tipo_dia = "{{ tipo_dias[0] }}" + ), + trips as ( + select distinct + feed_version, + feed_start_date, + tipo_os, + tipo_dia, + servico, + sentido, + trip_id_planejado, + trip_id, + shape_id, + from + {{ ref("ordem_servico_trips_shapes_gtfs") }} + -- `rj-smtr.gtfs.ordem_servico_trips_shapes` + where + tipo_os = "{{ tipo_oss[1] }}" + and feed_version = "{{ feed_versions[1] }}" + and feed_start_date = date("{{ feed_start_dates[1] }}") + and tipo_dia = "{{ tipo_dias[1] }}" + ), + combina_trips_shapes as ( + select + feed_version, + feed_start_date, + feed_end_date, + tipo_os, + tipo_dia, + servico, + vista, + consorcio, + sentido, + partidas_total_planejada, + distancia_planejada, + distancia_total_planejada, + inicio_periodo, + fim_periodo, + faixa_horaria_inicio, + faixa_horaria_fim, + shape_id, + shape_id_planejado, + sentido_shape, + id_tipo_trajeto, + from dia_atual + union all + select + feed_version, + feed_start_date, + feed_end_date, + tipo_os, + tipo_dia, + servico, + vista, + consorcio, + sentido, + partidas_total_planejada, + distancia_planejada, + distancia_total_planejada, + inicio_periodo, + fim_periodo, + faixa_horaria_inicio, + faixa_horaria_fim, + shape_id, + shape_id_planejado, + sentido_shape, + id_tipo_trajeto, + from dia_anterior + ), + data_trips_shapes as ( + select + d.data, + case + when subtipo_dia is not null + then concat(o.tipo_dia, " - ", subtipo_dia) + else o.tipo_dia + end as tipo_dia, + servico, + vista, + consorcio, + sentido, + partidas_total_planejada, + distancia_planejada, + distancia_total_planejada, + if( + inicio_periodo is not null + and array_length(split(inicio_periodo, ":")) = 3, + datetime_add( + datetime( + d.data, + parse_time( + "%T", + concat( + safe_cast( + mod( + safe_cast( + split(inicio_periodo, ":")[ + offset(0) + ] as int64 + ), + 24 + ) as int64 + ), + ":", + safe_cast( + split(inicio_periodo, ":")[offset(1)] as int64 + ), + ":", + safe_cast( + split(inicio_periodo, ":")[offset(2)] as int64 + ) + ) + ) + ), + interval div( + safe_cast(split(inicio_periodo, ":")[offset(0)] as int64), + 24 + ) day + ), + null + ) as inicio_periodo, + if( + fim_periodo is not null + and array_length(split(fim_periodo, ":")) = 3, + datetime_add( + datetime( + d.data, + parse_time( + "%T", + concat( + safe_cast( + mod( + safe_cast( + split(fim_periodo, ":")[ + offset(0) + ] as int64 + ), + 24 + ) as int64 + ), + ":", + safe_cast( + split(fim_periodo, ":")[offset(1)] as int64 + ), + ":", + safe_cast( + split(fim_periodo, ":")[offset(2)] as int64 + ) + ) + ) + ), + interval div( + safe_cast(split(fim_periodo, ":")[offset(0)] as int64), 24 + ) day + ), + null + ) as fim_periodo, + if( + d.data >= date("{{ var('DATA_SUBSIDIO_V9_INICIO') }}"), -- fmt: off + datetime_add( + datetime( + d.data, + parse_time( + "%T", + concat( + safe_cast( + mod( + safe_cast( + split(o.faixa_horaria_inicio, ":")[ + offset(0) + ] as int64 + ), + 24 + ) as int64 + ), + ":", + safe_cast( + split(o.faixa_horaria_inicio, ":")[ + offset(1) + ] as int64 + ), + ":", + safe_cast( + split(o.faixa_horaria_inicio, ":")[ + offset(2) + ] as int64 + ) + ) + ) + ), + interval div( + safe_cast( + split(o.faixa_horaria_inicio, ":")[offset(0)] as int64 + ), + 24 + ) day + ), + datetime_add( + datetime( + d.data, + parse_time( + "%T", + concat( + safe_cast( + mod( + safe_cast( + split("00:00:00", ":")[ + offset(0) + ] as int64 + ), + 24 + ) as int64 + ), + ":", + safe_cast( + split("00:00:00", ":")[offset(1)] as int64 + ), + ":", + safe_cast( + split("00:00:00", ":")[offset(2)] as int64 + ) + ) + ) + ), + interval div( + safe_cast(split("00:00:00", ":")[offset(0)] as int64), 24 + ) day + ) + ) as faixa_horaria_inicio, + if( + d.data >= date("{{ var('DATA_SUBSIDIO_V9_INICIO') }}"), -- fmt: off + datetime_add( + datetime( + d.data, + parse_time( + "%T", + concat( + safe_cast( + mod( + safe_cast( + split(o.faixa_horaria_fim, ":")[ + offset(0) + ] as int64 + ), + 24 + ) as int64 + ), + ":", + safe_cast( + split(o.faixa_horaria_fim, ":")[ + offset(1) + ] as int64 + ), + ":", + safe_cast( + split(o.faixa_horaria_fim, ":")[ + offset(2) + ] as int64 + ) + ) + ) + ), + interval div( + safe_cast( + split(o.faixa_horaria_fim, ":")[offset(0)] as int64 + ), + 24 + ) day + ), + datetime_add( + datetime( + d.data, + parse_time( + "%T", + concat( + safe_cast( + mod( + safe_cast( + split("23:59:59", ":")[ + offset(0) + ] as int64 + ), + 24 + ) as int64 + ), + ":", + safe_cast( + split("23:59:59", ":")[offset(1)] as int64 + ), + ":", + safe_cast( + split("23:59:59", ":")[offset(2)] as int64 + ) + ) + ) + ), + interval div( + safe_cast(split("23:59:59", ":")[offset(0)] as int64), 24 + ) day + ) + ) as faixa_horaria_fim, + t.trip_id_planejado, + t.trip_id, + shape_id, + shape_id_planejado, + safe_cast(null as date) as data_shape, + sentido_shape, + id_tipo_trajeto, + feed_version, + feed_start_date + from data_versao_efetiva as d + left join + combina_trips_shapes as o using ( + feed_start_date, feed_version, tipo_dia, tipo_os + ) + left join + trips as t using ( + feed_start_date, + feed_version, + tipo_dia, + tipo_os, + servico, + sentido, + shape_id + ) + where + data = date_sub("{{ var('run_date') }}", interval 1 day) -- fmt: off + and faixa_horaria_inicio != "24:00:00" + ), + shapes as ( + select * + from + {{ ref("shapes_geom_gtfs") }} + -- `rj-smtr.gtfs.shapes_geom` + where + feed_start_date in ( + select feed_start_date + from data_versao_efetiva + where + data between date_sub("{{ var('run_date') }}", interval 2 day) and date_sub("{{ var('run_date') }}", interval 1 day) -- fmt: off + ) + ), + dados_agregados as ( + select + data, + tipo_dia, + servico, + vista, + consorcio, + sentido, + sum(coalesce(partidas_total_planejada, 0)) as partidas_total_planejada, + distancia_planejada, + sum(distancia_total_planejada) as distancia_total_planejada, + inicio_periodo, + fim_periodo, + faixa_horaria_inicio, + faixa_horaria_fim, + trip_id_planejado, + trip_id, + shape_id, + shape_id_planejado, + data_shape, + sentido_shape, + id_tipo_trajeto, + feed_version, + feed_start_date + from data_trips_shapes + group by + data, + tipo_dia, + servico, + vista, + consorcio, + sentido, + distancia_planejada, + inicio_periodo, + fim_periodo, + faixa_horaria_inicio, + faixa_horaria_fim, + trip_id_planejado, + trip_id, + shape_id, + shape_id_planejado, + data_shape, + sentido_shape, + id_tipo_trajeto, + feed_version, + feed_start_date + ) + select + data, + tipo_dia, + servico, + vista, + consorcio, + sentido, + partidas_total_planejada, + distancia_planejada, + distancia_total_planejada, + inicio_periodo, + fim_periodo, + faixa_horaria_inicio, + faixa_horaria_fim, + trip_id_planejado, + trip_id, + shape_id, + shape_id_planejado, + data_shape, + s.shape, + sentido_shape, + s.start_pt, + s.end_pt, + id_tipo_trajeto, + feed_version, + feed_start_date, + current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao + from dados_agregados + left join shapes as s using (feed_version, feed_start_date, shape_id) + {% if var("run_date") == "2024-05-05" %} + -- Apuração "Madonna · The Celebration Tour in Rio" + where and servico != "SE001" + {% endif %} +{% endif %} diff --git a/queries/models/sources.yml b/queries/models/sources.yml index e61c4af1f..44fd47259 100644 --- a/queries/models/sources.yml +++ b/queries/models/sources.yml @@ -167,8 +167,26 @@ sources: - name: autuacoes_citran - name: receita_autuacao + - name: dados_mestres + database: datario + + tables: + - name: logradouro + - name: source_rioonibus database: rj-smtr-staging tables: - - name: viagem_informada \ No newline at end of file + - name: viagem_informada + + - name: source_jae + database: rj-smtr-staging + + tables: + - name: transacao_ordem + + - name: source_sonda + database: rj-smtr-staging + + tables: + - name: viagem_informada diff --git a/queries/models/subsidio/CHANGELOG.md b/queries/models/subsidio/CHANGELOG.md index d947700b9..6a60dff12 100644 --- a/queries/models/subsidio/CHANGELOG.md +++ b/queries/models/subsidio/CHANGELOG.md @@ -1,17 +1,35 @@ # Changelog - subsidio +## [2.0.0] - 2024-12-06 + +# Corrigido + +- Corrigido e refatorado o modelo `viagem_transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/357): + - Reformatação conforme padrão `sqlfmt` + - Passa a considerar registros de GPS do validador com coordenadas zeradas a partir de `DATA_SUBSIDIO_V12_INICIO` + - Alterada janela de dados da CTE `viagem`, de forma a não ocorrer sobreposição entre viagens finalizadas na partição do dia anterior ao `start_date` + - Passa a considerar uma transação RioCard ou Jaé para fins de validação do SBD a partir de `DATA_SUBSIDIO_V12_INICIO` + +## [1.0.3] - 2024-11-29 + +# Alterado + +- Alterada a janela de dados considerados no modelo `viagem_transacao.sql` para 6 dias (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/346) + ## [1.0.2] - 2024-10-24 -- cria exceção na verificação d viagens sem transacao para a eleição de 2024-10-06 em viagem_transacao de 06h as 20h (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/286) +### Adicionado + +- Adicionada exceção na verificação de viagens sem transação para a eleição de 2024-10-06 no modelo `viagem_transacao.sql` de 06h às 20h (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/286) ## [1.0.1] - 2024-09-12 ### Corrigido -- Corriido o tratamento de `viagem_transacao` para lidar com casos de mudança aberto/fechado ao longo da viagem, lat, long zerada do validador, mais de um validador associado ao veículo e viagem que inicia/encerra em dia diferente(https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/210) +- Corrigido o tratamento do modelo `viagem_transacao.sql` para lidar com casos de mudança aberto/fechado ao longo da viagem, lat, long zerada do validador, mais de um validador associado ao veículo e viagem que inicia/encerra em dia diferente (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/210) ## [1.0.0] - 2024-07-31 ### Adicionado -- Cria modelo `viagem_transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/121) \ No newline at end of file +- Adicionado modelo `viagem_transacao.sql` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/121) \ No newline at end of file diff --git a/queries/models/subsidio/viagem_transacao.sql b/queries/models/subsidio/viagem_transacao.sql index d90197c4a..eba9bd43b 100644 --- a/queries/models/subsidio/viagem_transacao.sql +++ b/queries/models/subsidio/viagem_transacao.sql @@ -6,245 +6,304 @@ ) }} -WITH - -- 1. Transações Jaé - transacao AS ( - SELECT - id_veiculo, - datetime_transacao - FROM - {{ ref("transacao") }} - -- rj-smtr.br_rj_riodejaneiro_bilhetagem.transacao - WHERE - data BETWEEN DATE("{{ var("start_date") }}") - AND DATE_ADD(DATE("{{ var("end_date") }}"), INTERVAL 1 DAY) - ), - -- 2. Transações RioCard - transacao_riocard AS ( - SELECT - id_veiculo, - datetime_transacao - FROM - {{ ref("transacao_riocard") }} - -- rj-smtr.br_rj_riodejaneiro_bilhetagem.transacao_riocard - WHERE - data BETWEEN DATE("{{ var("start_date") }}") - AND DATE_ADD(DATE("{{ var("end_date") }}"), INTERVAL 1 DAY) - ), - -- 3. GPS Validador - gps_validador AS ( - SELECT - data, - datetime_gps, - id_veiculo, - id_validador, - estado_equipamento, - latitude, - longitude - FROM - {{ ref("gps_validador") }} - -- rj-smtr.br_rj_riodejaneiro_bilhetagem.gps_validador - WHERE - data BETWEEN DATE("{{ var("start_date") }}") - AND DATE_ADD(DATE("{{ var("end_date") }}"), INTERVAL 1 DAY) - AND (latitude != 0 OR longitude != 0) - ), - -- 4. Viagens realizadas - viagem AS ( - SELECT - data, - servico_realizado AS servico, - datetime_partida, - datetime_chegada, - id_veiculo, - id_viagem, - distancia_planejada - FROM - {{ ref("viagem_completa") }} - -- rj-smtr.projeto_subsidio_sppo.viagem_completa - WHERE - data BETWEEN DATE("{{ var("start_date") }}") - AND DATE( "{{ var("end_date") }}" ) ), - -- 5. Status dos veículos - veiculos AS ( - SELECT - data, - id_veiculo, - status - FROM - {{ ref("sppo_veiculo_dia") }} - -- rj-smtr.veiculo.sppo_veiculo_dia - WHERE - data BETWEEN DATE("{{ var("start_date") }}") - AND DATE("{{ var("end_date") }}") ), - -- 6. Viagem, para fins de contagem de passageiros, com tolerância de 30 minutos, limitada pela viagem anterior - viagem_com_tolerancia AS ( - SELECT - v.*, - LAG(v.datetime_chegada) OVER (PARTITION BY v.id_veiculo ORDER BY v.datetime_partida) AS viagem_anterior_chegada, - CASE - WHEN LAG(v.datetime_chegada) OVER (PARTITION BY v.id_veiculo ORDER BY v.datetime_partida) IS NULL THEN - DATETIME(TIMESTAMP_SUB(datetime_partida, INTERVAL 30 MINUTE)) - ELSE - DATETIME(TIMESTAMP_ADD(GREATEST( - TIMESTAMP_SUB(datetime_partida, INTERVAL 30 MINUTE), - LAG(v.datetime_chegada) OVER (PARTITION BY v.id_veiculo ORDER BY v.datetime_partida) - ), INTERVAL 1 SECOND)) - END AS datetime_partida_com_tolerancia - FROM - viagem AS v - ), - -- 7. Contagem de transações Jaé - transacao_contagem AS ( - SELECT - v.data, - v.id_viagem, - COUNT(t.datetime_transacao) AS quantidade_transacao - FROM - transacao AS t - JOIN - viagem_com_tolerancia AS v - ON - t.id_veiculo = SUBSTR(v.id_veiculo, 2) - AND t.datetime_transacao BETWEEN v.datetime_partida_com_tolerancia AND v.datetime_chegada - GROUP BY - v.data, v.id_viagem - ), - -- 5. Contagem de transações RioCard - transacao_riocard_contagem AS ( - SELECT - v.data, - v.id_viagem, - COUNT(tr.datetime_transacao) AS quantidade_transacao_riocard - FROM - transacao_riocard AS tr - JOIN - viagem_com_tolerancia AS v - ON - tr.id_veiculo = SUBSTR(v.id_veiculo, 2) - AND tr.datetime_transacao BETWEEN v.datetime_partida_com_tolerancia AND v.datetime_chegada - GROUP BY - v.data, v.id_viagem - ), - -- 6. Ajusta estado do equipamento - -- Agrupa mesma posição para mesmo validador e veículo, mantendo preferencialmente o estado do equipamento "ABERTO" - estado_equipamento_aux AS ( - SELECT - data, - id_validador, - id_veiculo, - latitude, - longitude, - IF(COUNT(CASE WHEN estado_equipamento = "ABERTO" THEN 1 END) >= 1, "ABERTO", "FECHADO") AS estado_equipamento, - MIN(datetime_gps) AS datetime_gps, - FROM - gps_validador - GROUP BY - 1, - 2, - 3, - 4, - 5 - ), - -- 7. Relacionamento entre estado do equipamento e viagem - gps_validador_viagem AS ( - SELECT - v.data, - e.datetime_gps, - v.id_viagem, - e.id_validador, - e.estado_equipamento, - e.latitude, - e.longitude - FROM - estado_equipamento_aux AS e - JOIN - viagem AS v - ON - e.id_veiculo = SUBSTR(v.id_veiculo, 2) - AND e.datetime_gps BETWEEN v.datetime_partida AND v.datetime_chegada - ), - -- 8. Calcula a porcentagem de estado do equipamento "ABERTO" por validador e viagem - estado_equipamento_perc AS ( - SELECT - data, - id_viagem, - id_validador, - COUNTIF(estado_equipamento = "ABERTO") / COUNT(*) AS percentual_estado_equipamento_aberto - FROM - gps_validador_viagem - GROUP BY - 1, - 2, - 3 - ), - -- 9. Considera o validador com maior porcentagem de estado do equipamento "ABERTO" por viagem - estado_equipamento_max_perc AS ( - SELECT - data, - id_viagem, - MAX_BY(id_validador, percentual_estado_equipamento_aberto) AS id_validador, - MAX(percentual_estado_equipamento_aberto) AS percentual_estado_equipamento_aberto - FROM - estado_equipamento_perc - GROUP BY - 1, - 2 - ), - -- 10. Verifica se a viagem possui estado do equipamento "ABERTO" em pelo menos 80% dos registros - estado_equipamento_verificacao AS ( - SELECT - data, - id_viagem, - id_validador, - percentual_estado_equipamento_aberto, - IF(percentual_estado_equipamento_aberto >= 0.8 OR percentual_estado_equipamento_aberto IS NULL, TRUE, FALSE) AS indicador_estado_equipamento_aberto - FROM - viagem - LEFT JOIN - estado_equipamento_max_perc - USING - (data, id_viagem) - ) -SELECT - v.data, - v.id_viagem, - v.id_veiculo, - v.servico, - eev.id_validador, - CASE - WHEN v.data >= DATE("{{ var("DATA_SUBSIDIO_V8_INICIO") }}") - AND (COALESCE(tr.quantidade_transacao_riocard, 0) = 0 - OR COALESCE(eev.indicador_estado_equipamento_aberto, FALSE) = FALSE) - AND ve.status IN ("Licenciado com ar e não autuado", "Licenciado sem ar e não autuado") - AND v.datetime_partida NOT BETWEEN "2024-10-06 06:00:00" AND "2024-10-06 20:00:00" -- Eleição (2024-10-06) - THEN "Sem transação" - ELSE ve.status - END AS tipo_viagem, - v.distancia_planejada, - COALESCE(t.quantidade_transacao, 0) AS quantidade_transacao, - COALESCE(tr.quantidade_transacao_riocard, 0) AS quantidade_transacao_riocard, - eev.percentual_estado_equipamento_aberto, - eev.indicador_estado_equipamento_aberto, - v.datetime_partida_com_tolerancia AS datetime_partida_bilhetagem, - v.datetime_partida, - v.datetime_chegada, - CURRENT_DATETIME("America/Sao_Paulo") AS datetime_ultima_atualizacao -FROM - viagem_com_tolerancia AS v -LEFT JOIN - veiculos AS ve -USING - (data, id_veiculo) -LEFT JOIN - transacao_contagem AS t -USING - (data, id_viagem) -LEFT JOIN - transacao_riocard_contagem AS tr -USING - (data, id_viagem) -LEFT JOIN - estado_equipamento_verificacao AS eev -USING - (data, id_viagem) \ No newline at end of file +with + -- 1. Transações Jaé + transacao as ( + select id_veiculo, datetime_transacao + from {{ ref("transacao") }} + -- rj-smtr.br_rj_riodejaneiro_bilhetagem.transacao + where + data between date("{{ var('start_date') }}") and date_add( + date("{{ var('end_date') }}"), interval 1 day + ) + and date(datetime_processamento) + <= date_add(date("{{ var('end_date') }}"), interval 6 day) + ), + -- 2. Transações RioCard + transacao_riocard as ( + select id_veiculo, datetime_transacao + from {{ ref("transacao_riocard") }} + -- rj-smtr.br_rj_riodejaneiro_bilhetagem.transacao_riocard + where + data between date("{{ var('start_date') }}") and date_add( + date("{{ var('end_date') }}"), interval 1 day + ) + and date(datetime_processamento) + <= date_add(date("{{ var('end_date') }}"), interval 6 day) + ), + -- 3. GPS Validador + gps_validador as ( + select + data, + datetime_gps, + id_veiculo, + id_validador, + estado_equipamento, + latitude, + longitude + from {{ ref("gps_validador") }} + -- rj-smtr.br_rj_riodejaneiro_bilhetagem.gps_validador + where + data between date("{{ var('start_date') }}") and date_add( + date("{{ var('end_date') }}"), interval 1 day + ) + and ( + ( + data < date("{{ var('DATA_SUBSIDIO_V12_INICIO') }}") + and (latitude != 0 or longitude != 0) + ) + or data >= date("{{ var('DATA_SUBSIDIO_V12_INICIO') }}") + ) + and date(datetime_captura) + <= date_add(date("{{ var('end_date') }}"), interval 6 day) + ), + -- 4. Viagens realizadas + viagem as ( + select + data, + servico_realizado as servico, + datetime_partida, + datetime_chegada, + id_veiculo, + id_viagem, + distancia_planejada + from {{ ref("viagem_completa") }} + -- rj-smtr.projeto_subsidio_sppo.viagem_completa + where + data + between date_sub(date("{{ var('start_date') }}"), interval 1 day) and date( + "{{ var('end_date') }}" + ) + ), + -- 5. Status dos veículos + veiculos as ( + select data, id_veiculo, status + from {{ ref("sppo_veiculo_dia") }} + -- rj-smtr.veiculo.sppo_veiculo_dia + where + data + between date("{{ var('start_date') }}") and date("{{ var('end_date') }}") + ), + -- 6. Viagem, para fins de contagem de passageiros, com tolerância de 30 minutos, + -- limitada pela viagem anterior + viagem_com_tolerancia_previa as ( + select + v.*, + lag(v.datetime_chegada) over ( + partition by v.id_veiculo order by v.datetime_partida + ) as viagem_anterior_chegada, + case + when + lag(v.datetime_chegada) over ( + partition by v.id_veiculo order by v.datetime_partida + ) + is null + then datetime(timestamp_sub(datetime_partida, interval 30 minute)) + else + datetime( + timestamp_add( + greatest( + timestamp_sub(datetime_partida, interval 30 minute), + lag(v.datetime_chegada) over ( + partition by v.id_veiculo + order by v.datetime_partida + ) + ), + interval 1 second + ) + ) + end as datetime_partida_com_tolerancia + from viagem as v + ), + -- 7. Considera apenas as viagens realizadas no período de apuração + viagem_com_tolerancia as ( + select * + from viagem_com_tolerancia_previa + where + data + between date("{{ var('start_date') }}") and date("{{ var('end_date') }}") + ), + -- 8. Contagem de transações Jaé + transacao_contagem as ( + select v.data, v.id_viagem, count(t.datetime_transacao) as quantidade_transacao + from transacao as t + join + viagem_com_tolerancia as v + on t.id_veiculo = substr(v.id_veiculo, 2) + and t.datetime_transacao + between v.datetime_partida_com_tolerancia and v.datetime_chegada + group by v.data, v.id_viagem + ), + -- 9. Contagem de transações RioCard + transacao_riocard_contagem as ( + select + v.data, + v.id_viagem, + count(tr.datetime_transacao) as quantidade_transacao_riocard + from transacao_riocard as tr + join + viagem_com_tolerancia as v + on tr.id_veiculo = substr(v.id_veiculo, 2) + and tr.datetime_transacao + between v.datetime_partida_com_tolerancia and v.datetime_chegada + group by v.data, v.id_viagem + ), + -- 10. Ajusta estado do equipamento + -- Agrupa mesma posição para mesmo validador e veículo, mantendo preferencialmente + -- o estado do equipamento "ABERTO" quanto latitude e longitude for diferente de + -- (0,0) + estado_equipamento_aux as ( + select * + from + ( + ( + select + data, + id_validador, + id_veiculo, + latitude, + longitude, + if( + count(case when estado_equipamento = "ABERTO" then 1 end) + >= 1, + "ABERTO", + "FECHADO" + ) as estado_equipamento, + min(datetime_gps) as datetime_gps, + from gps_validador + where + ( + data >= date("{{ var('DATA_SUBSIDIO_V12_INICIO') }}") + and latitude != 0 + and longitude != 0 + ) + or data < date("{{ var('DATA_SUBSIDIO_V12_INICIO') }}") + group by 1, 2, 3, 4, 5 + ) + union all + ( + select + data, + id_validador, + id_veiculo, + latitude, + longitude, + estado_equipamento, + datetime_gps, + from gps_validador + where + data >= date("{{ var('DATA_SUBSIDIO_V12_INICIO') }}") + and latitude = 0 + and longitude = 0 + ) + ) + ), + -- 11. Relacionamento entre estado do equipamento e viagem + gps_validador_viagem as ( + select + v.data, + e.datetime_gps, + v.id_viagem, + e.id_validador, + e.estado_equipamento, + e.latitude, + e.longitude + from estado_equipamento_aux as e + join + viagem as v + on e.id_veiculo = substr(v.id_veiculo, 2) + and e.datetime_gps between v.datetime_partida and v.datetime_chegada + ), + -- 12. Calcula a porcentagem de estado do equipamento "ABERTO" por validador e + -- viagem + estado_equipamento_perc as ( + select + data, + id_viagem, + id_validador, + countif(estado_equipamento = "ABERTO") + / count(*) as percentual_estado_equipamento_aberto + from gps_validador_viagem + group by 1, 2, 3 + ), + -- 13. Considera o validador com maior porcentagem de estado do equipamento + -- "ABERTO" por viagem + estado_equipamento_max_perc as ( + select + data, + id_viagem, + max_by(id_validador, percentual_estado_equipamento_aberto) as id_validador, + max( + percentual_estado_equipamento_aberto + ) as percentual_estado_equipamento_aberto + from estado_equipamento_perc + group by 1, 2 + ), + -- 14. Verifica se a viagem possui estado do equipamento "ABERTO" em pelo menos + -- 80% dos registros + estado_equipamento_verificacao as ( + select + data, + id_viagem, + id_validador, + percentual_estado_equipamento_aberto, + if( + percentual_estado_equipamento_aberto >= 0.8 + or percentual_estado_equipamento_aberto is null, + true, + false + ) as indicador_estado_equipamento_aberto + from viagem + left join estado_equipamento_max_perc using (data, id_viagem) + ) +select + v.data, + v.id_viagem, + v.id_veiculo, + v.servico, + eev.id_validador, + case + when + v.data >= date("{{ var('DATA_SUBSIDIO_V8_INICIO') }}") + and ( + ( + v.data < date("{{ var('DATA_SUBSIDIO_V12_INICIO') }}") + and ( + coalesce(tr.quantidade_transacao_riocard, 0) = 0 + or coalesce(eev.indicador_estado_equipamento_aberto, false) + = false + ) + ) + or ( + v.data >= date("{{ var('DATA_SUBSIDIO_V12_INICIO') }}") + and ( + ( + coalesce(tr.quantidade_transacao_riocard, 0) = 0 + and coalesce(t.quantidade_transacao, 0) = 0 + ) + or coalesce(eev.indicador_estado_equipamento_aberto, false) + = false + ) + ) + ) + and ve.status + in ("Licenciado com ar e não autuado", "Licenciado sem ar e não autuado") + and v.datetime_partida not between "2024-10-06 06:00:00" + and "2024-10-06 20:00:00" -- Eleição (2024-10-06) + then "Sem transação" + else ve.status + end as tipo_viagem, + v.distancia_planejada, + coalesce(t.quantidade_transacao, 0) as quantidade_transacao, + coalesce(tr.quantidade_transacao_riocard, 0) as quantidade_transacao_riocard, + eev.percentual_estado_equipamento_aberto, + eev.indicador_estado_equipamento_aberto, + v.datetime_partida_com_tolerancia as datetime_partida_bilhetagem, + v.datetime_partida, + v.datetime_chegada, + current_datetime("America/Sao_Paulo") as datetime_ultima_atualizacao +from viagem_com_tolerancia as v +left join veiculos as ve using (data, id_veiculo) +left join transacao_contagem as t using (data, id_viagem) +left join transacao_riocard_contagem as tr using (data, id_viagem) +left join estado_equipamento_verificacao as eev using (data, id_viagem) diff --git a/queries/models/veiculo/CHANGELOG.md b/queries/models/veiculo/CHANGELOG.md index 6577475e9..18d85c828 100644 --- a/queries/models/veiculo/CHANGELOG.md +++ b/queries/models/veiculo/CHANGELOG.md @@ -1,5 +1,12 @@ # Changelog - veiculo + +## [1.1.4] - 2024-11-13 + +#### Adicionado + +- Adiciona testes do subsidio para `sppo_veiculo_dia` (https://github.com/prefeitura-rio/pipelines_rj_smtr/pull/256) + ## [1.1.3] - 2024-10-25 #### Alterado diff --git a/queries/models/veiculo/schema.yml b/queries/models/veiculo/schema.yml index a14206db7..74cb81b9b 100644 --- a/queries/models/veiculo/schema.yml +++ b/queries/models/veiculo/schema.yml @@ -148,6 +148,12 @@ models: Renumeração - Solicitação de alteração de número de ordem de veículo já incluído no Sistema de Transporte Urbano (STU)" - name: sppo_veiculo_dia description: "Tabela resumo dos veículos que operaram no SPPO e seus respectivos indicadores" + tests: + - dbt_utils.unique_combination_of_columns: + combination_of_columns: + - data + - id_veiculo + name: dbt_utils.unique_combination_of_columns__data_id_veiculo__sppo_veiculo_dia columns: - name: data description: "Data de operação" @@ -179,6 +185,9 @@ models: - Autuado por limpeza/equipamento - Veículo que operou, foi licenciado, mas foi autuado cumulativamente por infrações relacionadas à limpeza e equipamentos do veículo\n - Sem ar e não autuado - Veículo que operou, foi licenciado sem ar condicionado e não foi autuado\n - Com ar e não autuado - Veículo que operou, foi licenciado com ar condicionado e não foi autuado" + tests: + - not_null: + name: not_null__status__sppo_veiculo_dia - name: versao description: "SHA do repositório no Github" - name: sppo_registro_agente_verao diff --git a/queries/packages.yml b/queries/packages.yml index f6f4a01c5..f98fb15af 100644 --- a/queries/packages.yml +++ b/queries/packages.yml @@ -4,4 +4,6 @@ packages: - package: data-mie/dbt_profiler version: 0.8.2 - package: dbt-labs/dbt_utils - version: 1.3.0 \ No newline at end of file + version: 1.3.0 + - package: calogica/dbt_expectations + version: [">=0.10.0", "<0.11.0"] \ No newline at end of file diff --git a/queries/selectors.yml b/queries/selectors.yml index 6d28a8cbe..d6412f96f 100644 --- a/queries/selectors.yml +++ b/queries/selectors.yml @@ -30,4 +30,44 @@ selectors: definition: method: file value: viagem_informada_monitoramento - parents: true \ No newline at end of file + parents: true + exclude: + - method: fqn + value: gtfs + - method: fqn + value: planejamento + + - name: viagem_validacao + description: Materialização da tabela de validação das viagens informadas + definition: + union: + - method: fqn + value: gps_viagem + - method: fqn + value: gps_segmento_viagem + - method: fqn + value: viagem_validacao + + - name: planejamento_diario + description: Materialização das tabelas de planejamento que devem rodar diariamente + definition: + union: + - method: fqn + value: aux_calendario_manual + - method: fqn + value: calendario + + - name: transacao_ordem + description: Materialização da tabela auxiliar de relacionamento entre as ordens de pagamento e as transações + definition: + union: + - method: fqn + value: staging_transacao_ordem + - method: fqn + value: aux_transacao_id_ordem_pagamento + + - name: datario + description: Materialização das views para o datario + definition: + method: fqn + value: datario \ No newline at end of file diff --git a/requirements.txt b/requirements.txt index d6b61e6f5..34bbf891c 100644 --- a/requirements.txt +++ b/requirements.txt @@ -113,6 +113,7 @@ tornado==6.4 tqdm==4.66.1 typing_extensions==4.9.0 tzdata==2023.3 +unidecode==1.3.8 uritemplate==4.1.1 urllib3==1.26.18 yarl==1.9.4