From 44afc0620d19b3cc4fa1f282ebf98ff10717b8c1 Mon Sep 17 00:00:00 2001 From: Jules Huisman Date: Fri, 30 Dec 2022 15:19:49 +0100 Subject: [PATCH] New `dagster_meltano` package (#15) * added a meltano elt solid and unit tests * fixed tests * add meltano install step to ci/cd tests * add navigation to tests setp * Created initial version to consolidate the two solid variants * Created a new dagster type for environment variables * Delete solids_new.py * Replaced the solids file with the new MeltanoEltSolid * Updated the tests * Updated the readme * Removed the select from meltano.yaml * cleaned up logging for errors * Update devcontainer.json * Ignored the build folder * Made changes to overal structure * Ported `dagster-meltano` code * Added pip cache * Added dbt to the meltano project * Improved the extract load op * Meltano job notebook * Added notebook content * Add ipykernel / delete copilot * Added meltano jobs and schedules * Removed old run_streams * Correctly parsed logging * Add metadata to tap runs * Revert "Added meltano jobs and schedules" This reverts commit 52a4f141437dee1b78d5baff88ed7f290a5dcaa6. * Revert "Add ipykernel / delete copilot" This reverts commit 6602781b7e893b0bf7fc55e03072da5a5641b328. * Revert "Added notebook content" This reverts commit 664d3da3828391c4d154dabebc7b32208e3adb9c. * Revert "Meltano job notebook" This reverts commit 767b24b2763acbb5d9a44cfdca21e4dd8996a575. * Initial working set-up of dagster repo * Added a `request_duration` parser * Notebook contents * Add test jobs in meltano config * Add dagster jobs to the meltano resource * Add Job class that transforms meltano job to a dagster job * Add load_jobs_from_meltano_project func * Add auto recognition of meltano schedules * Disabled tests * Moved the schedule generation to the job file * Added meltano as extra * Locked versions * Add dagit as requirement * Removed dagit requirement * Added package finder * Moved to new `dagster_meltano` * Deleted output files * Added tests * Ignore pytest_cache * Added testing suite * Install in Meltano project * Added support for project_dir Co-authored-by: Josh Lloyd Co-authored-by: jlloyd-widen <82222659+jlloyd-widen@users.noreply.github.com> Co-authored-by: Josh Lloyd Co-authored-by: BernardWez --- .devcontainer/devcontainer.json | 76 +- .github/workflows/test.yaml | 64 +- .gitignore | 4 +- .pylintrc | 65 +- .vscode/settings.json | 17 + .vscode/tasks.json | 42 + Dockerfile | 7 - README.md | 113 +- dagster_meltano/__init__.py | 8 + dagster_meltano/assets.py | 40 + dagster_meltano/generation.py | 64 + dagster_meltano/job.py | 97 ++ dagster_meltano/log_processing.py | 10 + dagster_meltano/log_processing/__init__.py | 50 + .../log_processing/json_processor.py | 22 + .../log_processing/metadata_processor.py | 80 + .../log_processing/passthrough_processor.py | 15 + dagster_meltano/logging.yaml | 29 + dagster_meltano/meltano_elt.py | 94 -- dagster_meltano/meltano_invoker.py | 154 ++ dagster_meltano/meltano_resource.py | 105 ++ dagster_meltano/ops.py | 79 + dagster_meltano/schedule.py | 27 + dagster_meltano/solids.py | 140 -- dagster_meltano/tests/__init__.py | 0 dagster_meltano/tests/pipeline.py | 18 - dagster_meltano/tests/test_meltano_elt.py | 63 - dagster_meltano/utils.py | 36 + meltano/extract/sample.csv | 4 - meltano/meltano.yml | 21 - {meltano => meltano_project}/.gitignore | 0 {meltano => meltano_project}/README.md | 0 meltano_project/meltano.yml | 35 + .../orchestrate}/.gitkeep | 0 .../orchestrate/dagster/repository.py | 13 + meltano_project/output/.gitignore | 2 + .../tap-carbon-intensity--meltano.lock | 17 + .../loaders/target-jsonl--andyh1203.lock | 34 + {meltano => meltano_project}/requirements.txt | 0 poetry.lock | 1375 +++++++++++++++++ pyproject.toml | 50 +- pytest.ini | 5 - setup.py | 28 - tests/meltano_test_project/.gitignore | 3 + tests/meltano_test_project/meltano.yml | 33 + .../orchestrate}/.gitkeep | 0 .../orchestrate/dagster/repository.py | 13 + tests/meltano_test_project/output/.gitignore | 2 + .../tap-carbon-intensity--meltano.lock | 17 + .../loaders/target-jsonl--andyh1203.lock | 34 + tests/test_generation.py | 41 + tests/test_meltano_install.py | 31 + tests/test_meltano_run.py | 43 + 53 files changed, 2736 insertions(+), 584 deletions(-) create mode 100644 .vscode/settings.json create mode 100644 .vscode/tasks.json delete mode 100644 Dockerfile create mode 100644 dagster_meltano/assets.py create mode 100644 dagster_meltano/generation.py create mode 100644 dagster_meltano/job.py create mode 100644 dagster_meltano/log_processing.py create mode 100644 dagster_meltano/log_processing/__init__.py create mode 100644 dagster_meltano/log_processing/json_processor.py create mode 100644 dagster_meltano/log_processing/metadata_processor.py create mode 100644 dagster_meltano/log_processing/passthrough_processor.py create mode 100644 dagster_meltano/logging.yaml delete mode 100644 dagster_meltano/meltano_elt.py create mode 100644 dagster_meltano/meltano_invoker.py create mode 100644 dagster_meltano/meltano_resource.py create mode 100644 dagster_meltano/ops.py create mode 100644 dagster_meltano/schedule.py delete mode 100644 dagster_meltano/solids.py delete mode 100644 dagster_meltano/tests/__init__.py delete mode 100644 dagster_meltano/tests/pipeline.py delete mode 100644 dagster_meltano/tests/test_meltano_elt.py create mode 100644 dagster_meltano/utils.py delete mode 100644 meltano/extract/sample.csv delete mode 100644 meltano/meltano.yml rename {meltano => meltano_project}/.gitignore (100%) rename {meltano => meltano_project}/README.md (100%) create mode 100644 meltano_project/meltano.yml rename {meltano/extract => meltano_project/orchestrate}/.gitkeep (100%) create mode 100644 meltano_project/orchestrate/dagster/repository.py create mode 100644 meltano_project/output/.gitignore create mode 100644 meltano_project/plugins/extractors/tap-carbon-intensity--meltano.lock create mode 100644 meltano_project/plugins/loaders/target-jsonl--andyh1203.lock rename {meltano => meltano_project}/requirements.txt (100%) create mode 100644 poetry.lock delete mode 100644 pytest.ini delete mode 100644 setup.py create mode 100644 tests/meltano_test_project/.gitignore create mode 100644 tests/meltano_test_project/meltano.yml rename {meltano/load => tests/meltano_test_project/orchestrate}/.gitkeep (100%) create mode 100644 tests/meltano_test_project/orchestrate/dagster/repository.py create mode 100644 tests/meltano_test_project/output/.gitignore create mode 100644 tests/meltano_test_project/plugins/extractors/tap-carbon-intensity--meltano.lock create mode 100644 tests/meltano_test_project/plugins/loaders/target-jsonl--andyh1203.lock create mode 100644 tests/test_generation.py create mode 100644 tests/test_meltano_install.py create mode 100644 tests/test_meltano_run.py diff --git a/.devcontainer/devcontainer.json b/.devcontainer/devcontainer.json index cf47a8e..6702c24 100644 --- a/.devcontainer/devcontainer.json +++ b/.devcontainer/devcontainer.json @@ -1,53 +1,23 @@ -{ - "name": "dagster-meltano", - "build": { - "dockerfile": "../Dockerfile" - }, - "extensions": [ - "ms-python.vscode-pylance", - "mechatroner.rainbow-csv", - "njpwerner.autodocstring", - "github.autopilot", - "waderyan.gitblame" - ], - "postCreateCommand": "cd meltano && meltano install", - "settings": { - "python.defaultInterpreterPath": "/usr/local/bin/python", - "python.linting.ignorePatterns": [ - ".vscode/*.py", - "**/site-packages/**/*.py", - ".meltano" - ], - "files.exclude": { - "**/.git": true, - "**/.svn": true, - "**/.hg": true, - "**/CVS": true, - "**/.DS_Store": true, - "**/__pycache__": true, - ".pytest_cache": true - }, - "python.testing.unittestEnabled": false, - "python.testing.pytestEnabled": true, - "editor.formatOnSave": true, - "python.formatting.provider": "black", - "python.formatting.blackArgs": [ - "--line-length", - "100", - "-S", - "--fast", - ], - "python.linting.enabled": true, - "python.linting.pylintEnabled": true, - "python.linting.lintOnSave": true, - "python.sortImports.args": [ - "--profile", - "black" - ], - "[python]": { - "editor.codeActionsOnSave": { - "source.organizeImports": true - } - } - } -} \ No newline at end of file +{ + "name": "dagster-meltano", + "image": "ghcr.io/quantile-development/vscode-python-base:3.8", + "features": { + "ghcr.io/devcontainers-contrib/features/black:1": {}, + "ghcr.io/devcontainers-contrib/features/isort:1": {}, + "ghcr.io/devcontainers-contrib/features/meltano:1": { + "version": "2.12.0" + }, + "ghcr.io/devcontainers-contrib/features/poetry:1": {} + }, + "extensions": [ + "ms-python.vscode-pylance", + "njpwerner.autodocstring", + "zhuangtongfa.material-theme", + "bungcip.better-toml", + "redhat.vscode-yaml", + "waderyan.gitblame", + "esbenp.prettier-vscode" + ], + "postCreateCommand": "poetry config virtualenvs.in-project true && poetry install && git config --global --add safe.directory /workspaces/dagster-meltano", + "remoteUser": "root" +} diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index aed6515..38a932f 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -1,27 +1,37 @@ -name: test -on: - pull_request: - paths: - - 'dagster_meltano/**' - - 'tests/**' - -jobs: - test: - runs-on: ubuntu-latest - timeout-minutes: 10 - - steps: - - name: Check out repository code - uses: actions/checkout@v2 - - - name: Setup Python - uses: actions/setup-python@v2 - with: - python-version: "3.9" - - - name: Install dependencies - run: | - pip install ".[development]" - - name: Run test suite - run: | - pytest \ No newline at end of file +name: test +on: + pull_request: + +jobs: + test: + runs-on: ubuntu-latest + timeout-minutes: 10 + + steps: + - name: Check out repository code + uses: actions/checkout@v2 + + - name: Setup Python + uses: actions/setup-python@v4.3.0 + with: + python-version: 3.8 + architecture: x64 + cache: "pip" + cache-dependency-path: "poetry.lock" + + - name: Install Poetry + run: pipx install poetry + + - name: Install Dependencies + run: poetry install + + - name: Install Meltano + run: pipx install meltano==2.12.0 + + - name: Install Meltano plugins + run: | + cd tests/meltano_test_project + meltano install + + - name: Run test suite + run: poetry run pytest diff --git a/.gitignore b/.gitignore index fbc7d91..07f0698 100644 --- a/.gitignore +++ b/.gitignore @@ -3,6 +3,6 @@ dist/ .idea .env -.meltano -.vscode +**/.meltano/ +build/ meltano/load/sample.jsonl \ No newline at end of file diff --git a/.pylintrc b/.pylintrc index 991797a..fa860fa 100644 --- a/.pylintrc +++ b/.pylintrc @@ -1,32 +1,33 @@ -[MESSAGES CONTROL] - -# Enable the message, report, category or checker with the given id(s). You can -# either give multiple identifier separated by comma (,) or put this option -# multiple time. -#enable= - -# Disable the message, report, category or checker with the given id(s). You -# can either give multiple identifier separated by comma (,) or put this option -# multiple time (only on the command line, not in the configuration file where -# it should appear only once).# -# -# R - refactoring related checks -# C - convention related checks -# W0511 disable TODO warning -# W1201, W1202 disable log format warning. False positives (I think) -# W0707 disable raise-missing-from which we cant use because py2 back compat - -disable=C,R,duplicate-code,W0511,W1201,W1202,W0707,no-init - -# See: https://github.com/getsentry/responses/issues/74 -[TYPECHECK] -ignored-classes=responses -signature-mutators=solid,composite_solid,lambda_solid,configured,op,graph - -[MASTER] -ignore=snapshots -load-plugins=dagster.utils.linter - -# See: https://stackoverflow.com/questions/40163106/cannot-find-col-function-in-pyspark -generated-members=pyspark.* -ignored-modules=pyspark.sql.functions +[MESSAGES CONTROL] + +# Enable the message, report, category or checker with the given id(s). You can +# either give multiple identifier separated by comma (,) or put this option +# multiple time. +#enable= + +# Disable the message, report, category or checker with the given id(s). You +# can either give multiple identifier separated by comma (,) or put this option +# multiple time (only on the command line, not in the configuration file where +# it should appear only once).# +# +# R - refactoring related checks +# C - convention related checks +# W0511 disable TODO warning +# W1201, W1202 disable log format warning. False positives (I think) +# W0707 disable raise-missing-from which we cant use because py2 back compat + +disable=C,R,duplicate-code,W0511,W1201,W1202,W0707,no-init + +# See: https://github.com/getsentry/responses/issues/74 +[TYPECHECK] +ignored-classes=responses +signature-mutators=solid,composite_solid,lambda_solid,configured,op,graph + +[MASTER] +ignore=snapshots +load-plugins=dagster.utils.linter +init-hook='import sys; sys.path.append("/workspaces/dagster-meltano")' + +# See: https://stackoverflow.com/questions/40163106/cannot-find-col-function-in-pyspark +generated-members=pyspark.* +ignored-modules=pyspark.sql.functions diff --git a/.vscode/settings.json b/.vscode/settings.json new file mode 100644 index 0000000..ce4e37b --- /dev/null +++ b/.vscode/settings.json @@ -0,0 +1,17 @@ +{ + "python.linting.pylintPath": "/workspaces/dagster-meltano/.venv/bin/pylint", + "files.exclude": { + "**/.git": true, + "**/.svn": true, + "**/.hg": true, + "**/CVS": true, + "**/.DS_Store": true, + "**/Thumbs.db": true, + "**/.venv": true, + "**/__pycache__": true, + "**/.pytest_cache": true + }, + "python.testing.pytestArgs": ["tests"], + "python.testing.unittestEnabled": false, + "python.testing.pytestEnabled": true +} diff --git a/.vscode/tasks.json b/.vscode/tasks.json new file mode 100644 index 0000000..f616c52 --- /dev/null +++ b/.vscode/tasks.json @@ -0,0 +1,42 @@ +{ + "version": "2.0.0", + "tasks": [ + { + "label": "meltano-install", + "type": "shell", + "command": "meltano", + "args": [ + "install" + ], + "options": { + "cwd": "${workspaceFolder}/meltano" + }, + "isBackground": true, + "presentation": { + "reveal": "always", + "panel": "shared", + "close": true + }, + "problemMatcher": [] + }, + { + "label": "dagit-up", + "type": "shell", + "command": "meltano", + "args": [ + "run", + "dagit:up" + ], + "options": { + "cwd": "${workspaceFolder}/meltano" + }, + "isBackground": true, + "presentation": { + "reveal": "always", + "panel": "shared", + "close": true + }, + "problemMatcher": [] + } + ] + } \ No newline at end of file diff --git a/Dockerfile b/Dockerfile deleted file mode 100644 index 37a559f..0000000 --- a/Dockerfile +++ /dev/null @@ -1,7 +0,0 @@ -FROM mcr.microsoft.com/vscode/devcontainers/python:3.9 - -ENV MELTANO_PROJECT_ROOT=/workspaces/dagster-meltano/meltano - -COPY . /workspaces/dagster-meltano - -RUN pip install -e /workspaces/dagster-meltano[development] \ No newline at end of file diff --git a/README.md b/README.md index cf20aed..11055da 100644 --- a/README.md +++ b/README.md @@ -1,64 +1,49 @@ -# Dagster-meltano (Archived) - -## This repository is currently outdated. If you want to use Meltano inside Dagster, check out the [dagster-ext](https://github.com/quantile-development/dagster-ext) repository. Soon the `dagster_meltano` part of the `dagster-ext` will be extracted and placed in this repository. - -A dagster plugin that allows you to run Meltano pipelines using Dagster. - -[![Downloads](https://pepy.tech/badge/dagster-meltano/month)](https://pepy.tech/project/dagster-meltano) - -## Installation -1. Install using pip `pip install dagster-meltano`. -2. Make sure you have an installed Meltano project. -3. Point the plugin to the root of the Meltano project by defining `MELTANO_PROJECT_ROOT`. - -## Example -An example of a Dagster pipeline that runs a Meltano elt process. - -```python - -from dagster import OutputDefinition, Nothing -from dagster_meltano.tests import pipeline -from dagster_meltano.solids import meltano_elt_solid - - -@pipeline -def meltano_pipeline(): - meltano_elt_solid( - output_defs=[OutputDefinition(dagster_type=Nothing)], - tap='tap-csv', - target='target-jsonl', - job_id='csv-to-jsonl' # Optional - ) -``` - -## Development -### Setup using VSCode -1. Open this repository in Visual Studio Code. -2. Install the [Remote - Containers](https://marketplace.visualstudio.com/items?itemName=ms-vscode-remote.remote-containers) plugin for Visual Studio Code. -3. Wait for the container setup, it should automatically install all Meltano plugins. -4. Open the integrated terminal and start Dagit `dagit -f dagster_meltano/tests/pipeline.py` -4. Visit `localhost:3000` to access Dagit. - -### Setup using other IDEs -1. Create a virtual environment -2. Pip install dependencies: `pip install dagster meltano` -3. Install Meltano plugins: `cd meltano && meltano install && cd ..` -4. Set env vars: `export MELTANO_PROJECT_ROOT=` -5. Run dagit: `dagit -f dagster_meltano/tests/pipeline.py` - -## Testing and Linting -We use [Dagster's default setup](https://docs.dagster.io/community/contributing#developing-dagster) -for testing and linting. - -### Linting -Specifically linting can be accomplished by installing the appropriate linters: - -```shell -pip install black pylint isort -``` - -And then running them against the codebase: - -```shell -black dagster_meltano/ && isort dagster_meltano/ && pylint dagster_meltano/ -``` +# Dagster-meltano + +A dagster plugin that allows you to run Meltano using Dagster. + +[![Downloads](https://pepy.tech/badge/dagster-meltano/month)](https://pepy.tech/project/dagster-meltano) + +## Installation + +You can install using `pip install dagster-meltano`. + +## Examples + +An example of automatically loading all jobs and schedules from your Meltano project. + +```python +from dagster import repository +from dagster_meltano import load_jobs_from_meltano_project + +meltano_jobs = load_jobs_from_meltano_project("") + +@repository +def repository(): + return [meltano_jobs] +``` + +An example of running a abitrary `meltano run` command. + +```python +from dagster import repository, job +from dagster_meltano import meltano_resource, meltano_run_op + +@job(resource_defs={"meltano": meltano_resource}) +def meltano_run_job(): + tap_done = meltano_run_op("tap-1 target-1")() + meltano_run_op("tap-2 target-2")(tap_done) + +@repository() +def repository(): + return [meltano_run_job] +``` + +## Development using VSCode + +1. Open this repository in Visual Studio Code. +2. Install the [Remote - Containers](https://marketplace.visualstudio.com/items?itemName=ms-vscode-remote.remote-containers) plugin for Visual Studio Code. +3. Go to the example Meltano project root `cd meltano_project` +4. Install all plugins `meltano install` +5. Start dagit `meltano invoke dagster:start` +6. Visit `localhost:3000` to access Dagit. diff --git a/dagster_meltano/__init__.py b/dagster_meltano/__init__.py index e69de29..cd45475 100644 --- a/dagster_meltano/__init__.py +++ b/dagster_meltano/__init__.py @@ -0,0 +1,8 @@ +from dagster import resource + +from dagster_meltano.generation import ( + load_assets_from_meltano_project, + load_jobs_from_meltano_project, +) +from dagster_meltano.meltano_resource import MeltanoResource, meltano_resource +from dagster_meltano.ops import meltano_install_op, meltano_run_op diff --git a/dagster_meltano/assets.py b/dagster_meltano/assets.py new file mode 100644 index 0000000..dd5717e --- /dev/null +++ b/dagster_meltano/assets.py @@ -0,0 +1,40 @@ +from typing import List, Optional + +from dagster import AssetsDefinition +from dagster_dbt import load_assets_from_dbt_project + +# from dagster_meltano.meltano_resource import MeltanoResource +# from .utils import generate_dbt_group_name + + +def load_assets_from_meltano_project( + meltano_project_dir: str, + dbt_project_dir: Optional[str] = None, + dbt_profiles_dir: Optional[str] = None, + dbt_target_dir: Optional[str] = None, + dbt_use_build_command: bool = True, +) -> List[AssetsDefinition]: + """This function generates all Assets it can find in the supplied Meltano project. + This currently includes the taps and dbt assets. + + Args: + project_dir (Optional[str], optional): The location of the Meltano project. Defaults to os.getenv("MELTANO_PROJECT_ROOT"). + + Returns: + List[AssetsDefinition]: Returns a list of all Meltano assets + """ + return [] + meltano_resource = MeltanoResource(meltano_project_dir) + meltano_assets = [extractor.asset for extractor in meltano_resource.extractors] + + if dbt_project_dir: + dbt_assets = load_assets_from_dbt_project( + project_dir=dbt_project_dir, + profiles_dir=dbt_profiles_dir, + target_dir=dbt_target_dir, + use_build_command=dbt_use_build_command, + node_info_to_group_fn=generate_dbt_group_name, + ) + meltano_assets += dbt_assets + + return meltano_assets diff --git a/dagster_meltano/generation.py b/dagster_meltano/generation.py new file mode 100644 index 0000000..5229fa9 --- /dev/null +++ b/dagster_meltano/generation.py @@ -0,0 +1,64 @@ +import json +import logging +import subprocess +from typing import List, Optional, Union + +from dagster import AssetsDefinition, JobDefinition, ScheduleDefinition + +from dagster_meltano.meltano_resource import MeltanoResource +from dagster_meltano.utils import generate_dbt_group_name + + +def load_jobs_from_meltano_project( + meltano_project_dir: Optional[str], +) -> List[Union[JobDefinition, ScheduleDefinition]]: + """This function generates dagster jobs for all jobs defined in the Meltano project. If there are schedules connected + to the jobs, it also returns those. + + Args: + project_dir (Optional[str], optional): The location of the Meltano project. Defaults to os.getenv("MELTANO_PROJECT_ROOT"). + + Returns: + List[Union[JobDefinition, ScheduleDefinition]]: Returns a list of either Dagster JobDefinitions or ScheduleDefinitions + """ + meltano_resource = MeltanoResource( + project_dir=meltano_project_dir, + meltano_bin="meltano", + ) + + meltano_jobs = meltano_resource.jobs + + return list(meltano_jobs) + + +def load_assets_from_meltano_project( + meltano_project_dir: str, +) -> List[AssetsDefinition]: + """This function generates all Assets it can find in the supplied Meltano project. + This currently includes the taps and dbt assets. + + Args: + project_dir (Optional[str], optional): The location of the Meltano project. Defaults to os.getenv("MELTANO_PROJECT_ROOT"). + + Returns: + List[AssetsDefinition]: Returns a list of all Meltano assets + """ + # meltano_resource = MeltanoResource(meltano_project_dir) + meltano_assets = [] + # meltano_assets = [extractor.asset for extractor in meltano_resource.extractors] + + # if dbt_project_dir: + # dbt_assets = load_assets_from_dbt_project( + # project_dir=dbt_project_dir, + # profiles_dir=dbt_profiles_dir, + # target_dir=dbt_target_dir, + # use_build_command=dbt_use_build_command, + # node_info_to_group_fn=generate_dbt_group_name, + # ) + # meltano_assets += dbt_assets + + return meltano_assets + + +if __name__ == "__main__": + load_jobs_from_meltano_project("/workspace/meltano") diff --git a/dagster_meltano/job.py b/dagster_meltano/job.py new file mode 100644 index 0000000..90281b1 --- /dev/null +++ b/dagster_meltano/job.py @@ -0,0 +1,97 @@ +import asyncio +import json +import logging +from functools import lru_cache + +from dagster import ( + In, + JobDefinition, + Nothing, + OpDefinition, + OpExecutionContext, + get_dagster_logger, + job, + op, +) + +from dagster_meltano.meltano_invoker import MeltanoInvoker +from dagster_meltano.ops import meltano_run_op as meltano_run_op_factory +from dagster_meltano.utils import generate_dagster_name + + +class Job: + def __init__( + self, + meltano_job: dict, + meltano_invoker: MeltanoInvoker, + ) -> None: + self.name = meltano_job["job_name"] + self.tasks = meltano_job["tasks"] + self.meltano_invoker = meltano_invoker + + @property + def dagster_name(self) -> str: + return generate_dagster_name(self.name) + + def task_contains_tap(self, task: str) -> bool: + """Check whether the supplied task contains a tap.""" + return "tap-" in task + + @property + def dagster_job(self) -> JobDefinition: + # We need to import the `meltano_resource` here to prevent circular imports. + from dagster_meltano.meltano_resource import meltano_resource + + @job( + name=self.dagster_name, + description=f"Runs the `{self.name}` job from Meltano.", + resource_defs={"meltano": meltano_resource}, + ) + def dagster_job(): + op_layers = [[], []] + previous_task_contains_tap = None + for task in self.tasks: + meltano_run_op = meltano_run_op_factory(task) + current_task_contains_tap = self.task_contains_tap(task) + + # # If the task does not contain a tap, we generate a new layer + # if not self.task_contains_tap(task): + # op_layers.append([]) + + if not current_task_contains_tap or not previous_task_contains_tap: + op_layers.append([]) + + # if not current_task_contains_tap and previous_task_contains_tap: + + # op_layers.append([]) + + # # When we are in the first layer + # if len(op_layers) == 1: + # op_layers[-1].append(meltano_run_op()) + # continue + + op_layers[-1].append(meltano_run_op(op_layers[-2])) + previous_task_contains_tap = current_task_contains_tap + + # logging.warning(op_layers) + + # for op_layer_index, op_layer in enumerate(op_layers)[1:]: + # previous_layer = op_layers[op_layer_index - 1] + + # for + # [meltano_run_op(previous_layer) for meltano_run_op in op_layer] + + # meltano_run_op() + + # meltano_run = meltano_run_op(command=task) + + # if previous_done: + # previous_done = meltano_run(previous_done) + # else: + # previous_done = meltano_run() + + # if self.task_contains_tap(task): + # previous_done.append(previous_done) + # continue + + return dagster_job diff --git a/dagster_meltano/log_processing.py b/dagster_meltano/log_processing.py new file mode 100644 index 0000000..2c33f4e --- /dev/null +++ b/dagster_meltano/log_processing.py @@ -0,0 +1,10 @@ + + + + + + + + + + diff --git a/dagster_meltano/log_processing/__init__.py b/dagster_meltano/log_processing/__init__.py new file mode 100644 index 0000000..5a4c2e1 --- /dev/null +++ b/dagster_meltano/log_processing/__init__.py @@ -0,0 +1,50 @@ +import asyncio +from abc import ABC, abstractmethod +from typing import Any, Optional + +from dagster import get_dagster_logger + + +class LogProcessor(ABC): + """ + A generic log processor. + """ + + def __init__(self, reader: asyncio.streams.StreamReader, log_type: str) -> dict: + self.reader = reader + self.log_type = log_type + self.dagster_logger = get_dagster_logger() + + async def process_logs(self): + """ + A method that starts processing the logs. + """ + while True: + # If there are no more logs to process + if self.reader.at_eof(): + break + + log_line = await self.reader.readline() + log_line = log_line.decode("utf-8").rstrip() + self.process_line(log_line) + + await asyncio.sleep(0) + + return self.results + + @abstractmethod + def process_line(self, log_line: str): + """Process a single log line. + + Args: + log_line (str): A single log line. + """ + ... + + def results(self) -> Optional[Any]: + """Return optional results from the log processor. + + Returns: + Optional[Any]: _description_ + """ + return None diff --git a/dagster_meltano/log_processing/json_processor.py b/dagster_meltano/log_processing/json_processor.py new file mode 100644 index 0000000..c053f06 --- /dev/null +++ b/dagster_meltano/log_processing/json_processor.py @@ -0,0 +1,22 @@ +import json +from typing import Any, Optional + +from dagster_meltano.log_processing import LogProcessor + + +class JsonLogProcessor(LogProcessor): + """ + A log processor that tries to read JSON from the log lines. + """ + + log_lines = "" + + def process_line(self, log_line: str): + self.log_lines += log_line + + @property + def results(self) -> Optional[Any]: + try: + return json.loads(self.log_lines) + except json.decoder.JSONDecodeError: + raise ValueError(f"Could not process json: {self.log_lines}") diff --git a/dagster_meltano/log_processing/metadata_processor.py b/dagster_meltano/log_processing/metadata_processor.py new file mode 100644 index 0000000..7563b1e --- /dev/null +++ b/dagster_meltano/log_processing/metadata_processor.py @@ -0,0 +1,80 @@ +import asyncio +import json +from abc import ABC, abstractmethod +from dataclasses import dataclass +from json import JSONDecoder +from typing import Any, Iterator, List, Optional + +from dagster_meltano.log_processing import LogProcessor + + +def extract_json_objects(text, decoder=JSONDecoder()): + """Find JSON objects in text, and yield the decoded JSON data + Does not attempt to look for JSON arrays, text, or other JSON types outside + of a parent JSON object. + """ + pos = 0 + while True: + match = text.find('{', pos) + if match == -1: + break + try: + result, index = decoder.raw_decode(text[match:]) + yield result + pos = match + index + except ValueError: + pos = match + 1 + + +class Metric: + def __init__(self, body: dict) -> None: + self.body = body + + @property + def is_record_count(self) -> bool: + return self.body.get("metric") == "record_count" + + @property + def value(self) -> int: + return self.body.get("value") + + @property + def tags(self) -> dict: + return self.body.get("tags") + + @property + def stream_name(self) -> str: + return self.tags.get("stream") or self.tags.get("endpoint") + + +class MetadataLogProcessor(LogProcessor): + metrics: List[Metric] = [] + + def find_metrics(self, log_line: str) -> Iterator[Metric]: + for json_object in extract_json_objects(log_line): + yield Metric(json_object) + + def process_line(self, log_line: str): + if not log_line: + return + + log_function = self.dagster_logger.info + + try: + log_line = json.loads(log_line) + except json.decoder.JSONDecodeError: + log_function(log_line) + return + + event = log_line.get("event", log_line) + + if log_line.get("level") == "debug": + log_function = self.dagster_logger.debug + + log_function(event) + + self.metrics.extend(self.find_metrics(event)) + + @property + def results(self) -> Optional[Any]: + return self.metrics diff --git a/dagster_meltano/log_processing/passthrough_processor.py b/dagster_meltano/log_processing/passthrough_processor.py new file mode 100644 index 0000000..34014fd --- /dev/null +++ b/dagster_meltano/log_processing/passthrough_processor.py @@ -0,0 +1,15 @@ +from dagster_meltano.log_processing import LogProcessor + + +class PassthroughLogProcessor(LogProcessor): + """ + A log processor that passes all log lines to the Dagster logger. + """ + + def process_line(self, log_line: str): + """Pass the log line to Dagster. + + Args: + log_line (str): The log line to pass. + """ + self.dagster_logger.info(log_line) diff --git a/dagster_meltano/logging.yaml b/dagster_meltano/logging.yaml new file mode 100644 index 0000000..bddc856 --- /dev/null +++ b/dagster_meltano/logging.yaml @@ -0,0 +1,29 @@ +version: 1 +disable_existing_loggers: false + +formatters: + default: + format: "%(message)s" + structured_plain: + (): meltano.core.logging.console_log_formatter + colors: False + structured_colored: + (): meltano.core.logging.console_log_formatter + colors: True + key_value: + (): meltano.core.logging.key_value_formatter + sort_keys: False + json: + (): meltano.core.logging.json_formatter + +handlers: + console: + class: logging.StreamHandler + level: INFO + formatter: json + stream: "ext://sys.stderr" + +root: + level: INFO + propagate: yes + handlers: [console] diff --git a/dagster_meltano/meltano_elt.py b/dagster_meltano/meltano_elt.py deleted file mode 100644 index 23fbf3b..0000000 --- a/dagster_meltano/meltano_elt.py +++ /dev/null @@ -1,94 +0,0 @@ -"""Class for Meltano ELT command""" - -import os -from subprocess import PIPE, STDOUT, Popen -from typing import Generator, List, Optional - - -class MeltanoELT: - """Control `meltano elt` command.""" - - def __init__( - self, - tap: str, - target: str, - job_id: str, - full_refresh: bool, - env_vars: Optional[dict] = None, - ) -> None: - """Initialize a new Meltano ELT process. - - Args: - tap (str): The name of the Meltano tap. - target (str): The name of the Meltano target. - job_id (str): The id of the job. - full_refresh (bool): Whether to ignore existing state. - env_vars (Optional[dict]): Additional environment variables to pass to the - command context. - """ - if env_vars is None: - env_vars = {} - - self._tap = tap - self._target = target - self._job_id = job_id - self._full_refresh = full_refresh - self._elt_process = None - self._env_vars = env_vars - - @property - def elt_command(self) -> List[str]: - """Constructs all the parts of the ELT command. - - Returns: - List[str]: All parts of the ELT command. - """ - # All default parts of the command - elt_command = ["meltano", "elt", self._tap, self._target, "--job_id", self._job_id] - - # If the user specified a full refresh - if self._full_refresh: - elt_command += ["--full-refresh"] - - return elt_command - - @property - def elt_process(self) -> Popen: - """Creates a subprocess that runs the Meltano ELT command. - It is started in the Meltano project root, and inherits environment. - variables from the Dagster environment. - - It injects tap and target configuration by utilizing environment variables. - - Returns: - Popen: The ELT process. - """ - - # Create a Meltano ELT process if it does not already exists - if not self._elt_process: - self._elt_process = Popen( - self.elt_command, - stdout=PIPE, - stderr=STDOUT, - cwd=os.getenv( - "MELTANO_PROJECT_ROOT" - ), # Start the command in the root of the Meltano project - env={ - **os.environ, # Pass all environment variables from the Dagster environment - **self._env_vars, - }, - start_new_session=True, - ) - - return self._elt_process - - @property - def logs(self) -> Generator[str, None, None]: - """A generator that loops through the stdout and stderr (both routed to stdout). - - Yields: - Generator[str, None, None]: The lines the ELT command produces. - """ - # Loop through the stdout of the ELT process - for line in iter(self.elt_process.stdout.readline, b""): - yield line.decode("utf-8").rstrip() diff --git a/dagster_meltano/meltano_invoker.py b/dagster_meltano/meltano_invoker.py new file mode 100644 index 0000000..ca0d50e --- /dev/null +++ b/dagster_meltano/meltano_invoker.py @@ -0,0 +1,154 @@ +import asyncio +import json +import os +import subprocess +from asyncio.subprocess import Process +from pathlib import Path +from typing import IO, Any, Callable, Dict, List, Optional, Tuple, Union + +from dagster import get_dagster_logger + +from dagster_meltano.log_processing import LogProcessor +from dagster_meltano.log_processing.passthrough_processor import PassthroughLogProcessor + +# log = structlog.get_logger() +log = get_dagster_logger() + + +class MeltanoInvoker: + """Invoker utility class for invoking subprocesses.""" + + def __init__( + self, + bin: str = "meltano", + cwd: str = None, + log_level: str = "info", + env: Optional[Dict[str, Any]] = {}, + ) -> None: + """Minimal invoker for running subprocesses. + + Args: + bin: The path/name of the binary to run. + cwd: The working directory to run from. + env: Env to use when calling Popen, defaults to current os.environ if None. + """ + self.bin = bin + self.cwd = cwd + self.env = { + **os.environ.copy(), + "MELTANO_CLI_LOG_CONFIG": Path(__file__).parent / "logging.yaml", + "MELTANO_CLI_LOG_LEVEL": log_level, + "DBT_USE_COLORS": "false", + "NO_COLOR": "1", + **env, + } + + def run( + self, + *args: Union[str, bytes], + stdout: Union[None, int, IO] = subprocess.PIPE, + stderr: Union[None, int, IO] = subprocess.STDOUT, + text: bool = True, + **kwargs: Any, + ) -> subprocess.Popen: + """Run a subprocess. Simple wrapper around subprocess.run. + + Note that output from stdout and stderr is NOT logged automatically. Especially + useful when you want to run a command, but don't care about its output and only + care about its return code. + + stdout and stderr by default are set up to use subprocess.PIPE. If you do not + want to capture io from the subprocess use subprocess.DEVNULL to discard it. + + The Invoker's at env and cwd are used when calling `subprocess.run`. If you want + to override these you're likely better served using `subprocess.run` directly. + + Lastly note that this method is blocking AND `subprocess.run` is called with + `check=True`. This means that if the subprocess fails a `CalledProcessError` + will be raised. + + Args: + *args: The arguments to pass to the subprocess. + stdout: The stdout stream to use. + stderr: The stderr stream to use. + text: If true, decode stdin, stdout and stderr using the system default. + **kwargs: Additional keyword arguments to pass to subprocess.run. + + Returns: + The completed process. + """ + return subprocess.Popen( + [self.bin, *args], + cwd=self.cwd, + env=self.env, + stdout=stdout, + stderr=stderr, + text=text, + **kwargs, + ) + + async def exec( + self, + sub_command: Union[str, None] = None, + log_processor: Optional[LogProcessor] = PassthroughLogProcessor, + *args: Union[str, bytes], + ) -> Tuple[asyncio.subprocess.Process, List[any]]: + popen_args = [] + if sub_command: + popen_args.append(sub_command) + if args: + popen_args.extend(*args) + + process = await asyncio.create_subprocess_exec( + self.bin, + *popen_args, + stdout=asyncio.subprocess.PIPE, + stderr=asyncio.subprocess.PIPE, + cwd=self.cwd, + env=self.env, + ) + + log_results = await asyncio.gather( + asyncio.create_task( + log_processor(process.stderr, log_type="stderr").process_logs() + ), + asyncio.create_task( + log_processor(process.stdout, log_type="stdout").process_logs() + ), + return_exceptions=True, + ) + + # Raise first exception if any + for log_result in log_results: + if isinstance(log_result, Exception): + raise log_result + + await process.wait() + return process, log_results + + def run_and_log( + self, + sub_command: Union[str, None] = None, + log_processor: Optional[LogProcessor] = None, + *args: Union[str, bytes], + ) -> Tuple[Any, Any]: + """Run a subprocess and stream the output to the logger. + + Note that output from stdout and stderr IS logged. Best used when you want + to run a command and stream the output to a user. + + Args: + sub_command: The subcommand to run. + log_processor: Gets called for each log line that is being processed. + *args: The arguments to pass to the subprocess. + + Raises: + CalledProcessError: If the subprocess failed. + """ + process, log_results = asyncio.run(self.exec(sub_command, log_processor, *args)) + if process.returncode: + raise subprocess.CalledProcessError( + process.returncode, cmd=self.bin, stderr=None + ) + + return log_results diff --git a/dagster_meltano/meltano_resource.py b/dagster_meltano/meltano_resource.py new file mode 100644 index 0000000..f255c3e --- /dev/null +++ b/dagster_meltano/meltano_resource.py @@ -0,0 +1,105 @@ +import asyncio +import os +from functools import lru_cache +from typing import Dict, List, Optional + +from dagster import resource, Field + +from dagster_meltano.job import Job +from dagster_meltano.log_processing.json_processor import JsonLogProcessor +from dagster_meltano.meltano_invoker import MeltanoInvoker +from dagster_meltano.schedule import Schedule +from dagster_meltano.utils import Singleton + +STDOUT = 1 + + +class MeltanoResource(metaclass=Singleton): + def __init__( + self, + project_dir: str = None, + meltano_bin: Optional[str] = "meltano", + ): + self.project_dir = project_dir + self.meltano_bin = meltano_bin + self.meltano_invoker = MeltanoInvoker( + bin=meltano_bin, + cwd=project_dir, + log_level="info", # TODO: Get this from the resource config + ) + + async def load_json_from_cli(self, command: List[str]) -> dict: + _, log_results = await self.meltano_invoker.exec( + None, + JsonLogProcessor, + command, + ) + return log_results[STDOUT] + + async def gather_meltano_yaml_information(self): + jobs, schedules = await asyncio.gather( + self.load_json_from_cli(["job", "list", "--format=json"]), + self.load_json_from_cli(["schedule", "list", "--format=json"]), + ) + + return jobs, schedules + + @property + @lru_cache + def meltano_yaml(self): + jobs, schedules = asyncio.run(self.gather_meltano_yaml_information()) + return {"jobs": jobs["jobs"], "schedules": schedules["schedules"]} + + @property + @lru_cache + def meltano_jobs(self) -> List[Job]: + meltano_job_list = self.meltano_yaml["jobs"] + return [ + Job( + meltano_job, + self.meltano_invoker, + ) + for meltano_job in meltano_job_list + ] + + @property + @lru_cache + def meltano_schedules(self) -> List[Schedule]: + meltano_schedule_list = self.meltano_yaml["schedules"]["job"] + schedule_list = [ + Schedule(meltano_schedule) for meltano_schedule in meltano_schedule_list + ] + return schedule_list + + @property + def meltano_job_schedules(self) -> Dict[str, Schedule]: + return {schedule.job_name: schedule for schedule in self.meltano_schedules} + + @property + def jobs(self) -> List[dict]: + for meltano_job in self.meltano_jobs: + yield meltano_job.dagster_job + + for meltano_schedule in self.meltano_schedules: + yield meltano_schedule.dagster_schedule + + +@resource( + description="A resource that corresponds to a Meltano project.", + config_schema={ + "project_dir": Field( + str, + default_value=os.getenv("MELTANO_PROJECT_ROOT", os.getcwd()), + is_required=False, + ) + }, +) +def meltano_resource(init_context): + project_dir = init_context.resource_config["project_dir"] + return MeltanoResource(project_dir) + + +if __name__ == "__main__": + meltano_resource = MeltanoResource("/workspace/meltano") + print(list(meltano_resource.jobs)) + print(meltano_resource.jobs) diff --git a/dagster_meltano/ops.py b/dagster_meltano/ops.py new file mode 100644 index 0000000..48b7c80 --- /dev/null +++ b/dagster_meltano/ops.py @@ -0,0 +1,79 @@ +from __future__ import annotations + +from functools import lru_cache +from typing import TYPE_CHECKING + +from dagster import ( + In, + Nothing, + OpDefinition, + OpExecutionContext, + get_dagster_logger, + op, +) + +from dagster_meltano.log_processing.metadata_processor import MetadataLogProcessor +from dagster_meltano.utils import generate_dagster_name + +if TYPE_CHECKING: + from dagster_meltano.meltano_resource import MeltanoResource + +dagster_logger = get_dagster_logger() + +STDOUT = 1 + + +@lru_cache +def meltano_run_op(command: str) -> OpDefinition: + """ + Run `meltano run ` using a Dagster op. + + This factory is cached to make sure the same commands can be reused in the + same repository. + """ + dagster_name = generate_dagster_name(command) + + @op( + name=dagster_name, + description=f"Run `{command}` using Meltano.", + ins={"after": In(Nothing)}, + tags={"kind": "meltano"}, + required_resource_keys={"meltano"}, + ) + def dagster_op(context: OpExecutionContext): + meltano_resource: MeltanoResource = context.resources.meltano + log_results = meltano_resource.meltano_invoker.run_and_log( + "run", + MetadataLogProcessor, + command.split(), + ) + # dagster_logger.info(log_results[STDOUT]) + + # yield AssetMaterialization( + # asset_key="my_dataset", + # metadata={ + # "my_text_label": "hello", + # "dashboard_url": MetadataValue.url("http://mycoolsite.com/my_dashboard"), + # "num_rows": 0, + # }, + # ) + + return dagster_op + + +@op( + name=generate_dagster_name("meltano install"), + description="Install all Meltano plugins", + ins={"after": In(Nothing)}, + tags={"kind": "meltano"}, + required_resource_keys={"meltano"}, +) +def meltano_install_op(context: OpExecutionContext): + """ + Run `meltano install` using a Dagster op. + """ + meltano_resource: MeltanoResource = context.resources.meltano + meltano_resource.meltano_invoker.run_and_log( + "install", + MetadataLogProcessor, + ) diff --git a/dagster_meltano/schedule.py b/dagster_meltano/schedule.py new file mode 100644 index 0000000..67e485e --- /dev/null +++ b/dagster_meltano/schedule.py @@ -0,0 +1,27 @@ +from dagster import DefaultScheduleStatus, ScheduleDefinition + +from dagster_meltano.utils import generate_dagster_name + + +class Schedule: + def __init__(self, meltano_schedule: dict) -> None: + self.name = meltano_schedule["name"] + self.cron_interval = meltano_schedule["cron_interval"] + self.job_name = meltano_schedule["job"]["name"] + + @property + def dagster_name(self) -> str: + return generate_dagster_name(self.name) + + @property + def dagster_job_name(self) -> str: + return generate_dagster_name(self.job_name) + + @property + def dagster_schedule(self) -> ScheduleDefinition: + return ScheduleDefinition( + name=self.dagster_name, + job_name=self.dagster_job_name, + cron_schedule=self.cron_interval, + default_status=DefaultScheduleStatus.RUNNING, + ) diff --git a/dagster_meltano/solids.py b/dagster_meltano/solids.py deleted file mode 100644 index c4edd7e..0000000 --- a/dagster_meltano/solids.py +++ /dev/null @@ -1,140 +0,0 @@ -"""Composable solids for configuring Meltano commands in pipeline.""" - -from types import FunctionType -from typing import Generator, List - -from dagster import ( - AssetMaterialization, - Field, - InputDefinition, - Optional, - OutputDefinition, - SolidDefinition, - SolidExecutionContext, - check, -) - -from dagster_meltano.meltano_elt import MeltanoELT - - -def run_elt( - name: str, tap: str, target: str, job_id: str, env_vars: Optional[dict] -) -> FunctionType: - """Run `meltano elt` command yielding asset materialization and producing logs.""" - check.str_param(name, "name") - check.str_param(tap, "tap") - check.str_param(target, "target") - check.str_param(job_id, "job_id") - check.dict_param(env_vars, "env_vars", key_type=str, value_type=str) - - def command( - step_context: SolidExecutionContext, inputs # pylint: disable=W0613 - ) -> Generator[AssetMaterialization, None, None]: - check.inst_param(step_context, "step_context", SolidExecutionContext) - check.param_invariant( - isinstance(step_context.run_config, dict), - "context", - "StepExecutionContext must have valid run_config", - ) - - full_refresh = step_context.solid_config["full_refresh"] - - log = step_context.log - - meltano_elt = MeltanoELT( - tap=tap, - target=target, - job_id=job_id, - full_refresh=full_refresh, - env_vars=env_vars, - ) - - for line in meltano_elt.logs: - log.info(line) - - meltano_elt.elt_process.wait() - - return_code = meltano_elt.elt_process.returncode - - if return_code != 0: - error = f"The meltano elt failed with code {return_code}" - log.error(error) - raise Exception(error) - else: - log.info(f"Meltano exited with return code {return_code}") - - yield AssetMaterialization( - asset_key=name, - metadata={ - "Tap": tap, - "Target": target, - "Job ID": job_id, - "Full Refresh": "True" if full_refresh else "False", - }, - ) - - return command - - -def meltano_elt_solid( - tap: str, - target: str, - input_defs: Optional[List[InputDefinition]] = None, - output_defs: Optional[List[OutputDefinition]] = None, - name: Optional[str] = None, - job_id: Optional[str] = None, - env_vars: Optional[dict] = None, -) -> SolidDefinition: - """Create a solid for a meltano elt process. - - Args: - name (str): The name of the solid. - - Returns: - SolidDefinition: The solid that runs the Meltano ELT process. - """ - if input_defs is None: - input_defs = [] - - if output_defs is None: - output_defs = [] - - check.opt_str_param(name, "name") - check.opt_str_param(job_id, "job_id") - - # If no name is specified, create a name based on the tap and target name - if not name: - name = f"{tap}_{target}" - - # Solid names cannot contain dashes - name = name.replace("-", "_") - - # If no job id is defined, we base it on the tap and target name - if not job_id: - job_id = f"{tap}-{target}" - - # Add a default tag to indicate this is a Meltano solid - default_tags = {"kind": "meltano"} - - return SolidDefinition( - name=name, - input_defs=input_defs, - output_defs=output_defs, - compute_fn=run_elt( - name=name, - tap=tap, - target=target, - job_id=job_id, - env_vars=env_vars, - ), - config_schema={ - "full_refresh": Field( - bool, - default_value=False, - description="Whether to ignore state on this run", - ) - }, - required_resource_keys=set(), - description="", - tags={**default_tags}, - )() diff --git a/dagster_meltano/tests/__init__.py b/dagster_meltano/tests/__init__.py deleted file mode 100644 index e69de29..0000000 diff --git a/dagster_meltano/tests/pipeline.py b/dagster_meltano/tests/pipeline.py deleted file mode 100644 index 8c58477..0000000 --- a/dagster_meltano/tests/pipeline.py +++ /dev/null @@ -1,18 +0,0 @@ -"""Smoke test for testing Meltano commands in dagster pipelines""" - -import json - -from dagster import Nothing, OutputDefinition, pipeline - -from dagster_meltano.solids import meltano_elt_solid - - -@pipeline -def meltano_pipeline(): - meltano_elt_solid( - output_defs=[OutputDefinition(dagster_type=Nothing)], - tap="tap-csv", - target="target-jsonl", - job_id="csv-to-jsonl", - env_vars={"TAP_CSV__SELECT": json.dumps(["sample.id"])}, - ) diff --git a/dagster_meltano/tests/test_meltano_elt.py b/dagster_meltano/tests/test_meltano_elt.py deleted file mode 100644 index 863e931..0000000 --- a/dagster_meltano/tests/test_meltano_elt.py +++ /dev/null @@ -1,63 +0,0 @@ -from dagster_meltano.meltano_elt import MeltanoELT - -# Default parameters supplied to MeltanoELT -meltano_elt_params = { - "tap": "tap-csv", - "target": "target-json", - "job_id": "tap-csv-target-json", - "full_refresh": True, -} - - -def _create_meltano_elt(full_refresh: bool = True) -> MeltanoELT: - """This function creates an instance of the MeltanoELT class. Is used by the testing functions. - - Args: - full_refresh (bool, optional): Whether to create a full refresh command. Defaults to True. - - Returns: - MeltanoELT: The instance of the MeltanoELT class - """ - # Inject the full fresh parameter - meltano_elt_params["full_refresh"] = full_refresh - - # Create the instance - return MeltanoELT(**meltano_elt_params) - - -def test_meltano_elt_construction(): - """ - On successfull creation no errors should be raised. - """ - meltano_elt = _create_meltano_elt() - - # Test if the instance is of the right type - assert isinstance(meltano_elt, MeltanoELT) - - -def test_meltano_elt_command(): - """ - Test if the generated meltano elt command is of the correct format. - """ - meltano_elt = _create_meltano_elt() - - # Test if the command is of the right format - assert meltano_elt.elt_command == [ - "meltano", - "elt", - meltano_elt_params["tap"], - meltano_elt_params["target"], - "--job_id", - meltano_elt_params["job_id"], - "--full-refresh", - ] - - -def test_meltano_elt_command_no_refresh(): - """ - Test whether the '--full-refresh' flag is omitted if no full refresh is requested. - """ - meltano_elt = _create_meltano_elt(full_refresh=False) - - # Make sure that the full refresh flag is missing - assert "--full-refresh" not in meltano_elt.elt_command diff --git a/dagster_meltano/utils.py b/dagster_meltano/utils.py new file mode 100644 index 0000000..2c5edcc --- /dev/null +++ b/dagster_meltano/utils.py @@ -0,0 +1,36 @@ +import json +import subprocess +from typing import List + + +class Singleton(type): + _instances = {} + + def __call__(cls, *args, **kwargs): + if cls not in cls._instances: + cls._instances[cls] = super(Singleton, cls).__call__(*args, **kwargs) + return cls._instances[cls] + + +def generate_dagster_name(input_string) -> str: + """ + Generate a dagster safe name (^[A-Za-z0-9_]+$.) + """ + return input_string.replace("-", "_").replace(" ", "_").replace(":", "_") + + +def generate_dbt_group_name(node_info: dict) -> str: + """Generate the name of the Dagster asset group a DBT nodes lives in. + This will be namespaced with an extra group if the DBT models are nested in + sub-folders. + + Args: + node_info (dict): The information generated for this node by DBT. + + Returns: + str: The name of the asset group. + """ + if len(node_info.get("fqn", [])) >= 3: + return "_".join(node_info["fqn"][1:-1]) + + return "dbt" diff --git a/meltano/extract/sample.csv b/meltano/extract/sample.csv deleted file mode 100644 index 59efecd..0000000 --- a/meltano/extract/sample.csv +++ /dev/null @@ -1,4 +0,0 @@ -id,first_name,last_name -1,peter,veter -2,jan,pan -3,flim,vlam \ No newline at end of file diff --git a/meltano/meltano.yml b/meltano/meltano.yml deleted file mode 100644 index ce68800..0000000 --- a/meltano/meltano.yml +++ /dev/null @@ -1,21 +0,0 @@ -version: 1 -send_anonymous_usage_stats: true -project_id: 315567d7-95e5-463a-aeb1-343fa09e2e5c -plugins: - extractors: - - name: tap-csv - variant: meltanolabs - pip_url: git+https://github.com/MeltanoLabs/tap-csv.git - config: - files: - - entity: "sample" - path: extract/sample.csv - keys: ["id"] - select: - - sample.id - loaders: - - name: target-jsonl - variant: andyh1203 - pip_url: target-jsonl - config: - destination_path: load diff --git a/meltano/.gitignore b/meltano_project/.gitignore similarity index 100% rename from meltano/.gitignore rename to meltano_project/.gitignore diff --git a/meltano/README.md b/meltano_project/README.md similarity index 100% rename from meltano/README.md rename to meltano_project/README.md diff --git a/meltano_project/meltano.yml b/meltano_project/meltano.yml new file mode 100644 index 0000000..0d15d53 --- /dev/null +++ b/meltano_project/meltano.yml @@ -0,0 +1,35 @@ +version: 1 +default_environment: dev +project_id: 1b791a52-034d-4c11-a000-ef0eeed7b573 +send_anonymous_usage_stats: false +plugins: + extractors: + - name: tap-smoke-test + namespace: tap_smoke_test + pip_url: git+https://github.com/meltano/tap-smoke-test.git + config: + streams: + - stream_name: animals + input_filename: https://gitlab.com/meltano/tap-smoke-test/-/raw/main/demo-data/animals-data.jsonl + loaders: + - name: target-jsonl + variant: andyh1203 + pip_url: target-jsonl + env: + MELTANO_LOAD_SCHEMA: TAP_CARBON_INTENSITY + utilities: + - name: dagster + variant: quantile-development + pip_url: git+https://github.com/quantile-development/dagster-ext.git -e ${MELTANO_PROJECT_ROOT}/.. +jobs: + - name: load-jobs + tasks: + - tap-smoke-test target-jsonl +schedules: + - name: daily-load-jobs + interval: "@daily" + job: load-jobs +environments: + - name: dev + - name: staging + - name: prod diff --git a/meltano/extract/.gitkeep b/meltano_project/orchestrate/.gitkeep similarity index 100% rename from meltano/extract/.gitkeep rename to meltano_project/orchestrate/.gitkeep diff --git a/meltano_project/orchestrate/dagster/repository.py b/meltano_project/orchestrate/dagster/repository.py new file mode 100644 index 0000000..c627ebb --- /dev/null +++ b/meltano_project/orchestrate/dagster/repository.py @@ -0,0 +1,13 @@ +import os + +from dagster import repository + +from dagster_meltano import load_jobs_from_meltano_project + +MELTANO_PROJECT_DIR = os.getenv("MELTANO_PROJECT_ROOT", os.getcwd()) +MELTANO_BIN = os.getenv("MELTANO_BIN", "meltano") + + +@repository +def meltano_jobs(): + return [load_jobs_from_meltano_project(MELTANO_PROJECT_DIR)] diff --git a/meltano_project/output/.gitignore b/meltano_project/output/.gitignore new file mode 100644 index 0000000..d6b7ef3 --- /dev/null +++ b/meltano_project/output/.gitignore @@ -0,0 +1,2 @@ +* +!.gitignore diff --git a/meltano_project/plugins/extractors/tap-carbon-intensity--meltano.lock b/meltano_project/plugins/extractors/tap-carbon-intensity--meltano.lock new file mode 100644 index 0000000..e279be9 --- /dev/null +++ b/meltano_project/plugins/extractors/tap-carbon-intensity--meltano.lock @@ -0,0 +1,17 @@ +{ + "plugin_type": "extractors", + "name": "tap-carbon-intensity", + "namespace": "tap_carbon_intensity", + "variant": "meltano", + "label": "Carbon Emissions Intensity", + "docs": "https://hub.meltano.com/extractors/tap-carbon-intensity--meltano", + "repo": "https://gitlab.com/meltano/tap-carbon-intensity", + "pip_url": "git+https://gitlab.com/meltano/tap-carbon-intensity.git", + "description": "National Grid ESO's Carbon Emissions Intensity API", + "logo_url": "https://hub.meltano.com/assets/logos/extractors/carbon-intensity.png", + "capabilities": [ + "discover", + "state", + "catalog" + ] +} \ No newline at end of file diff --git a/meltano_project/plugins/loaders/target-jsonl--andyh1203.lock b/meltano_project/plugins/loaders/target-jsonl--andyh1203.lock new file mode 100644 index 0000000..5825fc4 --- /dev/null +++ b/meltano_project/plugins/loaders/target-jsonl--andyh1203.lock @@ -0,0 +1,34 @@ +{ + "plugin_type": "loaders", + "name": "target-jsonl", + "namespace": "target_jsonl", + "variant": "andyh1203", + "label": "JSON Lines (JSONL)", + "docs": "https://hub.meltano.com/loaders/target-jsonl--andyh1203", + "repo": "https://github.com/andyh1203/target-jsonl", + "pip_url": "target-jsonl", + "description": "JSONL loader", + "logo_url": "https://hub.meltano.com/assets/logos/loaders/jsonl.png", + "settings": [ + { + "name": "destination_path", + "kind": "string", + "value": "output", + "label": "Destination Path", + "description": "Sets the destination path the JSONL files are written to, relative\nto the project root.\n\nThe directory needs to exist already, it will not be created\nautomatically.\n\nTo write JSONL files to the project root, set an empty string (`\"\"`).\n" + }, + { + "name": "do_timestamp_file", + "kind": "boolean", + "value": false, + "label": "Include Timestamp in File Names", + "description": "Specifies if the files should get timestamped.\n\nBy default, the resulting file will not have a timestamp in the file name (i.e. `exchange_rate.jsonl`).\n\nIf this option gets set to `true`, the resulting file will have a timestamp associated with it (i.e. `exchange_rate-{timestamp}.jsonl`).\n" + }, + { + "name": "custom_name", + "kind": "string", + "label": "Custom File Name Override", + "description": "Specifies a custom name for the filename, instead of the stream name.\n\nThe file name will be `{custom_name}-{timestamp}.jsonl`, if `do_timestamp_file` is `true`.\nOtherwise the file name will be `{custom_name}.jsonl`.\n\nIf custom name is not provided, the stream name will be used.\n" + } + ] +} \ No newline at end of file diff --git a/meltano/requirements.txt b/meltano_project/requirements.txt similarity index 100% rename from meltano/requirements.txt rename to meltano_project/requirements.txt diff --git a/poetry.lock b/poetry.lock new file mode 100644 index 0000000..7320587 --- /dev/null +++ b/poetry.lock @@ -0,0 +1,1375 @@ +[[package]] +name = "alembic" +version = "1.9.1" +description = "A database migration tool for SQLAlchemy." +category = "main" +optional = false +python-versions = ">=3.7" + +[package.dependencies] +importlib-metadata = {version = "*", markers = "python_version < \"3.9\""} +importlib-resources = {version = "*", markers = "python_version < \"3.9\""} +Mako = "*" +SQLAlchemy = ">=1.3.0" + +[package.extras] +tz = ["python-dateutil"] + +[[package]] +name = "astroid" +version = "2.12.13" +description = "An abstract syntax tree for Python with inference support." +category = "dev" +optional = false +python-versions = ">=3.7.2" + +[package.dependencies] +lazy-object-proxy = ">=1.4.0" +typing-extensions = {version = ">=3.10", markers = "python_version < \"3.10\""} +wrapt = {version = ">=1.11,<2", markers = "python_version < \"3.11\""} + +[[package]] +name = "attrs" +version = "22.2.0" +description = "Classes Without Boilerplate" +category = "dev" +optional = false +python-versions = ">=3.6" + +[package.extras] +cov = ["attrs[tests]", "coverage-enable-subprocess", "coverage[toml] (>=5.3)"] +dev = ["attrs[docs,tests]"] +docs = ["furo", "myst-parser", "sphinx", "sphinx-notfound-page", "sphinxcontrib-towncrier", "towncrier", "zope.interface"] +tests = ["attrs[tests-no-zope]", "zope.interface"] +tests-no-zope = ["cloudpickle", "cloudpickle", "hypothesis", "hypothesis", "mypy (>=0.971,<0.990)", "mypy (>=0.971,<0.990)", "pympler", "pympler", "pytest (>=4.3.0)", "pytest (>=4.3.0)", "pytest-mypy-plugins", "pytest-mypy-plugins", "pytest-xdist[psutil]", "pytest-xdist[psutil]"] + +[[package]] +name = "certifi" +version = "2022.12.7" +description = "Python package for providing Mozilla's CA Bundle." +category = "main" +optional = false +python-versions = ">=3.6" + +[[package]] +name = "charset-normalizer" +version = "2.1.1" +description = "The Real First Universal Charset Detector. Open, modern and actively maintained alternative to Chardet." +category = "main" +optional = false +python-versions = ">=3.6.0" + +[package.extras] +unicode-backport = ["unicodedata2"] + +[[package]] +name = "click" +version = "8.1.3" +description = "Composable command line interface toolkit" +category = "main" +optional = false +python-versions = ">=3.7" + +[package.dependencies] +colorama = {version = "*", markers = "platform_system == \"Windows\""} + +[[package]] +name = "colorama" +version = "0.4.6" +description = "Cross-platform colored terminal text." +category = "main" +optional = false +python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,!=3.6.*,>=2.7" + +[[package]] +name = "coloredlogs" +version = "14.0" +description = "Colored terminal output for Python's logging module" +category = "main" +optional = false +python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*" + +[package.dependencies] +humanfriendly = ">=7.1" + +[package.extras] +cron = ["capturer (>=2.4)"] + +[[package]] +name = "croniter" +version = "1.3.8" +description = "croniter provides iteration for datetime object with cron like format" +category = "main" +optional = false +python-versions = ">=2.6, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*" + +[package.dependencies] +python-dateutil = "*" + +[[package]] +name = "dagster" +version = "1.1.7" +description = "The data orchestration platform built for productivity." +category = "main" +optional = false +python-versions = "*" + +[package.dependencies] +alembic = ">=1.2.1,<1.6.3 || >1.6.3,<1.7.0 || >1.7.0" +click = ">=5.0" +coloredlogs = ">=6.1,<=14.0" +croniter = ">=0.3.34" +docstring-parser = "*" +grpcio = ">=1.32.0,<1.48.1" +grpcio-health-checking = ">=1.32.0,<1.44.0" +Jinja2 = "*" +packaging = ">=20.9,<22" +pendulum = "*" +protobuf = ">=3.13.0,<4" +psutil = {version = ">=1.0", markers = "platform_system == \"Windows\""} +python-dateutil = "*" +python-dotenv = "*" +pytz = "*" +pywin32 = {version = "!=226", markers = "platform_system == \"Windows\""} +PyYAML = ">=5.1" +requests = "*" +setuptools = "*" +sqlalchemy = ">=1.0" +tabulate = "*" +tomli = "*" +toposort = ">=1.0" +tqdm = "*" +typing-extensions = ">=4.0.1" +universal-pathlib = "*" +watchdog = ">=0.8.3" + +[package.extras] +black = ["black[jupyter] (==22.3.0)"] +docker = ["docker"] +isort = ["isort (==5.10.1)"] +mypy = ["mypy (==0.991)", "types-PyYAML", "types-backports", "types-certifi", "types-chardet", "types-croniter", "types-cryptography", "types-mock", "types-paramiko", "types-pkg-resources", "types-protobuf (<=3.19.21)", "types-pyOpenSSL", "types-python-dateutil", "types-pytz", "types-requests", "types-simplejson", "types-six", "types-tabulate", "types-toml", "types-tzlocal"] +test = ["astroid", "buildkite-test-collector", "docker", "grpcio-tools (>=1.32.0,<1.44.0)", "mock (==3.0.5)", "objgraph", "protobuf (==3.13.0)", "pylint (==2.13.7)", "pytest (==7.0.1)", "pytest-cov (==2.10.1)", "pytest-dependency (==0.5.1)", "pytest-mock (==3.3.1)", "pytest-rerunfailures (==10.0)", "pytest-runner (==5.2)", "pytest-xdist (==2.1.0)", "responses (>=0.10.0,<0.11.0)", "snapshottest (==0.6.0)", "tox (==3.25.0)", "yamllint"] + +[[package]] +name = "dill" +version = "0.3.6" +description = "serialize all of python" +category = "dev" +optional = false +python-versions = ">=3.7" + +[package.extras] +graph = ["objgraph (>=1.7.2)"] + +[[package]] +name = "docstring-parser" +version = "0.15" +description = "Parse Python docstrings in reST, Google and Numpydoc format" +category = "main" +optional = false +python-versions = ">=3.6,<4.0" + +[[package]] +name = "exceptiongroup" +version = "1.1.0" +description = "Backport of PEP 654 (exception groups)" +category = "dev" +optional = false +python-versions = ">=3.7" + +[package.extras] +test = ["pytest (>=6)"] + +[[package]] +name = "fsspec" +version = "2022.11.0" +description = "File-system specification" +category = "main" +optional = false +python-versions = ">=3.7" + +[package.extras] +abfs = ["adlfs"] +adl = ["adlfs"] +arrow = ["pyarrow (>=1)"] +dask = ["dask", "distributed"] +dropbox = ["dropbox", "dropboxdrivefs", "requests"] +entrypoints = ["importlib-metadata"] +fuse = ["fusepy"] +gcs = ["gcsfs"] +git = ["pygit2"] +github = ["requests"] +gs = ["gcsfs"] +gui = ["panel"] +hdfs = ["pyarrow (>=1)"] +http = ["aiohttp (!=4.0.0a0,!=4.0.0a1)", "requests"] +libarchive = ["libarchive-c"] +oci = ["ocifs"] +s3 = ["s3fs"] +sftp = ["paramiko"] +smb = ["smbprotocol"] +ssh = ["paramiko"] +tqdm = ["tqdm"] + +[[package]] +name = "greenlet" +version = "2.0.1" +description = "Lightweight in-process concurrent programming" +category = "main" +optional = false +python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*" + +[package.extras] +docs = ["Sphinx", "docutils (<0.18)"] +test = ["faulthandler", "objgraph", "psutil"] + +[[package]] +name = "grpcio" +version = "1.47.2" +description = "HTTP/2-based RPC framework" +category = "main" +optional = false +python-versions = ">=3.6" + +[package.dependencies] +six = ">=1.5.2" + +[package.extras] +protobuf = ["grpcio-tools (>=1.47.2)"] + +[[package]] +name = "grpcio-health-checking" +version = "1.43.0" +description = "Standard Health Checking Service for gRPC" +category = "main" +optional = false +python-versions = ">=3.6" + +[package.dependencies] +grpcio = ">=1.43.0" +protobuf = ">=3.6.0" + +[[package]] +name = "humanfriendly" +version = "10.0" +description = "Human friendly output for text interfaces using Python" +category = "main" +optional = false +python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*" + +[package.dependencies] +pyreadline3 = {version = "*", markers = "sys_platform == \"win32\" and python_version >= \"3.8\""} + +[[package]] +name = "idna" +version = "3.4" +description = "Internationalized Domain Names in Applications (IDNA)" +category = "main" +optional = false +python-versions = ">=3.5" + +[[package]] +name = "importlib-metadata" +version = "5.2.0" +description = "Read metadata from Python packages" +category = "main" +optional = false +python-versions = ">=3.7" + +[package.dependencies] +zipp = ">=0.5" + +[package.extras] +docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] +perf = ["ipython"] +testing = ["flake8 (<5)", "flufl.flake8", "importlib-resources (>=1.3)", "packaging", "pyfakefs", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)", "pytest-perf (>=0.9.2)"] + +[[package]] +name = "importlib-resources" +version = "5.10.2" +description = "Read resources from Python packages" +category = "main" +optional = false +python-versions = ">=3.7" + +[package.dependencies] +zipp = {version = ">=3.1.0", markers = "python_version < \"3.10\""} + +[package.extras] +docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] +testing = ["flake8 (<5)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)"] + +[[package]] +name = "iniconfig" +version = "1.1.1" +description = "iniconfig: brain-dead simple config-ini parsing" +category = "dev" +optional = false +python-versions = "*" + +[[package]] +name = "isort" +version = "5.11.4" +description = "A Python utility / library to sort Python imports." +category = "dev" +optional = false +python-versions = ">=3.7.0" + +[package.extras] +colors = ["colorama (>=0.4.3,<0.5.0)"] +pipfile-deprecated-finder = ["pipreqs", "requirementslib"] +plugins = ["setuptools"] +requirements-deprecated-finder = ["pip-api", "pipreqs"] + +[[package]] +name = "jinja2" +version = "3.1.2" +description = "A very fast and expressive template engine." +category = "main" +optional = false +python-versions = ">=3.7" + +[package.dependencies] +MarkupSafe = ">=2.0" + +[package.extras] +i18n = ["Babel (>=2.7)"] + +[[package]] +name = "lazy-object-proxy" +version = "1.8.0" +description = "A fast and thorough lazy object proxy." +category = "dev" +optional = false +python-versions = ">=3.7" + +[[package]] +name = "mako" +version = "1.2.4" +description = "A super-fast templating language that borrows the best ideas from the existing templating languages." +category = "main" +optional = false +python-versions = ">=3.7" + +[package.dependencies] +MarkupSafe = ">=0.9.2" + +[package.extras] +babel = ["Babel"] +lingua = ["lingua"] +testing = ["pytest"] + +[[package]] +name = "markupsafe" +version = "2.1.1" +description = "Safely add untrusted strings to HTML/XML markup." +category = "main" +optional = false +python-versions = ">=3.7" + +[[package]] +name = "mccabe" +version = "0.7.0" +description = "McCabe checker, plugin for flake8" +category = "dev" +optional = false +python-versions = ">=3.6" + +[[package]] +name = "packaging" +version = "21.3" +description = "Core utilities for Python packages" +category = "main" +optional = false +python-versions = ">=3.6" + +[package.dependencies] +pyparsing = ">=2.0.2,<3.0.5 || >3.0.5" + +[[package]] +name = "pendulum" +version = "2.1.2" +description = "Python datetimes made easy" +category = "main" +optional = false +python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*" + +[package.dependencies] +python-dateutil = ">=2.6,<3.0" +pytzdata = ">=2020.1" + +[[package]] +name = "platformdirs" +version = "2.6.2" +description = "A small Python package for determining appropriate platform-specific dirs, e.g. a \"user data dir\"." +category = "dev" +optional = false +python-versions = ">=3.7" + +[package.extras] +docs = ["furo (>=2022.12.7)", "proselint (>=0.13)", "sphinx (>=5.3)", "sphinx-autodoc-typehints (>=1.19.5)"] +test = ["appdirs (==1.4.4)", "covdefaults (>=2.2.2)", "pytest (>=7.2)", "pytest-cov (>=4)", "pytest-mock (>=3.10)"] + +[[package]] +name = "pluggy" +version = "1.0.0" +description = "plugin and hook calling mechanisms for python" +category = "dev" +optional = false +python-versions = ">=3.6" + +[package.extras] +dev = ["pre-commit", "tox"] +testing = ["pytest", "pytest-benchmark"] + +[[package]] +name = "protobuf" +version = "3.20.3" +description = "Protocol Buffers" +category = "main" +optional = false +python-versions = ">=3.7" + +[[package]] +name = "psutil" +version = "5.9.4" +description = "Cross-platform lib for process and system monitoring in Python." +category = "main" +optional = false +python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*" + +[package.extras] +test = ["enum34", "ipaddress", "mock", "pywin32", "wmi"] + +[[package]] +name = "pylint" +version = "2.15.9" +description = "python code static checker" +category = "dev" +optional = false +python-versions = ">=3.7.2" + +[package.dependencies] +astroid = ">=2.12.13,<=2.14.0-dev0" +colorama = {version = ">=0.4.5", markers = "sys_platform == \"win32\""} +dill = {version = ">=0.2", markers = "python_version < \"3.11\""} +isort = ">=4.2.5,<6" +mccabe = ">=0.6,<0.8" +platformdirs = ">=2.2.0" +tomli = {version = ">=1.1.0", markers = "python_version < \"3.11\""} +tomlkit = ">=0.10.1" +typing-extensions = {version = ">=3.10.0", markers = "python_version < \"3.10\""} + +[package.extras] +spelling = ["pyenchant (>=3.2,<4.0)"] +testutils = ["gitpython (>3)"] + +[[package]] +name = "pyparsing" +version = "3.0.9" +description = "pyparsing module - Classes and methods to define and execute parsing grammars" +category = "main" +optional = false +python-versions = ">=3.6.8" + +[package.extras] +diagrams = ["jinja2", "railroad-diagrams"] + +[[package]] +name = "pyreadline3" +version = "3.4.1" +description = "A python implementation of GNU readline." +category = "main" +optional = false +python-versions = "*" + +[[package]] +name = "pytest" +version = "7.2.0" +description = "pytest: simple powerful testing with Python" +category = "dev" +optional = false +python-versions = ">=3.7" + +[package.dependencies] +attrs = ">=19.2.0" +colorama = {version = "*", markers = "sys_platform == \"win32\""} +exceptiongroup = {version = ">=1.0.0rc8", markers = "python_version < \"3.11\""} +iniconfig = "*" +packaging = "*" +pluggy = ">=0.12,<2.0" +tomli = {version = ">=1.0.0", markers = "python_version < \"3.11\""} + +[package.extras] +testing = ["argcomplete", "hypothesis (>=3.56)", "mock", "nose", "pygments (>=2.7.2)", "requests", "xmlschema"] + +[[package]] +name = "python-dateutil" +version = "2.8.2" +description = "Extensions to the standard Python datetime module" +category = "main" +optional = false +python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,>=2.7" + +[package.dependencies] +six = ">=1.5" + +[[package]] +name = "python-dotenv" +version = "0.21.0" +description = "Read key-value pairs from a .env file and set them as environment variables" +category = "main" +optional = false +python-versions = ">=3.7" + +[package.extras] +cli = ["click (>=5.0)"] + +[[package]] +name = "pytz" +version = "2022.7" +description = "World timezone definitions, modern and historical" +category = "main" +optional = false +python-versions = "*" + +[[package]] +name = "pytzdata" +version = "2020.1" +description = "The Olson timezone database for Python." +category = "main" +optional = false +python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*" + +[[package]] +name = "pywin32" +version = "305" +description = "Python for Window Extensions" +category = "main" +optional = false +python-versions = "*" + +[[package]] +name = "pyyaml" +version = "6.0" +description = "YAML parser and emitter for Python" +category = "main" +optional = false +python-versions = ">=3.6" + +[[package]] +name = "requests" +version = "2.28.1" +description = "Python HTTP for Humans." +category = "main" +optional = false +python-versions = ">=3.7, <4" + +[package.dependencies] +certifi = ">=2017.4.17" +charset-normalizer = ">=2,<3" +idna = ">=2.5,<4" +urllib3 = ">=1.21.1,<1.27" + +[package.extras] +socks = ["PySocks (>=1.5.6,!=1.5.7)"] +use-chardet-on-py3 = ["chardet (>=3.0.2,<6)"] + +[[package]] +name = "setuptools" +version = "65.6.3" +description = "Easily download, build, install, upgrade, and uninstall Python packages" +category = "main" +optional = false +python-versions = ">=3.7" + +[package.extras] +docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-hoverxref (<2)", "sphinx-inline-tabs", "sphinx-notfound-page (==0.8.3)", "sphinx-reredirects", "sphinxcontrib-towncrier"] +testing = ["build[virtualenv]", "filelock (>=3.4.0)", "flake8 (<5)", "flake8-2020", "ini2toml[lite] (>=0.9)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pip (>=19.1)", "pip-run (>=8.8)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)", "pytest-perf", "pytest-timeout", "pytest-xdist", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel"] +testing-integration = ["build[virtualenv]", "filelock (>=3.4.0)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pytest", "pytest-enabler", "pytest-xdist", "tomli", "virtualenv (>=13.0.0)", "wheel"] + +[[package]] +name = "six" +version = "1.16.0" +description = "Python 2 and 3 compatibility utilities" +category = "main" +optional = false +python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*" + +[[package]] +name = "sqlalchemy" +version = "1.4.45" +description = "Database Abstraction Library" +category = "main" +optional = false +python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,>=2.7" + +[package.dependencies] +greenlet = {version = "!=0.4.17", markers = "python_version >= \"3\" 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\")"} + +[package.extras] +aiomysql = ["aiomysql", "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)", "greenlet (!=0.4.17)"] +mariadb-connector = ["mariadb (>=1.0.1,!=1.1.2)"] +mssql = ["pyodbc"] +mssql-pymssql = ["pymssql"] +mssql-pyodbc = ["pyodbc"] +mypy = ["mypy (>=0.910)", "sqlalchemy2-stubs"] +mysql = ["mysqlclient (>=1.4.0)", "mysqlclient (>=1.4.0,<2)"] +mysql-connector = ["mysql-connector-python"] +oracle = ["cx_oracle (>=7)", "cx_oracle (>=7,<8)"] +postgresql = ["psycopg2 (>=2.7)"] +postgresql-asyncpg = ["asyncpg", "greenlet (!=0.4.17)"] +postgresql-pg8000 = ["pg8000 (>=1.16.6,!=1.29.0)"] +postgresql-psycopg2binary = ["psycopg2-binary"] +postgresql-psycopg2cffi = ["psycopg2cffi"] +pymysql = ["pymysql", "pymysql (<1)"] +sqlcipher = ["sqlcipher3_binary"] + +[[package]] +name = "tabulate" +version = "0.9.0" +description = "Pretty-print tabular data" +category = "main" +optional = false +python-versions = ">=3.7" + +[package.extras] +widechars = ["wcwidth"] + +[[package]] +name = "tomli" +version = "2.0.1" +description = "A lil' TOML parser" +category = "main" +optional = false +python-versions = ">=3.7" + +[[package]] +name = "tomlkit" +version = "0.11.6" +description = "Style preserving TOML library" +category = "dev" +optional = false +python-versions = ">=3.6" + +[[package]] +name = "toposort" +version = "1.7" +description = "\"Implements a topological sort algorithm.\"" +category = "main" +optional = false +python-versions = "*" + +[[package]] +name = "tqdm" +version = "4.64.1" +description = "Fast, Extensible Progress Meter" +category = "main" +optional = false +python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,>=2.7" + +[package.dependencies] +colorama = {version = "*", markers = "platform_system == \"Windows\""} + +[package.extras] +dev = ["py-make (>=0.1.0)", "twine", "wheel"] +notebook = ["ipywidgets (>=6)"] +slack = ["slack-sdk"] +telegram = ["requests"] + +[[package]] +name = "typing-extensions" +version = "4.4.0" +description = "Backported and Experimental Type Hints for Python 3.7+" +category = "main" +optional = false +python-versions = ">=3.7" + +[[package]] +name = "universal-pathlib" +version = "0.0.21" +description = "Pathlib API extended to use fsspec backends" +category = "main" +optional = false +python-versions = ">=3.7" + +[package.dependencies] +fsspec = "*" + +[package.extras] +test = ["adlfs", "aiohttp", "flake8", "gcsfs", "hadoop-test-cluster", "ipython", "jupyter", "moto", "pyarrow", "pylint", "pytest", "requests", "s3fs", "webdav4[fsspec]"] + +[[package]] +name = "urllib3" +version = "1.26.13" +description = "HTTP library with thread-safe connection pooling, file post, and more." +category = "main" +optional = false +python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*, !=3.5.*" + +[package.extras] +brotli = ["brotli (>=1.0.9)", "brotlicffi (>=0.8.0)", "brotlipy (>=0.6.0)"] +secure = ["certifi", "cryptography (>=1.3.4)", "idna (>=2.0.0)", "ipaddress", "pyOpenSSL (>=0.14)", "urllib3-secure-extra"] +socks = ["PySocks (>=1.5.6,!=1.5.7,<2.0)"] + +[[package]] +name = "watchdog" +version = "2.2.0" +description = "Filesystem events monitoring" +category = "main" +optional = false +python-versions = ">=3.6" + +[package.extras] +watchmedo = ["PyYAML (>=3.10)"] + +[[package]] +name = "wrapt" +version = "1.14.1" +description = "Module for decorators, wrappers and monkey patching." +category = "dev" +optional = false +python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,>=2.7" + +[[package]] +name = "zipp" +version = "3.11.0" +description = "Backport of pathlib-compatible object wrapper for zip files" +category = "main" +optional = false +python-versions = ">=3.7" + +[package.extras] +docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)"] +testing = ["flake8 (<5)", "func-timeout", "jaraco.functools", "jaraco.itertools", "more-itertools", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)"] + +[metadata] +lock-version = "1.1" +python-versions = "<3.11,>=3.8" +content-hash = "fe70d4b257399d82ef3e3f4b2df6f6ce935db4f5ede4fc7a60a332e07a591ecc" + +[metadata.files] +alembic = [ + {file = "alembic-1.9.1-py3-none-any.whl", hash = "sha256:a9781ed0979a20341c2cbb56bd22bd8db4fc1913f955e705444bd3a97c59fa32"}, + {file = "alembic-1.9.1.tar.gz", hash = "sha256:f9f76e41061f5ebe27d4fe92600df9dd612521a7683f904dab328ba02cffa5a2"}, +] +astroid = [ + {file = "astroid-2.12.13-py3-none-any.whl", hash = "sha256:10e0ad5f7b79c435179d0d0f0df69998c4eef4597534aae44910db060baeb907"}, + {file = "astroid-2.12.13.tar.gz", hash = "sha256:1493fe8bd3dfd73dc35bd53c9d5b6e49ead98497c47b2307662556a5692d29d7"}, +] +attrs = [ + {file = "attrs-22.2.0-py3-none-any.whl", hash = "sha256:29e95c7f6778868dbd49170f98f8818f78f3dc5e0e37c0b1f474e3561b240836"}, + {file = "attrs-22.2.0.tar.gz", hash = "sha256:c9227bfc2f01993c03f68db37d1d15c9690188323c067c641f1a35ca58185f99"}, +] +certifi = [ + {file = "certifi-2022.12.7-py3-none-any.whl", hash = "sha256:4ad3232f5e926d6718ec31cfc1fcadfde020920e278684144551c91769c7bc18"}, + {file = "certifi-2022.12.7.tar.gz", hash = "sha256:35824b4c3a97115964b408844d64aa14db1cc518f6562e8d7261699d1350a9e3"}, +] +charset-normalizer = [ + {file = "charset-normalizer-2.1.1.tar.gz", hash = "sha256:5a3d016c7c547f69d6f81fb0db9449ce888b418b5b9952cc5e6e66843e9dd845"}, + {file = "charset_normalizer-2.1.1-py3-none-any.whl", hash = "sha256:83e9a75d1911279afd89352c68b45348559d1fc0506b054b346651b5e7fee29f"}, +] +click = [ + {file = "click-8.1.3-py3-none-any.whl", hash = "sha256:bb4d8133cb15a609f44e8213d9b391b0809795062913b383c62be0ee95b1db48"}, + {file = "click-8.1.3.tar.gz", hash = "sha256:7682dc8afb30297001674575ea00d1814d808d6a36af415a82bd481d37ba7b8e"}, +] +colorama = [ + {file = "colorama-0.4.6-py2.py3-none-any.whl", hash = "sha256:4f1d9991f5acc0ca119f9d443620b77f9d6b33703e51011c16baf57afb285fc6"}, + {file = "colorama-0.4.6.tar.gz", hash = "sha256:08695f5cb7ed6e0531a20572697297273c47b8cae5a63ffc6d6ed5c201be6e44"}, +] +coloredlogs = [ + {file = "coloredlogs-14.0-py2.py3-none-any.whl", hash = "sha256:346f58aad6afd48444c2468618623638dadab76e4e70d5e10822676f2d32226a"}, + {file = "coloredlogs-14.0.tar.gz", hash = "sha256:a1fab193d2053aa6c0a97608c4342d031f1f93a3d1218432c59322441d31a505"}, +] +croniter = [ + {file = "croniter-1.3.8-py2.py3-none-any.whl", hash = "sha256:d6ed8386d5f4bbb29419dc1b65c4909c04a2322bd15ec0dc5b2877bfa1b75c7a"}, + {file = "croniter-1.3.8.tar.gz", hash = "sha256:32a5ec04e97ec0837bcdf013767abd2e71cceeefd3c2e14c804098ce51ad6cd9"}, +] +dagster = [ + {file = "dagster-1.1.7-py3-none-any.whl", hash = "sha256:7aac07b01129ba338426ee8dd31e7c8735fcf290ad5e34ff94e81eba45898e81"}, + {file = "dagster-1.1.7.tar.gz", hash = "sha256:25d15e0e6b6452bb33b7dc8d586421bb0df9ceb0cf64ef68b7af3f83a700faaa"}, +] +dill = [ + {file = "dill-0.3.6-py3-none-any.whl", hash = "sha256:a07ffd2351b8c678dfc4a856a3005f8067aea51d6ba6c700796a4d9e280f39f0"}, + {file = "dill-0.3.6.tar.gz", hash = "sha256:e5db55f3687856d8fbdab002ed78544e1c4559a130302693d839dfe8f93f2373"}, +] +docstring-parser = [ + {file = "docstring_parser-0.15-py3-none-any.whl", hash = "sha256:d1679b86250d269d06a99670924d6bce45adc00b08069dae8c47d98e89b667a9"}, + {file = "docstring_parser-0.15.tar.gz", hash = "sha256:48ddc093e8b1865899956fcc03b03e66bb7240c310fac5af81814580c55bf682"}, +] +exceptiongroup = [ + {file = "exceptiongroup-1.1.0-py3-none-any.whl", hash = "sha256:327cbda3da756e2de031a3107b81ab7b3770a602c4d16ca618298c526f4bec1e"}, + {file = "exceptiongroup-1.1.0.tar.gz", hash = "sha256:bcb67d800a4497e1b404c2dd44fca47d3b7a5e5433dbab67f96c1a685cdfdf23"}, +] +fsspec = [ + {file = "fsspec-2022.11.0-py3-none-any.whl", hash = "sha256:d6e462003e3dcdcb8c7aa84c73a228f8227e72453cd22570e2363e8844edfe7b"}, + {file = "fsspec-2022.11.0.tar.gz", hash = "sha256:259d5fd5c8e756ff2ea72f42e7613c32667dc2049a4ac3d84364a7ca034acb8b"}, +] +greenlet = [ + {file = "greenlet-2.0.1-cp27-cp27m-macosx_10_14_x86_64.whl", hash = "sha256:9ed358312e63bf683b9ef22c8e442ef6c5c02973f0c2a939ec1d7b50c974015c"}, + {file = "greenlet-2.0.1-cp27-cp27m-manylinux2010_x86_64.whl", hash = "sha256:4f09b0010e55bec3239278f642a8a506b91034f03a4fb28289a7d448a67f1515"}, + {file = "greenlet-2.0.1-cp27-cp27m-win32.whl", hash = "sha256:1407fe45246632d0ffb7a3f4a520ba4e6051fc2cbd61ba1f806900c27f47706a"}, + {file = "greenlet-2.0.1-cp27-cp27m-win_amd64.whl", hash = "sha256:3001d00eba6bbf084ae60ec7f4bb8ed375748f53aeaefaf2a37d9f0370558524"}, + {file = "greenlet-2.0.1-cp27-cp27mu-manylinux2010_x86_64.whl", hash = "sha256:d566b82e92ff2e09dd6342df7e0eb4ff6275a3f08db284888dcd98134dbd4243"}, + {file = "greenlet-2.0.1-cp310-cp310-macosx_10_15_x86_64.whl", hash = "sha256:0722c9be0797f544a3ed212569ca3fe3d9d1a1b13942d10dd6f0e8601e484d26"}, + {file = "greenlet-2.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4d37990425b4687ade27810e3b1a1c37825d242ebc275066cfee8cb6b8829ccd"}, + {file = "greenlet-2.0.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:be35822f35f99dcc48152c9839d0171a06186f2d71ef76dc57fa556cc9bf6b45"}, + {file = "greenlet-2.0.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c140e7eb5ce47249668056edf3b7e9900c6a2e22fb0eaf0513f18a1b2c14e1da"}, + {file = "greenlet-2.0.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:d21681f09e297a5adaa73060737e3aa1279a13ecdcfcc6ef66c292cb25125b2d"}, + {file = "greenlet-2.0.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:fb412b7db83fe56847df9c47b6fe3f13911b06339c2aa02dcc09dce8bbf582cd"}, + {file = "greenlet-2.0.1-cp310-cp310-win_amd64.whl", hash = "sha256:c6a08799e9e88052221adca55741bf106ec7ea0710bca635c208b751f0d5b617"}, + {file = "greenlet-2.0.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:9e112e03d37987d7b90c1e98ba5e1b59e1645226d78d73282f45b326f7bddcb9"}, + {file = "greenlet-2.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:56961cfca7da2fdd178f95ca407fa330c64f33289e1804b592a77d5593d9bd94"}, + {file = "greenlet-2.0.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:13ba6e8e326e2116c954074c994da14954982ba2795aebb881c07ac5d093a58a"}, + {file = "greenlet-2.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1bf633a50cc93ed17e494015897361010fc08700d92676c87931d3ea464123ce"}, + {file = "greenlet-2.0.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:9f2c221eecb7ead00b8e3ddb913c67f75cba078fd1d326053225a3f59d850d72"}, + {file = "greenlet-2.0.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:13ebf93c343dd8bd010cd98e617cb4c1c1f352a0cf2524c82d3814154116aa82"}, + {file = "greenlet-2.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:6f61d71bbc9b4a3de768371b210d906726535d6ca43506737682caa754b956cd"}, + {file = "greenlet-2.0.1-cp35-cp35m-macosx_10_14_x86_64.whl", hash = "sha256:2d0bac0385d2b43a7bd1d651621a4e0f1380abc63d6fb1012213a401cbd5bf8f"}, + {file = "greenlet-2.0.1-cp35-cp35m-manylinux2010_x86_64.whl", hash = "sha256:f6327b6907b4cb72f650a5b7b1be23a2aab395017aa6f1adb13069d66360eb3f"}, + {file = "greenlet-2.0.1-cp35-cp35m-win32.whl", hash = "sha256:81b0ea3715bf6a848d6f7149d25bf018fd24554a4be01fcbbe3fdc78e890b955"}, + {file = "greenlet-2.0.1-cp35-cp35m-win_amd64.whl", hash = "sha256:38255a3f1e8942573b067510f9611fc9e38196077b0c8eb7a8c795e105f9ce77"}, + {file = "greenlet-2.0.1-cp36-cp36m-macosx_10_14_x86_64.whl", hash = "sha256:04957dc96669be041e0c260964cfef4c77287f07c40452e61abe19d647505581"}, + {file = "greenlet-2.0.1-cp36-cp36m-manylinux2010_x86_64.whl", hash = "sha256:4aeaebcd91d9fee9aa768c1b39cb12214b30bf36d2b7370505a9f2165fedd8d9"}, + {file = "greenlet-2.0.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:974a39bdb8c90a85982cdb78a103a32e0b1be986d411303064b28a80611f6e51"}, + {file = "greenlet-2.0.1-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8dca09dedf1bd8684767bc736cc20c97c29bc0c04c413e3276e0962cd7aeb148"}, + {file = "greenlet-2.0.1-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a4c0757db9bd08470ff8277791795e70d0bf035a011a528ee9a5ce9454b6cba2"}, + {file = "greenlet-2.0.1-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:5067920de254f1a2dee8d3d9d7e4e03718e8fd2d2d9db962c8c9fa781ae82a39"}, + {file = "greenlet-2.0.1-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:5a8e05057fab2a365c81abc696cb753da7549d20266e8511eb6c9d9f72fe3e92"}, + {file = "greenlet-2.0.1-cp36-cp36m-win32.whl", hash = "sha256:3d75b8d013086b08e801fbbb896f7d5c9e6ccd44f13a9241d2bf7c0df9eda928"}, + {file = "greenlet-2.0.1-cp36-cp36m-win_amd64.whl", hash = "sha256:097e3dae69321e9100202fc62977f687454cd0ea147d0fd5a766e57450c569fd"}, + {file = "greenlet-2.0.1-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:cb242fc2cda5a307a7698c93173d3627a2a90d00507bccf5bc228851e8304963"}, + {file = "greenlet-2.0.1-cp37-cp37m-manylinux2010_x86_64.whl", hash = "sha256:72b00a8e7c25dcea5946692a2485b1a0c0661ed93ecfedfa9b6687bd89a24ef5"}, + {file = "greenlet-2.0.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d5b0ff9878333823226d270417f24f4d06f235cb3e54d1103b71ea537a6a86ce"}, + {file = "greenlet-2.0.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:be9e0fb2ada7e5124f5282d6381903183ecc73ea019568d6d63d33f25b2a9000"}, + {file = "greenlet-2.0.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0b493db84d124805865adc587532ebad30efa68f79ad68f11b336e0a51ec86c2"}, + {file = "greenlet-2.0.1-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:0459d94f73265744fee4c2d5ec44c6f34aa8a31017e6e9de770f7bcf29710be9"}, + {file = "greenlet-2.0.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:a20d33124935d27b80e6fdacbd34205732660e0a1d35d8b10b3328179a2b51a1"}, + {file = "greenlet-2.0.1-cp37-cp37m-win32.whl", hash = "sha256:ea688d11707d30e212e0110a1aac7f7f3f542a259235d396f88be68b649e47d1"}, + {file = "greenlet-2.0.1-cp37-cp37m-win_amd64.whl", hash = "sha256:afe07421c969e259e9403c3bb658968702bc3b78ec0b6fde3ae1e73440529c23"}, + {file = "greenlet-2.0.1-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:cd4ccc364cf75d1422e66e247e52a93da6a9b73cefa8cad696f3cbbb75af179d"}, + {file = "greenlet-2.0.1-cp38-cp38-manylinux2010_x86_64.whl", hash = "sha256:4c8b1c43e75c42a6cafcc71defa9e01ead39ae80bd733a2608b297412beede68"}, + {file = "greenlet-2.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:659f167f419a4609bc0516fb18ea69ed39dbb25594934bd2dd4d0401660e8a1e"}, + {file = "greenlet-2.0.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:356e4519d4dfa766d50ecc498544b44c0249b6de66426041d7f8b751de4d6b48"}, + {file = "greenlet-2.0.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:811e1d37d60b47cb8126e0a929b58c046251f28117cb16fcd371eed61f66b764"}, + {file = "greenlet-2.0.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:d38ffd0e81ba8ef347d2be0772e899c289b59ff150ebbbbe05dc61b1246eb4e0"}, + {file = "greenlet-2.0.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:0109af1138afbfb8ae647e31a2b1ab030f58b21dd8528c27beaeb0093b7938a9"}, + {file = "greenlet-2.0.1-cp38-cp38-win32.whl", hash = "sha256:88c8d517e78acdf7df8a2134a3c4b964415b575d2840a2746ddb1cc6175f8608"}, + {file = "greenlet-2.0.1-cp38-cp38-win_amd64.whl", hash = "sha256:d6ee1aa7ab36475035eb48c01efae87d37936a8173fc4d7b10bb02c2d75dd8f6"}, + {file = "greenlet-2.0.1-cp39-cp39-macosx_10_15_x86_64.whl", hash = "sha256:b1992ba9d4780d9af9726bbcef6a1db12d9ab1ccc35e5773685a24b7fb2758eb"}, + {file = "greenlet-2.0.1-cp39-cp39-manylinux2010_x86_64.whl", hash = "sha256:b5e83e4de81dcc9425598d9469a624826a0b1211380ac444c7c791d4a2137c19"}, + {file = "greenlet-2.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:505138d4fa69462447a562a7c2ef723c6025ba12ac04478bc1ce2fcc279a2db5"}, + {file = "greenlet-2.0.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:cce1e90dd302f45716a7715517c6aa0468af0bf38e814ad4eab58e88fc09f7f7"}, + {file = "greenlet-2.0.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9e9744c657d896c7b580455e739899e492a4a452e2dd4d2b3e459f6b244a638d"}, + {file = "greenlet-2.0.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:662e8f7cad915ba75d8017b3e601afc01ef20deeeabf281bd00369de196d7726"}, + {file = "greenlet-2.0.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:41b825d65f31e394b523c84db84f9383a2f7eefc13d987f308f4663794d2687e"}, + {file = "greenlet-2.0.1-cp39-cp39-win32.whl", hash = "sha256:db38f80540083ea33bdab614a9d28bcec4b54daa5aff1668d7827a9fc769ae0a"}, + {file = "greenlet-2.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:b23d2a46d53210b498e5b701a1913697671988f4bf8e10f935433f6e7c332fb6"}, + {file = "greenlet-2.0.1.tar.gz", hash = "sha256:42e602564460da0e8ee67cb6d7236363ee5e131aa15943b6670e44e5c2ed0f67"}, +] +grpcio = [ + {file = "grpcio-1.47.2-cp310-cp310-linux_armv7l.whl", hash = "sha256:b5ca2fb4b7e48585d5d90e3dbcfbb6efed763c73e6ab2419a2377611396f8a24"}, + {file = "grpcio-1.47.2-cp310-cp310-macosx_10_10_x86_64.whl", hash = "sha256:c17acc79c7280334f5db8af75af9b0861f0f613dca41c982f471d967142a0bad"}, + {file = "grpcio-1.47.2-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:d6aa4556fb4be906baf2e2501e9edff9fd4817d88a1f544b8d7a7948e48b1954"}, + {file = "grpcio-1.47.2-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:70b52128e4395356896bb4eb70c8f8de10f9ce4000eca192a53889a76f80c647"}, + {file = "grpcio-1.47.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:696fbb04702dc8d2fcf2a76b366378f2adc606a1c65f277ebbc00f78d5a71cc1"}, + {file = "grpcio-1.47.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:f020d31693c372b89bd479eceec000d9462107a3b17bd0b7aed32f06fe246efe"}, + {file = "grpcio-1.47.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:a8d4feb221082e91843bcdf16186014583d0fb8671ee0092f66c956752dd81ab"}, + {file = "grpcio-1.47.2-cp310-cp310-win32.whl", hash = "sha256:89b2696928d0db627a80a55158049f760046b90234bb5a9aeb23f9be2750a03e"}, + {file = "grpcio-1.47.2-cp310-cp310-win_amd64.whl", hash = "sha256:f13ae44d39c297b6633c86180a11e6c70cf5596074cd16d7dcee55634d9a70e6"}, + {file = "grpcio-1.47.2-cp36-cp36m-linux_armv7l.whl", hash = "sha256:42a87566378866b98e89d2a7c0b5142cf462a5acba041827902f90d561763d14"}, + {file = "grpcio-1.47.2-cp36-cp36m-macosx_10_10_x86_64.whl", hash = "sha256:bd9129344ccc0845e01134e87e8b7c11f2b6943d0aff016f88273a6b5910bfed"}, + {file = "grpcio-1.47.2-cp36-cp36m-manylinux_2_17_aarch64.whl", hash = "sha256:c96813980f8e6c3ca302221dfadb380dedca911cd7f74bbab66c98c83b310cee"}, + {file = "grpcio-1.47.2-cp36-cp36m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:56e593416e67ca4af7edd626dabe1531d9ef966f8a78aa3156c7d5618b5b270a"}, + {file = "grpcio-1.47.2-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ef1f17c47269391af42bc2d64997c9b62bd80ae44cf595b77cf047f4a00b566a"}, + {file = "grpcio-1.47.2-cp36-cp36m-musllinux_1_1_i686.whl", hash = "sha256:c867a2ad3fced11b75a27672a1c4d72c1c2d42ed11803bce3263e91eda4b2ce0"}, + {file = "grpcio-1.47.2-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:d3c701104bebc6c66f9fff2ac1f4704f3ab45e179f2be496b2689b6e35179bd4"}, + {file = "grpcio-1.47.2-cp36-cp36m-win32.whl", hash = "sha256:3d04a83e9348d464c789f39c15c30608dc51346a2c76502df444197b11463d90"}, + {file = "grpcio-1.47.2-cp36-cp36m-win_amd64.whl", hash = "sha256:1a669c2f52959c82e5994884ac8940fa626dff2f3a9398098d5cdded3ed0b214"}, + {file = "grpcio-1.47.2-cp37-cp37m-linux_armv7l.whl", hash = "sha256:a57f6cef560c7b9f56549d30ef3b3be0c9ec5f0db321062c67fca05756daedaa"}, + {file = "grpcio-1.47.2-cp37-cp37m-macosx_10_10_x86_64.whl", hash = "sha256:98ecdaea5d66df8ae4a846b50c253aa8bf7ee4162552f56ce37a5d6d561bb711"}, + {file = "grpcio-1.47.2-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:f23bf8eafa0265c12b3c54e0b3b77dc4f24dde4b6e9260aecffaed3cfe5d1fbd"}, + {file = "grpcio-1.47.2-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:07b14bd4e4c12942887a2b0a08d008929d744c47f6eacf83711c778b4555a943"}, + {file = "grpcio-1.47.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:32b28bfd6de8037ae214bd81af846cf29d39c6518e2005a544c28b10cdef42cb"}, + {file = "grpcio-1.47.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:2e76b9447e87faaca797f6c74469466a77c6a95aa5e0e36e0887687c204d471c"}, + {file = "grpcio-1.47.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:e9d86d03cc8c4dcd1e3d2593dccf12fc61b044358e3085bfd23e50d47686657b"}, + {file = "grpcio-1.47.2-cp37-cp37m-win32.whl", hash = "sha256:73f46fa86d33a3da2f071c65d1c0bf8b4cef49983e1518aeebaf7700b2a4c23f"}, + {file = "grpcio-1.47.2-cp37-cp37m-win_amd64.whl", hash = "sha256:e5abb83b78a337c7731feb4c734fcaa595b3a1f1bdbbd6233cfc08cbf0a83405"}, + {file = "grpcio-1.47.2-cp38-cp38-linux_armv7l.whl", hash = "sha256:a202f137d1e7f018aa28a62bb464fa773ebe37c68ae9761a1b55a00d9beaf2f0"}, + {file = "grpcio-1.47.2-cp38-cp38-macosx_10_10_x86_64.whl", hash = "sha256:0b78e060766967959b53c460ce3fa6d31d6c0dde9f801976875d2e1cac2098f5"}, + {file = "grpcio-1.47.2-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:8b79f5241acf4a3b3c063a7a6866b70289f5d5fbaa0fcb48e566ce55f85c26fd"}, + {file = "grpcio-1.47.2-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6ce3ab22060f1e04b06567334f07c11d37e01f019841df46910aa2680b4dbc89"}, + {file = "grpcio-1.47.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9ca6bdc2ca97419d9154a2858e7190109280143c87f3eda4ad6dada2729dea7e"}, + {file = "grpcio-1.47.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:c1900803f47c37fa7c055f223d88111181ef53c5fc902391ad4f452aa5c97d2a"}, + {file = "grpcio-1.47.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:92d5f9c7ecb31d6e2579744969f089b285fe45c725d26366a0c1e858d2d52c44"}, + {file = "grpcio-1.47.2-cp38-cp38-win32.whl", hash = "sha256:60967c0d7725a11978f1d3fe60e21e0e0ad10fac920a35d29721f2b01089604f"}, + {file = "grpcio-1.47.2-cp38-cp38-win_amd64.whl", hash = "sha256:c0ee8af10c65d66f1896b5da09abb4a7bddcf3b6e7a6ffc5661713525ca98704"}, + {file = "grpcio-1.47.2-cp39-cp39-linux_armv7l.whl", hash = "sha256:f3510688f6c185a9483ecd1580a0ff5e5b0a7301387d831ec6e31e1c1430bd5d"}, + {file = "grpcio-1.47.2-cp39-cp39-macosx_10_10_x86_64.whl", hash = "sha256:60c69dadc0e9f96cb031d1902f8db6e4ef94bbc17667fc673919dc57871744de"}, + {file = "grpcio-1.47.2-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:af911454d415eb9d18f011b10c66ac29de87266e47fab5d00481d6478c7cc814"}, + {file = "grpcio-1.47.2-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:36b215d89cadc11c156db8ae4b37ebc50675bae3f75cee230efaf399d19c00c4"}, + {file = "grpcio-1.47.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ef09be9118f82bc2ff6aac77d7436eef2434a9580b6ffb3b62ddafc9c1b2015f"}, + {file = "grpcio-1.47.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:cd4106bf10fa440c5dd7c5f21484c7b1e608f0b70a274dc6c2c29f3eee4827f6"}, + {file = "grpcio-1.47.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:9ad746fe50780b9b8fb3e15e758d96561758c3cf6f96356cee76721ccd76d254"}, + {file = "grpcio-1.47.2-cp39-cp39-win32.whl", hash = "sha256:346859e7c279e81611cf270e2ea67990d6a314e3e2aa75f924615268e0764d0f"}, + {file = "grpcio-1.47.2-cp39-cp39-win_amd64.whl", hash = "sha256:e578933d5b365506ae85cbe104c5156baa2307ecdcc1a57d80143c078dca4b8f"}, + {file = "grpcio-1.47.2.tar.gz", hash = "sha256:c7f888b067f2b1833a5670f20eeac2c6063039a23a99b799929de065c1b18692"}, +] +grpcio-health-checking = [ + {file = "grpcio-health-checking-1.43.0.tar.gz", hash = "sha256:23b3c6c3a6da85f7cd944cbebd965cdceb2a5e4e65f071a2b6cf1eb3510c993e"}, + {file = "grpcio_health_checking-1.43.0-py3-none-any.whl", hash = "sha256:802195ec77dd609ed9721dcb4d5916b6018dc49351b904d152fb697595debd33"}, +] +humanfriendly = [ + {file = "humanfriendly-10.0-py2.py3-none-any.whl", hash = "sha256:1697e1a8a8f550fd43c2865cd84542fc175a61dcb779b6fee18cf6b6ccba1477"}, + {file = "humanfriendly-10.0.tar.gz", hash = "sha256:6b0b831ce8f15f7300721aa49829fc4e83921a9a301cc7f606be6686a2288ddc"}, +] +idna = [ + {file = "idna-3.4-py3-none-any.whl", hash = "sha256:90b77e79eaa3eba6de819a0c442c0b4ceefc341a7a2ab77d7562bf49f425c5c2"}, + {file = "idna-3.4.tar.gz", hash = "sha256:814f528e8dead7d329833b91c5faa87d60bf71824cd12a7530b5526063d02cb4"}, +] +importlib-metadata = [ + {file = "importlib_metadata-5.2.0-py3-none-any.whl", hash = "sha256:0eafa39ba42bf225fc00e67f701d71f85aead9f878569caf13c3724f704b970f"}, + {file = "importlib_metadata-5.2.0.tar.gz", hash = "sha256:404d48d62bba0b7a77ff9d405efd91501bef2e67ff4ace0bed40a0cf28c3c7cd"}, +] +importlib-resources = [ + {file = "importlib_resources-5.10.2-py3-none-any.whl", hash = "sha256:7d543798b0beca10b6a01ac7cafda9f822c54db9e8376a6bf57e0cbd74d486b6"}, + {file = "importlib_resources-5.10.2.tar.gz", hash = "sha256:e4a96c8cc0339647ff9a5e0550d9f276fc5a01ffa276012b58ec108cfd7b8484"}, +] +iniconfig = [ + {file = "iniconfig-1.1.1-py2.py3-none-any.whl", hash = "sha256:011e24c64b7f47f6ebd835bb12a743f2fbe9a26d4cecaa7f53bc4f35ee9da8b3"}, + {file = "iniconfig-1.1.1.tar.gz", hash = "sha256:bc3af051d7d14b2ee5ef9969666def0cd1a000e121eaea580d4a313df4b37f32"}, +] +isort = [ + {file = "isort-5.11.4-py3-none-any.whl", hash = "sha256:c033fd0edb91000a7f09527fe5c75321878f98322a77ddcc81adbd83724afb7b"}, + {file = "isort-5.11.4.tar.gz", hash = "sha256:6db30c5ded9815d813932c04c2f85a360bcdd35fed496f4d8f35495ef0a261b6"}, +] +jinja2 = [ + {file = "Jinja2-3.1.2-py3-none-any.whl", hash = "sha256:6088930bfe239f0e6710546ab9c19c9ef35e29792895fed6e6e31a023a182a61"}, + {file = "Jinja2-3.1.2.tar.gz", hash = "sha256:31351a702a408a9e7595a8fc6150fc3f43bb6bf7e319770cbc0db9df9437e852"}, +] +lazy-object-proxy = [ + {file = "lazy-object-proxy-1.8.0.tar.gz", hash = "sha256:c219a00245af0f6fa4e95901ed28044544f50152840c5b6a3e7b2568db34d156"}, + {file = "lazy_object_proxy-1.8.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:4fd031589121ad46e293629b39604031d354043bb5cdf83da4e93c2d7f3389fe"}, + {file = "lazy_object_proxy-1.8.0-cp310-cp310-win32.whl", hash = "sha256:b70d6e7a332eb0217e7872a73926ad4fdc14f846e85ad6749ad111084e76df25"}, + {file = "lazy_object_proxy-1.8.0-cp310-cp310-win_amd64.whl", hash = "sha256:eb329f8d8145379bf5dbe722182410fe8863d186e51bf034d2075eb8d85ee25b"}, + {file = "lazy_object_proxy-1.8.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:4e2d9f764f1befd8bdc97673261b8bb888764dfdbd7a4d8f55e4fbcabb8c3fb7"}, + {file = "lazy_object_proxy-1.8.0-cp311-cp311-win32.whl", hash = "sha256:e20bfa6db17a39c706d24f82df8352488d2943a3b7ce7d4c22579cb89ca8896e"}, + {file = "lazy_object_proxy-1.8.0-cp311-cp311-win_amd64.whl", hash = "sha256:14010b49a2f56ec4943b6cf925f597b534ee2fe1f0738c84b3bce0c1a11ff10d"}, + {file = "lazy_object_proxy-1.8.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:6850e4aeca6d0df35bb06e05c8b934ff7c533734eb51d0ceb2d63696f1e6030c"}, + {file = "lazy_object_proxy-1.8.0-cp37-cp37m-win32.whl", hash = "sha256:5b51d6f3bfeb289dfd4e95de2ecd464cd51982fe6f00e2be1d0bf94864d58acd"}, + {file = "lazy_object_proxy-1.8.0-cp37-cp37m-win_amd64.whl", hash = "sha256:6f593f26c470a379cf7f5bc6db6b5f1722353e7bf937b8d0d0b3fba911998858"}, + {file = "lazy_object_proxy-1.8.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:0c1c7c0433154bb7c54185714c6929acc0ba04ee1b167314a779b9025517eada"}, + {file = "lazy_object_proxy-1.8.0-cp38-cp38-win32.whl", hash = "sha256:d176f392dbbdaacccf15919c77f526edf11a34aece58b55ab58539807b85436f"}, + {file = "lazy_object_proxy-1.8.0-cp38-cp38-win_amd64.whl", hash = "sha256:afcaa24e48bb23b3be31e329deb3f1858f1f1df86aea3d70cb5c8578bfe5261c"}, + {file = "lazy_object_proxy-1.8.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:71d9ae8a82203511a6f60ca5a1b9f8ad201cac0fc75038b2dc5fa519589c9288"}, + {file = "lazy_object_proxy-1.8.0-cp39-cp39-win32.whl", hash = "sha256:8f6ce2118a90efa7f62dd38c7dbfffd42f468b180287b748626293bf12ed468f"}, + {file = "lazy_object_proxy-1.8.0-cp39-cp39-win_amd64.whl", hash = "sha256:eac3a9a5ef13b332c059772fd40b4b1c3d45a3a2b05e33a361dee48e54a4dad0"}, + {file = "lazy_object_proxy-1.8.0-pp37-pypy37_pp73-any.whl", hash = "sha256:ae032743794fba4d171b5b67310d69176287b5bf82a21f588282406a79498891"}, + {file = "lazy_object_proxy-1.8.0-pp38-pypy38_pp73-any.whl", hash = "sha256:7e1561626c49cb394268edd00501b289053a652ed762c58e1081224c8d881cec"}, + {file = "lazy_object_proxy-1.8.0-pp39-pypy39_pp73-any.whl", hash = "sha256:ce58b2b3734c73e68f0e30e4e725264d4d6be95818ec0a0be4bb6bf9a7e79aa8"}, +] +mako = [ + {file = "Mako-1.2.4-py3-none-any.whl", hash = "sha256:c97c79c018b9165ac9922ae4f32da095ffd3c4e6872b45eded42926deea46818"}, + {file = "Mako-1.2.4.tar.gz", hash = "sha256:d60a3903dc3bb01a18ad6a89cdbe2e4eadc69c0bc8ef1e3773ba53d44c3f7a34"}, +] +markupsafe = [ + {file = "MarkupSafe-2.1.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:86b1f75c4e7c2ac2ccdaec2b9022845dbb81880ca318bb7a0a01fbf7813e3812"}, + {file = "MarkupSafe-2.1.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:f121a1420d4e173a5d96e47e9a0c0dcff965afdf1626d28de1460815f7c4ee7a"}, + {file = "MarkupSafe-2.1.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a49907dd8420c5685cfa064a1335b6754b74541bbb3706c259c02ed65b644b3e"}, + {file = "MarkupSafe-2.1.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:10c1bfff05d95783da83491be968e8fe789263689c02724e0c691933c52994f5"}, + {file = "MarkupSafe-2.1.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b7bd98b796e2b6553da7225aeb61f447f80a1ca64f41d83612e6139ca5213aa4"}, + {file = "MarkupSafe-2.1.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:b09bf97215625a311f669476f44b8b318b075847b49316d3e28c08e41a7a573f"}, + {file = "MarkupSafe-2.1.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:694deca8d702d5db21ec83983ce0bb4b26a578e71fbdbd4fdcd387daa90e4d5e"}, + {file = "MarkupSafe-2.1.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:efc1913fd2ca4f334418481c7e595c00aad186563bbc1ec76067848c7ca0a933"}, + {file = "MarkupSafe-2.1.1-cp310-cp310-win32.whl", hash = "sha256:4a33dea2b688b3190ee12bd7cfa29d39c9ed176bda40bfa11099a3ce5d3a7ac6"}, + {file = "MarkupSafe-2.1.1-cp310-cp310-win_amd64.whl", hash = "sha256:dda30ba7e87fbbb7eab1ec9f58678558fd9a6b8b853530e176eabd064da81417"}, + {file = "MarkupSafe-2.1.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:671cd1187ed5e62818414afe79ed29da836dde67166a9fac6d435873c44fdd02"}, + {file = "MarkupSafe-2.1.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3799351e2336dc91ea70b034983ee71cf2f9533cdff7c14c90ea126bfd95d65a"}, + {file = "MarkupSafe-2.1.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e72591e9ecd94d7feb70c1cbd7be7b3ebea3f548870aa91e2732960fa4d57a37"}, + {file = "MarkupSafe-2.1.1-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6fbf47b5d3728c6aea2abb0589b5d30459e369baa772e0f37a0320185e87c980"}, + {file = "MarkupSafe-2.1.1-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:d5ee4f386140395a2c818d149221149c54849dfcfcb9f1debfe07a8b8bd63f9a"}, + {file = "MarkupSafe-2.1.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:bcb3ed405ed3222f9904899563d6fc492ff75cce56cba05e32eff40e6acbeaa3"}, + {file = "MarkupSafe-2.1.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:e1c0b87e09fa55a220f058d1d49d3fb8df88fbfab58558f1198e08c1e1de842a"}, + {file = "MarkupSafe-2.1.1-cp37-cp37m-win32.whl", hash = "sha256:8dc1c72a69aa7e082593c4a203dcf94ddb74bb5c8a731e4e1eb68d031e8498ff"}, + {file = "MarkupSafe-2.1.1-cp37-cp37m-win_amd64.whl", hash = "sha256:97a68e6ada378df82bc9f16b800ab77cbf4b2fada0081794318520138c088e4a"}, + {file = "MarkupSafe-2.1.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:e8c843bbcda3a2f1e3c2ab25913c80a3c5376cd00c6e8c4a86a89a28c8dc5452"}, + {file = "MarkupSafe-2.1.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:0212a68688482dc52b2d45013df70d169f542b7394fc744c02a57374a4207003"}, + {file = "MarkupSafe-2.1.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8e576a51ad59e4bfaac456023a78f6b5e6e7651dcd383bcc3e18d06f9b55d6d1"}, + {file = "MarkupSafe-2.1.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4b9fe39a2ccc108a4accc2676e77da025ce383c108593d65cc909add5c3bd601"}, + {file = "MarkupSafe-2.1.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:96e37a3dc86e80bf81758c152fe66dbf60ed5eca3d26305edf01892257049925"}, + {file = "MarkupSafe-2.1.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:6d0072fea50feec76a4c418096652f2c3238eaa014b2f94aeb1d56a66b41403f"}, + {file = "MarkupSafe-2.1.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:089cf3dbf0cd6c100f02945abeb18484bd1ee57a079aefd52cffd17fba910b88"}, + {file = "MarkupSafe-2.1.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:6a074d34ee7a5ce3effbc526b7083ec9731bb3cbf921bbe1d3005d4d2bdb3a63"}, + {file = "MarkupSafe-2.1.1-cp38-cp38-win32.whl", hash = "sha256:421be9fbf0ffe9ffd7a378aafebbf6f4602d564d34be190fc19a193232fd12b1"}, + {file = "MarkupSafe-2.1.1-cp38-cp38-win_amd64.whl", hash = "sha256:fc7b548b17d238737688817ab67deebb30e8073c95749d55538ed473130ec0c7"}, + {file = "MarkupSafe-2.1.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:e04e26803c9c3851c931eac40c695602c6295b8d432cbe78609649ad9bd2da8a"}, + {file = "MarkupSafe-2.1.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:b87db4360013327109564f0e591bd2a3b318547bcef31b468a92ee504d07ae4f"}, + {file = "MarkupSafe-2.1.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:99a2a507ed3ac881b975a2976d59f38c19386d128e7a9a18b7df6fff1fd4c1d6"}, + {file = "MarkupSafe-2.1.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:56442863ed2b06d19c37f94d999035e15ee982988920e12a5b4ba29b62ad1f77"}, + {file = "MarkupSafe-2.1.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3ce11ee3f23f79dbd06fb3d63e2f6af7b12db1d46932fe7bd8afa259a5996603"}, + {file = "MarkupSafe-2.1.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:33b74d289bd2f5e527beadcaa3f401e0df0a89927c1559c8566c066fa4248ab7"}, + {file = "MarkupSafe-2.1.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:43093fb83d8343aac0b1baa75516da6092f58f41200907ef92448ecab8825135"}, + {file = "MarkupSafe-2.1.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8e3dcf21f367459434c18e71b2a9532d96547aef8a871872a5bd69a715c15f96"}, + {file = "MarkupSafe-2.1.1-cp39-cp39-win32.whl", hash = "sha256:d4306c36ca495956b6d568d276ac11fdd9c30a36f1b6eb928070dc5360b22e1c"}, + {file = "MarkupSafe-2.1.1-cp39-cp39-win_amd64.whl", hash = "sha256:46d00d6cfecdde84d40e572d63735ef81423ad31184100411e6e3388d405e247"}, + {file = "MarkupSafe-2.1.1.tar.gz", hash = "sha256:7f91197cc9e48f989d12e4e6fbc46495c446636dfc81b9ccf50bb0ec74b91d4b"}, +] +mccabe = [ + {file = "mccabe-0.7.0-py2.py3-none-any.whl", hash = "sha256:6c2d30ab6be0e4a46919781807b4f0d834ebdd6c6e3dca0bda5a15f863427b6e"}, + {file = "mccabe-0.7.0.tar.gz", hash = "sha256:348e0240c33b60bbdf4e523192ef919f28cb2c3d7d5c7794f74009290f236325"}, +] +packaging = [ + {file = "packaging-21.3-py3-none-any.whl", hash = "sha256:ef103e05f519cdc783ae24ea4e2e0f508a9c99b2d4969652eed6a2e1ea5bd522"}, + {file = "packaging-21.3.tar.gz", hash = "sha256:dd47c42927d89ab911e606518907cc2d3a1f38bbd026385970643f9c5b8ecfeb"}, +] +pendulum = [ + {file = "pendulum-2.1.2-cp27-cp27m-macosx_10_15_x86_64.whl", hash = "sha256:b6c352f4bd32dff1ea7066bd31ad0f71f8d8100b9ff709fb343f3b86cee43efe"}, + {file = "pendulum-2.1.2-cp27-cp27m-win_amd64.whl", hash = "sha256:318f72f62e8e23cd6660dbafe1e346950281a9aed144b5c596b2ddabc1d19739"}, + {file = "pendulum-2.1.2-cp35-cp35m-macosx_10_15_x86_64.whl", hash = "sha256:0731f0c661a3cb779d398803655494893c9f581f6488048b3fb629c2342b5394"}, + {file = "pendulum-2.1.2-cp35-cp35m-manylinux1_i686.whl", hash = "sha256:3481fad1dc3f6f6738bd575a951d3c15d4b4ce7c82dce37cf8ac1483fde6e8b0"}, + {file = "pendulum-2.1.2-cp35-cp35m-manylinux1_x86_64.whl", hash = "sha256:9702069c694306297ed362ce7e3c1ef8404ac8ede39f9b28b7c1a7ad8c3959e3"}, + {file = "pendulum-2.1.2-cp35-cp35m-win_amd64.whl", hash = "sha256:fb53ffa0085002ddd43b6ca61a7b34f2d4d7c3ed66f931fe599e1a531b42af9b"}, + {file = "pendulum-2.1.2-cp36-cp36m-macosx_10_15_x86_64.whl", hash = "sha256:c501749fdd3d6f9e726086bf0cd4437281ed47e7bca132ddb522f86a1645d360"}, + {file = "pendulum-2.1.2-cp36-cp36m-manylinux1_i686.whl", hash = "sha256:c807a578a532eeb226150d5006f156632df2cc8c5693d778324b43ff8c515dd0"}, + {file = "pendulum-2.1.2-cp36-cp36m-manylinux1_x86_64.whl", hash = "sha256:2d1619a721df661e506eff8db8614016f0720ac171fe80dda1333ee44e684087"}, + {file = "pendulum-2.1.2-cp36-cp36m-win_amd64.whl", hash = "sha256:f888f2d2909a414680a29ae74d0592758f2b9fcdee3549887779cd4055e975db"}, + {file = "pendulum-2.1.2-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:e95d329384717c7bf627bf27e204bc3b15c8238fa8d9d9781d93712776c14002"}, + {file = "pendulum-2.1.2-cp37-cp37m-manylinux1_i686.whl", hash = "sha256:4c9c689747f39d0d02a9f94fcee737b34a5773803a64a5fdb046ee9cac7442c5"}, + {file = "pendulum-2.1.2-cp37-cp37m-manylinux1_x86_64.whl", hash = "sha256:1245cd0075a3c6d889f581f6325dd8404aca5884dea7223a5566c38aab94642b"}, + {file = "pendulum-2.1.2-cp37-cp37m-win_amd64.whl", hash = "sha256:db0a40d8bcd27b4fb46676e8eb3c732c67a5a5e6bfab8927028224fbced0b40b"}, + {file = "pendulum-2.1.2-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:f5e236e7730cab1644e1b87aca3d2ff3e375a608542e90fe25685dae46310116"}, + {file = "pendulum-2.1.2-cp38-cp38-manylinux1_i686.whl", hash = "sha256:de42ea3e2943171a9e95141f2eecf972480636e8e484ccffaf1e833929e9e052"}, + {file = "pendulum-2.1.2-cp38-cp38-manylinux1_x86_64.whl", hash = "sha256:7c5ec650cb4bec4c63a89a0242cc8c3cebcec92fcfe937c417ba18277d8560be"}, + {file = "pendulum-2.1.2-cp38-cp38-win_amd64.whl", hash = "sha256:33fb61601083f3eb1d15edeb45274f73c63b3c44a8524703dc143f4212bf3269"}, + {file = "pendulum-2.1.2-cp39-cp39-manylinux1_i686.whl", hash = "sha256:29c40a6f2942376185728c9a0347d7c0f07905638c83007e1d262781f1e6953a"}, + {file = "pendulum-2.1.2-cp39-cp39-manylinux1_x86_64.whl", hash = "sha256:94b1fc947bfe38579b28e1cccb36f7e28a15e841f30384b5ad6c5e31055c85d7"}, + {file = "pendulum-2.1.2.tar.gz", hash = "sha256:b06a0ca1bfe41c990bbf0c029f0b6501a7f2ec4e38bfec730712015e8860f207"}, +] +platformdirs = [ + {file = "platformdirs-2.6.2-py3-none-any.whl", hash = "sha256:83c8f6d04389165de7c9b6f0c682439697887bca0aa2f1c87ef1826be3584490"}, + {file = "platformdirs-2.6.2.tar.gz", hash = "sha256:e1fea1fe471b9ff8332e229df3cb7de4f53eeea4998d3b6bfff542115e998bd2"}, +] +pluggy = [ + {file = "pluggy-1.0.0-py2.py3-none-any.whl", hash = "sha256:74134bbf457f031a36d68416e1509f34bd5ccc019f0bcc952c7b909d06b37bd3"}, + {file = "pluggy-1.0.0.tar.gz", hash = "sha256:4224373bacce55f955a878bf9cfa763c1e360858e330072059e10bad68531159"}, +] +protobuf = [ + {file = "protobuf-3.20.3-cp310-cp310-manylinux2014_aarch64.whl", hash = "sha256:f4bd856d702e5b0d96a00ec6b307b0f51c1982c2bf9c0052cf9019e9a544ba99"}, + {file = "protobuf-3.20.3-cp310-cp310-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:9aae4406ea63d825636cc11ffb34ad3379335803216ee3a856787bcf5ccc751e"}, + {file = "protobuf-3.20.3-cp310-cp310-win32.whl", hash = "sha256:28545383d61f55b57cf4df63eebd9827754fd2dc25f80c5253f9184235db242c"}, + {file = "protobuf-3.20.3-cp310-cp310-win_amd64.whl", hash = "sha256:67a3598f0a2dcbc58d02dd1928544e7d88f764b47d4a286202913f0b2801c2e7"}, + {file = "protobuf-3.20.3-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:899dc660cd599d7352d6f10d83c95df430a38b410c1b66b407a6b29265d66469"}, + {file = "protobuf-3.20.3-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:e64857f395505ebf3d2569935506ae0dfc4a15cb80dc25261176c784662cdcc4"}, + {file = "protobuf-3.20.3-cp37-cp37m-manylinux2014_aarch64.whl", hash = "sha256:d9e4432ff660d67d775c66ac42a67cf2453c27cb4d738fc22cb53b5d84c135d4"}, + {file = "protobuf-3.20.3-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:74480f79a023f90dc6e18febbf7b8bac7508420f2006fabd512013c0c238f454"}, + {file = "protobuf-3.20.3-cp37-cp37m-win32.whl", hash = "sha256:b6cc7ba72a8850621bfec987cb72623e703b7fe2b9127a161ce61e61558ad905"}, + {file = "protobuf-3.20.3-cp37-cp37m-win_amd64.whl", hash = "sha256:8c0c984a1b8fef4086329ff8dd19ac77576b384079247c770f29cc8ce3afa06c"}, + {file = "protobuf-3.20.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:de78575669dddf6099a8a0f46a27e82a1783c557ccc38ee620ed8cc96d3be7d7"}, + {file = "protobuf-3.20.3-cp38-cp38-manylinux2014_aarch64.whl", hash = "sha256:f4c42102bc82a51108e449cbb32b19b180022941c727bac0cfd50170341f16ee"}, + {file = "protobuf-3.20.3-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:44246bab5dd4b7fbd3c0c80b6f16686808fab0e4aca819ade6e8d294a29c7050"}, + {file = "protobuf-3.20.3-cp38-cp38-win32.whl", hash = "sha256:c02ce36ec760252242a33967d51c289fd0e1c0e6e5cc9397e2279177716add86"}, + {file = "protobuf-3.20.3-cp38-cp38-win_amd64.whl", hash = "sha256:447d43819997825d4e71bf5769d869b968ce96848b6479397e29fc24c4a5dfe9"}, + {file = "protobuf-3.20.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:398a9e0c3eaceb34ec1aee71894ca3299605fa8e761544934378bbc6c97de23b"}, + {file = "protobuf-3.20.3-cp39-cp39-manylinux2014_aarch64.whl", hash = "sha256:bf01b5720be110540be4286e791db73f84a2b721072a3711efff6c324cdf074b"}, + {file = "protobuf-3.20.3-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:daa564862dd0d39c00f8086f88700fdbe8bc717e993a21e90711acfed02f2402"}, + {file = "protobuf-3.20.3-cp39-cp39-win32.whl", hash = "sha256:819559cafa1a373b7096a482b504ae8a857c89593cf3a25af743ac9ecbd23480"}, + {file = "protobuf-3.20.3-cp39-cp39-win_amd64.whl", hash = "sha256:03038ac1cfbc41aa21f6afcbcd357281d7521b4157926f30ebecc8d4ea59dcb7"}, + {file = "protobuf-3.20.3-py2.py3-none-any.whl", hash = "sha256:a7ca6d488aa8ff7f329d4c545b2dbad8ac31464f1d8b1c87ad1346717731e4db"}, + {file = "protobuf-3.20.3.tar.gz", hash = "sha256:2e3427429c9cffebf259491be0af70189607f365c2f41c7c3764af6f337105f2"}, +] +psutil = [ + {file = "psutil-5.9.4-cp27-cp27m-macosx_10_9_x86_64.whl", hash = "sha256:c1ca331af862803a42677c120aff8a814a804e09832f166f226bfd22b56feee8"}, + {file = "psutil-5.9.4-cp27-cp27m-manylinux2010_i686.whl", hash = "sha256:68908971daf802203f3d37e78d3f8831b6d1014864d7a85937941bb35f09aefe"}, + {file = "psutil-5.9.4-cp27-cp27m-manylinux2010_x86_64.whl", hash = "sha256:3ff89f9b835100a825b14c2808a106b6fdcc4b15483141482a12c725e7f78549"}, + {file = "psutil-5.9.4-cp27-cp27m-win32.whl", hash = "sha256:852dd5d9f8a47169fe62fd4a971aa07859476c2ba22c2254d4a1baa4e10b95ad"}, + {file = "psutil-5.9.4-cp27-cp27m-win_amd64.whl", hash = "sha256:9120cd39dca5c5e1c54b59a41d205023d436799b1c8c4d3ff71af18535728e94"}, + {file = "psutil-5.9.4-cp27-cp27mu-manylinux2010_i686.whl", hash = "sha256:6b92c532979bafc2df23ddc785ed116fced1f492ad90a6830cf24f4d1ea27d24"}, + {file = "psutil-5.9.4-cp27-cp27mu-manylinux2010_x86_64.whl", hash = "sha256:efeae04f9516907be44904cc7ce08defb6b665128992a56957abc9b61dca94b7"}, + {file = "psutil-5.9.4-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:54d5b184728298f2ca8567bf83c422b706200bcbbfafdc06718264f9393cfeb7"}, + {file = "psutil-5.9.4-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:16653106f3b59386ffe10e0bad3bb6299e169d5327d3f187614b1cb8f24cf2e1"}, + {file = "psutil-5.9.4-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:54c0d3d8e0078b7666984e11b12b88af2db11d11249a8ac8920dd5ef68a66e08"}, + {file = "psutil-5.9.4-cp36-abi3-win32.whl", hash = "sha256:149555f59a69b33f056ba1c4eb22bb7bf24332ce631c44a319cec09f876aaeff"}, + {file = "psutil-5.9.4-cp36-abi3-win_amd64.whl", hash = "sha256:fd8522436a6ada7b4aad6638662966de0d61d241cb821239b2ae7013d41a43d4"}, + {file = "psutil-5.9.4-cp38-abi3-macosx_11_0_arm64.whl", hash = "sha256:6001c809253a29599bc0dfd5179d9f8a5779f9dffea1da0f13c53ee568115e1e"}, + {file = "psutil-5.9.4.tar.gz", hash = "sha256:3d7f9739eb435d4b1338944abe23f49584bde5395f27487d2ee25ad9a8774a62"}, +] +pylint = [ + {file = "pylint-2.15.9-py3-none-any.whl", hash = "sha256:349c8cd36aede4d50a0754a8c0218b43323d13d5d88f4b2952ddfe3e169681eb"}, + {file = "pylint-2.15.9.tar.gz", hash = "sha256:18783cca3cfee5b83c6c5d10b3cdb66c6594520ffae61890858fe8d932e1c6b4"}, +] +pyparsing = [ + {file = "pyparsing-3.0.9-py3-none-any.whl", hash = "sha256:5026bae9a10eeaefb61dab2f09052b9f4307d44aee4eda64b309723d8d206bbc"}, + {file = "pyparsing-3.0.9.tar.gz", hash = "sha256:2b020ecf7d21b687f219b71ecad3631f644a47f01403fa1d1036b0c6416d70fb"}, +] +pyreadline3 = [ + {file = "pyreadline3-3.4.1-py3-none-any.whl", hash = "sha256:b0efb6516fd4fb07b45949053826a62fa4cb353db5be2bbb4a7aa1fdd1e345fb"}, + {file = "pyreadline3-3.4.1.tar.gz", hash = "sha256:6f3d1f7b8a31ba32b73917cefc1f28cc660562f39aea8646d30bd6eff21f7bae"}, +] +pytest = [ + {file = "pytest-7.2.0-py3-none-any.whl", hash = "sha256:892f933d339f068883b6fd5a459f03d85bfcb355e4981e146d2c7616c21fef71"}, + {file = "pytest-7.2.0.tar.gz", hash = "sha256:c4014eb40e10f11f355ad4e3c2fb2c6c6d1919c73f3b5a433de4708202cade59"}, +] +python-dateutil = [ + {file = "python-dateutil-2.8.2.tar.gz", hash = "sha256:0123cacc1627ae19ddf3c27a5de5bd67ee4586fbdd6440d9748f8abb483d3e86"}, + {file = "python_dateutil-2.8.2-py2.py3-none-any.whl", hash = "sha256:961d03dc3453ebbc59dbdea9e4e11c5651520a876d0f4db161e8674aae935da9"}, +] +python-dotenv = [ + {file = "python-dotenv-0.21.0.tar.gz", hash = "sha256:b77d08274639e3d34145dfa6c7008e66df0f04b7be7a75fd0d5292c191d79045"}, + {file = "python_dotenv-0.21.0-py3-none-any.whl", hash = "sha256:1684eb44636dd462b66c3ee016599815514527ad99965de77f43e0944634a7e5"}, +] +pytz = [ + {file = "pytz-2022.7-py2.py3-none-any.whl", hash = "sha256:93007def75ae22f7cd991c84e02d434876818661f8df9ad5df9e950ff4e52cfd"}, + {file = "pytz-2022.7.tar.gz", hash = "sha256:7ccfae7b4b2c067464a6733c6261673fdb8fd1be905460396b97a073e9fa683a"}, +] +pytzdata = [ + {file = "pytzdata-2020.1-py2.py3-none-any.whl", hash = "sha256:e1e14750bcf95016381e4d472bad004eef710f2d6417240904070b3d6654485f"}, + {file = "pytzdata-2020.1.tar.gz", hash = "sha256:3efa13b335a00a8de1d345ae41ec78dd11c9f8807f522d39850f2dd828681540"}, +] +pywin32 = [ + {file = "pywin32-305-cp310-cp310-win32.whl", hash = "sha256:421f6cd86e84bbb696d54563c48014b12a23ef95a14e0bdba526be756d89f116"}, + {file = "pywin32-305-cp310-cp310-win_amd64.whl", hash = "sha256:73e819c6bed89f44ff1d690498c0a811948f73777e5f97c494c152b850fad478"}, + {file = "pywin32-305-cp310-cp310-win_arm64.whl", hash = "sha256:742eb905ce2187133a29365b428e6c3b9001d79accdc30aa8969afba1d8470f4"}, + {file = "pywin32-305-cp311-cp311-win32.whl", hash = "sha256:19ca459cd2e66c0e2cc9a09d589f71d827f26d47fe4a9d09175f6aa0256b51c2"}, + {file = "pywin32-305-cp311-cp311-win_amd64.whl", hash = "sha256:326f42ab4cfff56e77e3e595aeaf6c216712bbdd91e464d167c6434b28d65990"}, + {file = "pywin32-305-cp311-cp311-win_arm64.whl", hash = "sha256:4ecd404b2c6eceaca52f8b2e3e91b2187850a1ad3f8b746d0796a98b4cea04db"}, + {file = "pywin32-305-cp36-cp36m-win32.whl", hash = "sha256:48d8b1659284f3c17b68587af047d110d8c44837736b8932c034091683e05863"}, + {file = "pywin32-305-cp36-cp36m-win_amd64.whl", hash = "sha256:13362cc5aa93c2beaf489c9c9017c793722aeb56d3e5166dadd5ef82da021fe1"}, + {file = "pywin32-305-cp37-cp37m-win32.whl", hash = "sha256:a55db448124d1c1484df22fa8bbcbc45c64da5e6eae74ab095b9ea62e6d00496"}, + {file = "pywin32-305-cp37-cp37m-win_amd64.whl", hash = "sha256:109f98980bfb27e78f4df8a51a8198e10b0f347257d1e265bb1a32993d0c973d"}, + {file = "pywin32-305-cp38-cp38-win32.whl", hash = "sha256:9dd98384da775afa009bc04863426cb30596fd78c6f8e4e2e5bbf4edf8029504"}, + {file = "pywin32-305-cp38-cp38-win_amd64.whl", hash = "sha256:56d7a9c6e1a6835f521788f53b5af7912090674bb84ef5611663ee1595860fc7"}, + {file = "pywin32-305-cp39-cp39-win32.whl", hash = "sha256:9d968c677ac4d5cbdaa62fd3014ab241718e619d8e36ef8e11fb930515a1e918"}, + {file = "pywin32-305-cp39-cp39-win_amd64.whl", hash = "sha256:50768c6b7c3f0b38b7fb14dd4104da93ebced5f1a50dc0e834594bff6fbe1271"}, +] +pyyaml = [ + {file = "PyYAML-6.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d4db7c7aef085872ef65a8fd7d6d09a14ae91f691dec3e87ee5ee0539d516f53"}, + {file = "PyYAML-6.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9df7ed3b3d2e0ecfe09e14741b857df43adb5a3ddadc919a2d94fbdf78fea53c"}, + {file = "PyYAML-6.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:77f396e6ef4c73fdc33a9157446466f1cff553d979bd00ecb64385760c6babdc"}, + {file = "PyYAML-6.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a80a78046a72361de73f8f395f1f1e49f956c6be882eed58505a15f3e430962b"}, + {file = "PyYAML-6.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:f84fbc98b019fef2ee9a1cb3ce93e3187a6df0b2538a651bfb890254ba9f90b5"}, + {file = "PyYAML-6.0-cp310-cp310-win32.whl", hash = "sha256:2cd5df3de48857ed0544b34e2d40e9fac445930039f3cfe4bcc592a1f836d513"}, + {file = "PyYAML-6.0-cp310-cp310-win_amd64.whl", hash = "sha256:daf496c58a8c52083df09b80c860005194014c3698698d1a57cbcfa182142a3a"}, + {file = "PyYAML-6.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:d4b0ba9512519522b118090257be113b9468d804b19d63c71dbcf4a48fa32358"}, + {file = "PyYAML-6.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:81957921f441d50af23654aa6c5e5eaf9b06aba7f0a19c18a538dc7ef291c5a1"}, + {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:afa17f5bc4d1b10afd4466fd3a44dc0e245382deca5b3c353d8b757f9e3ecb8d"}, + {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:dbad0e9d368bb989f4515da330b88a057617d16b6a8245084f1b05400f24609f"}, + {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:432557aa2c09802be39460360ddffd48156e30721f5e8d917f01d31694216782"}, + {file = "PyYAML-6.0-cp311-cp311-win32.whl", hash = "sha256:bfaef573a63ba8923503d27530362590ff4f576c626d86a9fed95822a8255fd7"}, + {file = "PyYAML-6.0-cp311-cp311-win_amd64.whl", hash = "sha256:01b45c0191e6d66c470b6cf1b9531a771a83c1c4208272ead47a3ae4f2f603bf"}, + {file = "PyYAML-6.0-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:897b80890765f037df3403d22bab41627ca8811ae55e9a722fd0392850ec4d86"}, + {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:50602afada6d6cbfad699b0c7bb50d5ccffa7e46a3d738092afddc1f9758427f"}, + {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:48c346915c114f5fdb3ead70312bd042a953a8ce5c7106d5bfb1a5254e47da92"}, + {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:98c4d36e99714e55cfbaaee6dd5badbc9a1ec339ebfc3b1f52e293aee6bb71a4"}, + {file = "PyYAML-6.0-cp36-cp36m-win32.whl", hash = "sha256:0283c35a6a9fbf047493e3a0ce8d79ef5030852c51e9d911a27badfde0605293"}, + {file = "PyYAML-6.0-cp36-cp36m-win_amd64.whl", hash = "sha256:07751360502caac1c067a8132d150cf3d61339af5691fe9e87803040dbc5db57"}, + {file = "PyYAML-6.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:819b3830a1543db06c4d4b865e70ded25be52a2e0631ccd2f6a47a2822f2fd7c"}, + {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:473f9edb243cb1935ab5a084eb238d842fb8f404ed2193a915d1784b5a6b5fc0"}, + {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0ce82d761c532fe4ec3f87fc45688bdd3a4c1dc5e0b4a19814b9009a29baefd4"}, + {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:231710d57adfd809ef5d34183b8ed1eeae3f76459c18fb4a0b373ad56bedcdd9"}, + {file = "PyYAML-6.0-cp37-cp37m-win32.whl", hash = "sha256:c5687b8d43cf58545ade1fe3e055f70eac7a5a1a0bf42824308d868289a95737"}, + {file = "PyYAML-6.0-cp37-cp37m-win_amd64.whl", hash = "sha256:d15a181d1ecd0d4270dc32edb46f7cb7733c7c508857278d3d378d14d606db2d"}, + {file = "PyYAML-6.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:0b4624f379dab24d3725ffde76559cff63d9ec94e1736b556dacdfebe5ab6d4b"}, + {file = "PyYAML-6.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:213c60cd50106436cc818accf5baa1aba61c0189ff610f64f4a3e8c6726218ba"}, + {file = "PyYAML-6.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9fa600030013c4de8165339db93d182b9431076eb98eb40ee068700c9c813e34"}, + {file = "PyYAML-6.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:277a0ef2981ca40581a47093e9e2d13b3f1fbbeffae064c1d21bfceba2030287"}, + {file = "PyYAML-6.0-cp38-cp38-win32.whl", hash = "sha256:d4eccecf9adf6fbcc6861a38015c2a64f38b9d94838ac1810a9023a0609e1b78"}, + {file = "PyYAML-6.0-cp38-cp38-win_amd64.whl", hash = "sha256:1e4747bc279b4f613a09eb64bba2ba602d8a6664c6ce6396a4d0cd413a50ce07"}, + {file = "PyYAML-6.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:055d937d65826939cb044fc8c9b08889e8c743fdc6a32b33e2390f66013e449b"}, + {file = "PyYAML-6.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:e61ceaab6f49fb8bdfaa0f92c4b57bcfbea54c09277b1b4f7ac376bfb7a7c174"}, + {file = "PyYAML-6.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d67d839ede4ed1b28a4e8909735fc992a923cdb84e618544973d7dfc71540803"}, + {file = "PyYAML-6.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:cba8c411ef271aa037d7357a2bc8f9ee8b58b9965831d9e51baf703280dc73d3"}, + {file = "PyYAML-6.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:40527857252b61eacd1d9af500c3337ba8deb8fc298940291486c465c8b46ec0"}, + {file = "PyYAML-6.0-cp39-cp39-win32.whl", hash = "sha256:b5b9eccad747aabaaffbc6064800670f0c297e52c12754eb1d976c57e4f74dcb"}, + {file = "PyYAML-6.0-cp39-cp39-win_amd64.whl", hash = "sha256:b3d267842bf12586ba6c734f89d1f5b871df0273157918b0ccefa29deb05c21c"}, + {file = "PyYAML-6.0.tar.gz", hash = "sha256:68fb519c14306fec9720a2a5b45bc9f0c8d1b9c72adf45c37baedfcd949c35a2"}, +] +requests = [ + {file = "requests-2.28.1-py3-none-any.whl", hash = "sha256:8fefa2a1a1365bf5520aac41836fbee479da67864514bdb821f31ce07ce65349"}, + {file = "requests-2.28.1.tar.gz", hash = "sha256:7c5599b102feddaa661c826c56ab4fee28bfd17f5abca1ebbe3e7f19d7c97983"}, +] +setuptools = [ + {file = "setuptools-65.6.3-py3-none-any.whl", hash = "sha256:57f6f22bde4e042978bcd50176fdb381d7c21a9efa4041202288d3737a0c6a54"}, + {file = "setuptools-65.6.3.tar.gz", hash = "sha256:a7620757bf984b58deaf32fc8a4577a9bbc0850cf92c20e1ce41c38c19e5fb75"}, +] +six = [ + {file = "six-1.16.0-py2.py3-none-any.whl", hash = "sha256:8abb2f1d86890a2dfb989f9a77cfcfd3e47c2a354b01111771326f8aa26e0254"}, + {file = "six-1.16.0.tar.gz", hash = "sha256:1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926"}, +] +sqlalchemy = [ + {file = "SQLAlchemy-1.4.45-cp27-cp27m-macosx_10_14_x86_64.whl", hash = "sha256:f1d3fb02a4d0b07d1351a4a52f159e5e7b3045c903468b7e9349ebf0020ffdb9"}, + {file = "SQLAlchemy-1.4.45-cp27-cp27m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:9b7025d46aba946272f6b6b357a22f3787473ef27451f342df1a2a6de23743e3"}, + {file = "SQLAlchemy-1.4.45-cp27-cp27m-win32.whl", hash = "sha256:26b8424b32eeefa4faad21decd7bdd4aade58640b39407bf43e7d0a7c1bc0453"}, + {file = "SQLAlchemy-1.4.45-cp27-cp27m-win_amd64.whl", hash = "sha256:13578d1cda69bc5e76c59fec9180d6db7ceb71c1360a4d7861c37d87ea6ca0b1"}, + {file = "SQLAlchemy-1.4.45-cp27-cp27mu-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:6cd53b4c756a6f9c6518a3dc9c05a38840f9ae442c91fe1abde50d73651b6922"}, + {file = "SQLAlchemy-1.4.45-cp310-cp310-macosx_10_15_x86_64.whl", hash = "sha256:ca152ffc7f0aa069c95fba46165030267ec5e4bb0107aba45e5e9e86fe4d9363"}, + {file = "SQLAlchemy-1.4.45-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:06055476d38ed7915eeed22b78580556d446d175c3574a01b9eb04d91f3a8b2e"}, + {file = "SQLAlchemy-1.4.45-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:081e2a2d75466353c738ca2ee71c0cfb08229b4f9909b5fa085f75c48d021471"}, + {file = "SQLAlchemy-1.4.45-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:96821d806c0c90c68ce3f2ce6dd529c10e5d7587961f31dd5c30e3bfddc4545d"}, + {file = "SQLAlchemy-1.4.45-cp310-cp310-win32.whl", hash = "sha256:c8051bff4ce48cbc98f11e95ac46bfd1e36272401070c010248a3230d099663f"}, + {file = "SQLAlchemy-1.4.45-cp310-cp310-win_amd64.whl", hash = "sha256:16ad798fc121cad5ea019eb2297127b08c54e1aa95fe17b3fea9fdbc5c34fe62"}, + {file = "SQLAlchemy-1.4.45-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:099efef0de9fbda4c2d7cb129e4e7f812007901942259d4e6c6e19bd69de1088"}, + {file = "SQLAlchemy-1.4.45-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:29a29d02c9e6f6b105580c5ed7afb722b97bc2e2fdb85e1d45d7ddd8440cfbca"}, + {file = "SQLAlchemy-1.4.45-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dc10423b59d6d032d6dff0bb42aa06dc6a8824eb6029d70c7d1b6981a2e7f4d8"}, + {file = "SQLAlchemy-1.4.45-cp311-cp311-win32.whl", hash = "sha256:1a92685db3b0682776a5abcb5f9e9addb3d7d9a6d841a452a17ec2d8d457bea7"}, + {file = "SQLAlchemy-1.4.45-cp311-cp311-win_amd64.whl", hash = "sha256:db3ccbce4a861bf4338b254f95916fc68dd8b7aa50eea838ecdaf3a52810e9c0"}, + {file = "SQLAlchemy-1.4.45-cp36-cp36m-macosx_10_14_x86_64.whl", hash = "sha256:a62ae2ea3b940ce9c9cbd675489c2047921ce0a79f971d3082978be91bd58117"}, + {file = "SQLAlchemy-1.4.45-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a87f8595390764db333a1705591d0934973d132af607f4fa8b792b366eacbb3c"}, + {file = "SQLAlchemy-1.4.45-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:9a21c1fb71c69c8ec65430160cd3eee44bbcea15b5a4e556f29d03f246f425ec"}, + {file = "SQLAlchemy-1.4.45-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f7944b04e6fcf8d733964dd9ee36b6a587251a1a4049af3a9b846f6e64eb349a"}, + {file = "SQLAlchemy-1.4.45-cp36-cp36m-win32.whl", hash = "sha256:a3bcd5e2049ceb97e8c273e6a84ff4abcfa1dc47b6d8bbd36e07cce7176610d3"}, + {file = "SQLAlchemy-1.4.45-cp36-cp36m-win_amd64.whl", hash = "sha256:5953e225be47d80410ae519f865b5c341f541d8e383fb6d11f67fb71a45bf890"}, + {file = "SQLAlchemy-1.4.45-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:6a91b7883cb7855a27bc0637166eed622fdf1bb94a4d1630165e5dd88c7e64d3"}, + {file = "SQLAlchemy-1.4.45-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d458fd0566bc9e10b8be857f089e96b5ca1b1ef033226f24512f9ffdf485a8c0"}, + {file = "SQLAlchemy-1.4.45-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:88f4ad3b081c0dbb738886f8d425a5d983328670ee83b38192687d78fc82bd1e"}, + {file = "SQLAlchemy-1.4.45-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cd95a3e6ab46da2c5b0703e797a772f3fab44d085b3919a4f27339aa3b1f51d3"}, + {file = "SQLAlchemy-1.4.45-cp37-cp37m-win32.whl", hash = "sha256:715f5859daa3bee6ecbad64501637fa4640ca6734e8cda6135e3898d5f8ccadd"}, + {file = "SQLAlchemy-1.4.45-cp37-cp37m-win_amd64.whl", hash = "sha256:2d1539fbc82d2206380a86d6d7d0453764fdca5d042d78161bbfb8dd047c80ec"}, + {file = "SQLAlchemy-1.4.45-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:01aa76f324c9bbc0dcb2bc3d9e2a9d7ede4808afa1c38d40d5e2007e3163b206"}, + {file = "SQLAlchemy-1.4.45-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:416fe7d228937bd37990b5a429fd00ad0e49eabcea3455af7beed7955f192edd"}, + {file = "SQLAlchemy-1.4.45-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:7e32ce2584564d9e068bb7e0ccd1810cbb0a824c0687f8016fe67e97c345a637"}, + {file = "SQLAlchemy-1.4.45-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:561605cfc26273825ed2fb8484428faf36e853c13e4c90c61c58988aeccb34ed"}, + {file = "SQLAlchemy-1.4.45-cp38-cp38-win32.whl", hash = "sha256:55ddb5585129c5d964a537c9e32a8a68a8c6293b747f3fa164e1c034e1657a98"}, + {file = "SQLAlchemy-1.4.45-cp38-cp38-win_amd64.whl", hash = "sha256:445914dcadc0b623bd9851260ee54915ecf4e3041a62d57709b18a0eed19f33b"}, + {file = "SQLAlchemy-1.4.45-cp39-cp39-macosx_10_15_x86_64.whl", hash = "sha256:2db887dbf05bcc3151de1c4b506b14764c6240a42e844b4269132a7584de1e5f"}, + {file = "SQLAlchemy-1.4.45-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:52b90c9487e4449ad954624d01dea34c90cd8c104bce46b322c83654f37a23c5"}, + {file = "SQLAlchemy-1.4.45-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:f61e54b8c2b389de1a8ad52394729c478c67712dbdcdadb52c2575e41dae94a5"}, + {file = "SQLAlchemy-1.4.45-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e91a5e45a2ea083fe344b3503405978dff14d60ef3aa836432c9ca8cd47806b6"}, + {file = "SQLAlchemy-1.4.45-cp39-cp39-win32.whl", hash = "sha256:0e068b8414d60dd35d43c693555fc3d2e1d822cef07960bb8ca3f1ee6c4ff762"}, + {file = "SQLAlchemy-1.4.45-cp39-cp39-win_amd64.whl", hash = "sha256:2d6f178ff2923730da271c8aa317f70cf0df11a4d1812f1d7a704b1cf29c5fe3"}, + {file = "SQLAlchemy-1.4.45.tar.gz", hash = "sha256:fd69850860093a3f69fefe0ab56d041edfdfe18510b53d9a2eaecba2f15fa795"}, +] +tabulate = [ + {file = "tabulate-0.9.0-py3-none-any.whl", hash = "sha256:024ca478df22e9340661486f85298cff5f6dcdba14f3813e8830015b9ed1948f"}, + {file = "tabulate-0.9.0.tar.gz", hash = "sha256:0095b12bf5966de529c0feb1fa08671671b3368eec77d7ef7ab114be2c068b3c"}, +] +tomli = [ + {file = "tomli-2.0.1-py3-none-any.whl", hash = "sha256:939de3e7a6161af0c887ef91b7d41a53e7c5a1ca976325f429cb46ea9bc30ecc"}, + {file = "tomli-2.0.1.tar.gz", hash = "sha256:de526c12914f0c550d15924c62d72abc48d6fe7364aa87328337a31007fe8a4f"}, +] +tomlkit = [ + {file = "tomlkit-0.11.6-py3-none-any.whl", hash = "sha256:07de26b0d8cfc18f871aec595fda24d95b08fef89d147caa861939f37230bf4b"}, + {file = "tomlkit-0.11.6.tar.gz", hash = "sha256:71b952e5721688937fb02cf9d354dbcf0785066149d2855e44531ebdd2b65d73"}, +] +toposort = [ + {file = "toposort-1.7-py2.py3-none-any.whl", hash = "sha256:8ed8e109e96ae30bf66da2d2155e4eb9989d9c5c743c837e37d9774a4eddd804"}, + {file = "toposort-1.7.tar.gz", hash = "sha256:ddc2182c42912a440511bd7ff5d3e6a1cabc3accbc674a3258c8c41cbfbb2125"}, +] +tqdm = [ + {file = "tqdm-4.64.1-py2.py3-none-any.whl", hash = "sha256:6fee160d6ffcd1b1c68c65f14c829c22832bc401726335ce92c52d395944a6a1"}, + {file = "tqdm-4.64.1.tar.gz", hash = "sha256:5f4f682a004951c1b450bc753c710e9280c5746ce6ffedee253ddbcbf54cf1e4"}, +] +typing-extensions = [ + {file = "typing_extensions-4.4.0-py3-none-any.whl", hash = "sha256:16fa4864408f655d35ec496218b85f79b3437c829e93320c7c9215ccfd92489e"}, + {file = "typing_extensions-4.4.0.tar.gz", hash = "sha256:1511434bb92bf8dd198c12b1cc812e800d4181cfcb867674e0f8279cc93087aa"}, +] +universal-pathlib = [ + {file = "universal_pathlib-0.0.21-py3-none-any.whl", hash = "sha256:3ffba2574999ca6aca8c9f4f224499c6571c92cee612cd923a9e909bf23f586b"}, + {file = "universal_pathlib-0.0.21.tar.gz", hash = "sha256:ed18290f2ded33481a754aac3da94fb6bf78f628027b10c3e95ceb6075415e69"}, +] +urllib3 = [ + {file = "urllib3-1.26.13-py2.py3-none-any.whl", hash = "sha256:47cc05d99aaa09c9e72ed5809b60e7ba354e64b59c9c173ac3018642d8bb41fc"}, + {file = "urllib3-1.26.13.tar.gz", hash = "sha256:c083dd0dce68dbfbe1129d5271cb90f9447dea7d52097c6e0126120c521ddea8"}, +] +watchdog = [ + {file = "watchdog-2.2.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:ed91c3ccfc23398e7aa9715abf679d5c163394b8cad994f34f156d57a7c163dc"}, + {file = "watchdog-2.2.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:76a2743402b794629a955d96ea2e240bd0e903aa26e02e93cd2d57b33900962b"}, + {file = "watchdog-2.2.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:920a4bda7daa47545c3201a3292e99300ba81ca26b7569575bd086c865889090"}, + {file = "watchdog-2.2.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:ceaa9268d81205876bedb1069f9feab3eccddd4b90d9a45d06a0df592a04cae9"}, + {file = "watchdog-2.2.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:1893d425ef4fb4f129ee8ef72226836619c2950dd0559bba022b0818c63a7b60"}, + {file = "watchdog-2.2.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:9e99c1713e4436d2563f5828c8910e5ff25abd6ce999e75f15c15d81d41980b6"}, + {file = "watchdog-2.2.0-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:a5bd9e8656d07cae89ac464ee4bcb6f1b9cecbedc3bf1334683bed3d5afd39ba"}, + {file = "watchdog-2.2.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:3a048865c828389cb06c0bebf8a883cec3ae58ad3e366bcc38c61d8455a3138f"}, + {file = "watchdog-2.2.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:e722755d995035dd32177a9c633d158f2ec604f2a358b545bba5bed53ab25bca"}, + {file = "watchdog-2.2.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:af4b5c7ba60206759a1d99811b5938ca666ea9562a1052b410637bb96ff97512"}, + {file = "watchdog-2.2.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:619d63fa5be69f89ff3a93e165e602c08ed8da402ca42b99cd59a8ec115673e1"}, + {file = "watchdog-2.2.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:1f2b0665c57358ce9786f06f5475bc083fea9d81ecc0efa4733fd0c320940a37"}, + {file = "watchdog-2.2.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:441024df19253bb108d3a8a5de7a186003d68564084576fecf7333a441271ef7"}, + {file = "watchdog-2.2.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:1a410dd4d0adcc86b4c71d1317ba2ea2c92babaf5b83321e4bde2514525544d5"}, + {file = "watchdog-2.2.0-pp37-pypy37_pp73-macosx_10_9_x86_64.whl", hash = "sha256:28704c71afdb79c3f215c90231e41c52b056ea880b6be6cee035c6149d658ed1"}, + {file = "watchdog-2.2.0-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:2ac0bd7c206bb6df78ef9e8ad27cc1346f2b41b1fef610395607319cdab89bc1"}, + {file = "watchdog-2.2.0-pp39-pypy39_pp73-macosx_10_9_x86_64.whl", hash = "sha256:27e49268735b3c27310883012ab3bd86ea0a96dcab90fe3feb682472e30c90f3"}, + {file = "watchdog-2.2.0-py3-none-manylinux2014_aarch64.whl", hash = "sha256:2af1a29fd14fc0a87fb6ed762d3e1ae5694dcde22372eebba50e9e5be47af03c"}, + {file = "watchdog-2.2.0-py3-none-manylinux2014_armv7l.whl", hash = "sha256:c7bd98813d34bfa9b464cf8122e7d4bec0a5a427399094d2c17dd5f70d59bc61"}, + {file = "watchdog-2.2.0-py3-none-manylinux2014_i686.whl", hash = "sha256:56fb3f40fc3deecf6e518303c7533f5e2a722e377b12507f6de891583f1b48aa"}, + {file = "watchdog-2.2.0-py3-none-manylinux2014_ppc64.whl", hash = "sha256:74535e955359d79d126885e642d3683616e6d9ab3aae0e7dcccd043bd5a3ff4f"}, + {file = "watchdog-2.2.0-py3-none-manylinux2014_ppc64le.whl", hash = "sha256:cf05e6ff677b9655c6e9511d02e9cc55e730c4e430b7a54af9c28912294605a4"}, + {file = "watchdog-2.2.0-py3-none-manylinux2014_s390x.whl", hash = "sha256:d6ae890798a3560688b441ef086bb66e87af6b400a92749a18b856a134fc0318"}, + {file = "watchdog-2.2.0-py3-none-manylinux2014_x86_64.whl", hash = "sha256:e5aed2a700a18c194c39c266900d41f3db0c1ebe6b8a0834b9995c835d2ca66e"}, + {file = "watchdog-2.2.0-py3-none-win32.whl", hash = "sha256:d0fb5f2b513556c2abb578c1066f5f467d729f2eb689bc2db0739daf81c6bb7e"}, + {file = "watchdog-2.2.0-py3-none-win_amd64.whl", hash = "sha256:1f8eca9d294a4f194ce9df0d97d19b5598f310950d3ac3dd6e8d25ae456d4c8a"}, + {file = "watchdog-2.2.0-py3-none-win_ia64.whl", hash = "sha256:ad0150536469fa4b693531e497ffe220d5b6cd76ad2eda474a5e641ee204bbb6"}, + {file = "watchdog-2.2.0.tar.gz", hash = "sha256:83cf8bc60d9c613b66a4c018051873d6273d9e45d040eed06d6a96241bd8ec01"}, +] +wrapt = [ + {file = "wrapt-1.14.1-cp27-cp27m-macosx_10_9_x86_64.whl", hash = "sha256:1b376b3f4896e7930f1f772ac4b064ac12598d1c38d04907e696cc4d794b43d3"}, + {file = "wrapt-1.14.1-cp27-cp27m-manylinux1_i686.whl", hash = "sha256:903500616422a40a98a5a3c4ff4ed9d0066f3b4c951fa286018ecdf0750194ef"}, + {file = "wrapt-1.14.1-cp27-cp27m-manylinux1_x86_64.whl", hash = "sha256:5a9a0d155deafd9448baff28c08e150d9b24ff010e899311ddd63c45c2445e28"}, + {file = "wrapt-1.14.1-cp27-cp27m-manylinux2010_i686.whl", hash = "sha256:ddaea91abf8b0d13443f6dac52e89051a5063c7d014710dcb4d4abb2ff811a59"}, + {file = "wrapt-1.14.1-cp27-cp27m-manylinux2010_x86_64.whl", hash = "sha256:36f582d0c6bc99d5f39cd3ac2a9062e57f3cf606ade29a0a0d6b323462f4dd87"}, + {file = "wrapt-1.14.1-cp27-cp27mu-manylinux1_i686.whl", hash = "sha256:7ef58fb89674095bfc57c4069e95d7a31cfdc0939e2a579882ac7d55aadfd2a1"}, + {file = "wrapt-1.14.1-cp27-cp27mu-manylinux1_x86_64.whl", hash = "sha256:e2f83e18fe2f4c9e7db597e988f72712c0c3676d337d8b101f6758107c42425b"}, + {file = "wrapt-1.14.1-cp27-cp27mu-manylinux2010_i686.whl", hash = "sha256:ee2b1b1769f6707a8a445162ea16dddf74285c3964f605877a20e38545c3c462"}, + {file = "wrapt-1.14.1-cp27-cp27mu-manylinux2010_x86_64.whl", hash = "sha256:833b58d5d0b7e5b9832869f039203389ac7cbf01765639c7309fd50ef619e0b1"}, + {file = "wrapt-1.14.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:80bb5c256f1415f747011dc3604b59bc1f91c6e7150bd7db03b19170ee06b320"}, + {file = "wrapt-1.14.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:07f7a7d0f388028b2df1d916e94bbb40624c59b48ecc6cbc232546706fac74c2"}, + {file = "wrapt-1.14.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:02b41b633c6261feff8ddd8d11c711df6842aba629fdd3da10249a53211a72c4"}, + {file = "wrapt-1.14.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2fe803deacd09a233e4762a1adcea5db5d31e6be577a43352936179d14d90069"}, + {file = "wrapt-1.14.1-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:257fd78c513e0fb5cdbe058c27a0624c9884e735bbd131935fd49e9fe719d310"}, + {file = "wrapt-1.14.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:4fcc4649dc762cddacd193e6b55bc02edca674067f5f98166d7713b193932b7f"}, + {file = "wrapt-1.14.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:11871514607b15cfeb87c547a49bca19fde402f32e2b1c24a632506c0a756656"}, + {file = "wrapt-1.14.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:8ad85f7f4e20964db4daadcab70b47ab05c7c1cf2a7c1e51087bfaa83831854c"}, + {file = "wrapt-1.14.1-cp310-cp310-win32.whl", hash = "sha256:a9a52172be0b5aae932bef82a79ec0a0ce87288c7d132946d645eba03f0ad8a8"}, + {file = "wrapt-1.14.1-cp310-cp310-win_amd64.whl", hash = "sha256:6d323e1554b3d22cfc03cd3243b5bb815a51f5249fdcbb86fda4bf62bab9e164"}, + {file = "wrapt-1.14.1-cp35-cp35m-manylinux1_i686.whl", hash = "sha256:43ca3bbbe97af00f49efb06e352eae40434ca9d915906f77def219b88e85d907"}, + {file = "wrapt-1.14.1-cp35-cp35m-manylinux1_x86_64.whl", hash = "sha256:6b1a564e6cb69922c7fe3a678b9f9a3c54e72b469875aa8018f18b4d1dd1adf3"}, + {file = "wrapt-1.14.1-cp35-cp35m-manylinux2010_i686.whl", hash = "sha256:00b6d4ea20a906c0ca56d84f93065b398ab74b927a7a3dbd470f6fc503f95dc3"}, + {file = "wrapt-1.14.1-cp35-cp35m-manylinux2010_x86_64.whl", hash = "sha256:a85d2b46be66a71bedde836d9e41859879cc54a2a04fad1191eb50c2066f6e9d"}, + {file = "wrapt-1.14.1-cp35-cp35m-win32.whl", hash = "sha256:dbcda74c67263139358f4d188ae5faae95c30929281bc6866d00573783c422b7"}, + {file = "wrapt-1.14.1-cp35-cp35m-win_amd64.whl", hash = "sha256:b21bb4c09ffabfa0e85e3a6b623e19b80e7acd709b9f91452b8297ace2a8ab00"}, + {file = "wrapt-1.14.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:9e0fd32e0148dd5dea6af5fee42beb949098564cc23211a88d799e434255a1f4"}, + {file = "wrapt-1.14.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9736af4641846491aedb3c3f56b9bc5568d92b0692303b5a305301a95dfd38b1"}, + {file = "wrapt-1.14.1-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5b02d65b9ccf0ef6c34cba6cf5bf2aab1bb2f49c6090bafeecc9cd81ad4ea1c1"}, + {file = "wrapt-1.14.1-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:21ac0156c4b089b330b7666db40feee30a5d52634cc4560e1905d6529a3897ff"}, + {file = "wrapt-1.14.1-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:9f3e6f9e05148ff90002b884fbc2a86bd303ae847e472f44ecc06c2cd2fcdb2d"}, + {file = "wrapt-1.14.1-cp36-cp36m-musllinux_1_1_i686.whl", hash = "sha256:6e743de5e9c3d1b7185870f480587b75b1cb604832e380d64f9504a0535912d1"}, + {file = "wrapt-1.14.1-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:d79d7d5dc8a32b7093e81e97dad755127ff77bcc899e845f41bf71747af0c569"}, + {file = "wrapt-1.14.1-cp36-cp36m-win32.whl", hash = "sha256:81b19725065dcb43df02b37e03278c011a09e49757287dca60c5aecdd5a0b8ed"}, + {file = "wrapt-1.14.1-cp36-cp36m-win_amd64.whl", hash = "sha256:b014c23646a467558be7da3d6b9fa409b2c567d2110599b7cf9a0c5992b3b471"}, + {file = "wrapt-1.14.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:88bd7b6bd70a5b6803c1abf6bca012f7ed963e58c68d76ee20b9d751c74a3248"}, + {file = "wrapt-1.14.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b5901a312f4d14c59918c221323068fad0540e34324925c8475263841dbdfe68"}, + {file = "wrapt-1.14.1-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d77c85fedff92cf788face9bfa3ebaa364448ebb1d765302e9af11bf449ca36d"}, + {file = "wrapt-1.14.1-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8d649d616e5c6a678b26d15ece345354f7c2286acd6db868e65fcc5ff7c24a77"}, + {file = "wrapt-1.14.1-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:7d2872609603cb35ca513d7404a94d6d608fc13211563571117046c9d2bcc3d7"}, + {file = "wrapt-1.14.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:ee6acae74a2b91865910eef5e7de37dc6895ad96fa23603d1d27ea69df545015"}, + {file = "wrapt-1.14.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:2b39d38039a1fdad98c87279b48bc5dce2c0ca0d73483b12cb72aa9609278e8a"}, + {file = "wrapt-1.14.1-cp37-cp37m-win32.whl", hash = "sha256:60db23fa423575eeb65ea430cee741acb7c26a1365d103f7b0f6ec412b893853"}, + {file = "wrapt-1.14.1-cp37-cp37m-win_amd64.whl", hash = "sha256:709fe01086a55cf79d20f741f39325018f4df051ef39fe921b1ebe780a66184c"}, + {file = "wrapt-1.14.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:8c0ce1e99116d5ab21355d8ebe53d9460366704ea38ae4d9f6933188f327b456"}, + {file = "wrapt-1.14.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:e3fb1677c720409d5f671e39bac6c9e0e422584e5f518bfd50aa4cbbea02433f"}, + {file = "wrapt-1.14.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:642c2e7a804fcf18c222e1060df25fc210b9c58db7c91416fb055897fc27e8cc"}, + {file = "wrapt-1.14.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7b7c050ae976e286906dd3f26009e117eb000fb2cf3533398c5ad9ccc86867b1"}, + {file = "wrapt-1.14.1-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ef3f72c9666bba2bab70d2a8b79f2c6d2c1a42a7f7e2b0ec83bb2f9e383950af"}, + {file = "wrapt-1.14.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:01c205616a89d09827986bc4e859bcabd64f5a0662a7fe95e0d359424e0e071b"}, + {file = "wrapt-1.14.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:5a0f54ce2c092aaf439813735584b9537cad479575a09892b8352fea5e988dc0"}, + {file = "wrapt-1.14.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:2cf71233a0ed05ccdabe209c606fe0bac7379fdcf687f39b944420d2a09fdb57"}, + {file = "wrapt-1.14.1-cp38-cp38-win32.whl", hash = "sha256:aa31fdcc33fef9eb2552cbcbfee7773d5a6792c137b359e82879c101e98584c5"}, + {file = "wrapt-1.14.1-cp38-cp38-win_amd64.whl", hash = "sha256:d1967f46ea8f2db647c786e78d8cc7e4313dbd1b0aca360592d8027b8508e24d"}, + {file = "wrapt-1.14.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3232822c7d98d23895ccc443bbdf57c7412c5a65996c30442ebe6ed3df335383"}, + {file = "wrapt-1.14.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:988635d122aaf2bdcef9e795435662bcd65b02f4f4c1ae37fbee7401c440b3a7"}, + {file = "wrapt-1.14.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9cca3c2cdadb362116235fdbd411735de4328c61425b0aa9f872fd76d02c4e86"}, + {file = "wrapt-1.14.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d52a25136894c63de15a35bc0bdc5adb4b0e173b9c0d07a2be9d3ca64a332735"}, + {file = "wrapt-1.14.1-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:40e7bc81c9e2b2734ea4bc1aceb8a8f0ceaac7c5299bc5d69e37c44d9081d43b"}, + {file = "wrapt-1.14.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:b9b7a708dd92306328117d8c4b62e2194d00c365f18eff11a9b53c6f923b01e3"}, + {file = "wrapt-1.14.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:6a9a25751acb379b466ff6be78a315e2b439d4c94c1e99cb7266d40a537995d3"}, + {file = "wrapt-1.14.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:34aa51c45f28ba7f12accd624225e2b1e5a3a45206aa191f6f9aac931d9d56fe"}, + {file = "wrapt-1.14.1-cp39-cp39-win32.whl", hash = "sha256:dee0ce50c6a2dd9056c20db781e9c1cfd33e77d2d569f5d1d9321c641bb903d5"}, + {file = "wrapt-1.14.1-cp39-cp39-win_amd64.whl", hash = "sha256:dee60e1de1898bde3b238f18340eec6148986da0455d8ba7848d50470a7a32fb"}, + {file = "wrapt-1.14.1.tar.gz", hash = "sha256:380a85cf89e0e69b7cfbe2ea9f765f004ff419f34194018a6827ac0e3edfed4d"}, +] +zipp = [ + {file = "zipp-3.11.0-py3-none-any.whl", hash = "sha256:83a28fcb75844b5c0cdaf5aa4003c2d728c77e05f5aeabe8e95e56727005fbaa"}, + {file = "zipp-3.11.0.tar.gz", hash = "sha256:a7a22e05929290a67401440b39690ae6563279bced5f314609d9d03798f56766"}, +] diff --git a/pyproject.toml b/pyproject.toml index 643ef14..244213f 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -1,26 +1,24 @@ -[build-system] -requires = [ - "setuptools>=42", - "wheel" -] -build-backend = "setuptools.build_meta" - -[tool.black] -line-length = 100 -exclude = ''' -/( - build - | buck-out - | dist - | _build - | \.eggs - | \.git - | \.hg - | \.mypy_cache - | \.nox - | \.tox - | \.venv - | snapshots -)/ -''' -target_version = ['py36', 'py37', 'py38'] \ No newline at end of file +[tool.poetry] +name = "dagster-meltano" +version = "1.0.0" +description = "A dagster plugin that allows you to run your Meltano project inside Dagster." +authors = ["Jules Huisman"] +license = "Apache 2.0" +packages = [ + { include = "dagster_meltano" } +] + +[tool.poetry.dependencies] +python = "<3.11,>=3.8" +dagster = ">=1.0" + +[tool.poetry.group.dev.dependencies] +pylint = "^2.15.9" +pytest = "^7.2.0" + +[build-system] +requires = ["poetry-core>=1.0.8"] +build-backend = "poetry.core.masonry.api" + +[project.urls] +Homepage = "https://github.com/quantile-development/dagster-meltano" \ No newline at end of file diff --git a/pytest.ini b/pytest.ini deleted file mode 100644 index b4f089a..0000000 --- a/pytest.ini +++ /dev/null @@ -1,5 +0,0 @@ -[pytest] -filterwarnings = - ignore::DeprecationWarning -testpaths = - dagster_meltano/tests \ No newline at end of file diff --git a/setup.py b/setup.py deleted file mode 100644 index 8260a0d..0000000 --- a/setup.py +++ /dev/null @@ -1,28 +0,0 @@ -import setuptools - -with open("README.md", "r", encoding="utf-8") as fh: - long_description = fh.read() - -setuptools.setup( - name="dagster-meltano", - version="0.0.3", - author="Jules Huisman", - author_email="jules.huisman@quantile.nl", - description="A Dagster plugin that allows you to run Meltano in Dagster", - long_description=long_description, - long_description_content_type="text/markdown", - url="https://github.com/quantile-development/dagster-meltano", - project_urls={ - "Bug Tracker": "https://github.com/quantile-development/dagster-meltano/issues", - }, - classifiers=[ - "Programming Language :: Python :: 3", - "License :: OSI Approved :: MIT License", - "Operating System :: OS Independent", - ], - extras_require={ - "development": ["pytest==6.2", "meltano==1.85", "dagit==0.13", "black", "isort", "pylint"] - }, - packages=["dagster_meltano"], - python_requires=">=3.6", -) diff --git a/tests/meltano_test_project/.gitignore b/tests/meltano_test_project/.gitignore new file mode 100644 index 0000000..15e24c3 --- /dev/null +++ b/tests/meltano_test_project/.gitignore @@ -0,0 +1,3 @@ +/venv +/.meltano +.env diff --git a/tests/meltano_test_project/meltano.yml b/tests/meltano_test_project/meltano.yml new file mode 100644 index 0000000..f89f2ff --- /dev/null +++ b/tests/meltano_test_project/meltano.yml @@ -0,0 +1,33 @@ +version: 1 +default_environment: dev +project_id: 1b791a52-034d-4c11-a000-ef0eeed7b573 +send_anonymous_usage_stats: false +plugins: + extractors: + - name: tap-smoke-test + namespace: tap_smoke_test + pip_url: git+https://github.com/meltano/tap-smoke-test.git + config: + streams: + - stream_name: animals + input_filename: https://gitlab.com/meltano/tap-smoke-test/-/raw/main/demo-data/animals-data.jsonl + loaders: + - name: target-jsonl + variant: andyh1203 + pip_url: target-jsonl + config: + destination_path: /tmp + env: + MELTANO_LOAD_SCHEMA: TAP_CARBON_INTENSITY +jobs: + - name: smoke-job + tasks: + - tap-smoke-test target-jsonl +schedules: + - name: daily-smoke-job + interval: "@daily" + job: smoke-job +environments: + - name: dev + - name: staging + - name: prod diff --git a/meltano/load/.gitkeep b/tests/meltano_test_project/orchestrate/.gitkeep similarity index 100% rename from meltano/load/.gitkeep rename to tests/meltano_test_project/orchestrate/.gitkeep diff --git a/tests/meltano_test_project/orchestrate/dagster/repository.py b/tests/meltano_test_project/orchestrate/dagster/repository.py new file mode 100644 index 0000000..c627ebb --- /dev/null +++ b/tests/meltano_test_project/orchestrate/dagster/repository.py @@ -0,0 +1,13 @@ +import os + +from dagster import repository + +from dagster_meltano import load_jobs_from_meltano_project + +MELTANO_PROJECT_DIR = os.getenv("MELTANO_PROJECT_ROOT", os.getcwd()) +MELTANO_BIN = os.getenv("MELTANO_BIN", "meltano") + + +@repository +def meltano_jobs(): + return [load_jobs_from_meltano_project(MELTANO_PROJECT_DIR)] diff --git a/tests/meltano_test_project/output/.gitignore b/tests/meltano_test_project/output/.gitignore new file mode 100644 index 0000000..d6b7ef3 --- /dev/null +++ b/tests/meltano_test_project/output/.gitignore @@ -0,0 +1,2 @@ +* +!.gitignore diff --git a/tests/meltano_test_project/plugins/extractors/tap-carbon-intensity--meltano.lock b/tests/meltano_test_project/plugins/extractors/tap-carbon-intensity--meltano.lock new file mode 100644 index 0000000..e279be9 --- /dev/null +++ b/tests/meltano_test_project/plugins/extractors/tap-carbon-intensity--meltano.lock @@ -0,0 +1,17 @@ +{ + "plugin_type": "extractors", + "name": "tap-carbon-intensity", + "namespace": "tap_carbon_intensity", + "variant": "meltano", + "label": "Carbon Emissions Intensity", + "docs": "https://hub.meltano.com/extractors/tap-carbon-intensity--meltano", + "repo": "https://gitlab.com/meltano/tap-carbon-intensity", + "pip_url": "git+https://gitlab.com/meltano/tap-carbon-intensity.git", + "description": "National Grid ESO's Carbon Emissions Intensity API", + "logo_url": "https://hub.meltano.com/assets/logos/extractors/carbon-intensity.png", + "capabilities": [ + "discover", + "state", + "catalog" + ] +} \ No newline at end of file diff --git a/tests/meltano_test_project/plugins/loaders/target-jsonl--andyh1203.lock b/tests/meltano_test_project/plugins/loaders/target-jsonl--andyh1203.lock new file mode 100644 index 0000000..5825fc4 --- /dev/null +++ b/tests/meltano_test_project/plugins/loaders/target-jsonl--andyh1203.lock @@ -0,0 +1,34 @@ +{ + "plugin_type": "loaders", + "name": "target-jsonl", + "namespace": "target_jsonl", + "variant": "andyh1203", + "label": "JSON Lines (JSONL)", + "docs": "https://hub.meltano.com/loaders/target-jsonl--andyh1203", + "repo": "https://github.com/andyh1203/target-jsonl", + "pip_url": "target-jsonl", + "description": "JSONL loader", + "logo_url": "https://hub.meltano.com/assets/logos/loaders/jsonl.png", + "settings": [ + { + "name": "destination_path", + "kind": "string", + "value": "output", + "label": "Destination Path", + "description": "Sets the destination path the JSONL files are written to, relative\nto the project root.\n\nThe directory needs to exist already, it will not be created\nautomatically.\n\nTo write JSONL files to the project root, set an empty string (`\"\"`).\n" + }, + { + "name": "do_timestamp_file", + "kind": "boolean", + "value": false, + "label": "Include Timestamp in File Names", + "description": "Specifies if the files should get timestamped.\n\nBy default, the resulting file will not have a timestamp in the file name (i.e. `exchange_rate.jsonl`).\n\nIf this option gets set to `true`, the resulting file will have a timestamp associated with it (i.e. `exchange_rate-{timestamp}.jsonl`).\n" + }, + { + "name": "custom_name", + "kind": "string", + "label": "Custom File Name Override", + "description": "Specifies a custom name for the filename, instead of the stream name.\n\nThe file name will be `{custom_name}-{timestamp}.jsonl`, if `do_timestamp_file` is `true`.\nOtherwise the file name will be `{custom_name}.jsonl`.\n\nIf custom name is not provided, the stream name will be used.\n" + } + ] +} \ No newline at end of file diff --git a/tests/test_generation.py b/tests/test_generation.py new file mode 100644 index 0000000..d0ffcda --- /dev/null +++ b/tests/test_generation.py @@ -0,0 +1,41 @@ +from dagster import JobDefinition, ScheduleDefinition + +from dagster_meltano import load_jobs_from_meltano_project + +from pathlib import Path + +MELTANO_PROJECT_TEST_PATH = Path(__file__).parent / "meltano_test_project" + + +def test_job_and_schedule_returned(): + """ + Check if the generator function correctly returns the jobs and schedules. + """ + (job, schedule) = load_jobs_from_meltano_project(MELTANO_PROJECT_TEST_PATH) + + assert isinstance(job, JobDefinition) + assert isinstance(schedule, ScheduleDefinition) + + assert job.name == "smoke_job" + assert schedule.name == "daily_smoke_job" + + +def test_job(): + """ + Run the job using Dagster and check for failures. + """ + (job, _schedule) = load_jobs_from_meltano_project(MELTANO_PROJECT_TEST_PATH) + + job_response = job.execute_in_process() + + assert job_response.success + assert job_response.output_for_node("tap_smoke_test_target_jsonl") == None + + +def test_schedule(): + """ + Make sure the generated cron schedule has the correct interval + """ + (_job, schedule) = load_jobs_from_meltano_project(MELTANO_PROJECT_TEST_PATH) + + assert schedule.cron_schedule == "0 0 * * *" diff --git a/tests/test_meltano_install.py b/tests/test_meltano_install.py new file mode 100644 index 0000000..3c6d211 --- /dev/null +++ b/tests/test_meltano_install.py @@ -0,0 +1,31 @@ +from dagster import job + +from dagster_meltano import meltano_resource, meltano_install_op + +from pathlib import Path + +MELTANO_PROJECT_TEST_PATH = str(Path(__file__).parent / "meltano_test_project") + + +@job(resource_defs={"meltano": meltano_resource}) +def install_job(): + meltano_install_op() + + +def test_meltano_install(): + """ + Check if the meltano install command works + """ + job_response = install_job.execute_in_process( + { + "resources": { + "meltano": { + "config": { + "project_dir": MELTANO_PROJECT_TEST_PATH, + }, + } + } + } + ) + + assert job_response.success diff --git a/tests/test_meltano_run.py b/tests/test_meltano_run.py new file mode 100644 index 0000000..02563c3 --- /dev/null +++ b/tests/test_meltano_run.py @@ -0,0 +1,43 @@ +import os +from dagster import job + +from dagster_meltano import meltano_resource, meltano_run_op + +from pathlib import Path + +MELTANO_PROJECT_TEST_PATH = str(Path(__file__).parent / "meltano_test_project") + + +@job(resource_defs={"meltano": meltano_resource}) +def meltano_run_job(): + meltano_run_op("tap-smoke-test target-jsonl")() + + +def test_meltano_run(): + """ + Check if we can run abitrary `meltano run` commands. + """ + job_response = meltano_run_job.execute_in_process( + { + "resources": { + "meltano": { + "config": { + "project_dir": MELTANO_PROJECT_TEST_PATH, + }, + } + } + } + ) + + assert job_response.success + + +def test_meltano_run_using_env(): + """ + Check if we can run abitrary `meltano run` commands, with the project + root defined using an env variable. + """ + os.environ["MELTANO_PROJECT_ROOT"] = MELTANO_PROJECT_TEST_PATH + job_response = meltano_run_job.execute_in_process() + + assert job_response.success