diff --git a/.dockerignore b/.dockerignore index 1c22d361bab5c..ab7451f2be831 100644 --- a/.dockerignore +++ b/.dockerignore @@ -80,8 +80,6 @@ # Git version is dynamically generated airflow/git_version -# Exclude mode_modules pulled by "yarn" for compilation of www files generated by NPM -airflow/www/node_modules airflow/ui/node_modules airflow/auth/managers/simple/ui/node_modules diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 1079ccb5fd4cf..42fbebe3cce2f 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -26,9 +26,6 @@ /airflow/api_connexion/ @ephraimbuddy @pierrejeambrun /airflow/api_fastapi/ @ephraimbuddy @pierrejeambrun -# WWW -/airflow/www/ @ryanahamilton @ashb @bbovenzi @pierrejeambrun @jscheffl - # UI /airflow/ui/ @bbovenzi @pierrejeambrun @ryanahamilton @jscheffl diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml index e61760c7fcf49..3e185efb9de87 100644 --- a/.github/boring-cyborg.yml +++ b/.github/boring-cyborg.yml @@ -303,7 +303,6 @@ labelPRBasedOnFilePath: - airflow/api_fastapi/**/* - clients/**/* - docs/apache-airflow/stable-rest-api-ref.rst - - tests/api_connexion/**/* - tests/api_fastapi/**/* area:dev-tools: @@ -321,10 +320,6 @@ labelPRBasedOnFilePath: - .pre-commit-config.yaml - .rat-excludes - .readthedocs.yml - - airflow/www/.eslintignore - - airflow/www/.eslintrc - - airflow/www/.stylelintignore - - airflow/www/.stylelintrc kind:documentation: - docs/**/* @@ -333,22 +328,7 @@ labelPRBasedOnFilePath: - chart/**/* - docs/helm-chart/** - area:Webserver: - - airflow/cli/commands/local_commands/webserver_command.py - - airflow/www/**/* - - tests/cli/commands/local_commands/test_webserver_command.py - - tests/www/**/* - area:UI: - - airflow/www/static/**/* - - airflow/www/templates/**/* - - airflow/www/.eslintignore - - airflow/www/.eslintrc - - airflow/www/.stylelintignore - - airflow/www/.stylelintrc - - airflow/www/package.json - - airflow/www/webpack.config.js - - airflow/www/yarn.lock - docs/apache-airflow/ui.rst - airflow/ui/**/* diff --git a/.github/workflows/basic-tests.yml b/.github/workflows/basic-tests.yml index 62ac87721a3a3..4ec7a05fd3c65 100644 --- a/.github/workflows/basic-tests.yml +++ b/.github/workflows/basic-tests.yml @@ -151,44 +151,6 @@ jobs: retention-days: '2' if: steps.restore-eslint-cache-simple-am-ui.outputs.stash-hit != 'true' - tests-www: - timeout-minutes: 10 - name: React WWW tests - runs-on: ${{ fromJSON(inputs.runs-on-as-json-public) }} - if: inputs.run-www-tests == 'true' - steps: - - name: "Cleanup repo" - shell: bash - run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*" - - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )" - uses: actions/checkout@v4 - with: - persist-credentials: false - - name: "Cleanup docker" - run: ./scripts/ci/cleanup_docker.sh - - name: "Setup node" - uses: actions/setup-node@v4 - with: - node-version: 21 - - name: "Restore eslint cache (www)" - uses: apache/infrastructure-actions/stash/restore@1c35b5ccf8fba5d4c3fdf25a045ca91aa0cbc468 - with: - path: airflow/www/node_modules/ - key: cache-www-node-modules-v1-${{ runner.os }}-${{ hashFiles('airflow/www/**/yarn.lock') }} - id: restore-eslint-cache - - run: yarn --cwd airflow/www/ install --frozen-lockfile --non-interactive - - run: yarn --cwd airflow/www/ run test - env: - FORCE_COLOR: 2 - - name: "Save eslint cache (www)" - uses: apache/infrastructure-actions/stash/save@1c35b5ccf8fba5d4c3fdf25a045ca91aa0cbc468 - with: - path: airflow/www/node_modules/ - key: cache-www-node-modules-v1-${{ runner.os }}-${{ hashFiles('airflow/www/**/yarn.lock') }} - if-no-files-found: 'error' - retention-days: '2' - if: steps.restore-eslint-cache.outputs.stash-hit != 'true' - install-pre-commit: timeout-minutes: 5 name: "Install pre-commit for cache" diff --git a/.gitignore b/.gitignore index 84edc6d083fcb..4272415bff820 100644 --- a/.gitignore +++ b/.gitignore @@ -10,8 +10,6 @@ airflow.db # Airflow temporary artifacts airflow/git_version -airflow/www/static/coverage/ -airflow/www/*.log airflow/ui/coverage/ logs/ airflow-webserver.pid diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index f10e2c69a7c72..753d82257860a 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -67,23 +67,13 @@ repos: - id: insert-license name: Add license for CSS/JS/JSX/PUML/TS/TSX files: \.(css|jsx?|puml|tsx?)$ - exclude: ^\.github/.*$|^airflow/www/static/js/types/api-generated.ts$|ui/openapi-gen/|.*/dist/.* + exclude: ^\.github/.*$|ui/openapi-gen/|.*/dist/.* args: - --comment-style - "/*!| *| */" - --license-filepath - scripts/ci/license-templates/LICENSE.txt - --fuzzy-match-generates-todo - - id: insert-license - name: Add license for all JINJA template files - files: ^airflow/www/templates/.*\.html$ - exclude: ^\.github/.*$ - args: - - --comment-style - - "{#||#}" - - --license-filepath - - scripts/ci/license-templates/LICENSE.txt - - --fuzzy-match-generates-todo - id: insert-license name: Add license for all Shell files exclude: ^\.github/.*$|^dev/breeze/autocomplete/.*$ @@ -312,7 +302,7 @@ repos: exclude: material-icons\.css$|^images/.*$|^RELEASE_NOTES\.txt$|^.*package-lock\.json$|^.*/kinglear\.txt$|^.*pnpm-lock\.yaml$|.*/dist/.* args: - --ignore-words=docs/spelling_wordlist.txt - - --skip=providers/.*/src/airflow/providers/*/*.rst,providers/*/docs/changelog.rst,airflow/www/*.log,docs/*/commits.rst,providers/*/docs/commits.rst,providers/*/*/docs/commits.rst,docs/apache-airflow/tutorial/pipeline_example.csv,*.min.js,*.lock,INTHEWILD.md + - --skip=providers/.*/src/airflow/providers/*/*.rst,providers/*/docs/changelog.rst,docs/*/commits.rst,providers/*/docs/commits.rst,providers/*/*/docs/commits.rst,docs/apache-airflow/tutorial/pipeline_example.csv,*.min.js,*.lock,INTHEWILD.md - --exclude-file=.codespellignorelines - repo: https://github.com/woodruffw/zizmor-pre-commit rev: v1.0.0 @@ -616,7 +606,6 @@ repos: ^airflow/api_connexion/openapi/v1.yaml$| ^airflow/ui/openapi-gen/| ^airflow/cli/commands/local_commands/fastapi_api_command.py$| - ^airflow/cli/commands/local_commands/webserver_command.py$| ^airflow/config_templates/| ^airflow/models/baseoperator.py$| ^airflow/operators/__init__.py$| @@ -649,9 +638,6 @@ repos: ^airflow/ui/pnpm-lock.yaml$| ^airflow/utils/db.py$| ^airflow/utils/trigger_rule.py$| - ^airflow/www/static/css/bootstrap-theme.css$| - ^airflow/www/static/js/types/api-generated.ts$| - ^airflow/www/templates/appbuilder/flash.html$| ^chart/values.schema.json$| ^dev/| ^docs/README.rst$| @@ -858,13 +844,6 @@ repos: entry: koalaman/shellcheck:v0.8.0 -x -a files: \.(bash|sh)$|^hooks/build$|^hooks/push$ exclude: ^dev/breeze/autocomplete/.*$ - - id: lint-css - name: stylelint - entry: "stylelint" - language: node - files: ^airflow/www/.*\.(css|sass|scss)$ - # Keep dependency versions in sync w/ airflow/www/package.json - additional_dependencies: ['stylelint@13.3.1', 'stylelint-config-standard@20.0.0', 'stylelint-config-prettier@9.0.5'] - id: compile-ui-assets name: Compile ui assets (manual) language: node @@ -874,15 +853,6 @@ repos: entry: ./scripts/ci/pre_commit/compile_ui_assets.py pass_filenames: false additional_dependencies: ['pnpm@9.7.1'] - - id: compile-ui-assets-dev - name: Compile ui assets in dev mode (manual) - language: node - stages: ['manual'] - types_or: [javascript, ts, tsx] - files: ^airflow/ui/|^airflow/auth/managers/simple/ui/ - entry: ./scripts/ci/pre_commit/compile_ui_assets_dev.py - pass_filenames: false - additional_dependencies: ['pnpm@9.7.1'] - id: compile-fab-assets name: Compile FAB provider assets language: node @@ -891,24 +861,15 @@ repos: entry: ./scripts/ci/pre_commit/compile_fab_assets.py pass_filenames: false additional_dependencies: [ 'yarn@1.22.21' ] - - id: compile-www-assets - name: Compile www assets (manual) - language: node - stages: ['manual'] - 'types_or': [javascript, ts, tsx] - files: ^airflow/www/ - entry: ./scripts/ci/pre_commit/compile_www_assets.py - pass_filenames: false - additional_dependencies: ['yarn@1.22.21'] - - id: compile-www-assets-dev - name: Compile www assets in dev mode (manual) + - id: compile-ui-assets-dev + name: Compile ui assets in dev mode (manual) language: node stages: ['manual'] - 'types_or': [javascript, ts, tsx] - files: ^airflow/www/ - entry: ./scripts/ci/pre_commit/compile_www_assets_dev.py + types_or: [javascript, ts, tsx] + files: ^airflow/ui/|^airflow/auth/managers/simple/ui/ + entry: ./scripts/ci/pre_commit/compile_ui_assets_dev.py pass_filenames: false - additional_dependencies: ['yarn@1.22.21'] + additional_dependencies: ['pnpm@9.7.1'] - id: check-providers-subpackages-init-file-exist name: Provider subpackage init files are there pass_filenames: false @@ -1194,15 +1155,6 @@ repos: additional_dependencies: ['rich>=12.4.4'] pass_filenames: false files: ^tests/.*\.py$ - - id: ts-compile-format-lint-www - name: Compile / format / lint WWW - description: TS types generation / ESLint / Prettier against UI files - language: node - 'types_or': [javascript, ts, tsx, yaml, css, json] - files: ^airflow/www/static/(js|css)/|^airflow/api_connexion/openapi/v1\.yaml$ - entry: ./scripts/ci/pre_commit/lint_www.py - additional_dependencies: ['yarn@1.22.21', "openapi-typescript@>=6.7.4"] - pass_filenames: false - id: ts-compile-format-lint-ui name: Compile / format / lint UI description: TS types generation / ESLint / Prettier new UI files diff --git a/Dockerfile.ci b/Dockerfile.ci index 2f547a72ccedd..5d318267fe13b 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -1016,8 +1016,8 @@ function check_airflow_python_client_installation() { python "${IN_CONTAINER_DIR}/install_airflow_python_client.py" } -function start_webserver_with_examples(){ - if [[ ${START_WEBSERVER_WITH_EXAMPLES=} != "true" ]]; then +function start_api_server_with_examples(){ + if [[ ${START_API_SERVER_WITH_EXAMPLES=} != "true" ]]; then return fi export AIRFLOW__CORE__LOAD_EXAMPLES=True @@ -1038,22 +1038,22 @@ function start_webserver_with_examples(){ airflow users create -u admin -p admin -f Thor -l Administrator -r Admin -e admin@email.domain echo "Admin user created" echo - echo "${COLOR_BLUE}Starting airflow webserver${COLOR_RESET}" + echo "${COLOR_BLUE}Starting airflow api server${COLOR_RESET}" echo - airflow webserver --port 8080 --daemon + airflow api-server --port 9091 --daemon echo - echo "${COLOR_BLUE}Waiting for webserver to start${COLOR_RESET}" + echo "${COLOR_BLUE}Waiting for api-server to start${COLOR_RESET}" echo - check_service_connection "Airflow webserver" "run_nc localhost 8080" 100 + check_service_connection "Airflow api-server" "run_nc localhost 9091" 100 EXIT_CODE=$? if [[ ${EXIT_CODE} != 0 ]]; then echo - echo "${COLOR_RED}Webserver did not start properly${COLOR_RESET}" + echo "${COLOR_RED}Api server did not start properly${COLOR_RESET}" echo exit ${EXIT_CODE} fi echo - echo "${COLOR_BLUE}Airflow webserver started${COLOR_RESET}" + echo "${COLOR_BLUE}Airflow api-server started${COLOR_RESET}" } handle_mount_sources @@ -1064,7 +1064,7 @@ check_downgrade_sqlalchemy check_downgrade_pendulum check_force_lowest_dependencies check_airflow_python_client_installation -start_webserver_with_examples +start_api_server_with_examples check_run_tests "${@}" exec /bin/bash "${@}" @@ -1115,8 +1115,8 @@ ARG DEV_APT_COMMAND="" ARG ADDITIONAL_DEV_APT_COMMAND="" ARG ADDITIONAL_DEV_ENV_VARS="" ARG ADDITIONAL_DEV_APT_DEPS="bash-completion dumb-init git graphviz krb5-user \ -less libenchant-2-2 libgcc-11-dev libgeos-dev libpq-dev net-tools netcat-openbsd \ -openssh-server postgresql-client software-properties-common rsync tmux unzip vim xxd" + less libenchant-2-2 libgcc-11-dev libgeos-dev libpq-dev net-tools netcat-openbsd \ + openssh-server postgresql-client software-properties-common rsync tmux unzip vim xxd" ARG ADDITIONAL_DEV_APT_ENV="" @@ -1153,7 +1153,7 @@ RUN bash /scripts/docker/install_mysql.sh prod \ && bash /scripts/docker/install_postgres.sh dev \ # The user is added to allow ssh debugging (you can connect with airflow/airflow over ssh) && adduser --gecos "First Last,RoomNumber,WorkPhone,HomePhone" --disabled-password \ - --quiet "airflow" --home "/home/airflow" \ + --quiet "airflow" --home "/home/airflow" \ && echo -e "airflow\nairflow" | passwd airflow 2>&1 \ && echo "airflow ALL=(ALL) NOPASSWD: ALL" > /etc/sudoers.d/airflow \ && chmod 0440 /etc/sudoers.d/airflow @@ -1309,7 +1309,7 @@ ENV ADDITIONAL_PYTHON_DEPS=${ADDITIONAL_PYTHON_DEPS} RUN --mount=type=cache,id=ci-$TARGETARCH-$DEPENDENCY_CACHE_EPOCH,target=/root/.cache/ \ bash /scripts/docker/install_packaging_tools.sh; \ if [[ -n "${ADDITIONAL_PYTHON_DEPS}" ]]; then \ - bash /scripts/docker/install_additional_dependencies.sh; \ + bash /scripts/docker/install_additional_dependencies.sh; \ fi COPY --from=scripts entrypoint_ci.sh /entrypoint @@ -1319,7 +1319,7 @@ RUN chmod a+x /entrypoint /entrypoint-exec # Install autocomplete for airflow and kubectl RUN if command -v airflow; then \ - register-python-argcomplete airflow >> ~/.bashrc ; \ + register-python-argcomplete airflow >> ~/.bashrc ; \ fi; \ echo "source /etc/bash_completion" >> ~/.bashrc @@ -1346,27 +1346,27 @@ RUN ln -sf /usr/bin/dumb-init /usr/local/bin/dumb-init EXPOSE 8080 LABEL org.apache.airflow.distro="debian" \ - org.apache.airflow.module="airflow" \ - org.apache.airflow.component="airflow" \ - org.apache.airflow.image="airflow-ci" \ - org.apache.airflow.version="${AIRFLOW_VERSION}" \ - org.apache.airflow.uid="0" \ - org.apache.airflow.gid="0" \ - org.apache.airflow.build-id="${BUILD_ID}" \ - org.apache.airflow.commit-sha="${COMMIT_SHA}" \ - org.opencontainers.image.source="${AIRFLOW_IMAGE_REPOSITORY}" \ - org.opencontainers.image.created="${AIRFLOW_IMAGE_DATE_CREATED}" \ - org.opencontainers.image.authors="dev@airflow.apache.org" \ - org.opencontainers.image.url="https://airflow.apache.org" \ - org.opencontainers.image.documentation="https://airflow.apache.org/docs/docker-stack/index.html" \ - org.opencontainers.image.source="https://github.com/apache/airflow" \ - org.opencontainers.image.version="${AIRFLOW_VERSION}" \ - org.opencontainers.image.revision="${COMMIT_SHA}" \ - org.opencontainers.image.vendor="Apache Software Foundation" \ - org.opencontainers.image.licenses="Apache-2.0" \ - org.opencontainers.image.ref.name="airflow-ci-image" \ - org.opencontainers.image.title="Continuous Integration Airflow Image" \ - org.opencontainers.image.description="Installed Apache Airflow with Continuous Integration dependencies" + org.apache.airflow.module="airflow" \ + org.apache.airflow.component="airflow" \ + org.apache.airflow.image="airflow-ci" \ + org.apache.airflow.version="${AIRFLOW_VERSION}" \ + org.apache.airflow.uid="0" \ + org.apache.airflow.gid="0" \ + org.apache.airflow.build-id="${BUILD_ID}" \ + org.apache.airflow.commit-sha="${COMMIT_SHA}" \ + org.opencontainers.image.source="${AIRFLOW_IMAGE_REPOSITORY}" \ + org.opencontainers.image.created="${AIRFLOW_IMAGE_DATE_CREATED}" \ + org.opencontainers.image.authors="dev@airflow.apache.org" \ + org.opencontainers.image.url="https://airflow.apache.org" \ + org.opencontainers.image.documentation="https://airflow.apache.org/docs/docker-stack/index.html" \ + org.opencontainers.image.source="https://github.com/apache/airflow" \ + org.opencontainers.image.version="${AIRFLOW_VERSION}" \ + org.opencontainers.image.revision="${COMMIT_SHA}" \ + org.opencontainers.image.vendor="Apache Software Foundation" \ + org.opencontainers.image.licenses="Apache-2.0" \ + org.opencontainers.image.ref.name="airflow-ci-image" \ + org.opencontainers.image.title="Continuous Integration Airflow Image" \ + org.opencontainers.image.description="Installed Apache Airflow with Continuous Integration dependencies" ENTRYPOINT ["/usr/bin/dumb-init", "--", "/entrypoint"] CMD [] diff --git a/airflow/api_fastapi/core_api/app.py b/airflow/api_fastapi/core_api/app.py index 2b8440bf560bb..93326355f87f8 100644 --- a/airflow/api_fastapi/core_api/app.py +++ b/airflow/api_fastapi/core_api/app.py @@ -30,11 +30,11 @@ from starlette.staticfiles import StaticFiles from starlette.templating import Jinja2Templates +from airflow.api_fastapi.core_api.init_dagbag import get_dag_bag from airflow.api_fastapi.core_api.middleware import FlaskExceptionsMiddleware from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.settings import AIRFLOW_PATH -from airflow.www.extensions.init_dagbag import get_dag_bag log = logging.getLogger(__name__) diff --git a/tests/api_connexion/schemas/test_version_schema.py b/airflow/api_fastapi/core_api/init_dagbag.py similarity index 59% rename from tests/api_connexion/schemas/test_version_schema.py rename to airflow/api_fastapi/core_api/init_dagbag.py index fab3694ad380f..fc23a2f7e06b2 100644 --- a/tests/api_connexion/schemas/test_version_schema.py +++ b/airflow/api_fastapi/core_api/init_dagbag.py @@ -16,17 +16,23 @@ # under the License. from __future__ import annotations -import pytest +import os -from airflow.api_connexion.endpoints.version_endpoint import VersionInfo -from airflow.api_connexion.schemas.version_schema import version_info_schema +from airflow.models import DagBag +from airflow.settings import DAGS_FOLDER -class TestVersionInfoSchema: - @pytest.mark.parametrize("git_commit", ["GIT_COMMIT", None]) - def test_serialize(self, git_commit): - version_info = VersionInfo("VERSION", git_commit) - current_data = version_info_schema.dump(version_info) +def get_dag_bag() -> DagBag: + """Instantiate the appropriate DagBag based on the ``SKIP_DAGS_PARSING`` environment variable.""" + if os.environ.get("SKIP_DAGS_PARSING") == "True": + return DagBag(os.devnull, include_examples=False) + return DagBag(DAGS_FOLDER, read_dags_from_db=True) - expected_result = {"version": "VERSION", "git_version": git_commit} - assert expected_result == current_data + +def init_dagbag(app): + """ + Create global DagBag for webserver and API. + + To access it use ``flask.current_app.dag_bag``. + """ + app.dag_bag = get_dag_bag() diff --git a/airflow/cli/cli_config.py b/airflow/cli/cli_config.py index 3331e7f6142a0..a10c1d74bda8d 100644 --- a/airflow/cli/cli_config.py +++ b/airflow/cli/cli_config.py @@ -1857,29 +1857,6 @@ class GroupCommand(NamedTuple): ARG_VERBOSE, ), ), - ActionCommand( - name="webserver", - help="Start an Airflow webserver instance", - func=lazy_load_command("airflow.cli.commands.local_commands.webserver_command.webserver"), - args=( - ARG_PORT, - ARG_WORKERS, - ARG_WORKERCLASS, - ARG_WORKER_TIMEOUT, - ARG_HOSTNAME, - ARG_PID, - ARG_DAEMON, - ARG_STDOUT, - ARG_STDERR, - ARG_ACCESS_LOGFILE, - ARG_ERROR_LOGFILE, - ARG_ACCESS_LOGFORMAT, - ARG_LOG_FILE, - ARG_SSL_CERT, - ARG_SSL_KEY, - ARG_DEBUG, - ), - ), ActionCommand( name="api-server", help="Start an Airflow API server instance", diff --git a/airflow/cli/commands/local_commands/standalone_command.py b/airflow/cli/commands/local_commands/standalone_command.py index 88e680190fbfe..d8fdb6da7aedf 100644 --- a/airflow/cli/commands/local_commands/standalone_command.py +++ b/airflow/cli/commands/local_commands/standalone_command.py @@ -83,12 +83,6 @@ def run(self): command=["dag-processor"], env=env, ) - self.subcommands["webserver"] = SubCommand( - self, - name="webserver", - command=["webserver"], - env=env, - ) self.subcommands["api-server"] = SubCommand( self, name="api-server", @@ -102,7 +96,6 @@ def run(self): env=env, ) - self.web_server_port = conf.getint("webserver", "WEB_SERVER_PORT", fallback=8080) # Run subcommand threads for command in self.subcommands.values(): command.start() @@ -152,7 +145,6 @@ def print_output(self, name: str, output): """ color: dict[str, Color] = { "api-server": "magenta", - "webserver": "green", "scheduler": "blue", "dag-processor": "yellow", "triggerer": "cyan", diff --git a/airflow/cli/commands/local_commands/webserver_command.py b/airflow/cli/commands/local_commands/webserver_command.py deleted file mode 100644 index b97e6bd338659..0000000000000 --- a/airflow/cli/commands/local_commands/webserver_command.py +++ /dev/null @@ -1,491 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -"""Webserver command.""" - -from __future__ import annotations - -import logging -import os -import signal -import subprocess -import sys -import textwrap -import time -from contextlib import suppress -from pathlib import Path -from time import sleep -from typing import NoReturn - -import psutil -from lockfile.pidlockfile import read_pid_from_pidfile - -from airflow import settings -from airflow.cli.commands.local_commands.daemon_utils import run_command_with_daemon_option -from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowWebServerTimeout -from airflow.utils import cli as cli_utils -from airflow.utils.cli import setup_locations -from airflow.utils.hashlib_wrapper import md5 -from airflow.utils.log.logging_mixin import LoggingMixin -from airflow.utils.providers_configuration_loader import providers_configuration_loaded - -log = logging.getLogger(__name__) - - -class GunicornMonitor(LoggingMixin): - """ - Runs forever. - - Monitoring the child processes of @gunicorn_master_proc and restarting - workers occasionally or when files in the plug-in directory has been modified. - - Each iteration of the loop traverses one edge of this state transition - diagram, where each state (node) represents - [ num_ready_workers_running / num_workers_running ]. We expect most time to - be spent in [n / n]. `bs` is the setting webserver.worker_refresh_batch_size. - The horizontal transition at ? happens after the new worker parses all the - dags (so it could take a while!) - V ────────────────────────────────────────────────────────────────────────┐ - [n / n] ──TTIN──> [ [n, n+bs) / n + bs ] ────?───> [n + bs / n + bs] ──TTOU─┘ - ^ ^───────────────┘ - │ - │ ┌────────────────v - └──────┴────── [ [0, n) / n ] <─── start - We change the number of workers by sending TTIN and TTOU to the gunicorn - master process, which increases and decreases the number of child workers - respectively. Gunicorn guarantees that on TTOU workers are terminated - gracefully and that the oldest worker is terminated. - - :param gunicorn_master_pid: PID for the main Gunicorn process - :param num_workers_expected: Number of workers to run the Gunicorn web server - :param master_timeout: Number of seconds the webserver waits before killing gunicorn master that - doesn't respond - :param worker_refresh_interval: Number of seconds to wait before refreshing a batch of workers. - :param worker_refresh_batch_size: Number of workers to refresh at a time. When set to 0, worker - refresh is disabled. When nonzero, airflow periodically refreshes webserver workers by - bringing up new ones and killing old ones. - :param reload_on_plugin_change: If set to True, Airflow will track files in plugins_folder directory. - When it detects changes, then reload the gunicorn. - """ - - def __init__( - self, - gunicorn_master_pid: int, - num_workers_expected: int, - master_timeout: int, - worker_refresh_interval: int, - worker_refresh_batch_size: int, - reload_on_plugin_change: bool, - ): - super().__init__() - self.gunicorn_master_proc = psutil.Process(gunicorn_master_pid) - self.num_workers_expected = num_workers_expected - self.master_timeout = master_timeout - self.worker_refresh_interval = worker_refresh_interval - self.worker_refresh_batch_size = worker_refresh_batch_size - self.reload_on_plugin_change = reload_on_plugin_change - - self._num_workers_running = 0 - self._num_ready_workers_running = 0 - self._last_refresh_time = time.monotonic() if worker_refresh_interval > 0 else None - self._last_plugin_state = self._generate_plugin_state() if reload_on_plugin_change else None - self._restart_on_next_plugin_check = False - - def _generate_plugin_state(self) -> dict[str, float]: - """ - Get plugin states. - - Generate dict of filenames and last modification time of all files in settings.PLUGINS_FOLDER - directory. - """ - if not settings.PLUGINS_FOLDER: - return {} - - all_filenames: list[str] = [] - for root, _, filenames in os.walk(settings.PLUGINS_FOLDER): - all_filenames.extend(os.path.join(root, f) for f in filenames) - plugin_state = {f: self._get_file_hash(f) for f in sorted(all_filenames)} - return plugin_state - - @staticmethod - def _get_file_hash(fname: str): - """Calculate MD5 hash for file.""" - hash_md5 = md5() - with open(fname, "rb") as f: - for chunk in iter(lambda: f.read(4096), b""): - hash_md5.update(chunk) - return hash_md5.hexdigest() - - def _get_num_ready_workers_running(self) -> int: - """Return number of ready Gunicorn workers by looking for READY_PREFIX in process name.""" - workers = psutil.Process(self.gunicorn_master_proc.pid).children() - - def ready_prefix_on_cmdline(proc): - try: - cmdline = proc.cmdline() - if cmdline: - return settings.GUNICORN_WORKER_READY_PREFIX in cmdline[0] - except psutil.NoSuchProcess: - pass - return False - - nb_ready_workers = sum(1 for proc in workers if ready_prefix_on_cmdline(proc)) - return nb_ready_workers - - def _get_num_workers_running(self) -> int: - """Return number of running Gunicorn workers processes.""" - workers = psutil.Process(self.gunicorn_master_proc.pid).children() - return len(workers) - - def _wait_until_true(self, fn, timeout: int = 0) -> None: - """Sleep until fn is true.""" - start_time = time.monotonic() - while not fn(): - if 0 < timeout <= time.monotonic() - start_time: - raise AirflowWebServerTimeout(f"No response from gunicorn master within {timeout} seconds") - sleep(0.1) - - def _spawn_new_workers(self, count: int) -> None: - """ - Send signal to kill the worker. - - :param count: The number of workers to spawn - """ - excess = 0 - for _ in range(count): - # TTIN: Increment the number of processes by one - self.gunicorn_master_proc.send_signal(signal.SIGTTIN) - excess += 1 - self._wait_until_true( - lambda: self.num_workers_expected + excess == self._get_num_workers_running(), - timeout=self.master_timeout, - ) - - def _kill_old_workers(self, count: int) -> None: - """ - Send signal to kill the worker. - - :param count: The number of workers to kill - """ - for _ in range(count): - count -= 1 - # TTOU: Decrement the number of processes by one - self.gunicorn_master_proc.send_signal(signal.SIGTTOU) - self._wait_until_true( - lambda: self.num_workers_expected + count == self._get_num_workers_running(), - timeout=self.master_timeout, - ) - - def _reload_gunicorn(self) -> None: - """ - Send signal to reload the gunicorn configuration. - - When gunicorn receive signals, it reloads the configuration, - start the new worker processes with a new configuration and gracefully - shutdown older workers. - """ - # HUP: Reload the configuration. - self.gunicorn_master_proc.send_signal(signal.SIGHUP) - sleep(1) - self._wait_until_true( - lambda: self.num_workers_expected == self._get_num_workers_running(), timeout=self.master_timeout - ) - - def start(self) -> NoReturn: - """Start monitoring the webserver.""" - try: - self._wait_until_true( - lambda: self.num_workers_expected == self._get_num_workers_running(), - timeout=self.master_timeout, - ) - while True: - if not self.gunicorn_master_proc.is_running(): - sys.exit(1) - self._check_workers() - # Throttle loop - sleep(1) - - except (AirflowWebServerTimeout, OSError) as err: - self.log.error(err) - self.log.error("Shutting down webserver") - try: - self.gunicorn_master_proc.terminate() - self.gunicorn_master_proc.wait() - finally: - sys.exit(1) - - def _check_workers(self) -> None: - num_workers_running = self._get_num_workers_running() - num_ready_workers_running = self._get_num_ready_workers_running() - - # Whenever some workers are not ready, wait until all workers are ready - if num_ready_workers_running < num_workers_running: - self.log.debug( - "[%d / %d] Some workers are starting up, waiting...", - num_ready_workers_running, - num_workers_running, - ) - sleep(1) - return - - # If there are too many workers, then kill a worker gracefully by asking gunicorn to reduce - # number of workers - if num_workers_running > self.num_workers_expected: - excess = min(num_workers_running - self.num_workers_expected, self.worker_refresh_batch_size) - self.log.debug( - "[%d / %d] Killing %s workers", num_ready_workers_running, num_workers_running, excess - ) - self._kill_old_workers(excess) - return - - # If there are too few workers, start a new worker by asking gunicorn - # to increase number of workers - if num_workers_running < self.num_workers_expected: - self.log.error( - "[%d / %d] Some workers seem to have died and gunicorn did not restart them as expected", - num_ready_workers_running, - num_workers_running, - ) - sleep(10) - num_workers_running = self._get_num_workers_running() - if num_workers_running < self.num_workers_expected: - new_worker_count = min( - self.num_workers_expected - num_workers_running, self.worker_refresh_batch_size - ) - # log at info since we are trying fix an error logged just above - self.log.info( - "[%d / %d] Spawning %d workers", - num_ready_workers_running, - num_workers_running, - new_worker_count, - ) - self._spawn_new_workers(new_worker_count) - return - - # Now the number of running and expected worker should be equal - - # If workers should be restarted periodically. - if self.worker_refresh_interval > 0 and self._last_refresh_time: - # and we refreshed the workers a long time ago, refresh the workers - last_refresh_diff = time.monotonic() - self._last_refresh_time - if self.worker_refresh_interval < last_refresh_diff: - num_new_workers = self.worker_refresh_batch_size - self.log.debug( - "[%d / %d] Starting doing a refresh. Starting %d workers.", - num_ready_workers_running, - num_workers_running, - num_new_workers, - ) - self._spawn_new_workers(num_new_workers) - self._last_refresh_time = time.monotonic() - return - - # if we should check the directory with the plugin, - if self.reload_on_plugin_change: - # compare the previous and current contents of the directory - new_state = self._generate_plugin_state() - # If changed, wait until its content is fully saved. - if new_state != self._last_plugin_state: - self.log.debug( - "[%d / %d] Plugins folder changed. The gunicorn will be restarted the next time the " - "plugin directory is checked, if there is no change in it.", - num_ready_workers_running, - num_workers_running, - ) - self._restart_on_next_plugin_check = True - self._last_plugin_state = new_state - elif self._restart_on_next_plugin_check: - self.log.debug( - "[%d / %d] Starts reloading the gunicorn configuration.", - num_ready_workers_running, - num_workers_running, - ) - self._restart_on_next_plugin_check = False - self._last_refresh_time = time.monotonic() - self._reload_gunicorn() - - -@cli_utils.action_cli -@providers_configuration_loaded -def webserver(args): - """Start Airflow Webserver.""" - print(settings.HEADER) - - # Check for old/insecure config, and fail safe (i.e. don't launch) if the config is wildly insecure. - if conf.get("webserver", "secret_key") == "temporary_key": - from rich import print as rich_print - - rich_print( - "[red][bold]ERROR:[/bold] The `secret_key` setting under the webserver config has an insecure " - "value - Airflow has failed safe and refuses to start. Please change this value to a new, " - "per-environment, randomly generated string, for example using this command `[cyan]openssl rand " - "-hex 30[/cyan]`", - file=sys.stderr, - ) - sys.exit(1) - - access_logfile = args.access_logfile or conf.get("webserver", "access_logfile") - error_logfile = args.error_logfile or conf.get("webserver", "error_logfile") - access_logformat = args.access_logformat or conf.get("webserver", "access_logformat") - num_workers = args.workers or conf.get("webserver", "workers") - worker_timeout = args.worker_timeout or conf.get("webserver", "web_server_worker_timeout") - ssl_cert = args.ssl_cert or conf.get("webserver", "web_server_ssl_cert") - ssl_key = args.ssl_key or conf.get("webserver", "web_server_ssl_key") - if not ssl_cert and ssl_key: - raise AirflowException("An SSL certificate must also be provided for use with " + ssl_key) - if ssl_cert and not ssl_key: - raise AirflowException("An SSL key must also be provided for use with " + ssl_cert) - - from airflow.www.app import create_app - - if args.debug: - print(f"Starting the web server on port {args.port} and host {args.hostname}.") - app = create_app(testing=conf.getboolean("core", "unit_test_mode")) - app.run( - debug=True, - use_reloader=not app.config["TESTING"], - port=args.port, - host=args.hostname, - ssl_context=(ssl_cert, ssl_key) if ssl_cert and ssl_key else None, - ) - else: - print( - textwrap.dedent( - f"""\ - Running the Gunicorn Server with: - Workers: {num_workers} {args.workerclass} - Host: {args.hostname}:{args.port} - Timeout: {worker_timeout} - Logfiles: {access_logfile} {error_logfile} - Access Logformat: {access_logformat} - =================================================================""" - ) - ) - - pid_file, _, _, _ = setup_locations("webserver", pid=args.pid) - run_args = [ - sys.executable, - "-m", - "gunicorn", - "--workers", - str(num_workers), - "--worker-class", - str(args.workerclass), - "--timeout", - str(worker_timeout), - "--bind", - args.hostname + ":" + str(args.port), - "--name", - "airflow-webserver", - "--pid", - pid_file, - "--config", - "python:airflow.www.gunicorn_config", - ] - - if args.access_logfile: - run_args += ["--access-logfile", str(args.access_logfile)] - - if args.error_logfile: - run_args += ["--error-logfile", str(args.error_logfile)] - - if args.access_logformat and args.access_logformat.strip(): - run_args += ["--access-logformat", str(args.access_logformat)] - - if args.daemon: - run_args += ["--daemon"] - - if ssl_cert: - run_args += ["--certfile", ssl_cert, "--keyfile", ssl_key] - - run_args += ["airflow.www.app:cached_app()"] - - if conf.getboolean("webserver", "reload_on_plugin_change", fallback=False): - log.warning( - "Setting reload_on_plugin_change = true prevents running Gunicorn with preloading. " - "This means the app cannot be loaded before workers are forked, and each worker has a " - "separate copy of the app. This may cause IntegrityError during webserver startup, and " - "should be avoided in production." - ) - else: - # To prevent different workers creating the web app and - # all writing to the database at the same time, we use the --preload option. - run_args += ["--preload"] - - def kill_proc(signum: int, gunicorn_master_proc: psutil.Process | subprocess.Popen) -> NoReturn: - log.info("Received signal: %s. Closing gunicorn.", signum) - gunicorn_master_proc.terminate() - with suppress(TimeoutError): - gunicorn_master_proc.wait(timeout=30) - if isinstance(gunicorn_master_proc, subprocess.Popen): - still_running = gunicorn_master_proc.poll() is not None - else: - still_running = gunicorn_master_proc.is_running() - if still_running: - gunicorn_master_proc.kill() - sys.exit(0) - - def monitor_gunicorn(gunicorn_master_proc: psutil.Process | subprocess.Popen) -> NoReturn: - # Register signal handlers - signal.signal(signal.SIGINT, lambda signum, _: kill_proc(signum, gunicorn_master_proc)) - signal.signal(signal.SIGTERM, lambda signum, _: kill_proc(signum, gunicorn_master_proc)) - - # These run forever until SIG{INT, TERM, KILL, ...} signal is sent - GunicornMonitor( - gunicorn_master_pid=gunicorn_master_proc.pid, - num_workers_expected=num_workers, - master_timeout=conf.getint("webserver", "web_server_master_timeout"), - worker_refresh_interval=conf.getint("webserver", "worker_refresh_interval", fallback=30), - worker_refresh_batch_size=conf.getint("webserver", "worker_refresh_batch_size", fallback=1), - reload_on_plugin_change=conf.getboolean( - "webserver", "reload_on_plugin_change", fallback=False - ), - ).start() - - def start_and_monitor_gunicorn(args): - if args.daemon: - subprocess.Popen(run_args, close_fds=True) - - # Reading pid of gunicorn master as it will be different that - # the one of process spawned above. - gunicorn_master_proc_pid = None - while not gunicorn_master_proc_pid: - sleep(0.1) - gunicorn_master_proc_pid = read_pid_from_pidfile(pid_file) - - # Run Gunicorn monitor - gunicorn_master_proc = psutil.Process(gunicorn_master_proc_pid) - monitor_gunicorn(gunicorn_master_proc) - else: - with subprocess.Popen(run_args, close_fds=True) as gunicorn_master_proc: - monitor_gunicorn(gunicorn_master_proc) - - if args.daemon: - # This makes possible errors get reported before daemonization - os.environ["SKIP_DAGS_PARSING"] = "True" - create_app(None) - os.environ.pop("SKIP_DAGS_PARSING") - - pid_file_path = Path(pid_file) - monitor_pid_file = str(pid_file_path.with_name(f"{pid_file_path.stem}-monitor{pid_file_path.suffix}")) - run_command_with_daemon_option( - args=args, - process_name="webserver", - callback=lambda: start_and_monitor_gunicorn(args), - should_setup_logging=True, - pid_file=monitor_pid_file, - ) diff --git a/airflow/settings.py b/airflow/settings.py index 82168685aa7e0..fb9450ef741c2 100644 --- a/airflow/settings.py +++ b/airflow/settings.py @@ -48,8 +48,6 @@ from sqlalchemy.engine import Engine from sqlalchemy.orm import Session as SASession - from airflow.www.utils import UIAlert - log = logging.getLogger(__name__) try: @@ -707,21 +705,6 @@ def initialize(): # instances MASK_SECRETS_IN_LOGS = False -# Display alerts on the dashboard -# Useful for warning about setup issues or announcing changes to end users -# List of UIAlerts, which allows for specifying the message, category, and roles the -# message should be shown to. For example: -# from airflow.www.utils import UIAlert -# -# DASHBOARD_UIALERTS = [ -# UIAlert("Welcome to Airflow"), # All users -# UIAlert("Airflow update happening next week", roles=["User"]), # Only users with the User role -# # A flash message with html: -# UIAlert('Visit airflow.apache.org', html=True), -# ] -# -DASHBOARD_UIALERTS: list[UIAlert] = [] - # Prefix used to identify tables holding data moved during migration. AIRFLOW_MOVED_TABLE_PREFIX = "_airflow_moved" diff --git a/airflow/ui/CONTRIBUTING.md b/airflow/ui/CONTRIBUTING.md index 73ff3ee00df46..44e5a55570e27 100644 --- a/airflow/ui/CONTRIBUTING.md +++ b/airflow/ui/CONTRIBUTING.md @@ -28,7 +28,7 @@ Manually: - Have the `dev-mode` environment variable enabled - Run `pnpm install && pnpm dev` -- Note: Make sure to access the UI via the Airflow localhost port (8080 or 28080) and not the vite port (5173) +- Note: Make sure to access the UI via the Airflow localhost port (9091 or 29091) and not the vite port (5173) ## More diff --git a/airflow/utils/airflow_flask_app.py b/airflow/utils/airflow_flask_app.py index f18f02a0ecb4d..498f15ea3125e 100644 --- a/airflow/utils/airflow_flask_app.py +++ b/airflow/utils/airflow_flask_app.py @@ -22,13 +22,11 @@ if TYPE_CHECKING: from airflow.models.dagbag import DagBag - from airflow.www.extensions.init_appbuilder import AirflowAppBuilder class AirflowApp(Flask): """Airflow Flask Application.""" - appbuilder: AirflowAppBuilder dag_bag: DagBag api_auth: list[Any] diff --git a/airflow/utils/helpers.py b/airflow/utils/helpers.py index 0be76b9feebaa..371061069af2b 100644 --- a/airflow/utils/helpers.py +++ b/airflow/utils/helpers.py @@ -251,17 +251,15 @@ def partition(pred: Callable[[T], bool], iterable: Iterable[T]) -> tuple[Iterabl return itertools.filterfalse(pred, iter_1), filter(pred, iter_2) -def build_airflow_url_with_query(query: dict[str, Any]) -> str: +def build_airflow_dagrun_url(dag_id: str, run_id: str) -> str: """ - Build airflow url using base_url and default_view and provided query. + Build airflow dagrun url using base_url and provided dag_id and run_id. For example: - http://0.0.0.0:8000/base/graph?dag_id=my-task&root=&logical_date=2020-10-27T10%3A59%3A25.615587 + http://localhost:9091/webapp/dags/hi/runs/manual__2025-02-23T18:27:39.051358+00:00_RZa1at4Q """ - import flask - - view = conf.get_mandatory_value("webserver", "dag_default_view").lower() - return flask.url_for(f"Airflow.{view}", **query) + baseurl = conf.get("api", "base_url") + return f"{baseurl}/webapp/dags/{dag_id}/runs/{run_id}" # The 'template' argument is typed as Any because the jinja2.Template is too diff --git a/airflow/www/.eslintignore b/airflow/www/.eslintignore deleted file mode 100644 index f70244c96f6b9..0000000000000 --- a/airflow/www/.eslintignore +++ /dev/null @@ -1,7 +0,0 @@ -**/*{.,-}min.js -**/*.sh -**/*.py -jqClock.min.js -coverage/** -static/dist/* -static/docs/* diff --git a/airflow/www/.eslintrc b/airflow/www/.eslintrc deleted file mode 100644 index 29914c0cd847d..0000000000000 --- a/airflow/www/.eslintrc +++ /dev/null @@ -1,67 +0,0 @@ -{ - "extends": ["airbnb", "airbnb/hooks", "prettier"], - "parser": "@babel/eslint-parser", - "parserOptions": { - "babelOptions": { - "presets": [ - "@babel/preset-env", - "@babel/preset-react", - "@babel/preset-typescript" - ], - "plugins": ["@babel/plugin-transform-runtime"] - } - }, - "plugins": ["html", "react"], - "rules": { - "no-param-reassign": 1, - "react/prop-types": 0, - "react/jsx-props-no-spreading": 0, - "import/extensions": [ - "error", - "ignorePackages", - { - "js": "never", - "jsx": "never", - "ts": "never", - "tsx": "never" - } - ], - "import/no-extraneous-dependencies": [ - "error", - { - "devDependencies": true, - "optionalDependencies": false, - "peerDependencies": false - } - ], - "react/function-component-definition": [ - 0, - { - "namedComponents": "function-declaration" - } - ] - }, - "settings": { - "import/resolver": { - "node": { - "extensions": [".js", ".jsx", ".ts", ".tsx"] - } - } - }, - // eslint that apply only to typescript files - "overrides": [ - { - "files": ["*.ts", "*.tsx"], - "extends": ["airbnb-typescript", "prettier"], - "parser": "@typescript-eslint/parser", - "plugins": ["@typescript-eslint"], - "parserOptions": { - "project": "./tsconfig.json" - }, - "rules": { - "react/require-default-props": 0, - "@typescript-eslint/no-explicit-any": 1 - } - } - ] -} diff --git a/airflow/www/.prettierignore b/airflow/www/.prettierignore deleted file mode 100644 index a2bcd8157d69c..0000000000000 --- a/airflow/www/.prettierignore +++ /dev/null @@ -1,5 +0,0 @@ -.mypy_cache/ -templates/**/*.html -dist/ -*.md -*.yaml diff --git a/airflow/www/.prettierrc b/airflow/www/.prettierrc deleted file mode 100644 index 3574cf79c5c7e..0000000000000 --- a/airflow/www/.prettierrc +++ /dev/null @@ -1,10 +0,0 @@ -{ - "overrides": [ - { - "files": "*.json", - "options": { - "tabWidth": 2 - } - } - ] -} diff --git a/airflow/www/.stylelintignore b/airflow/www/.stylelintignore deleted file mode 100644 index 683379b45b24e..0000000000000 --- a/airflow/www/.stylelintignore +++ /dev/null @@ -1,2 +0,0 @@ -node_modules -static/dist/ diff --git a/airflow/www/.stylelintrc b/airflow/www/.stylelintrc deleted file mode 100644 index 2e8ff5864a48b..0000000000000 --- a/airflow/www/.stylelintrc +++ /dev/null @@ -1,3 +0,0 @@ -{ - "extends": ["stylelint-config-standard", "stylelint-config-prettier"] -} diff --git a/airflow/www/alias-rest-types.js b/airflow/www/alias-rest-types.js deleted file mode 100644 index cc6e53cd9650a..0000000000000 --- a/airflow/www/alias-rest-types.js +++ /dev/null @@ -1,214 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -const ts = require("typescript"); -const fs = require("fs"); - -/* This library does three things to make openapi-typescript generation easier to use. - * 1. Creates capitalized exports for Paths and Operations - * 2. Alias Variables based either on the Path name or the Operation ID, such as: - * export type ListProjectsVariables = operations['listProjects']['parameters']['query']; - * 3. Aliases the returned data types, such as: - * export type ConnectionArgument = components['schemas']['ConnectionArgument']; - */ - -/* Finds all words, capitalizes them, and removes all other characters. */ -const toPascalCase = (str) => - (str.match(/[a-zA-Z0-9]+/g) || []) - .map((w) => `${w.charAt(0).toUpperCase()}${w.slice(1)}`) - .join(""); - -/* Adds a prefix to a type prop as necessary. - * ('', 'components') => 'components' - */ -const prefixPath = (rootPrefix, prop) => - rootPrefix ? `${rootPrefix}['${prop}']` : prop; - -// Recursively find child nodes by name. -const findNode = (node, ...names) => { - if (!node || names.length === 0) return node; - - const children = node.members || node.type.members; - - if (!children) { - return undefined; - } - - const child = children.find((c) => c.name?.text === names[0]); - - return findNode(child, ...names.slice(1)); -}; - -// Generate Variable Type Aliases for a given path or operation -const generateVariableAliases = (node, operationPath, operationName) => { - const variableTypes = []; - const hasPath = !!findNode(node, "parameters", "path"); - const hasQuery = !!findNode(node, "parameters", "query"); - const hasBody = !!findNode( - node, - "requestBody", - "content", - "application/json" - ); - - if (hasPath) variableTypes.push(`${operationPath}['parameters']['path']`); - if (hasQuery) variableTypes.push(`${operationPath}['parameters']['query']`); - if (hasBody) - variableTypes.push( - `${operationPath}['requestBody']['content']['application/json']` - ); - - if (variableTypes.length === 0) return ""; - const typeName = `${toPascalCase(operationName)}Variables`; - return [ - typeName, - `export type ${typeName} = CamelCasedPropertiesDeep<${variableTypes.join( - " & " - )}>;`, - ]; -}; - -// Generate Type Aliases -const generateAliases = (rootNode, writeText, prefix = "") => { - // Loop through the root AST nodes of the file - ts.forEachChild(rootNode, (node) => { - // Response Data Types - if (ts.isInterfaceDeclaration(node) && node.name?.text === "components") { - const schemaMemberNames = findNode(node, "schemas").type.members.map( - (n) => n.name?.text - ); - - const types = schemaMemberNames.map((n) => [ - `${n}`, - `export type ${n} = CamelCasedPropertiesDeep<${prefixPath( - prefix, - "components" - )}['schemas']['${n}']>;`, - ]); - if (types.length) { - writeText.push(["comment", `Types for returned data ${prefix}`]); - writeText.push(...types); - } - } - - // Paths referencing an operation are skipped - if (node.name?.text === "paths") { - if (!prefix) { - writeText.push(["comment", "Alias paths to PascalCase."]); - writeText.push(["Paths", "export type Paths = paths;"]); - } - - const types = []; - - (node.members || node.type.members).forEach((path) => { - const methodNames = path.type.members.map((m) => m.name.text); - const methodTypes = methodNames.map((m) => - generateVariableAliases( - findNode(path, m), - `${prefixPath(prefix, "paths")}['${path.name?.text}']['${m}']`, - `${path.name.text}${toPascalCase(m)}` - ) - ); - types.push(...methodTypes.filter((m) => !!m)); - }); - - if (types.length) { - writeText.push([ - "comment", - `Types for path operation variables ${prefix}`, - ]); - writeText.push(...types); - } - } - - // operationIds are defined - if (node.name?.text === "operations") { - if (!prefix) { - writeText.push(["comment", "Alias operations to PascalCase."]); - writeText.push(["Operations", "export type Operations = operations;"]); - } - - const types = (node.members || node.type.members).map((operation) => - generateVariableAliases( - operation, - `${prefixPath(prefix, "operations")}['${operation.name.text}']`, - operation.name.text - ) - ); - if (types.length) { - writeText.push(["comment", `Types for operation variables ${prefix}`]); - writeText.push(...types); - writeText.push("\n"); - } - } - - // recursively call this for any externals - if (ts.isInterfaceDeclaration(node) && node.name?.text === "external") { - node.members.forEach((external) => { - generateAliases( - external.type, - writeText, - `external['${external.name.text}']` - ); - }); - } - }); -}; - -function generate(file) { - // Create a Program to represent the project, then pull out the - // source file to parse its AST. - const program = ts.createProgram([file], { allowJs: true }); - const sourceFile = program.getSourceFile(file); - const writeText = []; - writeText.push(["comment", "eslint-disable"]); - // eslint-disable-next-line quotes - writeText.push([ - "block", - `import type { CamelCasedPropertiesDeep } from 'type-fest';`, - ]); - writeText.push(["block", sourceFile.text]); - generateAliases(sourceFile, writeText); - - const finalText = writeText - // Deduplicate types - .map((pair) => { - // keep all comments and code blocks - if (pair[0] === "comment" || pair[0] === "block") return pair; - // return the first instance of this key only - const firstInstance = writeText.find((p) => p[0] === pair[0]); - return firstInstance === pair - ? pair - : ["comment", `Duplicate removed: ${pair[1]}`]; - }) - // Remove undefined created above - .filter((p) => !!p) - // Escape comments and flatten. - .map((pair) => (pair[0] === "comment" ? `\n/* ${pair[1]} */` : pair[1])) - .join("\n"); - - fs.writeFileSync(file, finalText, (err) => { - if (err) { - // eslint-disable-next-line no-console - console.error(err); - } - }); -} - -generate(process.argv[2]); diff --git a/airflow/www/app.py b/airflow/www/app.py deleted file mode 100644 index 06a0e14de040e..0000000000000 --- a/airflow/www/app.py +++ /dev/null @@ -1,183 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from datetime import timedelta -from os.path import isabs - -from flask import Flask -from flask_appbuilder import SQLA -from flask_wtf.csrf import CSRFProtect -from markupsafe import Markup -from sqlalchemy.engine.url import make_url - -from airflow import settings -from airflow.configuration import conf -from airflow.exceptions import AirflowConfigException -from airflow.logging_config import configure_logging -from airflow.models import import_all_models -from airflow.utils.json import AirflowJsonProvider -from airflow.www.extensions.init_appbuilder import init_appbuilder -from airflow.www.extensions.init_appbuilder_links import init_appbuilder_links -from airflow.www.extensions.init_cache import init_cache -from airflow.www.extensions.init_dagbag import init_dagbag -from airflow.www.extensions.init_jinja_globals import init_jinja_globals -from airflow.www.extensions.init_manifest_files import configure_manifest_files -from airflow.www.extensions.init_robots import init_robots -from airflow.www.extensions.init_security import ( - init_api_auth, - init_cache_control, - init_xframe_protection, -) -from airflow.www.extensions.init_session import init_airflow_session_interface -from airflow.www.extensions.init_views import ( - init_api_auth_provider, - init_api_connexion, - init_api_error_handlers, - init_api_internal, - init_appbuilder_views, - init_error_handlers, - init_flash_views, - init_plugins, -) -from airflow.www.extensions.init_wsgi_middlewares import init_wsgi_middleware - -app: Flask | None = None - -# Initializes at the module level, so plugins can access it. -# See: /docs/plugins.rst -csrf = CSRFProtect() - - -def create_app(config=None, testing=False): - """Create a new instance of Airflow WWW app.""" - flask_app = Flask(__name__) - flask_app.secret_key = conf.get("webserver", "SECRET_KEY") - - flask_app.config["PERMANENT_SESSION_LIFETIME"] = timedelta(minutes=settings.get_session_lifetime_config()) - - flask_app.config["MAX_CONTENT_LENGTH"] = conf.getfloat("webserver", "allowed_payload_size") * 1024 * 1024 - flask_app.config["MAX_FORM_PARTS"] = conf.getint("webserver", "max_form_parts") - flask_app.config["MAX_FORM_MEMORY_SIZE"] = conf.getint("webserver", "max_form_memory_size") - - webserver_config = conf.get_mandatory_value("webserver", "config_file") - # Enable customizations in webserver_config.py to be applied via Flask.current_app. - with flask_app.app_context(): - flask_app.config.from_pyfile(webserver_config, silent=True) - - flask_app.config["TESTING"] = testing - flask_app.config["SQLALCHEMY_DATABASE_URI"] = conf.get("database", "SQL_ALCHEMY_CONN") - - instance_name = conf.get(section="webserver", key="instance_name", fallback="Airflow") - require_confirmation_dag_change = conf.getboolean( - section="webserver", key="require_confirmation_dag_change", fallback=False - ) - instance_name_has_markup = conf.getboolean( - section="webserver", key="instance_name_has_markup", fallback=False - ) - if instance_name_has_markup: - instance_name = Markup(instance_name).striptags() - - flask_app.config["APP_NAME"] = instance_name - flask_app.config["REQUIRE_CONFIRMATION_DAG_CHANGE"] = require_confirmation_dag_change - - url = make_url(flask_app.config["SQLALCHEMY_DATABASE_URI"]) - if url.drivername == "sqlite" and url.database and not isabs(url.database): - raise AirflowConfigException( - f'Cannot use relative path: `{conf.get("database", "SQL_ALCHEMY_CONN")}` to connect to sqlite. ' - "Please use absolute path such as `sqlite:////tmp/airflow.db`." - ) - - flask_app.config["SQLALCHEMY_TRACK_MODIFICATIONS"] = False - - flask_app.config["SESSION_COOKIE_HTTPONLY"] = True - flask_app.config["SESSION_COOKIE_SECURE"] = conf.getboolean("webserver", "COOKIE_SECURE") - - # Note: Ensure "Lax" is the default if config not specified - flask_app.config["SESSION_COOKIE_SAMESITE"] = ( - conf.get("webserver", "COOKIE_SAMESITE", fallback=None) or "Lax" - ) - - # Above Flask 2.0.x, default value of SEND_FILE_MAX_AGE_DEFAULT changed 12 hours to None. - # for static file caching, it needs to set value explicitly. - flask_app.config["SEND_FILE_MAX_AGE_DEFAULT"] = timedelta(seconds=43200) - - if config: - flask_app.config.from_mapping(config) - - if "SQLALCHEMY_ENGINE_OPTIONS" not in flask_app.config: - flask_app.config["SQLALCHEMY_ENGINE_OPTIONS"] = settings.prepare_engine_args() - - # Configure the JSON encoder used by `|tojson` filter from Flask - flask_app.json_provider_class = AirflowJsonProvider - flask_app.json = AirflowJsonProvider(flask_app) - - csrf.init_app(flask_app) - - init_wsgi_middleware(flask_app) - - db = SQLA() - db.session = settings.Session - db.init_app(flask_app) - - init_dagbag(flask_app) - - init_api_auth(flask_app) - - init_robots(flask_app) - - init_cache(flask_app) - - init_flash_views(flask_app) - - configure_logging() - configure_manifest_files(flask_app) - - import_all_models() - - with flask_app.app_context(): - init_appbuilder(flask_app) - - init_appbuilder_views(flask_app) - init_appbuilder_links(flask_app) - init_plugins(flask_app) - init_error_handlers(flask_app) - init_api_connexion(flask_app) - if conf.getboolean("webserver", "run_internal_api", fallback=False): - init_api_internal(flask_app) - init_api_auth_provider(flask_app) - init_api_error_handlers(flask_app) # needs to be after all api inits to let them add their path first - init_jinja_globals(flask_app) - init_xframe_protection(flask_app) - init_cache_control(flask_app) - init_airflow_session_interface(flask_app) - return flask_app - - -def cached_app(config=None, testing=False): - """Return cached instance of Airflow WWW app.""" - global app - if not app: - app = create_app(config=config, testing=testing) - return app - - -def purge_cached_app(): - """Remove the cached version of the app in global state.""" - global app - app = None diff --git a/airflow/www/babel.config.js b/airflow/www/babel.config.js deleted file mode 100644 index 7f40a9fdf6b37..0000000000000 --- a/airflow/www/babel.config.js +++ /dev/null @@ -1,35 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -module.exports = (api) => { - api.cache(true); - - const presets = [ - "@babel/preset-env", - "@babel/preset-react", - "@babel/preset-typescript", - ]; - - const plugins = ["@babel/plugin-transform-runtime"]; - - return { - presets, - plugins, - }; -}; diff --git a/airflow/www/blueprints.py b/airflow/www/blueprints.py deleted file mode 100644 index fda6b65397b66..0000000000000 --- a/airflow/www/blueprints.py +++ /dev/null @@ -1,28 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from flask import Blueprint, redirect, url_for - -routes = Blueprint("routes", __name__) - - -@routes.route("/") -def index(): - """Return main Airflow page.""" - return redirect(url_for("Airflow.index")) diff --git a/airflow/www/constants.py b/airflow/www/constants.py deleted file mode 100644 index 263caf1576c4c..0000000000000 --- a/airflow/www/constants.py +++ /dev/null @@ -1,28 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from pathlib import Path - -from airflow.configuration import conf - -WWW = Path(__file__).resolve().parent -# There is a difference with configuring Swagger in Connexion 2.x and Connexion 3.x -# Connexion 2: https://connexion.readthedocs.io/en/2.14.2/quickstart.html#the-swagger-ui-console -# Connexion 3: https://connexion.readthedocs.io/en/stable/swagger_ui.html#configuring-the-swagger-ui -SWAGGER_ENABLED = conf.getboolean("webserver", "enable_swagger_ui", fallback=True) -SWAGGER_BUNDLE = WWW.joinpath("static", "dist", "swagger-ui") diff --git a/airflow/www/decorators.py b/airflow/www/decorators.py deleted file mode 100644 index 96a7f32e4c1a7..0000000000000 --- a/airflow/www/decorators.py +++ /dev/null @@ -1,200 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import functools -import gzip -import itertools -import json -import logging -from io import BytesIO -from typing import Callable, TypeVar, cast - -import pendulum -from flask import after_this_request, request -from pendulum.parsing.exceptions import ParserError - -from airflow.api_fastapi.app import get_auth_manager -from airflow.models import Log -from airflow.sdk.execution_time import secrets_masker -from airflow.utils.session import create_session - -T = TypeVar("T", bound=Callable) - -logger = logging.getLogger(__name__) - - -def _mask_variable_fields(extra_fields): - """ - Mask the 'val_content' field if 'key_content' is in the mask list. - - The variable requests values and args comes in this form: - {'key': 'key_content', 'val': 'val_content', 'description': 'description_content'} - """ - result = {} - keyname = None - for k, v in extra_fields.items(): - if k == "key": - keyname = v - result[k] = v - elif keyname and (k == "val" or k == "value"): - x = secrets_masker.redact(v, keyname) - result[k] = x - keyname = None - else: - result[k] = v - return result - - -def _mask_connection_fields(extra_fields): - """Mask connection fields.""" - result = {} - for k, v in extra_fields.items(): - if k == "extra" and v: - try: - extra = json.loads(v) - extra = {k: secrets_masker.redact(v, k) for k, v in extra.items()} - result[k] = dict(extra) - except json.JSONDecodeError: - result[k] = "Encountered non-JSON in `extra` field" - else: - result[k] = secrets_masker.redact(v, k) - return result - - -def action_logging(func: T | None = None, event: str | None = None) -> T | Callable: - """Log user actions.""" - - def log_action(f: T) -> T: - @functools.wraps(f) - def wrapper(*args, **kwargs): - __tracebackhide__ = True # Hide from pytest traceback. - - with create_session() as session: - event_name = event or f.__name__ - if not get_auth_manager().is_logged_in(): - user = "anonymous" - user_display = "" - else: - user = get_auth_manager().get_user_name() - user_display = get_auth_manager().get_user_display_name() - - isAPIRequest = request.blueprint == "/api/v1" - hasJsonBody = "application/json" in request.headers.get("content-type", "") and request.json - - fields_skip_logging = { - "csrf_token", - "_csrf_token", - "is_paused", - "dag_id", - "task_id", - "dag_run_id", - "run_id", - "logical_date", - } - extra_fields = { - k: secrets_masker.redact(v, k) - for k, v in itertools.chain(request.values.items(multi=True), request.view_args.items()) - if k not in fields_skip_logging - } - if event and event.startswith("variable."): - extra_fields = _mask_variable_fields( - request.json if isAPIRequest and hasJsonBody else extra_fields - ) - elif event and event.startswith("connection."): - extra_fields = _mask_connection_fields( - request.json if isAPIRequest and hasJsonBody else extra_fields - ) - elif hasJsonBody: - masked_json = {k: secrets_masker.redact(v, k) for k, v in request.json.items()} - extra_fields = {**extra_fields, **masked_json} - - params = {**request.values, **request.view_args} - if params and "is_paused" in params: - extra_fields["is_paused"] = params["is_paused"] == "false" - - if isAPIRequest: - if f"{request.origin}/" == request.root_url: - event_name = f"ui.{event_name}" - else: - event_name = f"api.{event_name}" - - log = Log( - event=event_name, - task_instance=None, - owner=user, - owner_display_name=user_display, - extra=json.dumps(extra_fields), - task_id=params.get("task_id"), - dag_id=params.get("dag_id"), - run_id=params.get("run_id") or params.get("dag_run_id"), - ) - - if "logical_date" in request.values: - logical_date_value = request.values.get("logical_date") - try: - log.logical_date = pendulum.parse(logical_date_value, strict=False) - except ParserError: - logger.exception( - "Failed to parse logical_date from the request: %s", logical_date_value - ) - - session.add(log) - - return f(*args, **kwargs) - - return cast(T, wrapper) - - if func: - return log_action(func) - return log_action - - -def gzipped(f: T) -> T: - """Make a view compressed.""" - - @functools.wraps(f) - def view_func(*args, **kwargs): - @after_this_request - def zipper(response): - accept_encoding = request.headers.get("Accept-Encoding", "") - - if "gzip" not in accept_encoding.lower(): - return response - - response.direct_passthrough = False - - if ( - response.status_code < 200 - or response.status_code >= 300 - or "Content-Encoding" in response.headers - ): - return response - with BytesIO() as gzip_buffer: - with gzip.GzipFile(mode="wb", fileobj=gzip_buffer) as gzip_file: - gzip_file.write(response.data) - response.data = gzip_buffer.getvalue() - response.headers["Content-Encoding"] = "gzip" - response.headers["Vary"] = "Accept-Encoding" - response.headers["Content-Length"] = len(response.data) - - return response - - return f(*args, **kwargs) - - return cast(T, view_func) diff --git a/airflow/www/extensions/init_appbuilder.py b/airflow/www/extensions/init_appbuilder.py deleted file mode 100644 index 2238066e6e953..0000000000000 --- a/airflow/www/extensions/init_appbuilder.py +++ /dev/null @@ -1,685 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -# mypy: disable-error-code=var-annotated -from __future__ import annotations - -import logging -from functools import reduce -from typing import TYPE_CHECKING - -from flask import Blueprint, current_app, url_for -from flask_appbuilder import __version__ -from flask_appbuilder.babel.manager import BabelManager -from flask_appbuilder.const import ( - LOGMSG_ERR_FAB_ADD_PERMISSION_MENU, - LOGMSG_ERR_FAB_ADD_PERMISSION_VIEW, - LOGMSG_ERR_FAB_ADDON_IMPORT, - LOGMSG_ERR_FAB_ADDON_PROCESS, - LOGMSG_INF_FAB_ADD_VIEW, - LOGMSG_INF_FAB_ADDON_ADDED, - LOGMSG_WAR_FAB_VIEW_EXISTS, -) -from flask_appbuilder.filters import TemplateFilters -from flask_appbuilder.menu import Menu -from flask_appbuilder.views import IndexView, UtilView - -from airflow import settings -from airflow.api_fastapi.app import create_auth_manager, get_auth_manager -from airflow.configuration import conf -from airflow.www.security_manager import AirflowSecurityManagerV2 - -if TYPE_CHECKING: - from flask import Flask - from flask_appbuilder import BaseView - from flask_appbuilder.security.manager import BaseSecurityManager - from sqlalchemy.orm import Session - - -# This product contains a modified portion of 'Flask App Builder' developed by Daniel Vaz Gaspar. -# (https://github.com/dpgaspar/Flask-AppBuilder). -# Copyright 2013, Daniel Vaz Gaspar -# This module contains code imported from FlaskAppbuilder, so lets use _its_ logger name -log = logging.getLogger("flask_appbuilder.base") - - -def dynamic_class_import(class_path): - """ - Will dynamically import a class from a string path. - - :param class_path: string with class path - :return: class - """ - # Split first occurrence of path - try: - tmp = class_path.split(".") - module_path = ".".join(tmp[0:-1]) - package = __import__(module_path) - return reduce(getattr, tmp[1:], package) - except Exception as e: - log.exception(e) - log.error(LOGMSG_ERR_FAB_ADDON_IMPORT, class_path, e) - - -class AirflowAppBuilder: - """ - This is the base class for all the framework. - - This is where you will register all your views - and create the menu structure. - Will hold your flask app object, all your views, and security classes. - Initialize your application like this for SQLAlchemy:: - from flask import Flask - from flask_appbuilder import SQLA, AppBuilder - app = Flask(__name__) - app.config.from_object('config') - db = SQLA(app) - appbuilder = AppBuilder(app, db.session) - When using MongoEngine:: - from flask import Flask - from flask_appbuilder import AppBuilder - from flask_appbuilder.security.mongoengine.manager import SecurityManager - from flask_mongoengine import MongoEngine - app = Flask(__name__) - app.config.from_object('config') - dbmongo = MongoEngine(app) - appbuilder = AppBuilder(app) - You can also create everything as an application factory. - """ - - baseviews: list[BaseView | Session] = [] - # Flask app - app = None - # Database Session - session = None - # Security Manager Class - sm: BaseSecurityManager - # Babel Manager Class - bm = None - # dict with addon name has key and instantiated class has value - addon_managers: dict - # temporary list that hold addon_managers config key - _addon_managers: list - - menu = None - indexview = None - - static_folder = None - static_url_path = None - - template_filters = None - - def __init__( - self, - app=None, - session: Session | None = None, - menu=None, - indexview=None, - base_template="airflow/main.html", - static_folder="static/appbuilder", - static_url_path="/appbuilder", - ): - """ - App-builder constructor. - - :param app: - The flask app object - :param session: - The SQLAlchemy session object - :param menu: - optional, a previous constructed menu - :param indexview: - optional, your customized indexview - :param static_folder: - optional, your override for the global static folder - :param static_url_path: - optional, your override for the global static url path - """ - from airflow.providers_manager import ProvidersManager - - providers_manager = ProvidersManager() - providers_manager.initialize_providers_configuration() - self.baseviews = [] - self._addon_managers = [] - self.addon_managers = {} - self.menu = menu - self.base_template = base_template - self.indexview = indexview - self.static_folder = static_folder - self.static_url_path = static_url_path - self.app = app - self.update_perms = conf.getboolean("fab", "UPDATE_FAB_PERMS") - self.auth_rate_limited = conf.getboolean("fab", "AUTH_RATE_LIMITED") - self.auth_rate_limit = conf.get("fab", "AUTH_RATE_LIMIT") - if app is not None: - self.init_app(app, session) - - def init_app(self, app, session): - """ - Will initialize the Flask app, supporting the app factory pattern. - - :param app: - :param session: The SQLAlchemy session - """ - app.config.setdefault("APP_NAME", "F.A.B.") - app.config.setdefault("APP_THEME", "") - app.config.setdefault("APP_ICON", "") - app.config.setdefault("LANGUAGES", {"en": {"flag": "gb", "name": "English"}}) - app.config.setdefault("ADDON_MANAGERS", []) - app.config.setdefault("RATELIMIT_ENABLED", self.auth_rate_limited) - app.config.setdefault("FAB_API_MAX_PAGE_SIZE", 100) - app.config.setdefault("FAB_BASE_TEMPLATE", self.base_template) - app.config.setdefault("FAB_STATIC_FOLDER", self.static_folder) - app.config.setdefault("FAB_STATIC_URL_PATH", self.static_url_path) - app.config.setdefault("AUTH_RATE_LIMITED", self.auth_rate_limited) - app.config.setdefault("AUTH_RATE_LIMIT", self.auth_rate_limit) - - self.app = app - - self.base_template = app.config.get("FAB_BASE_TEMPLATE", self.base_template) - self.static_folder = app.config.get("FAB_STATIC_FOLDER", self.static_folder) - self.static_url_path = app.config.get("FAB_STATIC_URL_PATH", self.static_url_path) - _index_view = app.config.get("FAB_INDEX_VIEW", None) - if _index_view is not None: - self.indexview = dynamic_class_import(_index_view) - else: - self.indexview = self.indexview or IndexView - _menu = app.config.get("FAB_MENU", None) - if _menu is not None: - self.menu = dynamic_class_import(_menu) - else: - self.menu = self.menu or Menu() - - if self.update_perms: # default is True, if False takes precedence from config - self.update_perms = app.config.get("FAB_UPDATE_PERMS", True) - - self._addon_managers = app.config["ADDON_MANAGERS"] - self.session = session - auth_manager = create_auth_manager() - auth_manager.appbuilder = self - auth_manager.init() - # FAB auth manager is used in both the old FAB UI and the new React UI backend by Fastapi. - # It can behave differently depending on the application. Setting this flag so that it knows this - # instance is run in FAB. - auth_manager.is_in_fab = True - if hasattr(auth_manager, "security_manager"): - self.sm = auth_manager.security_manager - else: - self.sm = AirflowSecurityManagerV2(self) - self.bm = BabelManager(self) - self._add_global_static() - self._add_global_filters() - app.before_request(self.sm.before_request) - self._add_admin_views() - self._add_addon_views() - if self.app: - self._add_menu_permissions() - else: - self.post_init() - self._init_extension(app) - self._swap_url_filter() - - def _init_extension(self, app): - app.appbuilder = self - if not hasattr(app, "extensions"): - app.extensions = {} - app.extensions["appbuilder"] = self - - def _swap_url_filter(self): - """Use our url filtering util function so there is consistency between FAB and Airflow routes.""" - from flask_appbuilder.security import views as fab_sec_views - - from airflow.www.views import get_safe_url - - fab_sec_views.get_safe_redirect = get_safe_url - - def post_init(self): - for baseview in self.baseviews: - # instantiate the views and add session - self._check_and_init(baseview) - # Register the views has blueprints - if baseview.__class__.__name__ not in self.get_app.blueprints.keys(): - self.register_blueprint(baseview) - # Add missing permissions where needed - self.add_permissions() - - @property - def get_app(self): - """ - Get current or configured flask app. - - :return: Flask App - """ - if self.app: - return self.app - else: - return current_app - - @property - def get_session(self): - """ - Get the current sqlalchemy session. - - :return: SQLAlchemy Session - """ - return self.session - - @property - def app_name(self): - """ - Get the App name. - - :return: String with app name - """ - return self.get_app.config["APP_NAME"] - - @property - def require_confirmation_dag_change(self): - """ - Get the value of the require_confirmation_dag_change configuration. - - The logic is: - - return True, in page dag.html, when user trigger/pause the dag from UI. - Once confirmation box will be shown before triggering the dag. - - Default value is False. - - :return: Boolean - """ - return self.get_app.config["REQUIRE_CONFIRMATION_DAG_CHANGE"] - - @property - def app_theme(self): - """ - Get the App theme name. - - :return: String app theme name - """ - return self.get_app.config["APP_THEME"] - - @property - def app_icon(self): - """ - Get the App icon location. - - :return: String with relative app icon location - """ - return self.get_app.config["APP_ICON"] - - @property - def languages(self): - return self.get_app.config["LANGUAGES"] - - @property - def version(self): - """ - Get the current F.A.B. version. - - :return: String with the current F.A.B. version - """ - return __version__ - - def _add_global_filters(self): - self.template_filters = TemplateFilters(self.get_app, self.sm) - - def _add_global_static(self): - bp = Blueprint( - "appbuilder", - "flask_appbuilder.base", - url_prefix="/static", - template_folder="templates", - static_folder=self.static_folder, - static_url_path=self.static_url_path, - ) - self.get_app.register_blueprint(bp) - - def _add_admin_views(self): - """Register indexview, utilview (back function), babel views and Security views.""" - self.indexview = self._check_and_init(self.indexview) - self.add_view_no_menu(self.indexview) - self.add_view_no_menu(UtilView()) - self.bm.register_views() - - try: - get_auth_manager().register_views() - except AttributeError: - # TODO: remove when min airflow version >= 3 - self.sm.register_views() - - def _add_addon_views(self): - """Register declared addons.""" - for addon in self._addon_managers: - addon_class = dynamic_class_import(addon) - if addon_class: - # Instantiate manager with appbuilder (self) - addon_class = addon_class(self) - try: - addon_class.pre_process() - addon_class.register_views() - addon_class.post_process() - self.addon_managers[addon] = addon_class - log.info(LOGMSG_INF_FAB_ADDON_ADDED, addon) - except Exception as e: - log.exception(e) - log.error(LOGMSG_ERR_FAB_ADDON_PROCESS, addon, e) - - def _check_and_init(self, baseview): - if hasattr(baseview, "datamodel"): - baseview.datamodel.session = self.session - if callable(baseview): - baseview = baseview() - return baseview - - def add_view( - self, - baseview, - name, - href="", - icon="", - label="", - category="", - category_icon="", - category_label="", - menu_cond=None, - ): - """ - Add your views associated with menus using this method. - - :param baseview: - A BaseView type class instantiated or not. - This method will instantiate the class for you if needed. - :param name: - The string name that identifies the menu. - :param href: - Override the generated href for the menu. - You can use an url string or an endpoint name - if non provided default_view from view will be set as href. - :param icon: - Font-Awesome icon name, optional. - :param label: - The label that will be displayed on the menu, - if absent param name will be used - :param category: - The menu category where the menu will be included, - if non provided the view will be accessible as a top menu. - :param category_icon: - Font-Awesome icon name for the category, optional. - :param category_label: - The label that will be displayed on the menu, - if absent param name will be used - :param menu_cond: - If a callable, :code:`menu_cond` will be invoked when - constructing the menu items. If it returns :code:`True`, - then this link will be a part of the menu. Otherwise, it - will not be included in the menu items. Defaults to - :code:`None`, meaning the item will always be present. - - Examples:: - - appbuilder = AppBuilder(app, db) - # Register a view, rendering a top menu without icon. - appbuilder.add_view(MyModelView(), "My View") - # or not instantiated - appbuilder.add_view(MyModelView, "My View") - # Register a view, a submenu "Other View" from "Other" with a phone icon. - appbuilder.add_view(MyOtherModelView, "Other View", icon="fa-phone", category="Others") - # Register a view, with category icon and translation. - appbuilder.add_view( - YetOtherModelView, - "Other View", - icon="fa-phone", - label=_("Other View"), - category="Others", - category_icon="fa-envelop", - category_label=_("Other View"), - ) - # Register a view whose menu item will be conditionally displayed - appbuilder.add_view( - YourFeatureView, - "Your Feature", - icon="fa-feature", - label=_("Your Feature"), - menu_cond=lambda: is_feature_enabled("your-feature"), - ) - # Add a link - appbuilder.add_link("google", href="www.google.com", icon="fa-google-plus") - """ - baseview = self._check_and_init(baseview) - log.info(LOGMSG_INF_FAB_ADD_VIEW, baseview.__class__.__name__, name) - - if not self._view_exists(baseview): - baseview.appbuilder = self - self.baseviews.append(baseview) - self._process_inner_views() - if self.app: - self.register_blueprint(baseview) - self._add_permission(baseview) - self.add_limits(baseview) - self.add_link( - name=name, - href=href, - icon=icon, - label=label, - category=category, - category_icon=category_icon, - category_label=category_label, - baseview=baseview, - cond=menu_cond, - ) - return baseview - - def add_link( - self, - name, - href, - icon="", - label="", - category="", - category_icon="", - category_label="", - baseview=None, - cond=None, - ): - """ - Add your own links to menu using this method. - - :param name: - The string name that identifies the menu. - :param href: - Override the generated href for the menu. - You can use an url string or an endpoint name - :param icon: - Font-Awesome icon name, optional. - :param label: - The label that will be displayed on the menu, - if absent param name will be used - :param category: - The menu category where the menu will be included, - if non provided the view will be accessible as a top menu. - :param category_icon: - Font-Awesome icon name for the category, optional. - :param category_label: - The label that will be displayed on the menu, - if absent param name will be used - :param baseview: - A BaseView type class instantiated. - :param cond: - If a callable, :code:`cond` will be invoked when - constructing the menu items. If it returns :code:`True`, - then this link will be a part of the menu. Otherwise, it - will not be included in the menu items. Defaults to - :code:`None`, meaning the item will always be present. - """ - self.menu.add_link( - name=name, - href=href, - icon=icon, - label=label, - category=category, - category_icon=category_icon, - category_label=category_label, - baseview=baseview, - cond=cond, - ) - if self.app: - self._add_permissions_menu(name) - if category: - self._add_permissions_menu(category) - - def add_separator(self, category, cond=None): - """ - Add a separator to the menu, you will sequentially create the menu. - - :param category: - The menu category where the separator will be included. - :param cond: - If a callable, :code:`cond` will be invoked when - constructing the menu items. If it returns :code:`True`, - then this separator will be a part of the menu. Otherwise, - it will not be included in the menu items. Defaults to - :code:`None`, meaning the separator will always be present. - """ - self.menu.add_separator(category, cond=cond) - - def add_view_no_menu(self, baseview, endpoint=None, static_folder=None): - """ - Add your views without creating a menu. - - :param baseview: A BaseView type class instantiated. - """ - baseview = self._check_and_init(baseview) - log.info(LOGMSG_INF_FAB_ADD_VIEW, baseview.__class__.__name__, "") - - if not self._view_exists(baseview): - baseview.appbuilder = self - self.baseviews.append(baseview) - self._process_inner_views() - if self.app: - self.register_blueprint(baseview, endpoint=endpoint, static_folder=static_folder) - self._add_permission(baseview) - else: - log.warning(LOGMSG_WAR_FAB_VIEW_EXISTS, baseview.__class__.__name__) - return baseview - - def security_cleanup(self): - """ - Clean up security. - - This method is useful if you have changed the name of your menus or - classes. Changing them leaves behind permissions that are not associated - with anything. You can use it always or just sometimes to perform a - security cleanup. - - .. warning:: - - This deletes any permission that is no longer part of any registered - view or menu. Only invoke AFTER YOU HAVE REGISTERED ALL VIEWS. - """ - if not hasattr(self.sm, "security_cleanup"): - raise NotImplementedError("The auth manager used does not support security_cleanup method.") - self.sm.security_cleanup(self.baseviews, self.menu) - - def security_converge(self, dry=False) -> dict: - """ - Migrates all permissions to the new names on all the Roles. - - This method is useful when you use: - - - ``class_permission_name`` - - ``previous_class_permission_name`` - - ``method_permission_name`` - - ``previous_method_permission_name`` - - :param dry: If True will not change DB - :return: Dict with all computed necessary operations - """ - return self.sm.security_converge(self.baseviews, self.menu, dry) - - def get_url_for_login_with(self, next_url: str | None = None) -> str: - return get_auth_manager().get_url_login(next_url=next_url) - - @property - def get_url_for_login(self): - return get_auth_manager().get_url_login() - - @property - def get_url_for_index(self): - return url_for(f"{self.indexview.endpoint}.{self.indexview.default_view}") - - def get_url_for_locale(self, lang): - return url_for( - f"{self.bm.locale_view.endpoint}.{self.bm.locale_view.default_view}", - locale=lang, - ) - - def add_limits(self, baseview) -> None: - if hasattr(baseview, "limits"): - self.sm.add_limit_view(baseview) - - def add_permissions(self, update_perms=False): - if self.update_perms or update_perms: - for baseview in self.baseviews: - self._add_permission(baseview, update_perms=update_perms) - self._add_menu_permissions(update_perms=update_perms) - - def _add_permission(self, baseview, update_perms=False): - if self.update_perms or update_perms: - try: - self.sm.add_permissions_view(baseview.base_permissions, baseview.class_permission_name) - except Exception as e: - log.exception(e) - log.error(LOGMSG_ERR_FAB_ADD_PERMISSION_VIEW, e) - - def _add_permissions_menu(self, name, update_perms=False): - if self.update_perms or update_perms: - try: - self.sm.add_permissions_menu(name) - except Exception as e: - log.exception(e) - log.error(LOGMSG_ERR_FAB_ADD_PERMISSION_MENU, e) - - def _add_menu_permissions(self, update_perms=False): - if self.update_perms or update_perms: - for category in self.menu.get_list(): - self._add_permissions_menu(category.name, update_perms=update_perms) - for item in category.childs: - # don't add permission for menu separator - if item.name != "-": - self._add_permissions_menu(item.name, update_perms=update_perms) - - def register_blueprint(self, baseview, endpoint=None, static_folder=None): - self.get_app.register_blueprint( - baseview.create_blueprint(self, endpoint=endpoint, static_folder=static_folder) - ) - - def _view_exists(self, view): - return any(baseview.__class__ == view.__class__ for baseview in self.baseviews) - - def _process_inner_views(self): - for view in self.baseviews: - for inner_class in view.get_uninit_inner_views(): - for v in self.baseviews: - if isinstance(v, inner_class) and v not in view.get_init_inner_views(): - view.get_init_inner_views().append(v) - - -def init_appbuilder(app: Flask) -> AirflowAppBuilder: - """Init `Flask App Builder `__.""" - return AirflowAppBuilder( - app=app, - session=settings.Session, - base_template="airflow/main.html", - ) diff --git a/airflow/www/extensions/init_views.py b/airflow/www/extensions/init_views.py deleted file mode 100644 index a45e008d77356..0000000000000 --- a/airflow/www/extensions/init_views.py +++ /dev/null @@ -1,336 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import logging -from functools import cached_property -from pathlib import Path -from typing import TYPE_CHECKING - -from connexion import FlaskApi, ProblemException, Resolver -from connexion.decorators.validation import RequestBodyValidator -from connexion.exceptions import BadRequestProblem -from flask import request -from werkzeug import Request - -from airflow.api_connexion.exceptions import common_error_handler -from airflow.api_fastapi.app import get_auth_manager -from airflow.configuration import conf -from airflow.security import permissions -from airflow.utils.yaml import safe_load -from airflow.www.constants import SWAGGER_BUNDLE, SWAGGER_ENABLED - -if TYPE_CHECKING: - from flask import Flask - -log = logging.getLogger(__name__) - -# airflow/www/extensions/init_views.py => airflow/ -ROOT_APP_DIR = Path(__file__).parents[2].resolve() - - -def init_flash_views(app): - """Init main app view - redirect to FAB.""" - from airflow.www.blueprints import routes - - app.register_blueprint(routes) - - -def init_appbuilder_views(app): - """Initialize Web UI views.""" - from airflow.models import import_all_models - - import_all_models() - - from airflow.www import views - - appbuilder = app.appbuilder - - # Remove the session from scoped_session registry to avoid - # reusing a session with a disconnected connection - appbuilder.session.remove() - appbuilder.add_view_no_menu(views.AutocompleteView()) - appbuilder.add_view_no_menu(views.Airflow()) - appbuilder.add_view( - views.DagRunModelView, - permissions.RESOURCE_DAG_RUN, - category=permissions.RESOURCE_BROWSE_MENU, - category_icon="fa-globe", - ) - appbuilder.add_view( - views.JobModelView, permissions.RESOURCE_JOB, category=permissions.RESOURCE_BROWSE_MENU - ) - appbuilder.add_view( - views.LogModelView, permissions.RESOURCE_AUDIT_LOG, category=permissions.RESOURCE_BROWSE_MENU - ) - appbuilder.add_view( - views.VariableModelView, permissions.RESOURCE_VARIABLE, category=permissions.RESOURCE_ADMIN_MENU - ) - appbuilder.add_view( - views.TaskInstanceModelView, - permissions.RESOURCE_TASK_INSTANCE, - category=permissions.RESOURCE_BROWSE_MENU, - ) - appbuilder.add_view( - views.TaskRescheduleModelView, - permissions.RESOURCE_TASK_RESCHEDULE, - category=permissions.RESOURCE_BROWSE_MENU, - ) - appbuilder.add_view( - views.TriggerModelView, - permissions.RESOURCE_TRIGGER, - category=permissions.RESOURCE_BROWSE_MENU, - ) - appbuilder.add_view( - views.ConfigurationView, - permissions.RESOURCE_CONFIG, - category=permissions.RESOURCE_ADMIN_MENU, - category_icon="fa-user", - ) - appbuilder.add_view( - views.ConnectionModelView, permissions.RESOURCE_CONNECTION, category=permissions.RESOURCE_ADMIN_MENU - ) - appbuilder.add_view( - views.PluginView, permissions.RESOURCE_PLUGIN, category=permissions.RESOURCE_ADMIN_MENU - ) - appbuilder.add_view( - views.ProviderView, permissions.RESOURCE_PROVIDER, category=permissions.RESOURCE_ADMIN_MENU - ) - appbuilder.add_view( - views.PoolModelView, permissions.RESOURCE_POOL, category=permissions.RESOURCE_ADMIN_MENU - ) - appbuilder.add_view( - views.XComModelView, permissions.RESOURCE_XCOM, category=permissions.RESOURCE_ADMIN_MENU - ) - appbuilder.add_view( - views.DagDependenciesView, - permissions.RESOURCE_DAG_DEPENDENCIES, - category=permissions.RESOURCE_BROWSE_MENU, - ) - # add_view_no_menu to change item position. - # I added link in extensions.init_appbuilder_links.init_appbuilder_links - appbuilder.add_view_no_menu(views.RedocView) - # Development views - appbuilder.add_view_no_menu(views.DevView) - appbuilder.add_view_no_menu(views.DocsView) - - -def init_plugins(app): - """Integrate Flask and FAB with plugins.""" - from airflow import plugins_manager - - plugins_manager.initialize_flask_plugins() - - appbuilder = app.appbuilder - - for view in plugins_manager.flask_appbuilder_views: - name = view.get("name") - if name: - filtered_view_kwargs = {k: v for k, v in view.items() if k not in ["view"]} - log.debug("Adding view %s with menu", name) - baseview = view.get("view") - if baseview: - appbuilder.add_view(baseview, **filtered_view_kwargs) - else: - log.error("'view' key is missing for the named view: %s", name) - else: - # if 'name' key is missing, intent is to add view without menu - log.debug("Adding view %s without menu", str(type(view["view"]))) - appbuilder.add_view_no_menu(view["view"]) - - for menu_link in sorted( - plugins_manager.flask_appbuilder_menu_links, key=lambda x: (x.get("category", ""), x["name"]) - ): - log.debug("Adding menu link %s to %s", menu_link["name"], menu_link["href"]) - appbuilder.add_link(**menu_link) - - for blue_print in plugins_manager.flask_blueprints: - log.debug("Adding blueprint %s:%s", blue_print["name"], blue_print["blueprint"].import_name) - app.register_blueprint(blue_print["blueprint"]) - - -def init_error_handlers(app: Flask): - """Add custom errors handlers.""" - from airflow.www import views - - app.register_error_handler(500, views.show_traceback) - app.register_error_handler(404, views.not_found) - - -def set_cors_headers_on_response(response): - """Add response headers.""" - allow_headers = conf.get("api", "access_control_allow_headers") - allow_methods = conf.get("api", "access_control_allow_methods") - allow_origins = conf.get("api", "access_control_allow_origins") - if allow_headers: - response.headers["Access-Control-Allow-Headers"] = allow_headers - if allow_methods: - response.headers["Access-Control-Allow-Methods"] = allow_methods - if allow_origins == "*": - response.headers["Access-Control-Allow-Origin"] = "*" - elif allow_origins: - allowed_origins = allow_origins.split(" ") - origin = request.environ.get("HTTP_ORIGIN", allowed_origins[0]) - if origin in allowed_origins: - response.headers["Access-Control-Allow-Origin"] = origin - return response - - -def init_data_form_parameters(): - """ - Initialize custom values for data form parameters. - - This is a workaround for Flask versions prior to 3.1.0. - In order to allow users customizing form data parameters, we need these two fields to be configurable. - Starting from Flask 3.1.0 these two parameters can be configured through Flask config, but unfortunately, - current version of flask supported in Airflow is way older. That's why this workaround was introduced. - See https://flask.palletsprojects.com/en/stable/api/#flask.Request.max_form_memory_size - # TODO: remove it when Flask upgraded to version 3.1.0 or higher. - """ - Request.max_form_parts = conf.getint("webserver", "max_form_parts") - Request.max_form_memory_size = conf.getint("webserver", "max_form_memory_size") - - -class _LazyResolution: - """ - OpenAPI endpoint that lazily resolves the function on first use. - - This is a stand-in replacement for ``connexion.Resolution`` that implements - its public attributes ``function`` and ``operation_id``, but the function - is only resolved when it is first accessed. - """ - - def __init__(self, resolve_func, operation_id): - self._resolve_func = resolve_func - self.operation_id = operation_id - - @cached_property - def function(self): - return self._resolve_func(self.operation_id) - - -class _LazyResolver(Resolver): - """ - OpenAPI endpoint resolver that loads lazily on first use. - - This re-implements ``connexion.Resolver.resolve()`` to not eagerly resolve - the endpoint function (and thus avoid importing it in the process), but only - return a placeholder that will be actually resolved when the contained - function is accessed. - """ - - def resolve(self, operation): - operation_id = self.resolve_operation_id(operation) - return _LazyResolution(self.resolve_function_from_operation_id, operation_id) - - -class _CustomErrorRequestBodyValidator(RequestBodyValidator): - """ - Custom request body validator that overrides error messages. - - By default, Connextion emits a very generic *None is not of type 'object'* - error when receiving an empty request body (with the view specifying the - body as non-nullable). We overrides it to provide a more useful message. - """ - - def validate_schema(self, data, url): - if not self.is_null_value_valid and data is None: - raise BadRequestProblem(detail="Request body must not be empty") - return super().validate_schema(data, url) - - -base_paths: list[str] = [] # contains the list of base paths that have api endpoints - - -def init_api_error_handlers(app: Flask) -> None: - """Add error handlers for 404 and 405 errors for existing API paths.""" - from airflow.www import views - - @app.errorhandler(404) - def _handle_api_not_found(ex): - if any([request.path.startswith(p) for p in base_paths]): - # 404 errors are never handled on the blueprint level - # unless raised from a view func so actual 404 errors, - # i.e. "no route for it" defined, need to be handled - # here on the application level - return common_error_handler(ex) - else: - return views.not_found(ex) - - @app.errorhandler(405) - def _handle_method_not_allowed(ex): - if any([request.path.startswith(p) for p in base_paths]): - return common_error_handler(ex) - else: - return views.method_not_allowed(ex) - - app.register_error_handler(ProblemException, common_error_handler) - - -def init_api_connexion(app: Flask) -> None: - """Initialize Stable API.""" - base_path = "/api/v1" - base_paths.append(base_path) - - with ROOT_APP_DIR.joinpath("api_connexion", "openapi", "v1.yaml").open() as f: - specification = safe_load(f) - api_bp = FlaskApi( - specification=specification, - resolver=_LazyResolver(), - base_path=base_path, - options={"swagger_ui": SWAGGER_ENABLED, "swagger_path": SWAGGER_BUNDLE.__fspath__()}, - strict_validation=True, - validate_responses=True, - validator_map={"body": _CustomErrorRequestBodyValidator}, - ).blueprint - api_bp.before_app_request(init_data_form_parameters) - api_bp.after_request(set_cors_headers_on_response) - - app.register_blueprint(api_bp) - app.extensions["csrf"].exempt(api_bp) - - -def init_api_internal(app: Flask, standalone_api: bool = False) -> None: - """Initialize Internal API.""" - if not standalone_api and not conf.getboolean("webserver", "run_internal_api", fallback=False): - return - - base_paths.append("/internal_api/v1") - with ROOT_APP_DIR.joinpath("api_internal", "openapi", "internal_api_v1.yaml").open() as f: - specification = safe_load(f) - api_bp = FlaskApi( - specification=specification, - base_path="/internal_api/v1", - options={"swagger_ui": SWAGGER_ENABLED, "swagger_path": SWAGGER_BUNDLE.__fspath__()}, - strict_validation=True, - validate_responses=True, - ).blueprint - api_bp.after_request(set_cors_headers_on_response) - - app.register_blueprint(api_bp) - app.after_request_funcs.setdefault(api_bp.name, []).append(set_cors_headers_on_response) - app.extensions["csrf"].exempt(api_bp) - - -def init_api_auth_provider(app): - """Initialize the API offered by the auth manager.""" - auth_mgr = get_auth_manager() - blueprint = auth_mgr.get_api_endpoints() - if blueprint: - base_paths.append(blueprint.url_prefix) - app.register_blueprint(blueprint) - app.extensions["csrf"].exempt(blueprint) diff --git a/airflow/www/forms.py b/airflow/www/forms.py deleted file mode 100644 index 7028e2026e449..0000000000000 --- a/airflow/www/forms.py +++ /dev/null @@ -1,231 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import datetime -import json -import operator -from collections.abc import Iterator -from functools import cache - -import pendulum -from flask_appbuilder.fieldwidgets import ( - BS3PasswordFieldWidget, - BS3TextAreaFieldWidget, - BS3TextFieldWidget, - Select2Widget, -) -from flask_appbuilder.forms import DynamicForm -from flask_babel import lazy_gettext -from flask_wtf import FlaskForm -from wtforms import widgets -from wtforms.fields import Field, IntegerField, PasswordField, SelectField, StringField, TextAreaField -from wtforms.validators import InputRequired, Optional - -from airflow.configuration import conf -from airflow.providers_manager import ProvidersManager -from airflow.utils.types import DagRunType -from airflow.www.validators import ReadOnly, ValidConnID -from airflow.www.widgets import ( - AirflowDateTimePickerROWidget, - AirflowDateTimePickerWidget, - BS3TextAreaROWidget, - BS3TextFieldROWidget, -) - - -class DateTimeWithTimezoneField(Field): - """A text field which stores a `datetime.datetime` matching a format.""" - - widget = widgets.TextInput() - - def __init__(self, label=None, validators=None, datetime_format="%Y-%m-%d %H:%M:%S%Z", **kwargs): - super().__init__(label, validators, **kwargs) - self.format = datetime_format - self.data = None - - def _value(self): - if self.raw_data: - return " ".join(self.raw_data) - if self.data: - return self.data.strftime(self.format) - return "" - - def process_formdata(self, valuelist): - if not valuelist: - return - date_str = " ".join(valuelist) - try: - # Check if the datetime string is in the format without timezone, if so convert it to the - # default timezone - if len(date_str) == 19: - parsed_datetime = datetime.datetime.strptime(date_str, "%Y-%m-%d %H:%M:%S") - default_timezone = self._get_default_timezone() - self.data = default_timezone.convert(parsed_datetime) - else: - self.data = pendulum.parse(date_str) - except ValueError: - self.data = None - raise ValueError(self.gettext("Not a valid datetime value")) - - def _get_default_timezone(self): - current_timezone = conf.get("core", "default_timezone") - if current_timezone == "system": - default_timezone = pendulum.local_timezone() - else: - default_timezone = pendulum.timezone(current_timezone) - return default_timezone - - -class DateTimeForm(FlaskForm): - """Date filter form needed for task views.""" - - logical_date = DateTimeWithTimezoneField("Logical date", widget=AirflowDateTimePickerWidget()) - - -class DagRunEditForm(DynamicForm): - """ - Form for editing DAG Run. - - Only note field is editable, so everything else is read-only here. - """ - - dag_id = StringField(lazy_gettext("Dag Id"), validators=[ReadOnly()], widget=BS3TextFieldROWidget()) - start_date = DateTimeWithTimezoneField( - lazy_gettext("Start Date"), validators=[ReadOnly()], widget=AirflowDateTimePickerROWidget() - ) - end_date = DateTimeWithTimezoneField( - lazy_gettext("End Date"), validators=[ReadOnly()], widget=AirflowDateTimePickerROWidget() - ) - run_id = StringField(lazy_gettext("Run Id"), validators=[ReadOnly()], widget=BS3TextFieldROWidget()) - state = StringField(lazy_gettext("State"), validators=[ReadOnly()], widget=BS3TextFieldROWidget()) - logical_date = DateTimeWithTimezoneField( - lazy_gettext("Logical Date"), - validators=[ReadOnly()], - widget=AirflowDateTimePickerROWidget(), - ) - conf = TextAreaField(lazy_gettext("Conf"), validators=[ReadOnly()], widget=BS3TextAreaROWidget()) - note = TextAreaField(lazy_gettext("User Note"), widget=BS3TextAreaFieldWidget()) - - def populate_obj(self, item): - """Populate the attributes of the passed obj with data from the form's not-read-only fields.""" - for name, field in self._fields.items(): - if not field.flags.readonly: - field.populate_obj(item, name) - item.run_type = DagRunType.from_run_id(item.run_id) - if item.conf: - item.conf = json.loads(item.conf) - - -class TaskInstanceEditForm(DynamicForm): - """ - Form for editing TaskInstance. - - Only note and state fields are editable, so everything else is read-only here. - """ - - dag_id = StringField( - lazy_gettext("Dag Id"), validators=[InputRequired(), ReadOnly()], widget=BS3TextFieldROWidget() - ) - task_id = StringField( - lazy_gettext("Task Id"), validators=[InputRequired(), ReadOnly()], widget=BS3TextFieldROWidget() - ) - start_date = DateTimeWithTimezoneField( - lazy_gettext("Start Date"), validators=[ReadOnly()], widget=AirflowDateTimePickerROWidget() - ) - end_date = DateTimeWithTimezoneField( - lazy_gettext("End Date"), validators=[ReadOnly()], widget=AirflowDateTimePickerROWidget() - ) - state = SelectField( - lazy_gettext("State"), - choices=( - ("success", "success"), - ("running", "running"), - ("failed", "failed"), - ("up_for_retry", "up_for_retry"), - ), - widget=Select2Widget(), - validators=[InputRequired()], - ) - logical_date = DateTimeWithTimezoneField( - lazy_gettext("Logical Date"), - widget=AirflowDateTimePickerROWidget(), - validators=[InputRequired(), ReadOnly()], - ) - note = TextAreaField(lazy_gettext("User Note"), widget=BS3TextAreaFieldWidget()) - - def populate_obj(self, item): - """Populate the attributes of the passed obj with data from the form's not-read-only fields.""" - for name, field in self._fields.items(): - if not field.flags.readonly: - field.populate_obj(item, name) - - -@cache -def create_connection_form_class() -> type[DynamicForm]: - """ - Create a form class for editing and adding Connection. - - This class is created dynamically because it relies heavily on run-time - provider discovery, which slows down webserver startup a lot. - By creating the class at runtime, we can delay loading the providers until - when the connection form is first used, which may as well be never for a - short-lived server. - """ - providers_manager = ProvidersManager() - - def _iter_connection_types() -> Iterator[tuple[str, str]]: - """List available connection types.""" - for connection_type, provider_info in providers_manager.hooks.items(): - if provider_info: - yield (connection_type, provider_info.hook_name) - - class ConnectionForm(DynamicForm): - def process(self, formdata=None, obj=None, **kwargs): - super().process(formdata=formdata, obj=obj, **kwargs) - for field in self._fields.values(): - if isinstance(getattr(field, "data", None), str): - field.data = field.data.strip() - - conn_id = StringField( - lazy_gettext("Connection Id"), - validators=[InputRequired(), ValidConnID()], - widget=BS3TextFieldWidget(), - ) - conn_type = SelectField( - lazy_gettext("Connection Type"), - choices=sorted(_iter_connection_types(), key=operator.itemgetter(1)), - widget=Select2Widget(), - validators=[InputRequired()], - description=( - "Connection Type missing? Make sure you've installed the " - "corresponding Airflow Provider Package." - ), - ) - description = StringField(lazy_gettext("Description"), widget=BS3TextAreaFieldWidget()) - host = StringField(lazy_gettext("Host"), widget=BS3TextFieldWidget()) - schema = StringField(lazy_gettext("Schema"), widget=BS3TextFieldWidget()) - login = StringField(lazy_gettext("Login"), widget=BS3TextFieldWidget()) - password = PasswordField(lazy_gettext("Password"), widget=BS3PasswordFieldWidget()) - port = IntegerField(lazy_gettext("Port"), validators=[Optional()], widget=BS3TextFieldWidget()) - extra = TextAreaField(lazy_gettext("Extra"), widget=BS3TextAreaFieldWidget()) - - for key, value in providers_manager.connection_form_widgets.items(): - setattr(ConnectionForm, key, value.field) - - return ConnectionForm diff --git a/airflow/www/jest-setup.js b/airflow/www/jest-setup.js deleted file mode 100644 index d12185f713548..0000000000000 --- a/airflow/www/jest-setup.js +++ /dev/null @@ -1,69 +0,0 @@ -// We need this lint rule for now because these are only dev-dependencies -/* eslint-disable import/no-extraneous-dependencies */ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import "@testing-library/jest-dom"; -import axios from "axios"; -import { setLogger } from "react-query"; -import "jest-canvas-mock"; - -// eslint-disable-next-line import/no-extraneous-dependencies -import moment from "moment-timezone"; - -axios.defaults.adapter = require("axios/lib/adapters/http"); - -axios.interceptors.response.use((res) => res.data || res); - -setLogger({ - /* eslint-disable no-console */ - log: console.log, - warn: console.warn, - /* eslint-enable no-console */ - // ✅ no more errors on the console - error: () => {}, -}); - -// Mock global objects we use across the app -global.stateColors = { - deferred: "mediumpurple", - failed: "red", - queued: "gray", - removed: "lightgrey", - restarting: "violet", - running: "lime", - scheduled: "tan", - skipped: "hotpink", - success: "green", - up_for_reschedule: "turquoise", - up_for_retry: "gold", - upstream_failed: "orange", -}; - -global.defaultDagRunDisplayNumber = 245; - -global.filtersOptions = { - // Must stay in sync with airflow/www/static/js/types/index.ts - dagStates: ["success", "running", "queued", "failed"], - runTypes: ["manual", "backfill", "scheduled", "asset_triggered"], -}; - -global.moment = moment; - -global.autoRefreshInterval = undefined; diff --git a/airflow/www/jest.config.js b/airflow/www/jest.config.js deleted file mode 100644 index 333a061268c0a..0000000000000 --- a/airflow/www/jest.config.js +++ /dev/null @@ -1,64 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -const config = { - verbose: true, - transform: { - "^.+\\.[jt]sx?$": "babel-jest", - }, - testEnvironment: "jsdom", - setupFilesAfterEnv: ["./jest-setup.js"], - moduleDirectories: ["node_modules"], - moduleNameMapper: { - // Listing all aliases - "^src/(.*)$": "/static/js/$1", - }, - transformIgnorePatterns: [ - `node_modules/(?!${[ - // specify modules that needs to be transformed for jest. (esm modules) - "ansi_up", - "axios", - "bail", - "ccount", - "character-entities", - "comma-separated-tokens", - "decode-named-character-reference", - "escape-string-regexp", - "hast", - "is-plain-obj", - "markdown-table", - "mdast", - "micromark", - "property-information", - "react-markdown", - "remark-gfm", - "remark-parse", - "remark-rehype", - "space-separated-tokens", - "trim-lines", - "trough", - "unified", - "unist", - "vfile", - "vfile-message", - ].join("|")})`, - ], -}; - -module.exports = config; diff --git a/airflow/www/package.json b/airflow/www/package.json deleted file mode 100644 index 638e7821396ad..0000000000000 --- a/airflow/www/package.json +++ /dev/null @@ -1,156 +0,0 @@ -{ - "name": "airflow-www", - "version": "1.0.0", - "description": "Apache Airflow is a platform to programmatically author, schedule and monitor workflows.", - "scripts": { - "test": "jest", - "dev": "NODE_ENV=development webpack --watch --progress --devtool eval-cheap-source-map --mode development", - "prod": "NODE_ENV=production node --max_old_space_size=4096 ./node_modules/webpack/bin/webpack.js --mode production --progress", - "build": "NODE_ENV=production webpack --progress --mode production", - "lint": "eslint --ignore-path=.eslintignore --max-warnings=0 --ext .js,.jsx,.ts,.tsx . && tsc", - "lint:fix": "eslint --fix --ignore-path=.eslintignore --ext .js,.jsx,.ts,.tsx . && tsc", - "format": "yarn prettier --write .", - "generate-api-types": "npx openapi-typescript \"../api_connexion/openapi/v1.yaml\" --output static/js/types/api-generated.ts && node alias-rest-types.js static/js/types/api-generated.ts" - }, - "author": "Apache", - "license": "Apache-2.0", - "repository": { - "type": "git", - "url": "git+https://github.com/apache/airflow.git" - }, - "homepage": "https://airflow.apache.org/", - "keywords": [ - "big", - "data", - "workflow", - "airflow", - "d3", - "nerds", - "database", - "flask" - ], - "browserslist": { - "production": [ - ">0.2%", - "not dead", - "not op_mini all" - ], - "development": [ - "last 1 chrome version", - "last 1 firefox version", - "last 1 safari version" - ] - }, - "devDependencies": { - "@babel/core": "^7.24.7", - "@babel/eslint-parser": "^7.24.7", - "@babel/plugin-transform-runtime": "^7.24.7", - "@babel/preset-env": "^7.24.7", - "@babel/preset-react": "^7.24.7", - "@babel/preset-typescript": "^7.24.7", - "@testing-library/jest-dom": "^5.16.0", - "@testing-library/react": "^13.0.0", - "@types/color": "^3.0.3", - "@types/json-to-pretty-yaml": "^1.2.1", - "@types/react": "^18.0.12", - "@types/react-dom": "^18.0.5", - "@types/react-syntax-highlighter": "^15.5.6", - "@types/react-table": "^7.7.12", - "@typescript-eslint/eslint-plugin": "^5.13.0", - "@typescript-eslint/parser": "^5.0.0", - "babel-jest": "^27.3.1", - "babel-loader": "^9.1.0", - "clean-webpack-plugin": "^3.0.0", - "copy-webpack-plugin": "^6.0.3", - "css-loader": "5.2.7", - "css-minimizer-webpack-plugin": "^4.0.0", - "eslint": "^8.6.0", - "eslint-config-airbnb": "^19.0.4", - "eslint-config-airbnb-typescript": "^17.0.0", - "eslint-config-prettier": "^8.6.0", - "eslint-plugin-html": "^6.0.2", - "eslint-plugin-import": "^2.27.5", - "eslint-plugin-jsx-a11y": "^6.5.0", - "eslint-plugin-node": "^11.1.0", - "eslint-plugin-promise": "^4.2.1", - "eslint-plugin-react": "^7.30.0", - "eslint-plugin-react-hooks": "^4.5.0", - "eslint-plugin-standard": "^4.0.1", - "file-loader": "^6.0.0", - "imports-loader": "^1.1.0", - "jest": "^27.3.1", - "jest-canvas-mock": "^2.5.1", - "mini-css-extract-plugin": "^1.6.2", - "moment": "^2.29.4", - "moment-locales-webpack-plugin": "^1.2.0", - "openapi-typescript": "^5.4.1", - "prettier": "^2.8.4", - "style-loader": "^1.2.1", - "stylelint": "^15.10.1", - "stylelint-config-prettier": "^9.0.5", - "stylelint-config-standard": "^20.0.0", - "terser-webpack-plugin": "<5.0.0", - "typescript": "^4.6.3", - "url-loader": "4.1.0", - "web-worker": "^1.2.0", - "webpack": "^5.94.0", - "webpack-cli": "^4.0.0", - "webpack-license-plugin": "^4.2.1", - "webpack-manifest-plugin": "^4.0.0" - }, - "dependencies": { - "@chakra-ui/anatomy": "^2.2.2", - "@chakra-ui/react": "2.4.2", - "@emotion/cache": "^11.9.3", - "@emotion/react": "^11.9.3", - "@emotion/styled": "^11", - "@tanstack/react-table": "^8.13.2", - "@visx/group": "^2.10.0", - "@visx/shape": "^2.12.2", - "ansi_up": "^6.0.2", - "axios": "^1.7.4", - "camelcase-keys": "^7.0.0", - "chakra-react-select": "^4.0.0", - "codemirror": "^5.59.1", - "color": "^4.2.3", - "d3": "^3.4.4", - "d3-selection": "^3.0.0", - "d3-shape": "^2.1.0", - "d3-tip": "^0.9.1", - "dagre-d3": "^0.6.4", - "echarts": "^5.4.2", - "elkjs": "^0.7.1", - "eonasdan-bootstrap-datetimepicker": "^4.17.47", - "framer-motion": "^6.0.0", - "jquery": ">=3.5.0", - "jquery-ui": "1.13.3", - "jshint": "^2.13.4", - "json-to-pretty-yaml": "^1.2.2", - "lodash": "^4.17.21", - "moment-timezone": "^0.5.43", - "react": "^18.0.0", - "react-dom": "^18.0.0", - "react-icons": "^5.2.1", - "react-json-view": "^1.21.3", - "react-markdown": "^8.0.4", - "react-query": "^3.39.1", - "react-router-dom": "^6.3.0", - "react-syntax-highlighter": "^15.5.0", - "react-table": "^7.8.0", - "react-textarea-autosize": "^8.3.4", - "reactflow": "^11.7.4", - "redoc": "^2.0.0-rc.72", - "remark-gfm": "^3.0.1", - "swagger-ui-dist": "4.1.3", - "tsconfig-paths": "^3.14.2", - "type-fest": "^2.17.0", - "url-search-params-polyfill": "^8.1.0", - "validator": "^13.9.0" - }, - "resolutions": { - "d3-color": "^3.1.0", - "braces": "^3.0.3", - "json5": ">=2.2.2", - "moment-timezone": ">=0.5.35" - } -} diff --git a/airflow/www/security_manager.py b/airflow/www/security_manager.py deleted file mode 100644 index d02d7efcf774a..0000000000000 --- a/airflow/www/security_manager.py +++ /dev/null @@ -1,314 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from functools import cached_property -from typing import TYPE_CHECKING, Callable - -from flask import g -from flask_limiter import Limiter -from flask_limiter.util import get_remote_address -from sqlalchemy import select - -from airflow.api_fastapi.app import get_auth_manager -from airflow.auth.managers.models.resource_details import ( - AccessView, - ConnectionDetails, - DagAccessEntity, - DagDetails, - PoolDetails, - VariableDetails, -) -from airflow.auth.managers.utils.fab import ( - get_method_from_fab_action_map, -) -from airflow.exceptions import AirflowException -from airflow.models import Connection, DagRun, Pool, TaskInstance, Variable -from airflow.security.permissions import ( - RESOURCE_ADMIN_MENU, - RESOURCE_ASSET, - RESOURCE_AUDIT_LOG, - RESOURCE_BROWSE_MENU, - RESOURCE_CLUSTER_ACTIVITY, - RESOURCE_CONFIG, - RESOURCE_CONNECTION, - RESOURCE_DAG, - RESOURCE_DAG_CODE, - RESOURCE_DAG_DEPENDENCIES, - RESOURCE_DAG_RUN, - RESOURCE_DOCS, - RESOURCE_DOCS_MENU, - RESOURCE_JOB, - RESOURCE_PLUGIN, - RESOURCE_POOL, - RESOURCE_PROVIDER, - RESOURCE_SLA_MISS, - RESOURCE_TASK_INSTANCE, - RESOURCE_TASK_RESCHEDULE, - RESOURCE_TRIGGER, - RESOURCE_VARIABLE, - RESOURCE_XCOM, -) -from airflow.utils.log.logging_mixin import LoggingMixin -from airflow.www.utils import CustomSQLAInterface - -EXISTING_ROLES = { - "Admin", - "Viewer", - "User", - "Op", - "Public", -} - -if TYPE_CHECKING: - from airflow.auth.managers.models.base_user import BaseUser - - -class AirflowSecurityManagerV2(LoggingMixin): - """ - Custom security manager, which introduces a permission model adapted to Airflow. - - It's named V2 to differentiate it from the obsolete airflow.www.security.AirflowSecurityManager. - """ - - def __init__(self, appbuilder) -> None: - super().__init__() - self.appbuilder = appbuilder - - # Setup Flask-Limiter - self.limiter = self.create_limiter() - - # Go and fix up the SQLAInterface used from the stock one to our subclass. - # This is needed to support the "hack" where we had to edit - # FieldConverter.conversion_table in place in airflow.www.utils - for attr in dir(self): - if attr.endswith("view"): - view = getattr(self, attr, None) - if view and getattr(view, "datamodel", None): - view.datamodel = CustomSQLAInterface(view.datamodel.obj) - - @staticmethod - def before_request(): - """Run hook before request.""" - g.user = get_auth_manager().get_user() - - def create_limiter(self) -> Limiter: - app = self.appbuilder.get_app - limiter = Limiter(key_func=app.config.get("RATELIMIT_KEY_FUNC", get_remote_address)) - limiter.init_app(app) - return limiter - - def register_views(self): - """Allow auth managers to register their own views. By default, do nothing.""" - pass - - def has_access( - self, action_name: str, resource_name: str, user=None, resource_pk: str | None = None - ) -> bool: - """ - Verify whether a given user could perform a certain action on the given resource. - - Example actions might include can_read, can_write, can_delete, etc. - - This function is called by FAB when accessing a view. See - https://github.com/dpgaspar/Flask-AppBuilder/blob/c6fecdc551629e15467fde5d06b4437379d90592/flask_appbuilder/security/decorators.py#L134 - - :param action_name: action_name on resource (e.g can_read, can_edit). - :param resource_name: name of view-menu or resource. - :param user: user - :param resource_pk: the resource primary key (e.g. the connection ID) - :return: Whether user could perform certain action on the resource. - """ - if not user: - user = g.user - - is_authorized_method = self._get_auth_manager_is_authorized_method(resource_name) - return is_authorized_method(action_name, resource_pk, user) - - def create_admin_standalone(self) -> tuple[str | None, str | None]: - """ - Perform the required steps when initializing airflow for standalone mode. - - If necessary, returns the username and password to be printed in the console for users to log in. - """ - return None, None - - def add_limit_view(self, baseview): - if not baseview.limits: - return - - for limit in baseview.limits: - self.limiter.limit( - limit_value=limit.limit_value, - key_func=limit.key_func, - per_method=limit.per_method, - methods=limit.methods, - error_message=limit.error_message, - exempt_when=limit.exempt_when, - override_defaults=limit.override_defaults, - deduct_when=limit.deduct_when, - on_breach=limit.on_breach, - cost=limit.cost, - )(baseview.blueprint) - - @cached_property - def _auth_manager_is_authorized_map( - self, - ) -> dict[str, Callable[[str, str | None, BaseUser | None], bool]]: - """ - Return the map associating a FAB resource name to the corresponding auth manager is_authorized_ API. - - The function returned takes the FAB action name and the user as parameter. - """ - auth_manager = get_auth_manager() - methods = get_method_from_fab_action_map() - - session = self.appbuilder.session - - def get_connection_id(resource_pk): - if not resource_pk: - return None - conn_id = session.scalar(select(Connection.conn_id).where(Connection.id == resource_pk).limit(1)) - if not conn_id: - raise AirflowException("Connection not found") - return conn_id - - def get_dag_id_from_dagrun_id(resource_pk): - if not resource_pk: - return None - dag_id = session.scalar(select(DagRun.dag_id).where(DagRun.id == resource_pk).limit(1)) - if not dag_id: - raise AirflowException("DagRun not found") - return dag_id - - def get_dag_id_from_task_instance(resource_pk): - if not resource_pk: - return None - dag_id = session.scalar( - select(TaskInstance.dag_id).where(TaskInstance.id == resource_pk).limit(1) - ) - if not dag_id: - raise AirflowException("Task instance not found") - return dag_id - - def get_pool_name(resource_pk): - if not resource_pk: - return None - pool = session.scalar(select(Pool).where(Pool.id == resource_pk).limit(1)) - if not pool: - raise AirflowException("Pool not found") - return pool.pool - - def get_variable_key(resource_pk): - if not resource_pk: - return None - variable = session.scalar(select(Variable).where(Variable.id == resource_pk).limit(1)) - if not variable: - raise AirflowException("Variable not found") - return variable.key - - def _is_authorized_view(view_): - return lambda action, resource_pk, user: auth_manager.is_authorized_view( - access_view=view_, - user=user, - ) - - def _is_authorized_dag(entity_=None, details_func_=None): - return lambda action, resource_pk, user: auth_manager.is_authorized_dag( - method=methods[action], - access_entity=entity_, - details=DagDetails(id=details_func_(resource_pk)) if details_func_ else None, - user=user, - ) - - mapping = { - RESOURCE_CONFIG: lambda action, resource_pk, user: auth_manager.is_authorized_configuration( - method=methods[action], - user=user, - ), - RESOURCE_CONNECTION: lambda action, resource_pk, user: auth_manager.is_authorized_connection( - method=methods[action], - details=ConnectionDetails(conn_id=get_connection_id(resource_pk)), - user=user, - ), - RESOURCE_ASSET: lambda action, resource_pk, user: auth_manager.is_authorized_asset( - method=methods[action], - user=user, - ), - RESOURCE_POOL: lambda action, resource_pk, user: auth_manager.is_authorized_pool( - method=methods[action], - details=PoolDetails(name=get_pool_name(resource_pk)), - user=user, - ), - RESOURCE_VARIABLE: lambda action, resource_pk, user: auth_manager.is_authorized_variable( - method=methods[action], - details=VariableDetails(key=get_variable_key(resource_pk)), - user=user, - ), - } - for resource, entity, details_func in [ - (RESOURCE_DAG, None, None), - (RESOURCE_AUDIT_LOG, DagAccessEntity.AUDIT_LOG, None), - (RESOURCE_DAG_CODE, DagAccessEntity.CODE, None), - (RESOURCE_DAG_DEPENDENCIES, DagAccessEntity.DEPENDENCIES, None), - (RESOURCE_SLA_MISS, DagAccessEntity.SLA_MISS, None), - (RESOURCE_TASK_RESCHEDULE, DagAccessEntity.TASK_RESCHEDULE, None), - (RESOURCE_XCOM, DagAccessEntity.XCOM, None), - (RESOURCE_DAG_RUN, DagAccessEntity.RUN, get_dag_id_from_dagrun_id), - (RESOURCE_TASK_INSTANCE, DagAccessEntity.TASK_INSTANCE, get_dag_id_from_task_instance), - ]: - mapping[resource] = _is_authorized_dag(entity, details_func) - for resource, view in [ - (RESOURCE_CLUSTER_ACTIVITY, AccessView.CLUSTER_ACTIVITY), - (RESOURCE_DOCS, AccessView.DOCS), - (RESOURCE_PLUGIN, AccessView.PLUGINS), - (RESOURCE_JOB, AccessView.JOBS), - (RESOURCE_PROVIDER, AccessView.PROVIDERS), - (RESOURCE_TRIGGER, AccessView.TRIGGERS), - ]: - mapping[resource] = _is_authorized_view(view) - return mapping - - def _get_auth_manager_is_authorized_method(self, fab_resource_name: str) -> Callable: - is_authorized_method = self._auth_manager_is_authorized_map.get(fab_resource_name) - if is_authorized_method: - return is_authorized_method - elif fab_resource_name in [RESOURCE_DOCS_MENU, RESOURCE_ADMIN_MENU, RESOURCE_BROWSE_MENU]: - # Display the "Browse", "Admin" and "Docs" dropdowns in the menu if the user has access to at - # least one dropdown child - return self._is_authorized_category_menu(fab_resource_name) - else: - # The user is trying to access a page specific to the auth manager - # (e.g. the user list view in FabAuthManager) or a page defined in a plugin - return lambda action, resource_pk, user: get_auth_manager().is_authorized_custom_view( - method=get_method_from_fab_action_map().get(action, action), - resource_name=fab_resource_name, - user=user, - ) - - def _is_authorized_category_menu(self, category: str) -> Callable: - items = {item.name for item in self.appbuilder.menu.find(category).childs} - return lambda action, resource_pk, user: any( - self._get_auth_manager_is_authorized_method(fab_resource_name=item)(action, resource_pk, user) - for item in items - ) - - def add_permissions_view(self, base_action_names, resource_name): - pass - - def add_permissions_menu(self, resource_name): - pass diff --git a/airflow/www/static/airflow.gif b/airflow/www/static/airflow.gif deleted file mode 100644 index 1889b868f708a..0000000000000 Binary files a/airflow/www/static/airflow.gif and /dev/null differ diff --git a/airflow/www/static/css/bootstrap-theme.css b/airflow/www/static/css/bootstrap-theme.css deleted file mode 100644 index 94e1ee5887715..0000000000000 --- a/airflow/www/static/css/bootstrap-theme.css +++ /dev/null @@ -1,6215 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/* stylelint-disable no-descending-specificity, rule-empty-line-before, no-duplicate-selectors */ - -/*! - * Bootstrap v3.2 - * - * Copyright 2014 Twitter, Inc - * Licensed under the Apache License v2.0 - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Designed and built with all the love in the world by @mdo and @fat. - * BootSwatchr built and provided by @DrewStrickland - */ - -/*! normalize.css v3.0.2 | MIT License | git.io/normalize */ -html { - font-family: sans-serif; - -ms-text-size-adjust: 100%; - -webkit-text-size-adjust: 100%; -} -html[data-color-scheme="dark"] { - filter: invert(100%) hue-rotate(180deg) saturate(90%) contrast(85%); -} - -/* Default icons to not display until the data-color-scheme has been set */ -#dark_icon, -#light_icon { - display: none; -} -html[data-color-scheme="dark"] #dark_icon { - display: block; -} -html[data-color-scheme="dark"] #light_icon { - display: none; -} -html[data-color-scheme="light"] #dark_icon { - display: none; -} -html[data-color-scheme="light"] #light_icon { - display: block; -} -body { - margin: 0; -} -article, -aside, -details, -figcaption, -figure, -footer, -header, -hgroup, -main, -menu, -nav, -section, -summary { - display: block; -} -audio, -canvas, -progress, -video { - display: inline-block; - vertical-align: baseline; -} -audio:not([controls]) { - display: none; - height: 0; -} -[hidden], -template { - display: none; -} -a { - background-color: transparent; -} -a:active, -a:hover { - outline: 0; -} -abbr[title] { - border-bottom: 1px dotted; -} -b, -strong { - font-weight: bold; -} -dfn { - font-style: italic; -} -h1 { - font-size: 2em; - margin: 0.67em 0; -} -mark { - background: #ff0; - color: #000; -} -small { - font-size: 80%; -} -sub, -sup { - font-size: 75%; - line-height: 0; - position: relative; - vertical-align: baseline; -} -sup { - top: -0.5em; -} -sub { - bottom: -0.25em; -} -img { - border: 0; -} -svg:not(:root) { - overflow: hidden; -} -figure { - margin: 1em 40px; -} -hr { - -moz-box-sizing: content-box; - box-sizing: content-box; - height: 0; -} -pre { - overflow: auto; -} -code, -kbd, -pre, -samp { - font-family: monospace; - font-size: 1em; -} -button, -input, -optgroup, -select, -textarea { - color: inherit; - font: inherit; - margin: 0; -} -button { - overflow: visible; -} -button, -select { - text-transform: none; -} -button, -html input[type="button"], -input[type="reset"], -input[type="submit"] { - -webkit-appearance: button; - cursor: pointer; -} -button[disabled], -html input[disabled] { - cursor: default; -} -button::-moz-focus-inner, -input::-moz-focus-inner { - border: 0; - padding: 0; -} -input { - line-height: normal; -} -input[type="checkbox"], -input[type="radio"] { - box-sizing: border-box; - padding: 0; -} -input[type="number"]::-webkit-inner-spin-button, -input[type="number"]::-webkit-outer-spin-button { - height: auto; -} -input[type="search"] { - box-sizing: border-box; - -webkit-appearance: none; -} -input[type="search"]::-webkit-search-cancel-button, -input[type="search"]::-webkit-search-decoration { - -webkit-appearance: none; -} -fieldset { - border: 1px solid #c0c0c0; - margin: 0 2px; - padding: 0.35em 0.625em 0.75em; -} -legend { - border: 0; - padding: 0; -} -textarea { - overflow: auto; -} -optgroup { - font-weight: bold; -} -table { - border-collapse: collapse; - border-spacing: 0; -} -td, -th { - padding: 0; -} - -/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */ -@media print { - *, - *::before, - *::after { - background: transparent !important; - color: #000 !important; - box-shadow: none !important; - text-shadow: none !important; - } - a, - a:visited { - text-decoration: underline; - } - a[href]::after { - content: " (" attr(href) ")"; - } - abbr[title]::after { - content: " (" attr(title) ")"; - } - a[href^="#"]::after, - a[href^="javascript:"]::after { - content: ""; - } - pre, - blockquote { - page-break-inside: avoid; - } - thead { - display: table-header-group; - } - tr, - img { - page-break-inside: avoid; - } - img { - max-width: 100% !important; - } - p, - h2, - h3 { - orphans: 3; - widows: 3; - } - h2, - h3 { - page-break-after: avoid; - } - select { - background: #fff !important; - } - .navbar { - display: none; - } - .btn > .caret, - .dropup > .btn > .caret { - border-top-color: #cbcbcb !important; - } - .label { - border: 1px solid #51504f; - } - .table { - border-collapse: collapse !important; - } - .table td, - .table th { - background-color: #fff !important; - } - .table-bordered th, - .table-bordered td { - border: 1px solid #f0f0f0 !important; - } -} -* { - -webkit-box-sizing: border-box; - -moz-box-sizing: border-box; - box-sizing: border-box; -} -*::before, -*::after { - -webkit-box-sizing: border-box; - -moz-box-sizing: border-box; - box-sizing: border-box; -} -html { - font-size: 12px; - -webkit-tap-highlight-color: rgba(0, 0, 0, 0); -} -body { - font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; - font-size: 14px; - line-height: 1.428571429; - color: #51504f; -} -input, -button, -select, -textarea { - font-family: inherit; - font-size: inherit; - line-height: inherit; -} -a, -a code { - color: #365f84; - text-decoration: none; -} -a:hover, -a:focus { - color: #017cee; - text-decoration: none; -} -a:focus { - outline: thin dotted; - outline: 5px auto -webkit-focus-ring-color; - outline-offset: -2px; -} -figure { - margin: 0; -} -img { - vertical-align: middle; -} -.img-responsive, -.thumbnail > img, -.thumbnail a > img, -.carousel-inner > .item > img, -.carousel-inner > .item > a > img { - display: block; - max-width: 100%; - height: auto; -} -.img-rounded { - border-radius: 6px; -} -.img-thumbnail { - padding: 4px; - line-height: 1.428571429; - background-color: #fafafa; - border: 1px solid #ddd; - border-radius: 4px; - -webkit-transition: all 0.2s ease-in-out; - -o-transition: all 0.2s ease-in-out; - transition: all 0.2s ease-in-out; - display: inline-block; - max-width: 100%; - height: auto; -} -.img-circle { - border-radius: 50%; -} -hr { - margin-top: 15px; - margin-bottom: 15px; - border: 0; - border-top: 2px solid #f0f0f0; -} -.sr-only { - position: absolute; - width: 1px; - height: 1px; - margin: -1px; - padding: 0; - overflow: hidden; - clip: rect(0, 0, 0, 0); - border: 0; -} -.sr-only-focusable:active, -.sr-only-focusable:focus { - position: static; - width: auto; - height: auto; - margin: 0; - overflow: visible; - clip: auto; -} -h1, -h2, -h3, -h4, -h5, -h6, -.h1, -.h2, -.h3, -.h4, -.h5, -.h6 { - font-family: inherit; - font-weight: 500; - line-height: 1.1; - color: inherit; -} -h1 small, -h2 small, -h3 small, -h4 small, -h5 small, -h6 small, -.h1 small, -.h2 small, -.h3 small, -.h4 small, -.h5 small, -.h6 small, -h1 .small, -h2 .small, -h3 .small, -h4 .small, -h5 .small, -h6 .small, -.h1 .small, -.h2 .small, -.h3 .small, -.h4 .small, -.h5 .small, -.h6 .small { - font-weight: normal; - line-height: 1; - color: #777; -} -h1, -.h1, -h2, -.h2, -h3, -.h3 { - margin-top: 20px; - margin-bottom: 10px; -} -h1 small, -.h1 small, -h2 small, -.h2 small, -h3 small, -.h3 small, -h1 .small, -.h1 .small, -h2 .small, -.h2 .small, -h3 .small, -.h3 .small { - font-size: 65%; -} -h4, -.h4, -h5, -.h5, -h6, -.h6 { - margin-top: 10px; - margin-bottom: 10px; -} -h4 small, -.h4 small, -h5 small, -.h5 small, -h6 small, -.h6 small, -h4 .small, -.h4 .small, -h5 .small, -.h5 .small, -h6 .small, -.h6 .small { - font-size: 75%; -} -h1, -.h1 { - font-size: 36px; -} -h2, -.h2 { - font-size: 30px; -} -h3, -.h3 { - font-size: 24px; -} -h4, -.h4 { - font-size: 18px; -} -h5, -.h5 { - font-size: 14px; -} -h6, -.h6 { - font-size: 12px; -} -p { - margin: 0 0 10px; -} -.lead { - margin-bottom: 20px; - font-size: 16px; - font-weight: 300; - line-height: 1.4; -} - -@media (min-width: 768px) { - .lead { - font-size: 21px; - } -} -small, -.small { - font-size: 85%; -} -mark, -.mark { - background-color: #fcf8e3; - padding: 0.2em; -} -.text-left { - text-align: left; -} -.text-right { - text-align: right; -} -.text-center { - text-align: center; -} -.text-justify { - text-align: justify; -} -.text-nowrap { - white-space: nowrap; -} -.text-lowercase { - text-transform: lowercase; -} -.text-uppercase { - text-transform: uppercase; -} -.text-capitalize { - text-transform: capitalize; -} -.text-muted { - color: #cbcbcb; -} -.text-primary { - color: #017cee; -} -a.text-primary:hover { - color: #002e33; -} -.text-success { - color: #1b8e49; -} -a.text-success:hover { - color: #366f4c; -} -.text-info { - color: #31708f; -} -a.text-info:hover { - color: #245269; -} -.text-warning { - color: #8a6d3b; -} -a.text-warning:hover { - color: #66512c; -} -.text-danger { - color: #e43921; -} -a.text-danger:hover { - color: #843534; -} -.bg-primary { - color: #fff; - background-color: #017cee; -} -a.bg-primary:hover { - background-color: #002e33; -} -.bg-success { - background-color: #bfebd1; -} -a.bg-success:hover { - background-color: #c1e2b3; -} -.bg-info { - background-color: #d9edf7; -} -a.bg-info:hover { - background-color: #afd9ee; -} -.bg-warning { - background-color: #fcf8e3; -} -a.bg-warning:hover { - background-color: #f7ecb5; -} -.bg-danger { - background-color: #f2dede; -} -a.bg-danger:hover { - background-color: #e4b9b9; -} -.page-header { - padding-bottom: 9px; - margin: 40px 0 20px; - border-bottom: 1px solid #eee; -} -ul, -ol { - margin-top: 0; - margin-bottom: 10px; -} -ul ul, -ol ul, -ul ol, -ol ol { - margin-bottom: 0; -} -.list-unstyled { - padding-left: 0; - list-style: none; -} -.list-inline { - padding-left: 0; - list-style: none; - margin-left: -5px; -} -.list-inline > li { - display: inline-block; - padding-left: 5px; - padding-right: 5px; -} -dl { - margin-top: 0; - margin-bottom: 20px; -} -dt, -dd { - line-height: 1.428571429; -} -dt { - font-weight: bold; -} -dd { - margin-left: 0; -} - -@media (min-width: 768px) { - .dl-horizontal dt { - float: left; - width: 160px; - clear: left; - text-align: right; - overflow: hidden; - text-overflow: ellipsis; - white-space: nowrap; - } - .dl-horizontal dd { - margin-left: 180px; - } -} -abbr[title], -abbr[data-original-title] { - cursor: help; - border-bottom: 1px dotted #777; -} -.initialism { - font-size: 90%; - text-transform: uppercase; -} -blockquote { - padding: 10px 20px; - margin: 0 0 20px; - font-size: 17.5px; - border-left: 5px solid #eee; -} -blockquote p:last-child, -blockquote ul:last-child, -blockquote ol:last-child { - margin-bottom: 0; -} -blockquote footer, -blockquote small, -blockquote .small { - display: block; - font-size: 80%; - line-height: 1.428571429; - color: #777; -} -blockquote footer::before, -blockquote small::before, -blockquote .small::before { - content: "\2014 \00A0"; -} -.blockquote-reverse, -blockquote.pull-right { - padding-right: 15px; - padding-left: 0; - border-right: 5px solid #eee; - border-left: 0; - text-align: right; -} -.blockquote-reverse footer::before, -blockquote.pull-right footer::before, -.blockquote-reverse small::before, -blockquote.pull-right small::before, -.blockquote-reverse .small::before, -blockquote.pull-right .small::before { - content: ""; -} -.blockquote-reverse footer::after, -blockquote.pull-right footer::after, -.blockquote-reverse small::after, -blockquote.pull-right small::after, -.blockquote-reverse .small::after, -blockquote.pull-right .small::after { - content: "\00A0 \2014"; -} -address { - margin-bottom: 20px; - font-style: normal; - line-height: 1.428571429; -} -code, -kbd, -pre, -samp { - font-family: Menlo, Monaco, Consolas, "Courier New", monospace; -} -code { - padding: 2px 4px; - font-size: 90%; - color: #51504f; - background-color: #e2e2e2; - border-radius: 4px; -} -kbd { - padding: 2px 4px; - font-size: 90%; - color: #fff; - background-color: #51504f; - border-radius: 3px; - box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.25); -} -kbd kbd { - padding: 0; - font-size: 100%; - font-weight: bold; - box-shadow: none; -} -pre { - display: block; - padding: 9.5px; - margin: 0 0 10px; - font-size: 13px; - line-height: 1.428571429; - word-break: break-all; - word-wrap: break-word; - color: #51504f; - background-color: #f0f0f0; - border: 0; - border-radius: 4px; -} -pre code { - padding: 0; - font-size: inherit; - color: inherit; - white-space: pre-wrap; - background-color: transparent; - border-radius: 0; -} -.pre-scrollable { - max-height: 340px; - overflow-y: auto; -} -.container { - margin-right: auto; - margin-left: auto; - padding-left: 15px; - padding-right: 15px; -} - -@media (min-width: 768px) { - .container { - width: 750px; - } -} - -@media (min-width: 992px) { - .container { - width: 970px; - } -} - -@media (min-width: 1200px) { - .container { - width: 1170px; - } -} -.container-fluid { - margin-right: auto; - margin-left: auto; - padding-left: 15px; - padding-right: 15px; -} -.row { - margin-left: -15px; - margin-right: -15px; -} -.col-xs-1, -.col-sm-1, -.col-md-1, -.col-lg-1, -.col-xs-2, -.col-sm-2, -.col-md-2, -.col-lg-2, -.col-xs-3, -.col-sm-3, -.col-md-3, -.col-lg-3, -.col-xs-4, -.col-sm-4, -.col-md-4, -.col-lg-4, -.col-xs-5, -.col-sm-5, -.col-md-5, -.col-lg-5, -.col-xs-6, -.col-sm-6, -.col-md-6, -.col-lg-6, -.col-xs-7, -.col-sm-7, -.col-md-7, -.col-lg-7, -.col-xs-8, -.col-sm-8, -.col-md-8, -.col-lg-8, -.col-xs-9, -.col-sm-9, -.col-md-9, -.col-lg-9, -.col-xs-10, -.col-sm-10, -.col-md-10, -.col-lg-10, -.col-xs-11, -.col-sm-11, -.col-md-11, -.col-lg-11, -.col-xs-12, -.col-sm-12, -.col-md-12, -.col-lg-12 { - position: relative; - min-height: 1px; - padding-left: 15px; - padding-right: 15px; -} -.col-xs-1, -.col-xs-2, -.col-xs-3, -.col-xs-4, -.col-xs-5, -.col-xs-6, -.col-xs-7, -.col-xs-8, -.col-xs-9, -.col-xs-10, -.col-xs-11, -.col-xs-12 { - float: left; -} -.col-xs-12 { - width: 100%; -} -.col-xs-11 { - width: 91.66666666666666%; -} -.col-xs-10 { - width: 83.33333333333334%; -} -.col-xs-9 { - width: 75%; -} -.col-xs-8 { - width: 66.66666666666666%; -} -.col-xs-7 { - width: 58.333333333333336%; -} -.col-xs-6 { - width: 50%; -} -.col-xs-5 { - width: 41.66666666666667%; -} -.col-xs-4 { - width: 33.33333333333333%; -} -.col-xs-3 { - width: 25%; -} -.col-xs-2 { - width: 16.666666666666664%; -} -.col-xs-1 { - width: 8.333333333333332%; -} -.col-xs-pull-12 { - right: 100%; -} -.col-xs-pull-11 { - right: 91.66666666666666%; -} -.col-xs-pull-10 { - right: 83.33333333333334%; -} -.col-xs-pull-9 { - right: 75%; -} -.col-xs-pull-8 { - right: 66.66666666666666%; -} -.col-xs-pull-7 { - right: 58.333333333333336%; -} -.col-xs-pull-6 { - right: 50%; -} -.col-xs-pull-5 { - right: 41.66666666666667%; -} -.col-xs-pull-4 { - right: 33.33333333333333%; -} -.col-xs-pull-3 { - right: 25%; -} -.col-xs-pull-2 { - right: 16.666666666666664%; -} -.col-xs-pull-1 { - right: 8.333333333333332%; -} -.col-xs-pull-0 { - right: auto; -} -.col-xs-push-12 { - left: 100%; -} -.col-xs-push-11 { - left: 91.66666666666666%; -} -.col-xs-push-10 { - left: 83.33333333333334%; -} -.col-xs-push-9 { - left: 75%; -} -.col-xs-push-8 { - left: 66.66666666666666%; -} -.col-xs-push-7 { - left: 58.333333333333336%; -} -.col-xs-push-6 { - left: 50%; -} -.col-xs-push-5 { - left: 41.66666666666667%; -} -.col-xs-push-4 { - left: 33.33333333333333%; -} -.col-xs-push-3 { - left: 25%; -} -.col-xs-push-2 { - left: 16.666666666666664%; -} -.col-xs-push-1 { - left: 8.333333333333332%; -} -.col-xs-push-0 { - left: auto; -} -.col-xs-offset-12 { - margin-left: 100%; -} -.col-xs-offset-11 { - margin-left: 91.66666666666666%; -} -.col-xs-offset-10 { - margin-left: 83.33333333333334%; -} -.col-xs-offset-9 { - margin-left: 75%; -} -.col-xs-offset-8 { - margin-left: 66.66666666666666%; -} -.col-xs-offset-7 { - margin-left: 58.333333333333336%; -} -.col-xs-offset-6 { - margin-left: 50%; -} -.col-xs-offset-5 { - margin-left: 41.66666666666667%; -} -.col-xs-offset-4 { - margin-left: 33.33333333333333%; -} -.col-xs-offset-3 { - margin-left: 25%; -} -.col-xs-offset-2 { - margin-left: 16.666666666666664%; -} -.col-xs-offset-1 { - margin-left: 8.333333333333332%; -} -.col-xs-offset-0 { - margin-left: 0%; -} - -@media (min-width: 768px) { - .col-sm-1, - .col-sm-2, - .col-sm-3, - .col-sm-4, - .col-sm-5, - .col-sm-6, - .col-sm-7, - .col-sm-8, - .col-sm-9, - .col-sm-10, - .col-sm-11, - .col-sm-12 { - float: left; - } - .col-sm-12 { - width: 100%; - } - .col-sm-11 { - width: 91.66666666666666%; - } - .col-sm-10 { - width: 83.33333333333334%; - } - .col-sm-9 { - width: 75%; - } - .col-sm-8 { - width: 66.66666666666666%; - } - .col-sm-7 { - width: 58.333333333333336%; - } - .col-sm-6 { - width: 50%; - } - .col-sm-5 { - width: 41.66666666666667%; - } - .col-sm-4 { - width: 33.33333333333333%; - } - .col-sm-3 { - width: 25%; - } - .col-sm-2 { - width: 16.666666666666664%; - } - .col-sm-1 { - width: 8.333333333333332%; - } - .col-sm-pull-12 { - right: 100%; - } - .col-sm-pull-11 { - right: 91.66666666666666%; - } - .col-sm-pull-10 { - right: 83.33333333333334%; - } - .col-sm-pull-9 { - right: 75%; - } - .col-sm-pull-8 { - right: 66.66666666666666%; - } - .col-sm-pull-7 { - right: 58.333333333333336%; - } - .col-sm-pull-6 { - right: 50%; - } - .col-sm-pull-5 { - right: 41.66666666666667%; - } - .col-sm-pull-4 { - right: 33.33333333333333%; - } - .col-sm-pull-3 { - right: 25%; - } - .col-sm-pull-2 { - right: 16.666666666666664%; - } - .col-sm-pull-1 { - right: 8.333333333333332%; - } - .col-sm-pull-0 { - right: auto; - } - .col-sm-push-12 { - left: 100%; - } - .col-sm-push-11 { - left: 91.66666666666666%; - } - .col-sm-push-10 { - left: 83.33333333333334%; - } - .col-sm-push-9 { - left: 75%; - } - .col-sm-push-8 { - left: 66.66666666666666%; - } - .col-sm-push-7 { - left: 58.333333333333336%; - } - .col-sm-push-6 { - left: 50%; - } - .col-sm-push-5 { - left: 41.66666666666667%; - } - .col-sm-push-4 { - left: 33.33333333333333%; - } - .col-sm-push-3 { - left: 25%; - } - .col-sm-push-2 { - left: 16.666666666666664%; - } - .col-sm-push-1 { - left: 8.333333333333332%; - } - .col-sm-push-0 { - left: auto; - } - .col-sm-offset-12 { - margin-left: 100%; - } - .col-sm-offset-11 { - margin-left: 91.66666666666666%; - } - .col-sm-offset-10 { - margin-left: 83.33333333333334%; - } - .col-sm-offset-9 { - margin-left: 75%; - } - .col-sm-offset-8 { - margin-left: 66.66666666666666%; - } - .col-sm-offset-7 { - margin-left: 58.333333333333336%; - } - .col-sm-offset-6 { - margin-left: 50%; - } - .col-sm-offset-5 { - margin-left: 41.66666666666667%; - } - .col-sm-offset-4 { - margin-left: 33.33333333333333%; - } - .col-sm-offset-3 { - margin-left: 25%; - } - .col-sm-offset-2 { - margin-left: 16.666666666666664%; - } - .col-sm-offset-1 { - margin-left: 8.333333333333332%; - } - .col-sm-offset-0 { - margin-left: 0%; - } -} - -@media (min-width: 992px) { - .col-md-1, - .col-md-2, - .col-md-3, - .col-md-4, - .col-md-5, - .col-md-6, - .col-md-7, - .col-md-8, - .col-md-9, - .col-md-10, - .col-md-11, - .col-md-12 { - float: left; - } - .col-md-12 { - width: 100%; - } - .col-md-11 { - width: 91.66666666666666%; - } - .col-md-10 { - width: 83.33333333333334%; - } - .col-md-9 { - width: 75%; - } - .col-md-8 { - width: 66.66666666666666%; - } - .col-md-7 { - width: 58.333333333333336%; - } - .col-md-6 { - width: 50%; - } - .col-md-5 { - width: 41.66666666666667%; - } - .col-md-4 { - width: 33.33333333333333%; - } - .col-md-3 { - width: 25%; - } - .col-md-2 { - width: 16.666666666666664%; - } - .col-md-1 { - width: 8.333333333333332%; - } - .col-md-pull-12 { - right: 100%; - } - .col-md-pull-11 { - right: 91.66666666666666%; - } - .col-md-pull-10 { - right: 83.33333333333334%; - } - .col-md-pull-9 { - right: 75%; - } - .col-md-pull-8 { - right: 66.66666666666666%; - } - .col-md-pull-7 { - right: 58.333333333333336%; - } - .col-md-pull-6 { - right: 50%; - } - .col-md-pull-5 { - right: 41.66666666666667%; - } - .col-md-pull-4 { - right: 33.33333333333333%; - } - .col-md-pull-3 { - right: 25%; - } - .col-md-pull-2 { - right: 16.666666666666664%; - } - .col-md-pull-1 { - right: 8.333333333333332%; - } - .col-md-pull-0 { - right: auto; - } - .col-md-push-12 { - left: 100%; - } - .col-md-push-11 { - left: 91.66666666666666%; - } - .col-md-push-10 { - left: 83.33333333333334%; - } - .col-md-push-9 { - left: 75%; - } - .col-md-push-8 { - left: 66.66666666666666%; - } - .col-md-push-7 { - left: 58.333333333333336%; - } - .col-md-push-6 { - left: 50%; - } - .col-md-push-5 { - left: 41.66666666666667%; - } - .col-md-push-4 { - left: 33.33333333333333%; - } - .col-md-push-3 { - left: 25%; - } - .col-md-push-2 { - left: 16.666666666666664%; - } - .col-md-push-1 { - left: 8.333333333333332%; - } - .col-md-push-0 { - left: auto; - } - .col-md-offset-12 { - margin-left: 100%; - } - .col-md-offset-11 { - margin-left: 91.66666666666666%; - } - .col-md-offset-10 { - margin-left: 83.33333333333334%; - } - .col-md-offset-9 { - margin-left: 75%; - } - .col-md-offset-8 { - margin-left: 66.66666666666666%; - } - .col-md-offset-7 { - margin-left: 58.333333333333336%; - } - .col-md-offset-6 { - margin-left: 50%; - } - .col-md-offset-5 { - margin-left: 41.66666666666667%; - } - .col-md-offset-4 { - margin-left: 33.33333333333333%; - } - .col-md-offset-3 { - margin-left: 25%; - } - .col-md-offset-2 { - margin-left: 16.666666666666664%; - } - .col-md-offset-1 { - margin-left: 8.333333333333332%; - } - .col-md-offset-0 { - margin-left: 0%; - } -} - -@media (min-width: 1200px) { - .col-lg-1, - .col-lg-2, - .col-lg-3, - .col-lg-4, - .col-lg-5, - .col-lg-6, - .col-lg-7, - .col-lg-8, - .col-lg-9, - .col-lg-10, - .col-lg-11, - .col-lg-12 { - float: left; - } - .col-lg-12 { - width: 100%; - } - .col-lg-11 { - width: 91.66666666666666%; - } - .col-lg-10 { - width: 83.33333333333334%; - } - .col-lg-9 { - width: 75%; - } - .col-lg-8 { - width: 66.66666666666666%; - } - .col-lg-7 { - width: 58.333333333333336%; - } - .col-lg-6 { - width: 50%; - } - .col-lg-5 { - width: 41.66666666666667%; - } - .col-lg-4 { - width: 33.33333333333333%; - } - .col-lg-3 { - width: 25%; - } - .col-lg-2 { - width: 16.666666666666664%; - } - .col-lg-1 { - width: 8.333333333333332%; - } - .col-lg-pull-12 { - right: 100%; - } - .col-lg-pull-11 { - right: 91.66666666666666%; - } - .col-lg-pull-10 { - right: 83.33333333333334%; - } - .col-lg-pull-9 { - right: 75%; - } - .col-lg-pull-8 { - right: 66.66666666666666%; - } - .col-lg-pull-7 { - right: 58.333333333333336%; - } - .col-lg-pull-6 { - right: 50%; - } - .col-lg-pull-5 { - right: 41.66666666666667%; - } - .col-lg-pull-4 { - right: 33.33333333333333%; - } - .col-lg-pull-3 { - right: 25%; - } - .col-lg-pull-2 { - right: 16.666666666666664%; - } - .col-lg-pull-1 { - right: 8.333333333333332%; - } - .col-lg-pull-0 { - right: auto; - } - .col-lg-push-12 { - left: 100%; - } - .col-lg-push-11 { - left: 91.66666666666666%; - } - .col-lg-push-10 { - left: 83.33333333333334%; - } - .col-lg-push-9 { - left: 75%; - } - .col-lg-push-8 { - left: 66.66666666666666%; - } - .col-lg-push-7 { - left: 58.333333333333336%; - } - .col-lg-push-6 { - left: 50%; - } - .col-lg-push-5 { - left: 41.66666666666667%; - } - .col-lg-push-4 { - left: 33.33333333333333%; - } - .col-lg-push-3 { - left: 25%; - } - .col-lg-push-2 { - left: 16.666666666666664%; - } - .col-lg-push-1 { - left: 8.333333333333332%; - } - .col-lg-push-0 { - left: auto; - } - .col-lg-offset-12 { - margin-left: 100%; - } - .col-lg-offset-11 { - margin-left: 91.66666666666666%; - } - .col-lg-offset-10 { - margin-left: 83.33333333333334%; - } - .col-lg-offset-9 { - margin-left: 75%; - } - .col-lg-offset-8 { - margin-left: 66.66666666666666%; - } - .col-lg-offset-7 { - margin-left: 58.333333333333336%; - } - .col-lg-offset-6 { - margin-left: 50%; - } - .col-lg-offset-5 { - margin-left: 41.66666666666667%; - } - .col-lg-offset-4 { - margin-left: 33.33333333333333%; - } - .col-lg-offset-3 { - margin-left: 25%; - } - .col-lg-offset-2 { - margin-left: 16.666666666666664%; - } - .col-lg-offset-1 { - margin-left: 8.333333333333332%; - } - .col-lg-offset-0 { - margin-left: 0%; - } -} -table { - background-color: transparent; -} -caption { - padding-top: 8px; - padding-bottom: 8px; - color: #777; - text-align: left; -} -th { - text-align: left; -} -.table { - width: 100%; - max-width: 100%; - margin-bottom: 20px; -} -.table > thead > tr > th, -.table > tbody > tr > th, -.table > tfoot > tr > th, -.table > thead > tr > td, -.table > tbody > tr > td, -.table > tfoot > tr > td { - padding: 8px; - line-height: 1.428571429; - vertical-align: middle; - border-top: 1px solid #ddd; -} -.table > thead > tr > th { - vertical-align: bottom; - background-color: #f0f0f0; - border-bottom: 1px solid #fff; - white-space: nowrap; -} -.table > caption + thead > tr:first-child > th, -.table > colgroup + thead > tr:first-child > th, -.table > thead:first-child > tr:first-child > th, -.table > caption + thead > tr:first-child > td, -.table > colgroup + thead > tr:first-child > td, -.table > thead:first-child > tr:first-child > td { - border-top: 0; -} -.table > tbody + tbody { - border-top: 0; -} -.table .table { - background-color: #f0f0f0; -} -.table-condensed > thead > tr > th, -.table-condensed > tbody > tr > th, -.table-condensed > tfoot > tr > th, -.table-condensed > thead > tr > td, -.table-condensed > tbody > tr > td, -.table-condensed > tfoot > tr > td { - padding: 5px; -} -.table-bordered { - border: 1px solid #f0f0f0; -} -.table-bordered > thead > tr > th, -.table-bordered > tbody > tr > th, -.table-bordered > tfoot > tr > th { - border: 0; -} -.table-bordered > thead > tr > td, -.table-bordered > tbody > tr > td, -.table-bordered > tfoot > tr > td { - border: 0; - border-top: 1px solid #f0f0f0; -} -.table-bordered > thead > tr > th, -.table-bordered > thead > tr > td { - border-bottom-width: 2px; -} -.table-striped > tbody > tr:nth-child(odd) { - background-color: #fff; -} -.table-striped > tbody > tr:nth-child(even) { - background-color: #f9f9f9; -} -.table-hover > tbody > tr:hover { - background-color: #f5f5f5; -} -table col[class*="col-"] { - position: static; - float: none; - display: table-column; -} -table td[class*="col-"], -table th[class*="col-"] { - position: static; - float: none; - display: table-cell; -} -.table > thead > tr > td.active, -.table > tbody > tr > td.active, -.table > tfoot > tr > td.active, -.table > thead > tr > th.active, -.table > tbody > tr > th.active, -.table > tfoot > tr > th.active, -.table > thead > tr.active > td, -.table > tbody > tr.active > td, -.table > tfoot > tr.active > td, -.table > thead > tr.active > th, -.table > tbody > tr.active > th, -.table > tfoot > tr.active > th { - background-color: #f5f5f5; -} -.table-hover > tbody > tr > td.active:hover, -.table-hover > tbody > tr > th.active:hover, -.table-hover > tbody > tr.active:hover > td, -.table-hover > tbody > tr:hover > .active, -.table-hover > tbody > tr.active:hover > th { - background-color: #e8e8e8; -} -.table > thead > tr > td.success, -.table > tbody > tr > td.success, -.table > tfoot > tr > td.success, -.table > thead > tr > th.success, -.table > tbody > tr > th.success, -.table > tfoot > tr > th.success, -.table > thead > tr.success > td, -.table > tbody > tr.success > td, -.table > tfoot > tr.success > td, -.table > thead > tr.success > th, -.table > tbody > tr.success > th, -.table > tfoot > tr.success > th { - background-color: #bfebd1; -} -.table-hover > tbody > tr > td.success:hover, -.table-hover > tbody > tr > th.success:hover, -.table-hover > tbody > tr.success:hover > td, -.table-hover > tbody > tr:hover > .success, -.table-hover > tbody > tr.success:hover > th { - background-color: #d0e9c6; -} -.table > thead > tr > td.info, -.table > tbody > tr > td.info, -.table > tfoot > tr > td.info, -.table > thead > tr > th.info, -.table > tbody > tr > th.info, -.table > tfoot > tr > th.info, -.table > thead > tr.info > td, -.table > tbody > tr.info > td, -.table > tfoot > tr.info > td, -.table > thead > tr.info > th, -.table > tbody > tr.info > th, -.table > tfoot > tr.info > th { - background-color: #d9edf7; -} -.table-hover > tbody > tr > td.info:hover, -.table-hover > tbody > tr > th.info:hover, -.table-hover > tbody > tr.info:hover > td, -.table-hover > tbody > tr:hover > .info, -.table-hover > tbody > tr.info:hover > th { - background-color: #c4e3f3; -} -.table > thead > tr > td.warning, -.table > tbody > tr > td.warning, -.table > tfoot > tr > td.warning, -.table > thead > tr > th.warning, -.table > tbody > tr > th.warning, -.table > tfoot > tr > th.warning, -.table > thead > tr.warning > td, -.table > tbody > tr.warning > td, -.table > tfoot > tr.warning > td, -.table > thead > tr.warning > th, -.table > tbody > tr.warning > th, -.table > tfoot > tr.warning > th { - background-color: #fcf8e3; -} -.table-hover > tbody > tr > td.warning:hover, -.table-hover > tbody > tr > th.warning:hover, -.table-hover > tbody > tr.warning:hover > td, -.table-hover > tbody > tr:hover > .warning, -.table-hover > tbody > tr.warning:hover > th { - background-color: #faf2cc; -} -.table > thead > tr > td.danger, -.table > tbody > tr > td.danger, -.table > tfoot > tr > td.danger, -.table > thead > tr > th.danger, -.table > tbody > tr > th.danger, -.table > tfoot > tr > th.danger, -.table > thead > tr.danger > td, -.table > tbody > tr.danger > td, -.table > tfoot > tr.danger > td, -.table > thead > tr.danger > th, -.table > tbody > tr.danger > th, -.table > tfoot > tr.danger > th { - background-color: #f2dede; -} -.table-hover > tbody > tr > td.danger:hover, -.table-hover > tbody > tr > th.danger:hover, -.table-hover > tbody > tr.danger:hover > td, -.table-hover > tbody > tr:hover > .danger, -.table-hover > tbody > tr.danger:hover > th { - background-color: #ebcccc; -} -.table-responsive { - overflow: visible; - min-height: 0.01%; -} - -@media screen and (max-width: 767px) { - .table-responsive { - width: 100%; - margin-bottom: 15px; - border: 1px solid #ddd; - } - .table-responsive > .table { - margin-bottom: 0; - } - .table-responsive > .table > thead > tr > th, - .table-responsive > .table > tbody > tr > th, - .table-responsive > .table > tfoot > tr > th, - .table-responsive > .table > thead > tr > td, - .table-responsive > .table > tbody > tr > td, - .table-responsive > .table > tfoot > tr > td { - white-space: nowrap; - } - .table-responsive > .table-bordered { - border: 0; - } - .table-responsive > .table-bordered > thead > tr > th:first-child, - .table-responsive > .table-bordered > tbody > tr > th:first-child, - .table-responsive > .table-bordered > tfoot > tr > th:first-child, - .table-responsive > .table-bordered > thead > tr > td:first-child, - .table-responsive > .table-bordered > tbody > tr > td:first-child, - .table-responsive > .table-bordered > tfoot > tr > td:first-child { - border-left: 0; - } - .table-responsive > .table-bordered > thead > tr > th:last-child, - .table-responsive > .table-bordered > tbody > tr > th:last-child, - .table-responsive > .table-bordered > tfoot > tr > th:last-child, - .table-responsive > .table-bordered > thead > tr > td:last-child, - .table-responsive > .table-bordered > tbody > tr > td:last-child, - .table-responsive > .table-bordered > tfoot > tr > td:last-child { - border-right: 0; - } - .table-responsive > .table-bordered > tbody > tr:last-child > th, - .table-responsive > .table-bordered > tfoot > tr:last-child > th, - .table-responsive > .table-bordered > tbody > tr:last-child > td, - .table-responsive > .table-bordered > tfoot > tr:last-child > td { - border-bottom: 0; - } -} - -fieldset { - padding: 0; - margin: 0; - border: 0; - min-width: 0; -} -legend { - display: block; - width: 100%; - padding: 0; - margin-bottom: 20px; - font-size: 21px; - line-height: inherit; - color: #51504f; - border: 0; - border-bottom: 1px solid #e5e5e5; -} -label { - display: inline-block; - max-width: 100%; - margin-bottom: 5px; - font-weight: bold; -} -input[type="radio"], -input[type="checkbox"] { - margin: 4px 0 0; - margin-top: 1px \9; - line-height: normal; -} -input[type="file"] { - display: block; -} -input[type="range"] { - display: block; - width: 100%; -} -select[multiple], -select[size] { - height: auto; -} -input[type="file"]:focus, -input[type="radio"]:focus, -input[type="checkbox"]:focus { - outline: thin dotted; - outline: 5px auto -webkit-focus-ring-color; - outline-offset: -2px; -} -output { - display: block; - padding-top: 7px; - font-size: 14px; - line-height: 1.428571429; - color: #555; -} -.form-control { - display: block; - width: 100%; - height: 34px; - padding: 6px 12px; - font-size: 14px; - line-height: 1.428571429; - color: #555; - background-color: #fff; - background-image: none; - border: 1px solid #ccc; - border-radius: 4px; - -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); - box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); - -webkit-transition: border-color ease-in-out 0.15s, - box-shadow ease-in-out 0.15s; - -o-transition: border-color ease-in-out 0.15s, box-shadow ease-in-out 0.15s; - transition: border-color ease-in-out 0.15s, box-shadow ease-in-out 0.15s; -} -.form-control:focus { - border-color: #66afe9; - outline: 0; - -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), - 0 0 8px rgba(102, 175, 233, 0.6); - box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), - 0 0 8px rgba(102, 175, 233, 0.6); -} -.form-control::-moz-placeholder { - color: #999; - opacity: 1; -} -.form-control:-ms-input-placeholder { - color: #999; -} -.form-control::-webkit-input-placeholder { - color: #999; -} -.form-control[disabled], -.form-control[readonly], -fieldset[disabled] .form-control { - cursor: not-allowed; - color: #cbcbcb; - background-color: #eee; - opacity: 1; -} -textarea.form-control { - height: auto; -} - -@media screen and (-webkit-min-device-pixel-ratio: 0) { - input[type="date"], - input[type="time"], - input[type="datetime-local"], - input[type="month"] { - line-height: 34px; - } - input[type="date"].input-sm, - input[type="time"].input-sm, - input[type="datetime-local"].input-sm, - input[type="month"].input-sm { - line-height: 30px; - } - input[type="date"].input-lg, - input[type="time"].input-lg, - input[type="datetime-local"].input-lg, - input[type="month"].input-lg { - line-height: 46px; - } -} -.form-group { - margin-bottom: 15px; -} -.radio, -.checkbox { - position: relative; - display: block; - margin-top: 10px; - margin-bottom: 10px; -} -.radio label, -.checkbox label { - min-height: 20px; - padding-left: 20px; - margin-bottom: 0; - font-weight: normal; - cursor: pointer; -} -.radio input[type="radio"], -.radio-inline input[type="radio"], -.checkbox input[type="checkbox"], -.checkbox-inline input[type="checkbox"] { - position: absolute; - margin-left: -20px; - margin-top: 4px \9; -} -.radio + .radio, -.checkbox + .checkbox { - margin-top: -5px; -} -.radio-inline, -.checkbox-inline { - display: inline-block; - padding-left: 20px; - margin-bottom: 0; - vertical-align: middle; - font-weight: normal; - cursor: pointer; -} -.radio-inline + .radio-inline, -.checkbox-inline + .checkbox-inline { - margin-top: 0; - margin-left: 10px; -} -input[type="radio"][disabled], -input[type="checkbox"][disabled], -input[type="radio"].disabled, -input[type="checkbox"].disabled, -fieldset[disabled] input[type="radio"], -fieldset[disabled] input[type="checkbox"] { - cursor: not-allowed; -} -.radio-inline.disabled, -.checkbox-inline.disabled, -fieldset[disabled] .radio-inline, -fieldset[disabled] .checkbox-inline { - cursor: not-allowed; -} -.radio.disabled label, -.checkbox.disabled label, -fieldset[disabled] .radio label, -fieldset[disabled] .checkbox label { - cursor: not-allowed; -} -.form-control-static { - padding-top: 7px; - padding-bottom: 7px; - margin-bottom: 0; -} -.form-control-static.input-lg, -.form-control-static.input-sm { - padding-left: 0; - padding-right: 0; -} -.input-sm { - height: 30px; - padding: 5px 10px; - font-size: 12px; - line-height: 1.5; - border-radius: 3px; -} -select.input-sm { - height: 30px; - line-height: 30px; -} -textarea.input-sm, -select[multiple].input-sm { - height: auto; -} -.form-group-sm .form-control { - height: 30px; - padding: 5px 10px; - font-size: 12px; - line-height: 1.5; - border-radius: 3px; -} -select.form-group-sm .form-control { - height: 30px; - line-height: 30px; -} -textarea.form-group-sm .form-control, -select[multiple].form-group-sm .form-control { - height: auto; -} -.input-lg { - height: 46px; - padding: 10px 16px; - font-size: 18px; - line-height: 1.33; - border-radius: 6px; -} -select.input-lg { - height: 46px; - line-height: 46px; -} -textarea.input-lg, -select[multiple].input-lg { - height: auto; -} -.form-group-lg .form-control { - height: 46px; - padding: 10px 16px; - font-size: 18px; - line-height: 1.33; - border-radius: 6px; -} -select.form-group-lg .form-control { - height: 46px; - line-height: 46px; -} -textarea.form-group-lg .form-control, -select[multiple].form-group-lg .form-control { - height: auto; -} -.has-feedback { - position: relative; -} -.has-feedback .form-control { - padding-right: 42.5px; -} -.form-control-feedback { - position: absolute; - top: 0; - right: 0; - z-index: 2; - display: block; - width: 34px; - height: 34px; - line-height: 34px; - text-align: center; - pointer-events: none; -} -.input-lg + .form-control-feedback { - width: 46px; - height: 46px; - line-height: 46px; -} -.input-sm + .form-control-feedback { - width: 30px; - height: 30px; - line-height: 30px; -} -.has-success .help-block, -.has-success .control-label, -.has-success .radio, -.has-success .checkbox, -.has-success .radio-inline, -.has-success .checkbox-inline, -.has-success.radio label, -.has-success.checkbox label, -.has-success.radio-inline label, -.has-success.checkbox-inline label { - color: #1b8e49; -} -.has-success .form-control { - border-color: #1b8e49; - -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); - box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); -} -.has-success .form-control:focus { - border-color: #366f4c; - -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168; - box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168; -} -.has-success .input-group-addon { - color: #1b8e49; - border-color: #1b8e49; - background-color: #bfebd1; -} -.has-success .form-control-feedback { - color: #1b8e49; -} -.has-warning .help-block, -.has-warning .control-label, -.has-warning .radio, -.has-warning .checkbox, -.has-warning .radio-inline, -.has-warning .checkbox-inline, -.has-warning.radio label, -.has-warning.checkbox label, -.has-warning.radio-inline label, -.has-warning.checkbox-inline label { - color: #8a6d3b; -} -.has-warning .form-control { - border-color: #8a6d3b; - -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); - box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); -} -.has-warning .form-control:focus { - border-color: #66512c; - -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b; - box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b; -} -.has-warning .input-group-addon { - color: #8a6d3b; - border-color: #8a6d3b; - background-color: #fcf8e3; -} -.has-warning .form-control-feedback { - color: #8a6d3b; -} -.has-error .help-block, -.has-error .control-label, -.has-error .radio, -.has-error .checkbox, -.has-error .radio-inline, -.has-error .checkbox-inline, -.has-error.radio label, -.has-error.checkbox label, -.has-error.radio-inline label, -.has-error.checkbox-inline label { - color: #e43921; -} -.has-error .form-control { - border-color: #e43921; - -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); - box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); -} -.has-error .form-control:focus { - border-color: #843534; - -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483; - box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483; -} -.has-error .input-group-addon { - color: #e43921; - border-color: #e43921; - background-color: #f2dede; -} -.has-error .form-control-feedback { - color: #e43921; -} -.has-feedback label ~ .form-control-feedback { - top: 25px; -} -.has-feedback label.sr-only ~ .form-control-feedback { - top: 0; -} -.help-block { - display: block; - margin-top: 5px; - margin-bottom: 10px; - color: #737373; -} - -@media (min-width: 768px) { - .form-inline .form-group { - display: inline-block; - margin-bottom: 0; - vertical-align: middle; - } - .form-inline .form-control { - display: inline-block; - width: auto; - vertical-align: middle; - } - .form-inline .form-control-static { - display: inline-block; - } - .form-inline .input-group { - display: inline-table; - vertical-align: middle; - } - .form-inline .input-group .input-group-addon, - .form-inline .input-group .input-group-btn, - .form-inline .input-group .form-control { - width: auto; - } - .form-inline .input-group > .form-control { - width: 100%; - } - .form-inline .control-label { - margin-bottom: 0; - vertical-align: middle; - } - .form-inline .radio, - .form-inline .checkbox { - display: inline-block; - margin-top: 0; - margin-bottom: 0; - vertical-align: middle; - } - .form-inline .radio label, - .form-inline .checkbox label { - padding-left: 0; - } - .form-inline .radio input[type="radio"], - .form-inline .checkbox input[type="checkbox"] { - position: relative; - margin-left: 0; - } - .form-inline .has-feedback .form-control-feedback { - top: 0; - } -} -.form-horizontal .radio, -.form-horizontal .checkbox, -.form-horizontal .radio-inline, -.form-horizontal .checkbox-inline { - margin-top: 0; - margin-bottom: 0; - padding-top: 7px; -} -.form-horizontal .radio, -.form-horizontal .checkbox { - min-height: 27px; -} -.form-horizontal .form-group { - margin-left: -15px; - margin-right: -15px; -} - -@media (min-width: 768px) { - .form-horizontal .control-label { - text-align: right; - margin-bottom: 0; - padding-top: 7px; - } -} -.form-horizontal .has-feedback .form-control-feedback { - right: 15px; -} - -@media (min-width: 768px) { - .form-horizontal .form-group-lg .control-label { - padding-top: 14.3px; - } -} - -@media (min-width: 768px) { - .form-horizontal .form-group-sm .control-label { - padding-top: 6px; - } -} -.btn { - display: inline-block; - margin-bottom: 0; - font-weight: normal; - text-align: center; - vertical-align: middle; - touch-action: manipulation; - cursor: pointer; - color: #017cee; - background: #fff; - border: 1px solid #017cee; - white-space: nowrap; - padding: 6px 12px; - font-size: 14px; - line-height: 1.428571429; - border-radius: 4px; - -webkit-user-select: none; - -moz-user-select: none; - -ms-user-select: none; - user-select: none; -} -.btn:focus, -.btn:active:focus, -.btn.active:focus, -.btn.focus, -.btn:active.focus, -.btn.active.focus { - outline: thin dotted; - outline: 5px auto -webkit-focus-ring-color; - outline-offset: -2px; -} -.btn:hover, -.btn:focus, -.btn.focus { - color: #017cee; - text-decoration: none; - background-color: #c0defb; -} -.btn:active, -.btn.active { - outline: 0; - background-image: none; - -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125); - box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125); -} -.btn.disabled, -.btn[disabled], -fieldset[disabled] .btn { - cursor: not-allowed; - pointer-events: none; - opacity: 0.5; - -webkit-box-shadow: none; - box-shadow: none; -} -.btn-icon-only { - padding: 6px; -} -.btn-default { - color: #365f84; - background-color: #fff; - border-color: #ccc; -} -.btn-default:hover, -.btn-default:focus, -.btn-default.focus, -.btn-default:active, -.btn-default.active, -.open > .dropdown-toggle.btn-default { - color: #017cee; - background-color: #f0f0f0; - border-color: #adadad; -} -.btn-default:active, -.btn-default.active, -.open > .dropdown-toggle.btn-default { - background-image: none; -} -.btn-default.disabled, -.btn-default[disabled], -fieldset[disabled] .btn-default, -.btn-default.disabled:hover, -.btn-default[disabled]:hover, -fieldset[disabled] .btn-default:hover, -.btn-default.disabled:focus, -.btn-default[disabled]:focus, -fieldset[disabled] .btn-default:focus, -.btn-default.disabled.focus, -.btn-default[disabled].focus, -fieldset[disabled] .btn-default.focus, -.btn-default.disabled:active, -.btn-default[disabled]:active, -fieldset[disabled] .btn-default:active, -.btn-default.disabled.active, -.btn-default[disabled].active, -fieldset[disabled] .btn-default.active { - background-color: #fff; - border-color: #ccc; -} -.btn-default .badge { - color: #fff; - background-color: #51504f; -} -.btn-primary { - color: #fff; - background-color: #017cee; - border-color: #017cee; -} -.btn-primary:hover, -.btn-primary:focus, -.btn-primary.focus, -.btn-primary:active, -.btn-primary.active, -.open > .dropdown-toggle.btn-primary { - color: #fff; - background-color: #0cb6ff; - border-color: #0cb6ff; -} -.btn-primary:active, -.btn-primary.active, -.open > .dropdown-toggle.btn-primary { - background-image: none; -} -.btn-primary.disabled, -.btn-primary[disabled], -fieldset[disabled] .btn-primary, -.btn-primary.disabled:hover, -.btn-primary[disabled]:hover, -fieldset[disabled] .btn-primary:hover, -.btn-primary.disabled:focus, -.btn-primary[disabled]:focus, -fieldset[disabled] .btn-primary:focus, -.btn-primary.disabled.focus, -.btn-primary[disabled].focus, -fieldset[disabled] .btn-primary.focus, -.btn-primary.disabled:active, -.btn-primary[disabled]:active, -fieldset[disabled] .btn-primary:active, -.btn-primary.disabled.active, -.btn-primary[disabled].active, -fieldset[disabled] .btn-primary.active { - background-color: #017cee; - border-color: #00454c; -} -.btn-primary .badge { - color: #017cee; - background-color: #fff; -} -.btn-primary:hover .badge, -.btn-primary:focus .badge, -.btn-primary.focus .badge, -.btn-primary:active .badge, -.btn-primary.active .badge, -.open > .dropdown-toggle.btn-primary .badge { - color: #0cb6ff; -} -.btn-success { - color: #fff; - background-color: #00ad46; - border-color: #38a047; -} -.btn-success:hover, -.btn-success:focus, -.btn-success.focus, -.btn-success:active, -.btn-success.active, -.open > .dropdown-toggle.btn-success { - color: #fff; - background-color: #328d3e; - border-color: #287333; -} -.btn-success:active, -.btn-success.active, -.open > .dropdown-toggle.btn-success { - background-image: none; -} -.btn-success.disabled, -.btn-success[disabled], -fieldset[disabled] .btn-success, -.btn-success.disabled:hover, -.btn-success[disabled]:hover, -fieldset[disabled] .btn-success:hover, -.btn-success.disabled:focus, -.btn-success[disabled]:focus, -fieldset[disabled] .btn-success:focus, -.btn-success.disabled.focus, -.btn-success[disabled].focus, -fieldset[disabled] .btn-success.focus, -.btn-success.disabled:active, -.btn-success[disabled]:active, -fieldset[disabled] .btn-success:active, -.btn-success.disabled.active, -.btn-success[disabled].active, -fieldset[disabled] .btn-success.active { - background-color: #00ad46; - border-color: #00ad46; -} -.btn-success .badge { - color: #00ad46; - background-color: #fff; -} -.btn-info { - color: #fff; - background-color: #00d1c1; - border-color: #00b8a9; -} -.btn-info:hover, -.btn-info:focus, -.btn-info.focus, -.btn-info:active, -.btn-info.active, -.open > .dropdown-toggle.btn-info { - color: #fff; - background-color: #009e92; - border-color: #007a71; -} -.btn-info:active, -.btn-info.active, -.open > .dropdown-toggle.btn-info { - background-image: none; -} -.btn-info.disabled, -.btn-info[disabled], -fieldset[disabled] .btn-info, -.btn-info.disabled:hover, -.btn-info[disabled]:hover, -fieldset[disabled] .btn-info:hover, -.btn-info.disabled:focus, -.btn-info[disabled]:focus, -fieldset[disabled] .btn-info:focus, -.btn-info.disabled.focus, -.btn-info[disabled].focus, -fieldset[disabled] .btn-info.focus, -.btn-info.disabled:active, -.btn-info[disabled]:active, -fieldset[disabled] .btn-info:active, -.btn-info.disabled.active, -.btn-info[disabled].active, -fieldset[disabled] .btn-info.active { - background-color: #00d1c1; - border-color: #00b8a9; -} -.btn-info .badge { - color: #00d1c1; - background-color: #fff; -} -.btn-warning { - color: #fff; - background-color: #ffb400; - border-color: #e6a200; -} -.btn-warning:hover, -.btn-warning:focus, -.btn-warning.focus, -.btn-warning:active, -.btn-warning.active, -.open > .dropdown-toggle.btn-warning { - color: #fff; - background-color: #cc9000; - border-color: #a87700; -} -.btn-warning:active, -.btn-warning.active, -.open > .dropdown-toggle.btn-warning { - background-image: none; -} -.btn-warning.disabled, -.btn-warning[disabled], -fieldset[disabled] .btn-warning, -.btn-warning.disabled:hover, -.btn-warning[disabled]:hover, -fieldset[disabled] .btn-warning:hover, -.btn-warning.disabled:focus, -.btn-warning[disabled]:focus, -fieldset[disabled] .btn-warning:focus, -.btn-warning.disabled.focus, -.btn-warning[disabled].focus, -fieldset[disabled] .btn-warning.focus, -.btn-warning.disabled:active, -.btn-warning[disabled]:active, -fieldset[disabled] .btn-warning:active, -.btn-warning.disabled.active, -.btn-warning[disabled].active, -fieldset[disabled] .btn-warning.active { - background-color: #ffb400; - border-color: #e6a200; -} -.btn-warning .badge { - color: #ffb400; - background-color: #fff; -} -.btn-danger { - color: #fff; - background-color: #ff5a5f; - border-color: #ff4146; -} -.btn-danger:hover, -.btn-danger:focus, -.btn-danger.focus, -.btn-danger:active, -.btn-danger.active, -.open > .dropdown-toggle.btn-danger { - color: #fff; - background-color: #ff272e; - border-color: #ff030b; -} -.btn-danger:active, -.btn-danger.active, -.open > .dropdown-toggle.btn-danger { - background-image: none; -} -.btn-danger.disabled, -.btn-danger[disabled], -fieldset[disabled] .btn-danger, -.btn-danger.disabled:hover, -.btn-danger[disabled]:hover, -fieldset[disabled] .btn-danger:hover, -.btn-danger.disabled:focus, -.btn-danger[disabled]:focus, -fieldset[disabled] .btn-danger:focus, -.btn-danger.disabled.focus, -.btn-danger[disabled].focus, -fieldset[disabled] .btn-danger.focus, -.btn-danger.disabled:active, -.btn-danger[disabled]:active, -fieldset[disabled] .btn-danger:active, -.btn-danger.disabled.active, -.btn-danger[disabled].active, -fieldset[disabled] .btn-danger.active { - background-color: #ff5a5f; - border-color: #ff4146; -} -.btn-danger .badge { - color: #ff5a5f; - background-color: #fff; -} -.btn-link { - color: #017cee; - font-weight: normal; - border-radius: 0; -} -.btn-link, -.btn-link:active, -.btn-link.active, -.btn-link[disabled], -fieldset[disabled] .btn-link { - background-color: transparent; - -webkit-box-shadow: none; - box-shadow: none; -} -.btn-link, -.btn-link:hover, -.btn-link:focus, -.btn-link:active { - border-color: transparent; -} -.btn-link:hover, -.btn-link:focus { - color: #004c54; - text-decoration: underline; - background-color: transparent; -} -.btn-link[disabled]:hover, -fieldset[disabled] .btn-link:hover, -.btn-link[disabled]:focus, -fieldset[disabled] .btn-link:focus { - color: #777; - text-decoration: none; -} -.btn-lg, -.btn-group-lg > .btn { - padding: 10px 16px; - font-size: 18px; - line-height: 1.33; - border-radius: 6px; -} -.btn-lg.btn-icon-only, -.btn-group-lg > .btn.btn-icon-only { - padding: 10px; -} -.btn-sm, -.btn-group-sm > .btn { - padding: 5px 10px; - font-size: 12px; - line-height: 1.5; - border-radius: 3px; -} -.btn-sm.btn-icon-only, -.btn-group-sm > .btn.btn-icon-only { - padding: 5px; -} -.btn-xs, -.btn-group-xs > .btn { - padding: 1px 5px; - font-size: 12px; - line-height: 1.5; - border-radius: 3px; -} -.btn-xs.btn-icon-only, -.btn-group-xs > .btn.btn-icon-only { - padding: 1px; -} -.btn-block { - display: block; - width: 100%; -} -.btn-block + .btn-block { - margin-top: 5px; -} -input[type="submit"].btn-block, -input[type="reset"].btn-block, -input[type="button"].btn-block { - width: 100%; -} -.fade { - opacity: 0; - -webkit-transition: opacity 0.15s linear; - -o-transition: opacity 0.15s linear; - transition: opacity 0.15s linear; -} -.fade.in { - opacity: 1; -} -.collapse { - display: none; - visibility: hidden; -} -.collapse.in { - display: block; - visibility: visible; -} -tr.collapse.in { - display: table-row; -} -tbody.collapse.in { - display: table-row-group; -} -.collapsing { - position: relative; - height: 0; - overflow: hidden; - -webkit-transition-property: height, visibility; - transition-property: height, visibility; - -webkit-transition-duration: 0.35s; - transition-duration: 0.35s; - -webkit-transition-timing-function: ease; - transition-timing-function: ease; -} -.caret { - display: inline-block; - width: 0; - height: 0; - margin-left: 2px; - vertical-align: middle; - border-top: 4px solid; - border-right: 4px solid transparent; - border-left: 4px solid transparent; -} -.dropdown { - position: relative; -} -.dropdown-toggle:focus { - outline: 0; -} -.dropdown-menu { - position: absolute; - top: 100%; - left: 0; - z-index: 1000; - display: none; - float: left; - min-width: 160px; - padding: 5px 0; - margin: 2px 0 0; - list-style: none; - font-size: 14px; - text-align: left; - background-color: #fff; - border: 1px solid #ddd; - border-radius: 4px; - -webkit-box-shadow: 0 6px 12px rgba(0, 0, 0, 0.175); - box-shadow: 0 6px 12px rgba(0, 0, 0, 0.175); - background-clip: padding-box; -} -.dropdown-menu.pull-right { - right: 0; - left: auto; -} -.dropdown-menu .divider { - height: 1px; - margin: 9px 0; - overflow: hidden; - background-color: #e5e5e5; -} - -.dropdown-menu > li > a, -.dropdown-menu > li > form > button { - display: block; - padding: 3px 20px; - clear: both; - font-weight: normal; - line-height: 1.428571429; - color: #51504f; - white-space: nowrap; - width: 100%; - max-width: 100%; - text-align: left; -} -.dropdown-menu > li > a:hover, -.dropdown-menu > li > a:focus, -.dropdown-menu > li > form > button:hover, -.dropdown-menu > li > form > button:focus { - text-decoration: none; - color: #262626; - background-color: #f5f5f5; -} -.dropdown-menu > .active > a, -.dropdown-menu > .active > a:hover, -.dropdown-menu > .active > a:focus { - color: #fff; - text-decoration: none; - outline: 0; - background-color: #017cee; -} -.dropdown-menu > .disabled > a, -.dropdown-menu > .disabled > a:hover, -.dropdown-menu > .disabled > a:focus { - color: #777; -} -.dropdown-menu > .disabled > a:hover, -.dropdown-menu > .disabled > a:focus { - text-decoration: none; - background-color: transparent; - background-image: none; - cursor: not-allowed; -} -.open > .dropdown-menu { - display: block; -} -.open > a { - outline: 0; -} -.dropdown-menu-right { - left: auto; - right: 0; -} -.dropdown-menu-left { - left: 0; - right: auto; -} -.dropdown-header { - display: block; - padding: 3px 20px; - font-size: 12px; - line-height: 1.428571429; - color: #777; - white-space: nowrap; -} -.dropdown-backdrop { - position: fixed; - left: 0; - right: 0; - bottom: 0; - top: 0; - z-index: 990; -} -.pull-right > .dropdown-menu { - right: 0; - left: auto; -} -.dropup .caret, -.navbar-fixed-bottom .dropdown .caret { - border-top: 0; - border-bottom: 4px solid; - content: ""; -} -.dropup .dropdown-menu, -.navbar-fixed-bottom .dropdown .dropdown-menu { - top: auto; - bottom: 100%; - margin-bottom: 2px; -} - -@media (min-width: 768px) { - .navbar-right .dropdown-menu { - left: auto; - right: 0; - } - .navbar-right .dropdown-menu-left { - left: 0; - right: auto; - } -} -.btn-group, -.btn-group-vertical { - position: relative; - display: inline-block; - vertical-align: middle; -} -.btn-group > .btn, -.btn-group-vertical > .btn { - position: relative; - float: left; -} -.btn-group > .btn:hover, -.btn-group-vertical > .btn:hover, -.btn-group > .btn:focus, -.btn-group-vertical > .btn:focus, -.btn-group > .btn:active, -.btn-group-vertical > .btn:active, -.btn-group > .btn.active, -.btn-group-vertical > .btn.active { - z-index: 2; -} -.btn-group .btn + .btn, -.btn-group .btn + .btn-group, -.btn-group .btn-group + .btn, -.btn-group .btn-group + .btn-group { - margin-left: -1px; -} -.btn-toolbar { - margin-left: -5px; -} -.btn-toolbar .btn-group, -.btn-toolbar .input-group { - float: left; -} -.btn-toolbar > .btn, -.btn-toolbar > .btn-group, -.btn-toolbar > .input-group { - margin-left: 5px; -} -.btn-group > .btn:not(:first-child):not(:last-child):not(.dropdown-toggle) { - border-radius: 0; -} -.btn-group > .btn:first-child { - margin-left: 0; -} -.btn-group > .btn:first-child:not(:last-child):not(.dropdown-toggle) { - border-bottom-right-radius: 0; - border-top-right-radius: 0; -} -.btn-group > .btn:last-child:not(:first-child), -.btn-group > .dropdown-toggle:not(:first-child) { - border-bottom-left-radius: 0; - border-top-left-radius: 0; -} -.btn-group > .btn-group { - float: left; -} -.btn-group > .btn-group:not(:first-child):not(:last-child) > .btn { - border-radius: 0; -} -.btn-group > .btn-group:first-child > .btn:last-child, -.btn-group > .btn-group:first-child > .dropdown-toggle { - border-bottom-right-radius: 0; - border-top-right-radius: 0; -} -.btn-group > .btn-group:last-child > .btn:first-child { - border-bottom-left-radius: 0; - border-top-left-radius: 0; -} -.btn-group .dropdown-toggle:active, -.btn-group.open .dropdown-toggle { - outline: 0; -} -.btn-group > .btn + .dropdown-toggle { - padding-left: 8px; - padding-right: 8px; -} -.btn-group > .btn-lg + .dropdown-toggle { - padding-left: 12px; - padding-right: 12px; -} -.btn-group.open .dropdown-toggle { - -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125); - box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125); -} -.btn-group.open .dropdown-toggle.btn-link { - -webkit-box-shadow: none; - box-shadow: none; -} -.btn .caret { - margin-left: 0; -} -.btn-lg .caret { - border-width: 5px 5px 0; - border-bottom-width: 0; -} -.dropup .btn-lg .caret { - border-width: 0 5px 5px; -} -.btn-group-vertical > .btn, -.btn-group-vertical > .btn-group, -.btn-group-vertical > .btn-group > .btn { - display: block; - float: none; - width: 100%; - max-width: 100%; -} -.btn-group-vertical > .btn-group > .btn { - float: none; -} -.btn-group-vertical > .btn + .btn, -.btn-group-vertical > .btn + .btn-group, -.btn-group-vertical > .btn-group + .btn, -.btn-group-vertical > .btn-group + .btn-group { - margin-top: -1px; - margin-left: 0; -} -.btn-group-vertical > .btn:not(:first-child):not(:last-child) { - border-radius: 0; -} -.btn-group-vertical > .btn:first-child:not(:last-child) { - border-top-right-radius: 4px; - border-bottom-right-radius: 0; - border-bottom-left-radius: 0; -} -.btn-group-vertical > .btn:last-child:not(:first-child) { - border-bottom-left-radius: 4px; - border-top-right-radius: 0; - border-top-left-radius: 0; -} -.btn-group-vertical > .btn-group:not(:first-child):not(:last-child) > .btn { - border-radius: 0; -} -.btn-group-vertical > .btn-group:first-child:not(:last-child) > .btn:last-child, -.btn-group-vertical - > .btn-group:first-child:not(:last-child) - > .dropdown-toggle { - border-bottom-right-radius: 0; - border-bottom-left-radius: 0; -} -.btn-group-vertical - > .btn-group:last-child:not(:first-child) - > .btn:first-child { - border-top-right-radius: 0; - border-top-left-radius: 0; -} -.btn-group-justified { - display: table; - width: 100%; - table-layout: fixed; - border-collapse: separate; -} -.btn-group-justified > .btn, -.btn-group-justified > .btn-group { - float: none; - display: table-cell; - width: 1%; -} -.btn-group-justified > .btn-group .btn { - width: 100%; -} -.btn-group-justified > .btn-group .dropdown-menu { - left: auto; -} -[data-toggle="buttons"] > .btn input[type="radio"], -[data-toggle="buttons"] > .btn-group > .btn input[type="radio"], -[data-toggle="buttons"] > .btn input[type="checkbox"], -[data-toggle="buttons"] > .btn-group > .btn input[type="checkbox"] { - position: absolute; - clip: rect(0, 0, 0, 0); - pointer-events: none; -} -.input-group { - position: relative; - display: table; - border-collapse: separate; -} -.input-group[class*="col-"] { - float: none; - padding-left: 0; - padding-right: 0; -} -.input-group .form-control { - position: relative; - z-index: 2; - float: left; - width: 100%; - margin-bottom: 0; -} -.input-group-lg > .form-control, -.input-group-lg > .input-group-addon, -.input-group-lg > .input-group-btn > .btn { - height: 46px; - padding: 10px 16px; - font-size: 18px; - line-height: 1.33; - border-radius: 6px; -} -select.input-group-lg > .form-control, -select.input-group-lg > .input-group-addon, -select.input-group-lg > .input-group-btn > .btn { - height: 46px; - line-height: 46px; -} -textarea.input-group-lg > .form-control, -textarea.input-group-lg > .input-group-addon, -textarea.input-group-lg > .input-group-btn > .btn, -select[multiple].input-group-lg > .form-control, -select[multiple].input-group-lg > .input-group-addon, -select[multiple].input-group-lg > .input-group-btn > .btn { - height: auto; -} -.input-group-sm > .form-control, -.input-group-sm > .input-group-addon, -.input-group-sm > .input-group-btn > .btn { - height: 30px; - padding: 5px 10px; - font-size: 12px; - line-height: 1.5; - border-radius: 3px; -} -select.input-group-sm > .form-control, -select.input-group-sm > .input-group-addon, -select.input-group-sm > .input-group-btn > .btn { - height: 30px; - line-height: 30px; -} -textarea.input-group-sm > .form-control, -textarea.input-group-sm > .input-group-addon, -textarea.input-group-sm > .input-group-btn > .btn, -select[multiple].input-group-sm > .form-control, -select[multiple].input-group-sm > .input-group-addon, -select[multiple].input-group-sm > .input-group-btn > .btn { - height: auto; -} -.input-group-addon, -.input-group-btn, -.input-group .form-control { - display: table-cell; -} -.input-group-addon:not(:first-child):not(:last-child), -.input-group-btn:not(:first-child):not(:last-child), -.input-group .form-control:not(:first-child):not(:last-child) { - border-radius: 0; -} -.input-group-addon, -.input-group-btn { - width: 1%; - white-space: nowrap; - vertical-align: middle; -} -.input-group-addon { - padding: 6px 12px; - font-size: 14px; - font-weight: normal; - line-height: 1; - color: #555; - text-align: center; - background-color: #eee; - border: 1px solid #ccc; - border-radius: 4px; -} -.input-group-addon.input-sm { - padding: 5px 10px; - font-size: 12px; - border-radius: 3px; -} -.input-group-addon.input-lg { - padding: 10px 16px; - font-size: 18px; - border-radius: 6px; -} -.input-group-addon input[type="radio"], -.input-group-addon input[type="checkbox"] { - margin-top: 0; -} -.input-group .form-control:first-child, -.input-group-addon:first-child, -.input-group-btn:first-child > .btn, -.input-group-btn:first-child > .btn-group > .btn, -.input-group-btn:first-child > .dropdown-toggle, -.input-group-btn:last-child > .btn:not(:last-child):not(.dropdown-toggle), -.input-group-btn:last-child > .btn-group:not(:last-child) > .btn { - border-bottom-right-radius: 0; - border-top-right-radius: 0; -} -.input-group-addon:first-child { - border-right: 0; -} -.input-group .form-control:last-child, -.input-group-addon:last-child, -.input-group-btn:last-child > .btn, -.input-group-btn:last-child > .btn-group > .btn, -.input-group-btn:last-child > .dropdown-toggle, -.input-group-btn:first-child > .btn:not(:first-child), -.input-group-btn:first-child > .btn-group:not(:first-child) > .btn { - border-bottom-left-radius: 0; - border-top-left-radius: 0; -} -.input-group-addon:last-child { - border-left: 0; -} -.input-group-btn { - position: relative; - font-size: 0; - white-space: nowrap; -} -.input-group-btn > .btn { - position: relative; -} -.input-group-btn > .btn + .btn { - margin-left: -1px; -} -.input-group-btn > .btn:hover, -.input-group-btn > .btn:focus, -.input-group-btn > .btn:active { - z-index: 2; -} -.input-group-btn:first-child > .btn, -.input-group-btn:first-child > .btn-group { - margin-right: -1px; -} -.input-group-btn:last-child > .btn, -.input-group-btn:last-child > .btn-group { - margin-left: -1px; -} -.nav { - margin-bottom: 0; - padding-left: 0; - list-style: none; -} -.nav > li { - position: relative; - display: block; -} -.nav > li > a { - position: relative; - display: block; - padding: 10px 15px; -} -.nav > li > a:hover, -.nav > li:focus-within > a, -.nav > li > a:focus { - text-decoration: none; - background-color: #eee; -} -.nav > li.disabled > a { - color: #777; -} -.nav > li.disabled > a:hover, -.nav > li.disabled > a:focus { - color: #777; - text-decoration: none; - background-color: transparent; - cursor: not-allowed; -} -.nav .open > a, -.nav .open > a:hover, -.nav .open > a:focus { - background-color: #eee; - border-color: #017cee; -} -.nav .nav-divider { - height: 1px; - margin: 9px 0; - overflow: hidden; - background-color: #e5e5e5; -} -.nav > li > a > img { - max-width: none; -} -.nav-tabs { - border-bottom: 1px solid #ddd; -} -.nav-tabs > li { - float: left; - margin-bottom: -1px; -} -.nav-tabs > li > a { - margin-right: 2px; - line-height: 1.428571429; - border: 1px solid transparent; - border-radius: 4px 4px 0 0; -} -.nav-tabs > li > a:hover { - border-color: #eee #eee #ddd; -} -.nav-tabs > li.active > a, -.nav-tabs > li.active > a:hover, -.nav-tabs > li.active > a:focus { - color: #555; - background-color: #fafafa; - border: 1px solid #ddd; - border-bottom-color: transparent; - cursor: default; -} -.nav-tabs.nav-justified { - width: 100%; - border-bottom: 0; -} -.nav-tabs.nav-justified > li { - float: none; -} -.nav-tabs.nav-justified > li > a { - border-radius: 4px; - margin-right: 0; - margin-bottom: 5px; - text-align: center; -} -.nav-tabs.nav-justified > .dropdown .dropdown-menu { - top: auto; - left: auto; -} - -@media (min-width: 768px) { - .nav-tabs.nav-justified > li { - display: table-cell; - width: 1%; - } - .nav-tabs.nav-justified > li > a { - margin-bottom: 0; - } -} -.nav-tabs.nav-justified > .active > a, -.nav-tabs.nav-justified > .active > a:hover, -.nav-tabs.nav-justified > .active > a:focus { - border: 1px solid #ddd; -} - -@media (min-width: 768px) { - .nav-tabs.nav-justified > li > a { - border-bottom: 1px solid #ddd; - border-radius: 4px 4px 0 0; - } - .nav-tabs.nav-justified > .active > a, - .nav-tabs.nav-justified > .active > a:hover, - .nav-tabs.nav-justified > .active > a:focus { - border-bottom-color: #fafafa; - } -} -.nav-pills > li { - float: left; -} -.nav-pills > li > a { - border-radius: 4px; - color: #365f84; -} -.nav-pills > li > a:hover, -.nav-pills > li > a:focus { - color: #365f84; - background-color: #c0defb; -} -.nav-pills > li + li { - margin-left: 2px; -} -.nav-pills > li.active > a, -.nav-pills > li.active > a:hover, -.nav-pills > li.active > a:focus { - color: #fff; - background-color: #365f84; -} -.nav-stacked > li { - float: none; -} -.nav-stacked > li + li { - margin-top: 2px; - margin-left: 0; -} -.nav-justified { - width: 100%; -} -.nav-justified > li { - float: none; -} -.nav-justified > li > a { - text-align: center; - margin-bottom: 5px; -} -.nav-justified > .dropdown .dropdown-menu { - top: auto; - left: auto; -} - -@media (min-width: 768px) { - .nav-justified > li { - display: table-cell; - width: 1%; - } - .nav-justified > li > a { - margin-bottom: 0; - } -} -.nav-tabs-justified { - border-bottom: 0; -} -.nav-tabs-justified > li > a { - margin-right: 0; - border-radius: 4px; -} -.nav-tabs-justified > .active > a, -.nav-tabs-justified > .active > a:hover, -.nav-tabs-justified > .active > a:focus { - border: 1px solid #ddd; -} - -@media (min-width: 768px) { - .nav-tabs-justified > li > a { - border-bottom: 1px solid #ddd; - border-radius: 4px 4px 0 0; - } - .nav-tabs-justified > .active > a, - .nav-tabs-justified > .active > a:hover, - .nav-tabs-justified > .active > a:focus { - border-bottom-color: #fafafa; - } -} -.tab-content > .tab-pane { - display: none; - visibility: hidden; -} -.tab-content > .active { - display: block; - visibility: visible; -} -.nav-tabs .dropdown-menu { - margin-top: -1px; - border-top-right-radius: 0; - border-top-left-radius: 0; -} -.navbar { - position: relative; - min-height: 50px; - background-color: #fff; - margin: 0; - border: 0; - box-shadow: 0 0 16px rgba(29, 32, 25, 0.25); -} - -@media (min-width: 768px) { - .navbar { - border-radius: 4px; - } -} - -@media (min-width: 768px) { - .navbar-header { - float: left; - } -} -.navbar-collapse { - overflow-x: visible; - padding-right: 15px; - padding-left: 15px; - border-top: 1px solid transparent; - box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1); - -webkit-overflow-scrolling: touch; -} -.navbar-collapse.in { - overflow-y: visible; -} - -@media (min-width: 768px) { - .navbar-collapse { - width: auto; - border-top: 0; - box-shadow: none; - } - .navbar-collapse.collapse { - display: block !important; - visibility: visible !important; - height: auto !important; - padding-bottom: 0; - overflow: visible !important; - } - .navbar-collapse.in { - overflow-y: visible; - } - .navbar-fixed-top .navbar-collapse, - .navbar-static-top .navbar-collapse, - .navbar-fixed-bottom .navbar-collapse { - padding-left: 0; - padding-right: 0; - } -} -.navbar-fixed-top .navbar-collapse, -.navbar-fixed-bottom .navbar-collapse { - max-height: none; -} - -@media (max-device-width: 480px) and (orientation: landscape) { - .navbar-fixed-top .navbar-collapse, - .navbar-fixed-bottom .navbar-collapse { - max-height: 200px; - } -} -.container > .navbar-header, -.container-fluid > .navbar-header, -.container > .navbar-collapse, -.container-fluid > .navbar-collapse { - margin-right: -15px; - margin-left: -15px; -} - -@media (min-width: 768px) { - .container > .navbar-header, - .container-fluid > .navbar-header, - .container > .navbar-collapse, - .container-fluid > .navbar-collapse { - margin-right: 0; - margin-left: 0; - } -} -.navbar-static-top { - z-index: 1000; - border-width: 0 0 1px; -} - -@media (min-width: 768px) { - .navbar-static-top { - border-radius: 0; - } -} -.navbar-fixed-top, -.navbar-fixed-bottom { - position: fixed; - right: 0; - left: 0; - z-index: 1030; -} - -@media (min-width: 768px) { - .navbar-fixed-top, - .navbar-fixed-bottom { - border-radius: 0; - } -} -.navbar-fixed-top { - top: 0; - border-width: 0 0 1px; -} -.navbar-fixed-bottom { - bottom: 0; - margin-bottom: 0; - border-width: 1px 0 0; -} -.navbar-brand { - float: left; - padding: 10px 15px; - height: 50px; -} - -@media (min-width: 768px) { - .navbar > .container .navbar-brand, - .navbar > .container-fluid .navbar-brand { - margin-left: -15px; - } -} -.navbar-toggle { - position: relative; - float: right; - margin-right: 15px; - padding: 9px 10px; - margin-top: 8px; - margin-bottom: 8px; - background-color: transparent; - background-image: none; - border: 1px solid transparent; - border-radius: 4px; -} -.navbar-toggle:focus { - outline: 0; -} -.navbar-toggle .icon-bar { - display: block; - width: 22px; - height: 2px; - border-radius: 1px; - background: currentColor; -} -.navbar-toggle .icon-bar + .icon-bar { - margin-top: 4px; -} - -@media (min-width: 768px) { - .navbar-toggle { - display: none; - } -} -.navbar-nav { - margin: 7.5px -15px; -} -.navbar-nav > li > a { - padding-top: 10px; - padding-bottom: 10px; - line-height: 20px; -} - -@media (max-width: 767px) { - .navbar-nav .dropdown:hover > .dropdown-menu, - .navbar-nav .dropdown:focus-within > .dropdown-menu { - position: static; - float: none; - width: auto; - margin-top: 0; - background-color: transparent; - border: 0; - box-shadow: none; - } - .navbar-nav .dropdown:hover > .dropdown-menu > li > a, - .navbar-nav .dropdown:hover > .dropdown-menu > li > form > button, - .navbar-nav .dropdown:hover > .dropdown-menu .dropdown-header, - .navbar-nav .dropdown:focus-within > .dropdown-menu > li > a, - .navbar-nav .dropdown:focus-within > .dropdown-menu > li > form > button, - .navbar-nav .dropdown:focus-within > .dropdown-menu .dropdown-header { - padding: 5px 15px 5px 25px; - } - .navbar-nav .dropdown:hover > .dropdown-menu > li > a, - .navbar-nav .dropdown:hover > .dropdown-menu > li > form > button, - .navbar-nav .dropdown:focus-within > .dropdown-menu > li > a, - .navbar-nav .dropdown:focus-within > .dropdown-menu > li > form > button { - line-height: 20px; - } - .navbar-nav .dropdown:hover > .dropdown-menu > li > a:hover, - .navbar-nav .dropdown:hover > .dropdown-menu > li > a:focus, - .navbar-nav .dropdown:focus-within > .dropdown-menu > li > a:hover, - .navbar-nav .dropdown:focus-within > .dropdown-menu > li > a:focus { - background-image: none; - } -} - -@media (min-width: 768px) { - .navbar-nav { - float: left; - margin: 0; - } - .navbar-nav > li { - float: left; - } - .navbar-nav > li > a { - padding-top: 20px; - padding-bottom: 20px; - } -} -.navbar-form { - margin-left: -15px; - margin-right: -15px; - padding: 10px 15px; - border-top: 1px solid transparent; - border-bottom: 1px solid transparent; - -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), - 0 1px 0 rgba(255, 255, 255, 0.1); - box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), - 0 1px 0 rgba(255, 255, 255, 0.1); - margin-top: 8px; - margin-bottom: 8px; -} - -@media (min-width: 768px) { - .navbar-form .form-group { - display: inline-block; - margin-bottom: 0; - vertical-align: middle; - } - .navbar-form .form-control { - display: inline-block; - width: auto; - vertical-align: middle; - } - .navbar-form .form-control-static { - display: inline-block; - } - .navbar-form .input-group { - display: inline-table; - vertical-align: middle; - } - .navbar-form .input-group .input-group-addon, - .navbar-form .input-group .input-group-btn, - .navbar-form .input-group .form-control { - width: auto; - } - .navbar-form .input-group > .form-control { - width: 100%; - } - .navbar-form .control-label { - margin-bottom: 0; - vertical-align: middle; - } - .navbar-form .radio, - .navbar-form .checkbox { - display: inline-block; - margin-top: 0; - margin-bottom: 0; - vertical-align: middle; - } - .navbar-form .radio label, - .navbar-form .checkbox label { - padding-left: 0; - } - .navbar-form .radio input[type="radio"], - .navbar-form .checkbox input[type="checkbox"] { - position: relative; - margin-left: 0; - } - .navbar-form .has-feedback .form-control-feedback { - top: 0; - } -} - -@media (max-width: 767px) { - .navbar-form .form-group { - margin-bottom: 5px; - } - .navbar-form .form-group:last-child { - margin-bottom: 0; - } -} - -@media (min-width: 768px) { - .navbar-form { - width: auto; - border: 0; - margin-left: 0; - margin-right: 0; - padding-top: 0; - padding-bottom: 0; - -webkit-box-shadow: none; - box-shadow: none; - } -} -.navbar-nav > li > .dropdown-menu { - margin-top: 0; - border-top-right-radius: 0; - border-top-left-radius: 0; -} -.navbar-fixed-bottom .navbar-nav > li > .dropdown-menu { - border-radius: 4px 4px 0 0; -} -.navbar-btn { - margin-top: 8px; - margin-bottom: 8px; -} -.navbar-btn.btn-sm { - margin-top: 10px; - margin-bottom: 10px; -} -.navbar-btn.btn-xs { - margin-top: 14px; - margin-bottom: 14px; -} -.navbar-text { - margin-top: 15px; - margin-bottom: 15px; -} - -@media (min-width: 768px) { - .navbar-text { - float: left; - margin-left: 15px; - margin-right: 15px; - } -} - -@media (min-width: 768px) { - .navbar-left { - float: left !important; - } - .navbar-right { - float: right !important; - margin-right: -15px; - } - .navbar-right ~ .navbar-right { - margin-right: 0; - } -} -.navbar-default { - background-color: #ffb400; - border-color: #de9d00; -} -.navbar-default .navbar-brand { - color: #007a87; -} -.navbar-default .navbar-brand:hover, -.navbar-default .navbar-brand:focus { - color: #004c54; - background-color: transparent; -} -.navbar-default .navbar-text { - color: #777; -} -.navbar-default .navbar-nav > li > a { - color: #007a87; -} -.navbar-default .navbar-nav > li > a:hover, -.navbar-default .navbar-nav > li > a:focus { - color: #51504f; - background-color: transparent; -} -.navbar-default .navbar-nav > .active > a, -.navbar-default .navbar-nav > .active > a:hover, -.navbar-default .navbar-nav > .active > a:focus { - color: #555; - background-color: #de9d00; -} -.navbar-default .navbar-nav > .disabled > a, -.navbar-default .navbar-nav > .disabled > a:hover, -.navbar-default .navbar-nav > .disabled > a:focus { - color: #ccc; - background-color: transparent; -} -.navbar-default .navbar-toggle { - border-color: #ddd; -} -.navbar-default .navbar-toggle:hover, -.navbar-default .navbar-toggle:focus { - background-color: #ddd; -} -.navbar-default .navbar-toggle .icon-bar { - background-color: #888; -} -.navbar-default .navbar-collapse, -.navbar-default .navbar-form { - border-color: #de9d00; -} -.navbar-default .navbar-nav > .open > a, -.navbar-default .navbar-nav > .open > a:hover, -.navbar-default .navbar-nav > .open > a:focus { - background-color: #de9d00; - color: #555; -} - -@media (max-width: 767px) { - .navbar-default .navbar-nav .open .dropdown-menu > li > a { - color: #007a87; - } - .navbar-default .navbar-nav .open .dropdown-menu > li > a:hover, - .navbar-default .navbar-nav .open .dropdown-menu > li > a:focus { - color: #51504f; - background-color: transparent; - } - .navbar-default .navbar-nav .open .dropdown-menu > .active > a, - .navbar-default .navbar-nav .open .dropdown-menu > .active > a:hover, - .navbar-default .navbar-nav .open .dropdown-menu > .active > a:focus { - color: #555; - background-color: #de9d00; - } - .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a, - .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:hover, - .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:focus { - color: #ccc; - background-color: transparent; - } -} -.navbar-default .navbar-link { - color: #007a87; -} -.navbar-default .navbar-link:hover { - color: #51504f; -} -.navbar-default .btn-link { - color: #007a87; -} -.navbar-default .btn-link:hover, -.navbar-default .btn-link:focus { - color: #51504f; -} -.navbar-default .btn-link[disabled]:hover, -fieldset[disabled] .navbar-default .btn-link:hover, -.navbar-default .btn-link[disabled]:focus, -fieldset[disabled] .navbar-default .btn-link:focus { - color: #ccc; -} -.navbar-inverse { - background-color: #007a87; - border-color: #004c54; - background-image: none; -} -.navbar-inverse .navbar-nav > .active > a, -.navbar-inverse .navbar-nav > .open > a { - background-image: none; -} -/* stylelint-disable selector-type-no-unknown */ -.dropdown-menu > .active > a, -ee.dropdown-menu > .active > a:focus, -.dropdown-menu > .active > a:hover { - background-image: none; -} -/* stylelint-enable selector-type-no-unknown */ -.navbar-inverse .navbar-brand { - color: #ddd; -} -.navbar-inverse .navbar-brand:hover, -.navbar-inverse .navbar-brand:focus { - color: #fff; - background-color: transparent; -} -.navbar-inverse .navbar-text { - color: #ffb400; -} -.navbar-inverse .navbar-nav > li > a { - color: #ddd; -} -.navbar-inverse .navbar-nav > li > a:hover, -.navbar-inverse .navbar-nav > li > a:focus { - color: #fff; - background-color: transparent; -} -.navbar-inverse .navbar-nav > .active > a, -.navbar-inverse .navbar-nav > .active > a:hover, -.navbar-inverse .navbar-nav > .active > a:focus { - color: #fff; - background-color: #004c54; -} -.navbar-inverse .navbar-nav > .disabled > a, -.navbar-inverse .navbar-nav > .disabled > a:hover, -.navbar-inverse .navbar-nav > .disabled > a:focus { - color: #444; - background-color: transparent; -} -.navbar-inverse .navbar-toggle { - border-color: #51504f; -} -.navbar-inverse .navbar-toggle:hover, -.navbar-inverse .navbar-toggle:focus { - background-color: #51504f; -} -.navbar-inverse .navbar-toggle .icon-bar { - background-color: #fff; -} -.navbar-inverse .navbar-collapse, -.navbar-inverse .navbar-form { - border-color: #005a63; -} -.navbar-inverse .navbar-nav > .open > a, -.navbar-inverse .navbar-nav > .open > a:hover, -.navbar-inverse .navbar-nav > .open > a:focus { - background-color: #004c54; - color: #fff; -} - -@media (max-width: 767px) { - .navbar-inverse .navbar-nav .open .dropdown-menu > .dropdown-header { - border-color: #004c54; - } - .navbar-inverse .navbar-nav .open .dropdown-menu .divider { - background-color: #004c54; - } - .navbar-inverse .navbar-nav .open .dropdown-menu > li > a { - color: #ddd; - } - .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:hover, - .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:focus { - color: #fff; - background-color: transparent; - } - .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a, - .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:hover, - .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:focus { - color: #fff; - background-color: #004c54; - } - .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a, - .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:hover, - .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:focus { - color: #444; - background-color: transparent; - } -} -.navbar-inverse .navbar-link { - color: #ddd; -} -.navbar-inverse .navbar-link:hover { - color: #fff; -} -.navbar-inverse .btn-link { - color: #ddd; -} -.navbar-inverse .btn-link:hover, -.navbar-inverse .btn-link:focus { - color: #fff; -} -.navbar-inverse .btn-link[disabled]:hover, -fieldset[disabled] .navbar-inverse .btn-link:hover, -.navbar-inverse .btn-link[disabled]:focus, -fieldset[disabled] .navbar-inverse .btn-link:focus { - color: #444; -} -.breadcrumb { - padding: 8px 15px; - margin-bottom: 20px; - list-style: none; - background-color: #f5f5f5; - border-radius: 4px; -} -.breadcrumb > li { - display: inline-block; -} -.breadcrumb > li + li::before { - content: "/\00a0"; - padding: 0 5px; - color: #ccc; -} -.breadcrumb > .active { - color: #777; -} -.pagination { - display: inline-block; - padding-left: 0; - margin: 20px 0; - border-radius: 4px; -} -.pagination > li { - display: inline; -} -.pagination > li > a, -.pagination > li > span { - position: relative; - float: left; - padding: 6px 12px; - line-height: 1.428571429; - text-decoration: none; - color: #017cee; - background-color: #fff; - border: 1px solid #ddd; - margin-left: -1px; -} -.pagination > li:first-child > a, -.pagination > li:first-child > span { - margin-left: 0; - border-bottom-left-radius: 4px; - border-top-left-radius: 4px; -} -.pagination > li:last-child > a, -.pagination > li:last-child > span { - border-bottom-right-radius: 4px; - border-top-right-radius: 4px; -} -.pagination > li > a:hover, -.pagination > li > span:hover, -.pagination > li > a:focus, -.pagination > li > span:focus { - color: #0cb6ff; - background-color: #eee; - border-color: #ddd; -} -.pagination > .active > a, -.pagination > .active > span, -.pagination > .active > a:hover, -.pagination > .active > span:hover, -.pagination > .active > a:focus, -.pagination > .active > span:focus { - z-index: 2; - color: #fff; - background-color: ##017cee; - border-color: ##017cee; - cursor: default; -} -.pagination > .disabled > span, -.pagination > .disabled > span:hover, -.pagination > .disabled > span:focus, -.pagination > .disabled > a, -.pagination > .disabled > a:hover, -.pagination > .disabled > a:focus { - color: #777; - background-color: #fff; - border-color: #ddd; - cursor: not-allowed; -} -.pagination-lg > li > a, -.pagination-lg > li > span { - padding: 10px 16px; - font-size: 18px; -} -.pagination-lg > li:first-child > a, -.pagination-lg > li:first-child > span { - border-bottom-left-radius: 6px; - border-top-left-radius: 6px; -} -.pagination-lg > li:last-child > a, -.pagination-lg > li:last-child > span { - border-bottom-right-radius: 6px; - border-top-right-radius: 6px; -} -.pagination-sm > li > a, -.pagination-sm > li > span { - padding: 5px 10px; - font-size: 12px; -} -.pagination-sm > li:first-child > a, -.pagination-sm > li:first-child > span { - border-bottom-left-radius: 3px; - border-top-left-radius: 3px; -} -.pagination-sm > li:last-child > a, -.pagination-sm > li:last-child > span { - border-bottom-right-radius: 3px; - border-top-right-radius: 3px; -} -.pager { - padding-left: 0; - margin: 20px 0; - list-style: none; - text-align: center; -} -.pager li { - display: inline; -} -.pager li > a, -.pager li > span { - display: inline-block; - padding: 5px 14px; - background-color: #fff; - border: 1px solid #ddd; - border-radius: 15px; -} -.pager li > a:hover, -.pager li > a:focus { - text-decoration: none; - background-color: #eee; -} -.pager .next > a, -.pager .next > span { - float: right; -} -.pager .previous > a, -.pager .previous > span { - float: left; -} -.pager .disabled > a, -.pager .disabled > a:hover, -.pager .disabled > a:focus, -.pager .disabled > span { - color: #777; - background-color: #fff; - cursor: not-allowed; -} -.label { - display: inline; - padding: 0.2em 0.6em 0.3em; - font-size: 75%; - font-weight: bold; - line-height: 1; - color: #51504f; - text-align: center; - white-space: nowrap; - vertical-align: baseline; - border-radius: 0.25em; -} -a.label:hover, -a.label:focus { - color: #51504f; - text-decoration: none; - cursor: pointer; -} -.label:empty { - display: none; -} -.btn .label { - position: relative; - top: -1px; -} -.label-default { - background-color: #e2e2e2; -} -.label-default[href]:hover, -.label-default[href]:focus { - background-color: #cbcbcb; -} -.label-primary { - color: #365f84; - background-color: #c0defb; -} -.label-primary[href]:hover, -.label-primary[href]:focus { - color: #365f84; - background-color: #0cb6ff; -} -.label-success { - color: #366f4c; - background-color: #bfebd1; -} -.label-success[href]:hover, -.label-success[href]:focus { - color: #366f4c; - background-color: #04d659; -} -.label-info { - color: #36787b; - background-color: #bff1f4; -} -.label-info[href]:hover, -.label-info[href]:focus { - color: #36787b; - background-color: #11e1ee; -} -.label-warning { - background-color: #ffb400; -} -.label-warning[href]:hover, -.label-warning[href]:focus { - background-color: #cc9000; -} -.label-danger { - color: #824840; - background-color: #f8cec8; -} -.label-danger[href]:hover, -.label-danger[href]:focus { - color: #824840; - background-color: #ff7557; -} -.badge { - display: inline-block; - min-width: 10px; - padding: 3px 7px; - font-size: 12px; - font-weight: bold; - color: #fff; - line-height: 1; - vertical-align: baseline; - white-space: nowrap; - text-align: center; - background-color: #777; - border-radius: 10px; -} -.badge:empty { - display: none; -} -.btn .badge { - position: relative; - top: -1px; -} -.btn-xs .badge { - top: 0; - padding: 1px 5px; -} -a.badge:hover, -a.badge:focus { - color: #fff; - text-decoration: none; - cursor: pointer; -} -.list-group-item.active > .badge, -.nav-pills > .active > a > .badge { - color: #017cee; - background-color: #fff; -} -.list-group-item > .badge { - float: right; -} -.list-group-item > .badge + .badge { - margin-right: 5px; -} -.nav-pills > li > a > .badge { - margin-left: 3px; -} -.jumbotron { - padding: 30px 15px; - margin-bottom: 30px; - color: inherit; - background-color: #eee; -} -.jumbotron h1, -.jumbotron .h1 { - color: inherit; -} -.jumbotron p { - margin-bottom: 15px; - font-size: 21px; - font-weight: 200; -} -.jumbotron > hr { - border-top-color: #d5d5d5; -} -.container .jumbotron, -.container-fluid .jumbotron { - border-radius: 6px; -} -.jumbotron .container { - max-width: 100%; -} - -@media screen and (min-width: 768px) { - .jumbotron { - padding: 48px 0; - } - .container .jumbotron, - .container-fluid .jumbotron { - padding-left: 60px; - padding-right: 60px; - } - .jumbotron h1, - .jumbotron .h1 { - font-size: 63px; - } -} -.thumbnail { - display: block; - padding: 4px; - margin-bottom: 20px; - line-height: 1.428571429; - background-color: #fafafa; - border: 1px solid #ddd; - border-radius: 4px; - -webkit-transition: border 0.2s ease-in-out; - -o-transition: border 0.2s ease-in-out; - transition: border 0.2s ease-in-out; -} -.thumbnail > img, -.thumbnail a > img { - margin-left: auto; - margin-right: auto; -} -a.thumbnail:hover, -a.thumbnail:focus, -a.thumbnail.active { - border-color: #017cee; -} -.thumbnail .caption { - padding: 9px; - color: #51504f; -} -.alert { - padding: 16px; - margin-bottom: 16px; - border: 0; - border-radius: 4px; -} -.alert h4 { - margin-top: 0; - color: inherit; -} -.alert .alert-link { - font-weight: bold; -} -.alert > p, -.alert > ul { - margin-bottom: 0; -} -.alert > p + p { - margin-top: 5px; -} -.alert-dismissable, -.alert-dismissible { - padding-right: 35px; -} -.alert-dismissable .close, -.alert-dismissible .close { - position: relative; - top: -2px; - right: -21px; - color: inherit; -} -.alert-success { - background-color: #bfebd1; - color: #1b8e49; -} -.alert-success hr { - border-top-color: #bfebd1; -} -.alert-success .alert-link { - color: #366f4c; -} -.alert-info { - background-color: #bff1f4; - color: #36787b; -} -.alert-info hr { - border-top-color: #a6e1ec; -} -.alert-info .alert-link { - color: #245269; -} -.alert-warning { - background-color: #fcf8e3; - color: #8a6d3b; -} -.alert-warning hr { - border-top-color: #f7e1b5; -} -.alert-warning .alert-link { - color: #66512c; -} -.alert-danger { - background-color: #f2dede; - color: #e43921; -} -.alert-danger hr { - border-top-color: #e4b9c0; -} -.alert-danger .alert-link { - color: #843534; -} -.alert-message { - background-color: #d9edf7; - color: #31708f; -} -.alert-message hr { - border-top-color: #a6e1ec; -} -.alert-message .alert-link { - color: #245269; -} -.alert-error { - background-color: #f2dede; - color: #e43921; -} -.alert-error hr { - border-top-color: #e4b9c0; -} -.alert-error .alert-link { - color: #843534; -} - -@-webkit-keyframes progress-bar-stripes { - from { - background-position: 40px 0; - } - to { - background-position: 0 0; - } -} - -@keyframes progress-bar-stripes { - from { - background-position: 40px 0; - } - to { - background-position: 0 0; - } -} -.progress { - overflow: hidden; - height: 20px; - margin-bottom: 20px; - background-color: #f5f5f5; - border-radius: 4px; - -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1); - box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1); -} -.progress-bar { - float: left; - width: 0%; - height: 100%; - font-size: 12px; - line-height: 20px; - color: #fff; - text-align: center; - background-color: #017cee; - -webkit-box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.15); - box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.15); - -webkit-transition: width 0.6s ease; - -o-transition: width 0.6s ease; - transition: width 0.6s ease; -} -.progress-striped .progress-bar, -.progress-bar-striped { - background-image: -webkit-linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-image: -o-linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-image: linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-size: 40px 40px; -} -.progress.active .progress-bar, -.progress-bar.active { - -webkit-animation: progress-bar-stripes 2s linear infinite; - -o-animation: progress-bar-stripes 2s linear infinite; - animation: progress-bar-stripes 2s linear infinite; -} -.progress-bar-success { - background-color: #00ad46; -} -.progress-striped .progress-bar-success { - background-image: -webkit-linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-image: -o-linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-image: linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); -} -.progress-bar-info { - background-color: #00d1c1; -} -.progress-striped .progress-bar-info { - background-image: -webkit-linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-image: -o-linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-image: linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); -} -.progress-bar-warning { - background-color: #ffb400; -} -.progress-striped .progress-bar-warning { - background-image: -webkit-linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-image: -o-linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-image: linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); -} -.progress-bar-danger { - background-color: #ff5a5f; -} -.progress-striped .progress-bar-danger { - background-image: -webkit-linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-image: -o-linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); - background-image: linear-gradient( - 45deg, - rgba(255, 255, 255, 0.15) 25%, - transparent 25%, - transparent 50%, - rgba(255, 255, 255, 0.15) 50%, - rgba(255, 255, 255, 0.15) 75%, - transparent 75%, - transparent - ); -} -.media { - margin-top: 15px; -} -.media:first-child { - margin-top: 0; -} -.media, -.media-body { - zoom: 1; - overflow: hidden; -} -.media-object { - display: block; -} -.media-right, -.media > .pull-right { - padding-left: 10px; -} -.media-left, -.media > .pull-left { - padding-right: 10px; -} -.media-left, -.media-right, -.media-body { - display: table-cell; - vertical-align: top; -} -.media-middle { - vertical-align: middle; -} -.media-bottom { - vertical-align: bottom; -} -.media-heading { - margin-top: 0; - margin-bottom: 5px; -} -.media-list { - padding-left: 0; - list-style: none; -} -.list-group { - margin-bottom: 20px; - padding-left: 0; -} -.list-group-item { - position: relative; - display: block; - padding: 10px 15px; - margin-bottom: -1px; - background-color: #fff; - border: 1px solid #ddd; -} -.list-group-item:first-child { - border-top-right-radius: 4px; - border-top-left-radius: 4px; -} -.list-group-item:last-child { - margin-bottom: 0; - border-bottom-right-radius: 4px; - border-bottom-left-radius: 4px; -} -a.list-group-item { - color: #555; -} -a.list-group-item .list-group-item-heading { - color: #51504f; -} -a.list-group-item:hover, -a.list-group-item:focus { - text-decoration: none; - color: #555; - background-color: #f5f5f5; -} -.list-group-item.disabled, -.list-group-item.disabled:hover, -.list-group-item.disabled:focus { - background-color: #eee; - color: #777; - cursor: not-allowed; -} -.list-group-item.disabled .list-group-item-heading, -.list-group-item.disabled:hover .list-group-item-heading, -.list-group-item.disabled:focus .list-group-item-heading { - color: inherit; -} -.list-group-item.disabled .list-group-item-text, -.list-group-item.disabled:hover .list-group-item-text, -.list-group-item.disabled:focus .list-group-item-text { - color: #777; -} -.list-group-item.active, -.list-group-item.active:hover, -.list-group-item.active:focus { - z-index: 2; - color: #fff; - background-color: #017cee; - border-color: #017cee; -} -.list-group-item.active .list-group-item-heading, -.list-group-item.active:hover .list-group-item-heading, -.list-group-item.active:focus .list-group-item-heading, -.list-group-item.active .list-group-item-heading > small, -.list-group-item.active:hover .list-group-item-heading > small, -.list-group-item.active:focus .list-group-item-heading > small, -.list-group-item.active .list-group-item-heading > .small, -.list-group-item.active:hover .list-group-item-heading > .small, -.list-group-item.active:focus .list-group-item-heading > .small { - color: inherit; -} -.list-group-item.active .list-group-item-text, -.list-group-item.active:hover .list-group-item-text, -.list-group-item.active:focus .list-group-item-text { - color: #33ebff; -} -.list-group-item-success { - color: #1b8e49; - background-color: #bfebd1; -} -a.list-group-item-success { - color: #1b8e49; -} -a.list-group-item-success .list-group-item-heading { - color: inherit; -} -a.list-group-item-success:hover, -a.list-group-item-success:focus { - color: #1b8e49; - background-color: #d0e9c6; -} -a.list-group-item-success.active, -a.list-group-item-success.active:hover, -a.list-group-item-success.active:focus { - color: #fff; - background-color: #1b8e49; - border-color: #1b8e49; -} -.list-group-item-info { - color: #31708f; - background-color: #d9edf7; -} -a.list-group-item-info { - color: #31708f; -} -a.list-group-item-info .list-group-item-heading { - color: inherit; -} -a.list-group-item-info:hover, -a.list-group-item-info:focus { - color: #31708f; - background-color: #c4e3f3; -} -a.list-group-item-info.active, -a.list-group-item-info.active:hover, -a.list-group-item-info.active:focus { - color: #fff; - background-color: #31708f; - border-color: #31708f; -} -.list-group-item-warning { - color: #8a6d3b; - background-color: #fcf8e3; -} -a.list-group-item-warning { - color: #8a6d3b; -} -a.list-group-item-warning .list-group-item-heading { - color: inherit; -} -a.list-group-item-warning:hover, -a.list-group-item-warning:focus { - color: #8a6d3b; - background-color: #faf2cc; -} -a.list-group-item-warning.active, -a.list-group-item-warning.active:hover, -a.list-group-item-warning.active:focus { - color: #fff; - background-color: #8a6d3b; - border-color: #8a6d3b; -} -.list-group-item-danger { - color: #e43921; - background-color: #f2dede; -} -a.list-group-item-danger { - color: #e43921; -} -a.list-group-item-danger .list-group-item-heading { - color: inherit; -} -a.list-group-item-danger:hover, -a.list-group-item-danger:focus { - color: #e43921; - background-color: #ebcccc; -} -a.list-group-item-danger.active, -a.list-group-item-danger.active:hover, -a.list-group-item-danger.active:focus { - color: #fff; - background-color: #e43921; - border-color: #e43921; -} -.list-group-item-heading { - margin-top: 0; - margin-bottom: 5px; -} -.list-group-item-text { - margin-bottom: 0; - line-height: 1.3; -} -.panel { - margin-bottom: 20px; - background-color: #fff; - border: 2px solid transparent; - border-radius: 4px; - box-shadow: none; -} -.panel-body { - padding: 15px; -} -.panel-heading { - padding: 10px 15px; - border-bottom: 1px solid transparent; - border-top-right-radius: 2px; - border-top-left-radius: 2px; -} -.panel-heading > .dropdown .dropdown-toggle { - color: inherit; -} -.panel-title { - margin-top: 0; - margin-bottom: 0; - font-size: 16px; - color: inherit; -} -.panel-title > a { - color: inherit; -} -.panel-footer { - padding: 10px 15px; - background-color: #f5f5f5; - border-top: 1px solid #ddd; - border-bottom-right-radius: 4px; - border-bottom-left-radius: 4px; -} -.panel > .list-group, -.panel > .panel-collapse > .list-group { - margin-bottom: 0; -} -.panel > .list-group .list-group-item, -.panel > .panel-collapse > .list-group .list-group-item { - border-width: 1px 0; - border-radius: 0; -} -.panel > .list-group:first-child .list-group-item:first-child, -.panel - > .panel-collapse - > .list-group:first-child - .list-group-item:first-child { - border-top: 0; - border-top-right-radius: 3px; - border-top-left-radius: 3px; -} -.panel > .list-group:last-child .list-group-item:last-child, -.panel > .panel-collapse > .list-group:last-child .list-group-item:last-child { - border-bottom: 0; - border-bottom-right-radius: 3px; - border-bottom-left-radius: 3px; -} -.panel-heading + .list-group .list-group-item:first-child { - border-top-width: 0; -} -.list-group + .panel-footer { - border-top-width: 0; -} -.panel > .table, -.panel > .table-responsive > .table, -.panel > .panel-collapse > .table { - margin-bottom: 0; -} -.panel > .table caption, -.panel > .table-responsive > .table caption, -.panel > .panel-collapse > .table caption { - padding-left: 15px; - padding-right: 15px; -} -.panel > .table:first-child, -.panel > .table-responsive:first-child > .table:first-child { - border-top-right-radius: 3px; - border-top-left-radius: 3px; -} -.panel > .table:first-child > thead:first-child > tr:first-child, -.panel - > .table-responsive:first-child - > .table:first-child - > thead:first-child - > tr:first-child, -.panel > .table:first-child > tbody:first-child > tr:first-child, -.panel - > .table-responsive:first-child - > .table:first-child - > tbody:first-child - > tr:first-child { - border-top-left-radius: 3px; - border-top-right-radius: 3px; -} -.panel > .table:first-child > thead:first-child > tr:first-child td:first-child, -.panel - > .table-responsive:first-child - > .table:first-child - > thead:first-child - > tr:first-child - td:first-child, -.panel > .table:first-child > tbody:first-child > tr:first-child td:first-child, -.panel - > .table-responsive:first-child - > .table:first-child - > tbody:first-child - > tr:first-child - td:first-child, -.panel > .table:first-child > thead:first-child > tr:first-child th:first-child, -.panel - > .table-responsive:first-child - > .table:first-child - > thead:first-child - > tr:first-child - th:first-child, -.panel > .table:first-child > tbody:first-child > tr:first-child th:first-child, -.panel - > .table-responsive:first-child - > .table:first-child - > tbody:first-child - > tr:first-child - th:first-child { - border-top-left-radius: 3px; -} -.panel > .table:first-child > thead:first-child > tr:first-child td:last-child, -.panel - > .table-responsive:first-child - > .table:first-child - > thead:first-child - > tr:first-child - td:last-child, -.panel > .table:first-child > tbody:first-child > tr:first-child td:last-child, -.panel - > .table-responsive:first-child - > .table:first-child - > tbody:first-child - > tr:first-child - td:last-child, -.panel > .table:first-child > thead:first-child > tr:first-child th:last-child, -.panel - > .table-responsive:first-child - > .table:first-child - > thead:first-child - > tr:first-child - th:last-child, -.panel > .table:first-child > tbody:first-child > tr:first-child th:last-child, -.panel - > .table-responsive:first-child - > .table:first-child - > tbody:first-child - > tr:first-child - th:last-child { - border-top-right-radius: 3px; -} -.panel > .table:last-child, -.panel > .table-responsive:last-child > .table:last-child { - border-bottom-right-radius: 3px; - border-bottom-left-radius: 3px; -} -.panel > .table:last-child > tbody:last-child > tr:last-child, -.panel - > .table-responsive:last-child - > .table:last-child - > tbody:last-child - > tr:last-child, -.panel > .table:last-child > tfoot:last-child > tr:last-child, -.panel - > .table-responsive:last-child - > .table:last-child - > tfoot:last-child - > tr:last-child { - border-bottom-left-radius: 3px; - border-bottom-right-radius: 3px; -} -.panel > .table:last-child > tbody:last-child > tr:last-child td:first-child, -.panel - > .table-responsive:last-child - > .table:last-child - > tbody:last-child - > tr:last-child - td:first-child, -.panel > .table:last-child > tfoot:last-child > tr:last-child td:first-child, -.panel - > .table-responsive:last-child - > .table:last-child - > tfoot:last-child - > tr:last-child - td:first-child, -.panel > .table:last-child > tbody:last-child > tr:last-child th:first-child, -.panel - > .table-responsive:last-child - > .table:last-child - > tbody:last-child - > tr:last-child - th:first-child, -.panel > .table:last-child > tfoot:last-child > tr:last-child th:first-child, -.panel - > .table-responsive:last-child - > .table:last-child - > tfoot:last-child - > tr:last-child - th:first-child { - border-bottom-left-radius: 3px; -} -.panel > .table:last-child > tbody:last-child > tr:last-child td:last-child, -.panel - > .table-responsive:last-child - > .table:last-child - > tbody:last-child - > tr:last-child - td:last-child, -.panel > .table:last-child > tfoot:last-child > tr:last-child td:last-child, -.panel - > .table-responsive:last-child - > .table:last-child - > tfoot:last-child - > tr:last-child - td:last-child, -.panel > .table:last-child > tbody:last-child > tr:last-child th:last-child, -.panel - > .table-responsive:last-child - > .table:last-child - > tbody:last-child - > tr:last-child - th:last-child, -.panel > .table:last-child > tfoot:last-child > tr:last-child th:last-child, -.panel - > .table-responsive:last-child - > .table:last-child - > tfoot:last-child - > tr:last-child - th:last-child { - border-bottom-right-radius: 3px; -} -.panel > .panel-body + .table, -.panel > .panel-body + .table-responsive, -.panel > .table + .panel-body, -.panel > .table-responsive + .panel-body { - border-top: 1px solid #ddd; -} -.panel > .table > tbody:first-child > tr:first-child th, -.panel > .table > tbody:first-child > tr:first-child td { - border-top: 0; -} -.panel > .table-bordered, -.panel > .table-responsive > .table-bordered { - border: 0; -} -.panel > .table-bordered > thead > tr > th:first-child, -.panel > .table-responsive > .table-bordered > thead > tr > th:first-child, -.panel > .table-bordered > tbody > tr > th:first-child, -.panel > .table-responsive > .table-bordered > tbody > tr > th:first-child, -.panel > .table-bordered > tfoot > tr > th:first-child, -.panel > .table-responsive > .table-bordered > tfoot > tr > th:first-child, -.panel > .table-bordered > thead > tr > td:first-child, -.panel > .table-responsive > .table-bordered > thead > tr > td:first-child, -.panel > .table-bordered > tbody > tr > td:first-child, -.panel > .table-responsive > .table-bordered > tbody > tr > td:first-child, -.panel > .table-bordered > tfoot > tr > td:first-child, -.panel > .table-responsive > .table-bordered > tfoot > tr > td:first-child { - border-left: 0; -} -.panel > .table-bordered > thead > tr > th:last-child, -.panel > .table-responsive > .table-bordered > thead > tr > th:last-child, -.panel > .table-bordered > tbody > tr > th:last-child, -.panel > .table-responsive > .table-bordered > tbody > tr > th:last-child, -.panel > .table-bordered > tfoot > tr > th:last-child, -.panel > .table-responsive > .table-bordered > tfoot > tr > th:last-child, -.panel > .table-bordered > thead > tr > td:last-child, -.panel > .table-responsive > .table-bordered > thead > tr > td:last-child, -.panel > .table-bordered > tbody > tr > td:last-child, -.panel > .table-responsive > .table-bordered > tbody > tr > td:last-child, -.panel > .table-bordered > tfoot > tr > td:last-child, -.panel > .table-responsive > .table-bordered > tfoot > tr > td:last-child { - border-right: 0; -} -.panel > .table-bordered > thead > tr:first-child > td, -.panel > .table-responsive > .table-bordered > thead > tr:first-child > td, -.panel > .table-bordered > tbody > tr:first-child > td, -.panel > .table-responsive > .table-bordered > tbody > tr:first-child > td, -.panel > .table-bordered > thead > tr:first-child > th, -.panel > .table-responsive > .table-bordered > thead > tr:first-child > th, -.panel > .table-bordered > tbody > tr:first-child > th, -.panel > .table-responsive > .table-bordered > tbody > tr:first-child > th { - border-bottom: 0; -} -.panel > .table-bordered > tbody > tr:last-child > td, -.panel > .table-responsive > .table-bordered > tbody > tr:last-child > td, -.panel > .table-bordered > tfoot > tr:last-child > td, -.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > td, -.panel > .table-bordered > tbody > tr:last-child > th, -.panel > .table-responsive > .table-bordered > tbody > tr:last-child > th, -.panel > .table-bordered > tfoot > tr:last-child > th, -.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > th { - border-bottom: 0; -} -.panel > .table-responsive { - border: 0; - margin-bottom: 0; -} -.panel-group { - margin-bottom: 20px; -} -.panel-group .panel { - margin-bottom: 0; - border-radius: 4px; -} -.panel-group .panel + .panel { - margin-top: 5px; -} -.panel-group .panel-heading { - border-bottom: 0; -} -.panel-group .panel-heading + .panel-collapse > .panel-body, -.panel-group .panel-heading + .panel-collapse > .list-group { - border-top: 1px solid #ddd; -} -.panel-group .panel-footer { - border-top: 0; -} -.panel-group .panel-footer + .panel-collapse .panel-body { - border-bottom: 1px solid #ddd; -} -.panel-default { - border-color: #ddd; -} -.panel-default > .panel-heading { - color: #51504f; - background-color: #f5f5f5; - border-color: #ddd; -} -.panel-default > .panel-heading + .panel-collapse > .panel-body { - border-top-color: #ddd; -} -.panel-default > .panel-heading .badge { - color: #f5f5f5; - background-color: #51504f; -} -.panel-default > .panel-footer + .panel-collapse > .panel-body { - border-bottom-color: #ddd; -} -.panel-primary { - border-color: #e2e2e2; -} -.panel-primary > .panel-heading { - color: #51504f; - background-color: #e2e2e2; - border-color: #e2e2e2; -} -.panel-primary > .panel-heading + .panel-collapse > .panel-body { - border-top-color: #e2e2e2; -} -.panel-primary > .panel-heading .badge { - color: #e2e2e2; -} -.panel-primary > .panel-footer + .panel-collapse > .panel-body { - border-bottom-color: #e2e2e2; -} -.panel-success { - border-color: #d6e9c6; -} -.panel-success > .panel-heading { - color: #1b8e49; - background-color: #bfebd1; - border-color: #d6e9c6; -} -.panel-success > .panel-heading + .panel-collapse > .panel-body { - border-top-color: #d6e9c6; -} -.panel-success > .panel-heading .badge { - color: #bfebd1; - background-color: #1b8e49; -} -.panel-success > .panel-footer + .panel-collapse > .panel-body { - border-bottom-color: #d6e9c6; -} -.panel-info { - border-color: #bce8f1; -} -.panel-info > .panel-heading { - color: #31708f; - background-color: #d9edf7; - border-color: #bce8f1; -} -.panel-info > .panel-heading + .panel-collapse > .panel-body { - border-top-color: #bce8f1; -} -.panel-info > .panel-heading .badge { - color: #d9edf7; - background-color: #31708f; -} -.panel-info > .panel-footer + .panel-collapse > .panel-body { - border-bottom-color: #bce8f1; -} -.panel-warning { - border-color: #faebcc; -} -.panel-warning > .panel-heading { - color: #8a6d3b; - background-color: #fcf8e3; - border-color: #faebcc; -} -.panel-warning > .panel-heading + .panel-collapse > .panel-body { - border-top-color: #faebcc; -} -.panel-warning > .panel-heading .badge { - color: #fcf8e3; - background-color: #8a6d3b; -} -.panel-warning > .panel-footer + .panel-collapse > .panel-body { - border-bottom-color: #faebcc; -} -.panel-danger { - border-color: #ebccd1; -} -.panel-danger > .panel-heading { - color: #e43921; - background-color: #f2dede; - border-color: #ebccd1; -} -.panel-danger > .panel-heading + .panel-collapse > .panel-body { - border-top-color: #ebccd1; -} -.panel-danger > .panel-heading .badge { - color: #f2dede; - background-color: #e43921; -} -.panel-danger > .panel-footer + .panel-collapse > .panel-body { - border-bottom-color: #ebccd1; -} -.well { - min-height: 20px; - padding: 19px; - margin-bottom: 20px; - background-color: #f5f5f5; - border: 0; - border-radius: 4px; - box-shadow: none; -} -.well blockquote { - border-color: #ddd; - border-color: rgba(0, 0, 0, 0.15); -} -.well-lg { - padding: 24px; - border-radius: 6px; -} -.well-sm { - padding: 9px; -} -.close { - float: right; - font-size: 21px; - font-weight: bold; - line-height: 1; - color: currentColor; - opacity: 0.4; -} -.close:hover, -.close:focus { - color: currentColor; - text-decoration: none; - cursor: pointer; - opacity: 0.8; -} -button.close { - padding: 0; - cursor: pointer; - background: transparent; - border: 0; - -webkit-appearance: none; -} -.modal-open { - overflow: hidden; -} -.modal { - display: none; - overflow: hidden; - position: fixed; - top: 0; - right: 0; - bottom: 0; - left: 0; - z-index: 1040; - -webkit-overflow-scrolling: touch; - outline: 0; -} -.modal.fade .modal-dialog { - -webkit-transform: translate(0, -25%); - -ms-transform: translate(0, -25%); - -o-transform: translate(0, -25%); - transform: translate(0, -25%); - -webkit-transition: -webkit-transform 0.3s ease-out; - -moz-transition: -moz-transform 0.3s ease-out; - -o-transition: -o-transform 0.3s ease-out; - transition: transform 0.3s ease-out; -} -.modal.in .modal-dialog { - -webkit-transform: translate(0, 0); - -ms-transform: translate(0, 0); - -o-transform: translate(0, 0); - transform: translate(0, 0); -} -.modal-open .modal { - overflow-x: hidden; - overflow-y: auto; -} -.modal-dialog { - position: relative; - width: auto; - margin: 10px; -} -.modal-content { - position: relative; - background-color: #fff; - border: 0; - border-radius: 6px; - -webkit-box-shadow: 0 3px 9px rgba(0, 0, 0, 0.5); - box-shadow: 0 3px 9px rgba(0, 0, 0, 0.5); - background-clip: padding-box; - outline: 0; - z-index: 5; -} -.modal-backdrop { - z-index: 2; - position: fixed; - top: 0; - right: 0; - left: 0; - background-color: #000; -} -.modal-backdrop.fade { - opacity: 0; -} -.modal-backdrop.in { - opacity: 0.5; -} -.modal-header { - padding: 15px; - border-bottom: 1px solid #e5e5e5; - min-height: 16.428571429px; -} -.modal-header .close { - margin-top: -2px; -} -.modal-title { - margin: 0; - line-height: 1.428571429; -} -.modal-body { - position: relative; - padding: 15px; -} -.modal-footer { - padding: 15px; - text-align: right; - border-top: 1px solid #e5e5e5; -} -.modal-footer .btn + .btn { - margin-left: 5px; - margin-bottom: 0; -} -.modal-footer .btn-group .btn + .btn { - margin-left: -1px; -} -.modal-footer .btn-block + .btn-block { - margin-left: 0; -} -.modal-scrollbar-measure { - position: absolute; - top: -9999px; - width: 50px; - height: 50px; - overflow: auto; -} - -@media (min-width: 768px) { - .modal-dialog { - width: 600px; - margin: 30px auto; - } - .modal-content { - -webkit-box-shadow: 0 5px 15px rgba(0, 0, 0, 0.5); - box-shadow: 0 5px 15px rgba(0, 0, 0, 0.5); - } - .modal-sm { - width: 300px; - } -} - -@media (min-width: 992px) { - .modal-lg { - width: 900px; - } -} -.tooltip { - position: absolute; - z-index: 1070; - display: block; - visibility: visible; - font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; - font-size: 12px; - font-weight: normal; - line-height: 1.4; - opacity: 0; -} -.tooltip.in { - opacity: 1; -} -.tooltip.top { - margin-top: -3px; - padding: 5px 0; -} -.tooltip.right { - margin-left: 3px; - padding: 0 5px; -} -.tooltip.bottom { - margin-top: 3px; - padding: 5px 0; -} -.tooltip.left { - margin-left: -3px; - padding: 0 5px; -} -.tooltip-inner { - max-width: 500px; - padding: 3px 8px; - color: #fff; - text-align: left; - text-decoration: none; - background-color: #51504f; - border-radius: 4px; - font-size: 13px; -} -.tooltip-arrow { - position: absolute; - width: 0; - height: 0; - border-color: transparent; - border-style: solid; -} -.tooltip.top .tooltip-arrow { - bottom: 0; - left: 50%; - margin-left: -5px; - border-width: 5px 5px 0; - border-top-color: #51504f; -} -.tooltip.top-left .tooltip-arrow { - bottom: 0; - right: 5px; - margin-bottom: -5px; - border-width: 5px 5px 0; - border-top-color: #51504f; -} -.tooltip.top-right .tooltip-arrow { - bottom: 0; - left: 5px; - margin-bottom: -5px; - border-width: 5px 5px 0; - border-top-color: #51504f; -} -.tooltip.right .tooltip-arrow { - top: 50%; - left: 0; - margin-top: -5px; - border-width: 5px 5px 5px 0; - border-right-color: #51504f; -} -.tooltip.left .tooltip-arrow { - top: 50%; - right: 0; - margin-top: -5px; - border-width: 5px 0 5px 5px; - border-left-color: #51504f; -} -.tooltip.bottom .tooltip-arrow { - top: 0; - left: 50%; - margin-left: -5px; - border-width: 0 5px 5px; - border-bottom-color: #51504f; -} -.tooltip.bottom-left .tooltip-arrow { - top: 0; - right: 5px; - margin-top: -5px; - border-width: 0 5px 5px; - border-bottom-color: #51504f; -} -.tooltip.bottom-right .tooltip-arrow { - top: 0; - left: 5px; - margin-top: -5px; - border-width: 0 5px 5px; - border-bottom-color: #51504f; -} -.popover { - position: absolute; - top: 0; - left: 0; - z-index: 1060; - display: none; - max-width: 276px; - padding: 1px; - font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; - font-size: 14px; - font-weight: normal; - line-height: 1.428571429; - text-align: left; - background-color: #fff; - background-clip: padding-box; - border: 1px solid #ccc; - border: 1px solid rgba(0, 0, 0, 0.2); - border-radius: 6px; - -webkit-box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2); - box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2); - white-space: normal; -} -.popover.top { - margin-top: -10px; -} -.popover.right { - margin-left: 10px; -} -.popover.bottom { - margin-top: 10px; -} -.popover.left { - margin-left: -10px; -} -.popover-title { - margin: 0; - padding: 8px 14px; - font-size: 14px; - background-color: #f7f7f7; - border-bottom: 1px solid #ebebeb; - border-radius: 5px 5px 0 0; -} -.popover-content { - padding: 9px 14px; -} -.popover > .arrow, -.popover > .arrow::after { - position: absolute; - display: block; - width: 0; - height: 0; - border-color: transparent; - border-style: solid; -} -.popover > .arrow { - border-width: 11px; -} -.popover > .arrow::after { - border-width: 10px; - content: ""; -} -.popover.top > .arrow { - left: 50%; - margin-left: -11px; - border-bottom-width: 0; - border-top-color: #999; - border-top-color: rgba(0, 0, 0, 0.25); - bottom: -11px; -} -.popover.top > .arrow::after { - content: " "; - bottom: 1px; - margin-left: -10px; - border-bottom-width: 0; - border-top-color: #fff; -} -.popover.right > .arrow { - top: 50%; - left: -11px; - margin-top: -11px; - border-left-width: 0; - border-right-color: #999; - border-right-color: rgba(0, 0, 0, 0.25); -} -.popover.right > .arrow::after { - content: " "; - left: 1px; - bottom: -10px; - border-left-width: 0; - border-right-color: #fff; -} -.popover.bottom > .arrow { - left: 50%; - margin-left: -11px; - border-top-width: 0; - border-bottom-color: #999; - border-bottom-color: rgba(0, 0, 0, 0.25); - top: -11px; -} -.popover.bottom > .arrow::after { - content: " "; - top: 1px; - margin-left: -10px; - border-top-width: 0; - border-bottom-color: #fff; -} -.popover.left > .arrow { - top: 50%; - right: -11px; - margin-top: -11px; - border-right-width: 0; - border-left-color: #999; - border-left-color: rgba(0, 0, 0, 0.25); -} -.popover.left > .arrow::after { - content: " "; - right: 1px; - border-right-width: 0; - border-left-color: #fff; - bottom: -10px; -} -.carousel { - position: relative; -} -.carousel-inner { - position: relative; - overflow: hidden; - width: 100%; -} -.carousel-inner > .item { - display: none; - position: relative; - -webkit-transition: 0.6s ease-in-out left; - -o-transition: 0.6s ease-in-out left; - transition: 0.6s ease-in-out left; -} -.carousel-inner > .item > img, -.carousel-inner > .item > a > img { - line-height: 1; -} - -@supports (transform: translate3d) { - .carousel-inner > .item { - -webkit-transition: -webkit-transform 0.6s ease-in-out; - -moz-transition: -moz-transform 0.6s ease-in-out; - -o-transition: -o-transform 0.6s ease-in-out; - transition: transform 0.6s ease-in-out; - -webkit-backface-visibility: hidden; - -moz-backface-visibility: hidden; - backface-visibility: hidden; - -webkit-perspective: 1000; - -moz-perspective: 1000; - perspective: 1000; - } - .carousel-inner > .item.next, - .carousel-inner > .item.active.right { - -webkit-transform: translate3d(100%, 0, 0); - transform: translate3d(100%, 0, 0); - left: 0; - } - .carousel-inner > .item.prev, - .carousel-inner > .item.active.left { - -webkit-transform: translate3d(-100%, 0, 0); - transform: translate3d(-100%, 0, 0); - left: 0; - } - .carousel-inner > .item.next.left, - .carousel-inner > .item.prev.right, - .carousel-inner > .item.active { - -webkit-transform: translate3d(0, 0, 0); - transform: translate3d(0, 0, 0); - left: 0; - } -} -.carousel-inner > .active, -.carousel-inner > .next, -.carousel-inner > .prev { - display: block; -} -.carousel-inner > .active { - left: 0; -} -.carousel-inner > .next, -.carousel-inner > .prev { - position: absolute; - top: 0; - width: 100%; -} -.carousel-inner > .next { - left: 100%; -} -.carousel-inner > .prev { - left: -100%; -} -.carousel-inner > .next.left, -.carousel-inner > .prev.right { - left: 0; -} -.carousel-inner > .active.left { - left: -100%; -} -.carousel-inner > .active.right { - left: 100%; -} -.carousel-control { - position: absolute; - top: 0; - left: 0; - bottom: 0; - width: 15%; - opacity: 0.5; - font-size: 20px; - color: #fff; - text-align: center; - text-shadow: 0 1px 2px rgba(0, 0, 0, 0.6); -} -.carousel-control.left { - background-image: -webkit-linear-gradient( - left, - rgba(0, 0, 0, 0.5) 0%, - rgba(0, 0, 0, 0.0001) 100% - ); - background-image: -o-linear-gradient( - left, - rgba(0, 0, 0, 0.5) 0%, - rgba(0, 0, 0, 0.0001) 100% - ); - background-image: linear-gradient( - to right, - rgba(0, 0, 0, 0.5) 0%, - rgba(0, 0, 0, 0.0001) 100% - ); - background-repeat: repeat-x; -} -.carousel-control.right { - left: auto; - right: 0; - background-image: -webkit-linear-gradient( - left, - rgba(0, 0, 0, 0.0001) 0%, - rgba(0, 0, 0, 0.5) 100% - ); - background-image: -o-linear-gradient( - left, - rgba(0, 0, 0, 0.0001) 0%, - rgba(0, 0, 0, 0.5) 100% - ); - background-image: linear-gradient( - to right, - rgba(0, 0, 0, 0.0001) 0%, - rgba(0, 0, 0, 0.5) 100% - ); - background-repeat: repeat-x; -} -.carousel-control:hover, -.carousel-control:focus { - outline: 0; - color: #fff; - text-decoration: none; - opacity: 0.9; -} -.carousel-control .icon-prev, -.carousel-control .icon-next, -.carousel-control .glyphicon-chevron-left, -.carousel-control .glyphicon-chevron-right { - position: absolute; - top: 50%; - z-index: 5; - display: inline-block; -} -.carousel-control .icon-prev, -.carousel-control .glyphicon-chevron-left { - left: 50%; - margin-left: -10px; -} -.carousel-control .icon-next, -.carousel-control .glyphicon-chevron-right { - right: 50%; - margin-right: -10px; -} -.carousel-control .icon-prev, -.carousel-control .icon-next { - width: 20px; - height: 20px; - margin-top: -10px; - line-height: 1; - font-family: serif; -} -.carousel-control .icon-prev::before { - content: "\2039"; -} -.carousel-control .icon-next::before { - content: "\203a"; -} -.carousel-indicators { - position: absolute; - bottom: 10px; - left: 50%; - z-index: 15; - width: 60%; - margin-left: -30%; - padding-left: 0; - list-style: none; - text-align: center; -} -.carousel-indicators li { - display: inline-block; - width: 10px; - height: 10px; - margin: 1px; - text-indent: -999px; - border: 1px solid #fff; - border-radius: 10px; - cursor: pointer; - background-color: #000 \9; - background-color: rgba(0, 0, 0, 0); -} -.carousel-indicators .active { - margin: 0; - width: 12px; - height: 12px; - background-color: #fff; -} -.carousel-caption { - position: absolute; - left: 15%; - right: 15%; - bottom: 20px; - z-index: 10; - padding-top: 20px; - padding-bottom: 20px; - color: #fff; - text-align: center; - text-shadow: 0 1px 2px rgba(0, 0, 0, 0.6); -} -.carousel-caption .btn { - text-shadow: none; -} - -@media screen and (min-width: 768px) { - .carousel-control .glyphicon-chevron-left, - .carousel-control .glyphicon-chevron-right, - .carousel-control .icon-prev, - .carousel-control .icon-next { - width: 30px; - height: 30px; - margin-top: -15px; - font-size: 30px; - } - .carousel-control .glyphicon-chevron-left, - .carousel-control .icon-prev { - margin-left: -15px; - } - .carousel-control .glyphicon-chevron-right, - .carousel-control .icon-next { - margin-right: -15px; - } - .carousel-caption { - left: 20%; - right: 20%; - padding-bottom: 30px; - } - .carousel-indicators { - bottom: 20px; - } -} -.clearfix::before, -.clearfix::after, -.dl-horizontal dd::before, -.dl-horizontal dd::after, -.container::before, -.container::after, -.container-fluid::before, -.container-fluid::after, -.row::before, -.row::after, -.form-horizontal .form-group::before, -.form-horizontal .form-group::after, -.btn-toolbar::before, -.btn-toolbar::after, -.btn-group-vertical > .btn-group::before, -.btn-group-vertical > .btn-group::after, -.nav::before, -.nav::after, -.navbar::before, -.navbar::after, -.navbar-header::before, -.navbar-header::after, -.navbar-collapse::before, -.navbar-collapse::after, -.pager::before, -.pager::after, -.panel-body::before, -.panel-body::after, -.modal-footer::before, -.modal-footer::after { - content: " "; - display: table; -} -.clearfix::after, -.dl-horizontal dd::after, -.container::after, -.container-fluid::after, -.row::after, -.form-horizontal .form-group::after, -.btn-toolbar::after, -.btn-group-vertical > .btn-group::after, -.nav::after, -.navbar::after, -.navbar-header::after, -.navbar-collapse::after, -.pager::after, -.panel-body::after, -.modal-footer::after { - clear: both; -} -.center-block { - display: block; - margin-left: auto; - margin-right: auto; -} -.pull-right { - float: right !important; -} -.pull-left { - float: left !important; -} -.hide { - display: none !important; -} -.show { - display: block !important; -} -.invisible { - visibility: hidden; -} -.hidden { - display: none !important; - visibility: hidden !important; -} -.affix { - position: fixed; -} - -@-ms-viewport { - width: device-width; -} -.visible-xs, -.visible-sm, -.visible-md, -.visible-lg { - display: none !important; -} -.visible-xs-block, -.visible-xs-inline, -.visible-xs-inline-block, -.visible-sm-block, -.visible-sm-inline, -.visible-sm-inline-block, -.visible-md-block, -.visible-md-inline, -.visible-md-inline-block, -.visible-lg-block, -.visible-lg-inline, -.visible-lg-inline-block { - display: none !important; -} - -@media (max-width: 767px) { - .visible-xs { - display: block !important; - } - table.visible-xs { - display: table; - } - tr.visible-xs { - display: table-row !important; - } - th.visible-xs, - td.visible-xs { - display: table-cell !important; - } -} - -@media (max-width: 767px) { - .visible-xs-block { - display: block !important; - } -} - -@media (max-width: 767px) { - .visible-xs-inline { - display: inline !important; - } -} - -@media (max-width: 767px) { - .visible-xs-inline-block { - display: inline-block !important; - } -} - -@media (min-width: 768px) and (max-width: 991px) { - .visible-sm { - display: block !important; - } - table.visible-sm { - display: table; - } - tr.visible-sm { - display: table-row !important; - } - th.visible-sm, - td.visible-sm { - display: table-cell !important; - } -} - -@media (min-width: 768px) and (max-width: 991px) { - .visible-sm-block { - display: block !important; - } -} - -@media (min-width: 768px) and (max-width: 991px) { - .visible-sm-inline { - display: inline !important; - } -} - -@media (min-width: 768px) and (max-width: 991px) { - .visible-sm-inline-block { - display: inline-block !important; - } -} - -@media (min-width: 992px) and (max-width: 1199px) { - .visible-md { - display: block !important; - } - table.visible-md { - display: table; - } - tr.visible-md { - display: table-row !important; - } - th.visible-md, - td.visible-md { - display: table-cell !important; - } -} - -@media (min-width: 992px) and (max-width: 1199px) { - .visible-md-block { - display: block !important; - } -} - -@media (min-width: 992px) and (max-width: 1199px) { - .visible-md-inline { - display: inline !important; - } -} - -@media (min-width: 992px) and (max-width: 1199px) { - .visible-md-inline-block { - display: inline-block !important; - } -} - -@media (min-width: 1200px) { - .visible-lg { - display: block !important; - } - table.visible-lg { - display: table; - } - tr.visible-lg { - display: table-row !important; - } - th.visible-lg, - td.visible-lg { - display: table-cell !important; - } -} - -@media (min-width: 1200px) { - .visible-lg-block { - display: block !important; - } -} - -@media (min-width: 1200px) { - .visible-lg-inline { - display: inline !important; - } -} - -@media (min-width: 1200px) { - .visible-lg-inline-block { - display: inline-block !important; - } -} - -@media (max-width: 767px) { - .hidden-xs { - display: none !important; - } -} - -@media (min-width: 768px) and (max-width: 991px) { - .hidden-sm { - display: none !important; - } -} - -@media (min-width: 992px) and (max-width: 1199px) { - .hidden-md { - display: none !important; - } -} - -@media (min-width: 1200px) { - .hidden-lg { - display: none !important; - } -} -.visible-print { - display: none !important; -} - -@media print { - .visible-print { - display: block !important; - } - table.visible-print { - display: table; - } - tr.visible-print { - display: table-row !important; - } - th.visible-print, - td.visible-print { - display: table-cell !important; - } -} -.visible-print-block { - display: none !important; -} - -@media print { - .visible-print-block { - display: block !important; - } -} -.visible-print-inline { - display: none !important; -} - -@media print { - .visible-print-inline { - display: inline !important; - } -} -.visible-print-inline-block { - display: none !important; -} - -@media print { - .visible-print-inline-block { - display: inline-block !important; - } -} - -@media print { - .hidden-print { - display: none !important; - } -} diff --git a/airflow/www/static/css/chart.css b/airflow/www/static/css/chart.css deleted file mode 100644 index b7968d55ca3bc..0000000000000 --- a/airflow/www/static/css/chart.css +++ /dev/null @@ -1,79 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -circle { - stroke: #017cee; - fill: #fff; - stroke-width: 1.5px; -} - -.node text { - font: 11px sans-serif; - pointer-events: none; -} - -path.link { - fill: none; - stroke: #3182bd; - stroke-width: 1.5px; - opacity: 0.5; -} - -rect.state { - stroke: #51504f; - shape-rendering: crispEdges; - cursor: pointer; -} - -.tooltip.in { - opacity: 1; - filter: alpha(opacity=100); -} - -.axis path, -.axis line { - fill: none; - stroke: #51504f; -} - -rect.null, -rect.undefined { - fill: #fff; -} - -.axis text { - font: 11px sans-serif; -} - -g.parent circle { - cursor: pointer; -} - -g.node.collapsed circle { - stroke: #444; - stroke-width: 4px; -} - -g.axis path { - shape-rendering: crispEdges; -} - -g.tick line { - shape-rendering: crispEdges; -} diff --git a/airflow/www/static/css/dags.css b/airflow/www/static/css/dags.css deleted file mode 100644 index a8b804cf49fbb..0000000000000 --- a/airflow/www/static/css/dags.css +++ /dev/null @@ -1,145 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -.dags-table-wrap { - margin-bottom: 16px; - border-radius: 4px 4px 0 0; - background-color: #f0f0f0; -} - -.dags-table-header { - display: flex; - flex-wrap: wrap; - justify-content: space-between; - align-items: center; - margin: 0; - padding-top: 16px; -} - -.dags-table-body { - margin: 0 1px; - overflow-x: auto; - padding-bottom: 34px; -} - -.no-x-padding { - padding-left: 0; - padding-right: 0; -} - -.dags-table-more { - position: relative; -} - -.dags-table-more__toggle, -.dags-table-more__toggle:hover, -.dags-table-more__toggle:focus { - display: block; - padding: 8px 16px; - cursor: default; - border-color: transparent; - background: transparent; -} - -.dags-table-more__menu { - position: absolute; - top: 0; - right: 0; - bottom: 0; - padding-right: 4px; - white-space: nowrap; - display: flex; - align-items: center; - overflow: hidden; -} - -.dags-table-more__menu:hover { - overflow: visible; -} - -.dags-table-more__menu::before { - width: 0; - height: 100%; - content: ""; - pointer-events: none; - transition: width 0.3s ease-in; -} - -.dags-table-more__menu:hover::before { - margin-left: -200px; - width: 200px; - background: linear-gradient(to right, rgba(255, 255, 255, 0) 0%, #fff 100%); -} - -tr:nth-child(odd) .dags-table-more__menu:hover { - background-color: #fff; -} - -tr:nth-child(2n) .dags-table-more__menu:hover { - background-color: #f9f9f9; -} - -tr:nth-child(2n) .dags-table-more__menu:hover::before { - background: linear-gradient( - to right, - rgba(255, 255, 255, 0) 0%, - #f9f9f9 100% - ); -} - -tr:nth-child(odd):hover .dags-table-more__menu:hover, -tr:nth-child(2n):hover .dags-table-more__menu:hover { - background-color: #f5f5f5; -} - -tr:nth-child(odd):hover .dags-table-more__menu:hover::before, -tr:nth-child(2n):hover .dags-table-more__menu:hover::before { - background: linear-gradient( - to right, - rgba(255, 255, 255, 0) 0%, - #f5f5f5 100% - ); -} - -.dags-table-more__links { - margin-right: 16px; - width: 0; - overflow: hidden; -} - -.dags-table-more__menu:hover .dags-table-more__links { - width: auto; - overflow: visible; -} - -.dags-table-more__link { - display: inline-block; - max-width: 0; - overflow: hidden; - margin-right: 16px; - transition: max-width 0.3s ease-in; -} - -.dags-table-more__link:last-child { - margin-right: 0; -} - -.dags-table-more__menu:hover .dags-table-more__link { - max-width: 200px; -} diff --git a/airflow/www/static/css/flash.css b/airflow/www/static/css/flash.css deleted file mode 100644 index 9a2c21654d551..0000000000000 --- a/airflow/www/static/css/flash.css +++ /dev/null @@ -1,57 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -.panel-heading #alerts-accordion-toggle::after { - float: right; - color: grey; -} - -.panel-heading #alerts-accordion-toggle.collapsed::after { - float: right; - color: grey; -} - -#errorHeading { - background-color: #d6d8d9; /* same color as Bootstrap's Dark Alert */ -} - -.dag-import-error { - position: relative; - white-space: pre; - height: 14px; - line-height: 14px; /* show only one line of 14px text */ - overflow: hidden; - cursor: pointer; -} - -.expanded-error { - height: 100%; -} - -.dag-import-error::after { - float: right; - color: #e43921; - position: absolute; - top: 0; - right: 0; -} - -.expanded-error .toggle-direction { - transform: rotate(180deg); -} diff --git a/airflow/www/static/css/graph.css b/airflow/www/static/css/graph.css deleted file mode 100644 index 16dc5186af14f..0000000000000 --- a/airflow/www/static/css/graph.css +++ /dev/null @@ -1,192 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -svg { - overflow: hidden; -} - -.node rect { - stroke: #51504f; - stroke-width: 1px; - fill: #fff; - transition: stroke 0.2s ease-in-out, opacity 0.2s ease-in-out; -} - -.node rect[data-highlight="highlight"] { - stroke: #017cee !important; -} - -.edgeLabel { - transition: opacity 0.2s ease-in-out; -} - -.edgeLabel > .label { - font-weight: normal; - font-size: 10px; -} - -.edgeLabel text { - transform: translate(16px, 0); -} - -.edgeLabel rect { - fill: #fff; -} - -.edgePath { - stroke: #51504f; - stroke-width: 1px; - fill: none; - transition: stroke 0.2s ease-in-out, opacity 0.2s ease-in-out; -} - -.edgePath[data-highlight="fade"], -.edgePaths[data-highlight="fade"] > .edgePath, -.edgeLabel[data-highlight="fade"] { - opacity: 0.2 !important; -} - -.edgePath[data-highlight="highlight"] { - stroke: #017cee; -} - -.edgePath .arrowhead { - stroke: none !important; - fill: #51504f; - stroke-width: 0 !important; - transition: fill 0.2s ease-in-out, opacity 0.2s ease-in-out; -} - -.edgePath[data-highlight="highlight"] .arrowhead { - stroke: #017cee; - fill: #017cee; -} - -.not-allowed rect, -.not-allowed text { - cursor: not-allowed !important; -} - -g.cluster rect { - stroke: #fff; - stroke-dasharray: 5; - rx: 5; - ry: 5; - opacity: 0.5; -} - -g.node { - transition: opacity 0.2s ease-in-out; -} - -g.node[data-highlight="fade"] { - opacity: 0.2 !important; -} - -g.node rect { - stroke: #fff; - stroke-width: 1.5px; - cursor: pointer; -} - -g.node circle { - stroke: #51504f; - stroke-width: 1.5px; - cursor: pointer; -} - -g.node .label { - font-size: 0.7em; - font-weight: normal; - pointer-events: none; -} - -g.node text { - cursor: pointer; -} - -.svg-wrapper { - border-radius: 4px; - background-color: #f0f0f0; - cursor: move; - overflow: hidden; - height: 100%; - width: 100%; -} - -#graph-svg { - overflow: visible; -} - -.refresh-actions { - justify-content: flex-end; - min-width: 225px; - display: inline-flex; - align-items: center; - right: 20px; - margin-top: 10px; - margin-bottom: 15px; - position: absolute; - background-color: #f0f0f0ee; /* the last two chars apply an opacity to the background color */ -} - -.legend-item.dag { - float: left; - background-color: #e8f7e4; -} - -.legend-item.trigger { - float: left; - background-color: #ffefeb; -} - -.legend-item.sensor { - float: left; - background-color: #e6f1f2; -} - -.legend-item.asset { - float: left; - background-color: #fcecd4; -} - -.legend-item.asset-alias { - float: left; - background-color: #e8cfe4; -} - -g.node.dag rect { - fill: #e8f7e4; -} - -g.node.trigger rect { - fill: #ffefeb; -} - -g.node.sensor rect { - fill: #e6f1f2; -} - -g.node.asset rect { - fill: #fcecd4; -} - -g.node.asset-alias rect { - fill: #e8cfe4; -} diff --git a/airflow/www/static/css/loading-dots.css b/airflow/www/static/css/loading-dots.css deleted file mode 100644 index b726eeff63a7c..0000000000000 --- a/airflow/www/static/css/loading-dots.css +++ /dev/null @@ -1,60 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -@keyframes loadingDotBlink { - 0% { - opacity: 0.2; - } - - 40% { - opacity: 1; - } - - 100% { - opacity: 0.2; - } -} - -.loading-dots { - display: inline-block; -} - -.loading-dot { - display: inline-block; - margin-left: 0.5rem; - border-radius: 50%; - width: 10px; - height: 10px; - background-color: currentColor; - animation: loadingDotBlink 1.2s infinite; - animation-fill-mode: both; - will-change: opacity; -} - -.loading-dot:first-child { - margin-left: 0; -} - -.loading-dot:nth-child(2) { - animation-delay: 0.2s; -} - -.loading-dot:nth-child(3) { - animation-delay: 0.4s; -} diff --git a/airflow/www/static/css/main.css b/airflow/www/static/css/main.css deleted file mode 100644 index 9d9ed1e9fcf9e..0000000000000 --- a/airflow/www/static/css/main.css +++ /dev/null @@ -1,676 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -body { - position: relative; - padding: 70px 0; - min-height: 100vh; - overflow-y: scroll; - background-color: #fff; -} - -nav { - box-shadow: 0 3px 3px #aaa; - z-index: 999; -} - -pre { - overflow: auto; - word-wrap: normal; - white-space: pre; -} - -pre code { - overflow-wrap: normal; - white-space: pre; -} - -input, -select { - margin: 0; -} - -div.container { - width: 98%; - padding-left: 15px; - padding-right: 15px; -} - -.navbar a { - color: #51504f; - font-weight: bold; -} - -.navbar a .caret { - color: #e2d2e2; -} - -/* If a category has a lot of menu items, we cap it, and - add a scroll bar */ -.navbar li.dropdown .dropdown-menu { - max-height: 85vh; - overflow-y: auto; -} - -.navbar-nav li.dropdown:hover > .dropdown-menu, -.navbar-nav li.dropdown:focus-within > .dropdown-menu { - display: block; -} - -/* If a menu remains open due to focus, a menu - opened via hover will always be displayed on top */ -.navbar-nav li.dropdown:hover > .dropdown-menu { - z-index: 1001; -} - -.navbar-brand { - padding: 0 15px; - height: 60px; - display: inline-flex; - align-items: center; -} - -.brand-logo { - width: 104px; - height: 40px; - overflow: visible !important; /* Allow for animation */ -} - -@keyframes pinSpin { - from { - transform: rotate(0) translateX(0); - } - - to { - transform: rotate(360deg) translateX(0); - } -} - -@media (prefers-reduced-motion: no-preference) { - .navbar-brand:hover .brand-logo-pinwheel { - transform-origin: 17.66px 17.66px; - animation: pinSpin 1.5s linear; - } -} - -.navbar-user-icon { - display: inline-flex; - justify-content: center; - align-items: center; - vertical-align: middle; - margin: -5px 0; - width: 30px; - height: 30px; - border-radius: 50%; - color: #017cee; - background-color: #c0defb; -} - -span.status_square { - width: 10px; - height: 10px; - border: 1px solid grey; - display: inline-block; - padding-left: 0; - cursor: pointer; -} - -div.squares { - float: right; - font-size: 1; -} - -.d3-tip { - background: rgba(0, 0, 0, 0.85); - color: #fff; - border: 0; - border-radius: 5px; - padding: 10px; - margin-top: -4px; -} - -input#execution_date { - width: 220px; - margin-bottom: 0; -} - -table.highlighttable { - width: 100%; - table-layout: fixed; -} - -div.linenodiv { - padding-right: 1px !important; -} - -.linenos { - width: 50px; - border: 0; -} - -div.linenodiv pre { - padding-left: 0; - padding-right: 4px; - color: #8e8e8d; - background-color: transparent; - text-align: right; -} - -.wrap { - white-space: pre-wrap; -} - -.code { - font-family: monospace; -} - -#sql { - border: 1px solid #ccc; - border-radius: 5px; -} - -.ace_editor div { - font: inherit !important; -} - -#ace_container { - margin: 10px 0; -} - -#sql_ace { - visibility: hidden; -} - -table.dataframe { - font-size: 12px; -} - -table.dataframe tbody tr td { - padding: 2px; -} - -table.dataframe.dataTable thead > tr > th { - padding: 10px 20px 10px 10px; -} - -table.dataTable.dataframe thead .sorting { - background: url("./../sort_both.png") no-repeat center right; -} - -table.dataTable.dataframe thead .sorting_desc { - background: url("./../sort_desc.png") no-repeat center right; -} - -table.dataTable.dataframe thead .sorting_asc { - background: url("./../sort_asc.png") no-repeat center right; -} - -.no-wrap { - white-space: nowrap; -} - -div.form-inline { - margin-bottom: 5px; -} - -body div.panel { - padding: 0; -} - -.legend-row { - display: flex; - align-items: center; - justify-content: space-between; -} - -.legend-item { - display: inline-flex; - align-items: center; - margin-right: 4px; - border-radius: 4px; - border: solid 2px #cbcbcb; - padding: 0 3px; - font-size: 11px; -} - -.legend-item--interactive { - cursor: pointer; -} - -.legend-item--no-border { - border-color: transparent; - padding: 0; -} - -.legend-item:last-child { - margin-right: 0; -} - -.legend-item__swatch { - display: inline-block; - margin-right: 3px; - width: 12px; - height: 12px; - border-radius: 2px; - border: solid 1px #8e8e8d; - background: #fff; -} - -.legend-item__swatch--circle { - border-radius: 50%; - width: 14px; - height: 14px; -} - -.legend-item__swatch--no-border { - border-color: transparent; -} - -label[for="timezone-other"], -#timezone-other { - margin: 3px 20px; - font-weight: normal; -} - -#timezone-menu { - overflow-x: hidden; -} - -#timezone-menu ul.typeahead.dropdown-menu { - max-height: 200px; - overflow-y: auto; - overflow-x: hidden; - border: none; - left: 0 !important; -} - -#timezone-menu .dropdown-item { - padding: 3px 20px; -} - -.ui-menu .ui-menu-item a.ui-state-focus, -.ui-menu .ui-menu-item a.ui-state-active { - border: 1px solid white; - border-left: 1px solid #ccc; - background: #f6f6f6; - color: #454545; -} - -/* depending on the version of FAB in use, we may have a style conflict */ -.bootstrap-datetimepicker-widget .datepicker > div { - display: block; -} - -.accordion-toggle { - display: block; -} - -.accordion-toggle.collapsed > .toggle-direction { - transform: rotate(180deg); -} - -.hll { - background-color: #ffc; -} - -.c { - color: #408080; - font-style: italic; -} /* Comment */ - -.err { - border: 1px solid #f00; -} /* Error */ - -.k { - color: #008000; - font-weight: bold; -} /* Keyword */ - -.o { - color: #666; -} /* Operator */ - -.cm { - color: #408080; - font-style: italic; -} /* Comment.Multiline */ - -.cp { - color: #bc7a00; -} /* Comment.Preproc */ - -.c1 { - color: #408080; - font-style: italic; -} /* Comment.Single */ - -.cs { - color: #408080; - font-style: italic; -} /* Comment.Special */ - -.gd { - color: #a00000; -} /* Generic.Deleted */ -.ge { - font-style: italic; -} /* Generic.Emph */ - -.gr { - color: #f00; -} /* Generic.Error */ - -.gh { - color: #000080; - font-weight: bold; -} /* Generic.Heading */ - -.gi { - color: #00a000; -} /* Generic.Inserted */ - -.go { - color: #888; -} /* Generic.Output */ - -.gp { - color: #000080; - font-weight: bold; -} /* Generic.Prompt */ - -.gs { - font-weight: bold; -} /* Generic.Strong */ - -.gu { - color: #800080; - font-weight: bold; -} /* Generic.Subheading */ - -.gt { - color: #04d; -} /* Generic.Traceback */ - -.kc { - color: #008000; - font-weight: bold; -} /* Keyword.Constant */ - -.kd { - color: #008000; - font-weight: bold; -} /* Keyword.Declaration */ - -.kn { - color: #008000; - font-weight: bold; -} /* Keyword.Namespace */ - -.kp { - color: #008000; -} /* Keyword.Pseudo */ - -.kr { - color: #008000; - font-weight: bold; -} /* Keyword.Reserved */ - -.kt { - color: #b00040; -} /* Keyword.Type */ - -.m { - color: #666; -} /* Literal.Number */ - -.s { - color: #ba2121; -} /* Literal.String */ - -.na { - color: #7d9029; -} /* Name.Attribute */ - -.nb { - color: #008000; -} /* Name.Builtin */ - -.nc { - color: #00f; - font-weight: bold; -} /* Name.Class */ - -.no { - color: #800; -} /* Name.Constant */ - -.nd { - color: #a2f; -} /* Name.Decorator */ - -.ni { - color: #999; - font-weight: bold; -} /* Name.Entity */ - -.ne { - color: #d2413a; - font-weight: bold; -} /* Name.Exception */ - -.nf { - color: #00f; -} /* Name.Function */ - -.nl { - color: #a0a000; -} /* Name.Label */ - -.nn { - color: #00f; - font-weight: bold; -} /* Name.Namespace */ - -.nt { - color: #008000; - font-weight: bold; -} /* Name.Tag */ - -.nv { - color: #19177c; -} /* Name.Variable */ - -.ow { - color: #a2f; - font-weight: bold; -} /* Operator.Word */ - -.w { - color: #bbb; -} /* Text.Whitespace */ - -.mb { - color: #666; -} /* Literal.Number.Bin */ - -.mf { - color: #666; -} /* Literal.Number.Float */ - -.mh { - color: #666; -} /* Literal.Number.Hex */ - -.mi { - color: #666; -} /* Literal.Number.Integer */ - -.mo { - color: #666; -} /* Literal.Number.Oct */ - -.sb { - color: #ba2121; -} /* Literal.String.Backtick */ - -.sc { - color: #ba2121; -} /* Literal.String.Char */ - -.sd { - color: #ba2121; - font-style: italic; -} /* Literal.String.Doc */ - -.s2 { - color: #ba2121; -} /* Literal.String.Double */ - -.s1 { - color: #ba2121; -} /* Literal.String.Single */ - -.footer { - display: flex; - align-items: center; - justify-content: space-between; - position: absolute; - bottom: 0; - width: 100%; - height: 60px; - color: #8e8e8d; - background-color: #f5f5f5; -} - -.dag-view-tools { - margin: 16px 0; - padding: 16px 0; - background-color: #f0f0f0; -} - -.code-wrap { - position: relative; - margin-top: 30px; -} - -.code-wrap-toggle { - position: absolute; - top: 15px; - right: 15px; -} - -.search-input { - position: relative; -} - -.search-input__input { - padding-right: 40px; -} - -.search-input__clear-btn { - position: absolute; - right: 2px; - top: 2px; - border: 0; -} - -/* Override default Bootstrap behavior to prevent wrapping */ -.btn-group { - display: inline-flex; -} - -/* Override FAB table views where table width extends beyond parent containers */ -.panel-body .panel-group + div { - overflow-x: auto; -} - -.task-instance-modal-column { - margin-top: 8px; - overflow-x: auto; -} - -.refresh-actions { - justify-content: flex-end; - min-width: 225px; - float: right; - display: inline-flex; - align-items: center; - right: 10px; - margin-bottom: 15px; - position: relative; -} - -.refresh-actions > .switch-label { - margin: 0 10px; -} - -.loading-dots.refresh-loading { - display: none; -} - -.trigger-dropdown-btn { - border-bottom-right-radius: 0; - border-top-right-radius: 0; - border-right-width: 0; -} - -.trigger-dropdown-menu { - left: -112px; -} - -.dropdown-form-btn { - padding: 3px 20px; - line-height: 1.428571429; - color: #51504f; - background-color: white; - border: none; - width: 100%; - text-align: left; -} - -.dropdown-form-btn:hover, -.dropdown-form-btn:focus { - color: #262626; - background-color: #f5f5f5; -} - -.tooltip { - z-index: 0; - max-width: 300px; -} - -.tooltip.in, -.tooltip.d3-tip { - z-index: 1070; -} - -details summary { - display: list-item; -} - -.menu-scroll { - max-height: 300px; - overflow-y: auto; -} - -.next-asset-triggered:hover { - cursor: pointer; - background-color: #cbcbcb; -} diff --git a/airflow/www/static/css/material-icons.css b/airflow/www/static/css/material-icons.css deleted file mode 100644 index 691f30fd5538a..0000000000000 --- a/airflow/www/static/css/material-icons.css +++ /dev/null @@ -1,84 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -@font-face { - font-family: "Material Icons"; - font-style: normal; - font-weight: 400; - src: url(data:font/woff2;base64,d09GMgABAAAAAUI8AA4AAAADjOAAAUHkAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAGhwbEByCzyoGYACwGhEICorJeIiAFAuwWAABNgIkA5gwBCAFgnoHIFuvznIDpyBryeWaoZ2jbV/h8F/QPjQVyrZdLHXexkto0GNMDdumQfSgOyhSwX2lL/v//89PKjJm20uyrrBtgAcBf70XMBlJPMc4ThIlIFVaT46i3HvL0aKMLvNMab5I12Ve1V2SfJsb7np3ScN7mPsRax9dxfBqO6Fz6tPkk0QcgdrGiFygr7T3/dJeUfRMt8IuknUFvw4YHYSVSIvQCSiexI3nU/9ARmj+Cg1SQI1DUV+UvEIe7pLu8yPlrKtRXgomrPXYSnkbMbh+g8Ucb37qPw1546MhtGBP9j9wq+diYLqeCtIikuEgaZG0SCpIJCZokFQUQCVIyLm75TABEy9afFFXgRxnk0W2HmoTLIBa1ceuw1sH2K4DQqQQe9iYKA55REI8/V/2ddA9TuaOI+4+il+QlYX8s+n/G+Ac1uOykvAChBtykz+WsN9+N9q6tYTewlf4ClljtpCtboVtj+GytroWbAI0RI6BbjkVexOVQ7v///yzk2SQCXfvngeWHypaqIbEiuAAbNvthBMhboioqIg3Vzn3292kmtP3PoXWy39PuvEZ4W8+nxFKM49OGMXnNKfWKYwwMraysi38zvd/pZRl+rbJ1WUyTm4um3Fu7VfEG14jL+RKCK+RlyMSHhJC+KVde5393d4k3JxCXiqbCllZV1cpWreUrPFrc4u/+Itwiz+OEPIC/H8c8Tt3e20UeGAJlEWcYFSCgf/xqH18I/nDk27+CyQhm9wldxkkm5U7QNZQckCYgZ071g6SOBBRnE0cFdQOB6DQqa1awu8UJ1hw/irRLuBba9B2a0vstuubhbltNzXEc4SQGgS5u1/PrQjDjH6BNA8Q46kaPu11z+z+i+QsXFISHx0OizyhQCgk3lL8mNN6eeYm7d4CbRcoYMuBIbCd/TywEJCefn8+tZluLtfk2l5lyYkLCwkXFgu4DIWPAAWtaRgsOhPjIpwFgMLtPU6JVf8PlrNJ6wtFJ4ilorOth4pNaH2BdxeYzb0E0/wUD6J/D3wR+4imgn6ob0WnolvwIa2HE2xKwwml9Q8fcU/jf+/t7k1wE3w4DoLyOsCklJNU4h/yJv9tg0GYdhcGIIll0s/zudkfVHZZEvIChCWPsEjuDYtCUAHJA0KABAX03cfiArJpXlyrLK7vfWs12E5rOwkjtBW17bS8tLWLth1pG7S1M1U62g5JZ2m03Qf8xFm6jJ2pDwS3MQKhaYiG0A01+YOgCEO4AXsov9Syt37hofuh0QSbJMjhnBpJu5J2/3Eczu0gdBo5yFJf5fy3DoIdjxL2T1jdQdaE8P1KdU/CCu6uRyG6zSz7HVMlvYSdU6o0WmTrusAaLbK9yPrhner/OfLDc6SHhQFdlqWWN47twoXSNsc+kFg/dbRWP7/yr/xLadN1FGg7oNL4UneJkvgC9wFEsS3kltbcGqWSIiExxEIL/ibpH3Q4UUrmY/BSxFI+/p+xwTybzKGc2v/MO87844BsFT6YYqeLxaTtMgna6NJU7z1drtLVvgIVFpxYlpM4BIYkXQi1dZKR5gP3AzyvS30x/E5Yp/jCBawAVvKd07UJfQH8wlo4pngJn8uqabnzx2a00DnSjBKArpJurZLTeetKhLppN3XILAdE/jNyPlVf/7ge1xTXIokuTU9uXYYzGSBFB0pXEi0rufLgMZkyLKW137oo/TJ/5P/vVa22+KRpk45URbl7gtRRrurg6rC33TGuejt5m/+/P7/3H8L7+BDwQUggQNIIoowPkBIIijIRSIMQqw4IUW6acveR5K4ZW+5Aq0I8Z8AP0UWCkpukLCs4jCyVq0NIkqya4zTdlSaksOxlLfvMqlfrWWxTWM1y04vl/KTq68qTzW+nts0ZM2YYoynZMozeMyzCu/J0d3jAJ0BSX4A6f6ukpG933gFQIEiuXW6lT6ljqcve6pZxiP+9qfWmfMs1lB9+5+cbG8nGayPZ4U4k+SBTOLj93jtCP9xuEa0eigQa0BKma2eWy28NXjdGajRGKpI7qgKaGBXQxP4iuet/JIX6kYzxoepHE8pFsUqpT0LlgYI0EQ/6B6n/2Z0NtLisANJbBr+pdMADWWbEg/9qVkmd7mgfbC9ABhDmSWXVi1bGV+KxQtHYx0WNpnyWFqALEDNggMHa49qbThGkba36WXi7XGLfqgq5eRNqtsSHaNR5cRDOFHMylT92lwjqMsYIkVfHfu29sPg3Xbrv6PL6rlVVFRFjRIyIiBjzr8N3e3t67J3Z28+ZvUgQEREpoYiEcAhBOvMWv+ul59MZ7v1JPt/b3b2ilVIiRoyIiNZaK+UopY3FmMb/A4GNOZNjGRAVUVGIv7Q6rM3Mi3QZKgg4EFDSuPjWMJbamTbNtdtvW5Jm1qDi80VEFJRXdBtCN/8/ClKftUPUoIishKGMhKy75NbM6L8FwkCeQVVtxHhEE4/73QRWs7XVEq6PfaUPTcf8pNQAAgyQHd0n/f7i+awNWxQAVof8z8DjP4JFzPC5XMSMyo/OFF6pk2/VyMe7f3TEOWxk8j75tnvuh3+ptZR082RhFl59ter3/WszGX+pOXtW/o9G/SggrPRhGBOsaS2b2M4eDnKM05yXq5RZIDQMHiwfrRgKRXlp5W5leFo1w92qGK5WyXC2CoajlTPsrciwtQLD2srWsdBy8Uv5aWahQsRLm22jPC7HJdz8GsA1b64RukaQiL2xSPlHUxuGkyGMYUrbSZ0AYSPHQdnR5GQF8mnvqj3O8aSC+VpN4i1IlqASiHiVuM6+hGArup9Qj0pKqCSxITDF0x97+URsk7kj6RapU9wT5sKIg2BpUDacmDt6fxVkamUl3xGUyBNuL3dHCm2KrbD/Oz80pqRTREQlQrSU082XtrVfwapRWFLUZqHvpm4rG2QqMz5qXo1a470u6SrbuT07u3LDOnAPx02Dh2TN1r0KgrIjgiQIJExymgQYStEuj7J2qkglNn1Tnd9klhEUWrUfLq18MemZrLF8x+/aP+/7xwv9wwmt9DGOJuGi7ZbKKqiLdfrBOZMfS9pDg8JjsoqI7VzUZS1tQ4XS4saKDZcz64Spm10ztcOyVDgg7GhSABswVW44cixABtAYJdE9ROiwqvUb4yjUUXBXTI88Jh45bMG16k8+hZJHBlFn31miRkr9FQ8AxhzuzpEQQ6blZNfJJatkd+YxaIHrMmLKLELrI+AjwpRldeJlLtAN39tC8SSdJ+xXxTQCygNDsBpVG3AuJwEiyQpaPiiInzoIvUN/VzdCso06fWvCOEP31qk6JDqIBinAwT1ulcSHy4vUY7pe20PXsay4JTWNlAe05djZXaodUWYizrAgyXrlDso9tj8t5OkmsoCISdDeId6rvENQ7j5AGP6FDH14yFsoppM8jIgYta0XUcDAhuULIkSsIC5AajYyAgTbabkj863RtUYcMuIOgS2bbcPfeUEj5x4oSgYDw83pMgufbsa5n2A73Ych4NRx+/5BxY6xkiwpxLgcXbEV41d1OWzDKyzE3r3LzCw23HQmxqNCgXvsUol4wN0ihoCScOXhV7othYx7NDwChs2/cyB0JqACt1SCEfugjZYnzumdwHAxV3K2M38quBjQLo5gEfCANrYzw3Ic/M0SB1nnC5tFQxo6M2yUbSRDVLMx3DcszKJd2uTCRIPlYcJWx3WH0bBEhzJhW/NU+32BA+A3hoGCNMf2awKgTLVlVCOGDq8wyg/p0MgglYKz57C5k5M371NuEgjqGEuHw8mpwVBRmJhPjC/LOeBnZvi6rxd0NUmAGNHCgp1VRbBm6aTrOiMTW0MIGEBLSSDtkRdHzPwIvTYAXZcjogGndS3PnR4hiWYdbPcsJ0fFoN6bI2DeML738PC53woEf4O+tMFQPZLmDZXcg2Fqib3Rty1E9aZG8OEIvr6tJSAiQFun51KYooI8xRodZZhxAqMNeIzIOw/5fcW1UYGRJoTDvyS/QcZITKU/rIHMxlglO2b6HRrNorzRl+Zc0VUtg+jT8eFz0J5ANn3hPy01zUhXZJfT0eZuIhvp+ignbXBN4OHvL4rPzKmyJYrTG9/scnkjzKObEcoZLU367O0dj17Ox6Z10Yg5jM05Y/q6rSOF3BnKkJNJERNepetm/y+M6I8iwM3mMSZRP0NtHIGOrLjm+g+62ckwmYyNzYUbk5nuwhVP3xbVSkFMTC8Jigt40cKpfJeLo/gd1b9to4AdG+0/sNAE0kjCf2URJRqFXMhvErxMvxVkXKW/IxihWcIKCTZrr1alD0RBWCrvsv0tyzGC/MeMMUuMgPo/iJEG7iVVJoXsYpKNbSNHyPU6YdbiB1S0h7TOfPjimPsxoV84AmAl3yIGrFv9JZzdwOBsdovu4xdn2aJ3q8iSt9Ff74K+J7USsVTfuVf3CDz0vs58AdzMLcCf6cAqCfjuPT//Sqcshg8K2l0uhJc6osWWhgcNSYlS56p7ZVw9HTO0dLhhmZ1iElMM0GtYSRICnpALnFGSvwqh9O+CI0hL6ObHUOmJX0DoQXM58nO+dQOVYYafINAv2jMez/r/b851kYc8z8f8iVGYREwQaQ7KaiCfYntVadWrXKO0pquWk0zSjEYeY+iYWUy+yQEOX2H7dmTlOr7aeX8kBxYi4GwkardPOhAIAEIRjhoN7sOIMMA3oWZF6m7Z1LiSCyVs1seNZJJ+FxwAU7vVQJE1ooFVzKex2IUJ74yFSJxzzdz0I8uSQWTKWCqPUvXAvzefDQDma1NjYrbmDdWTAgmTrMwgZyAaMaMA2mCHUp/YaYZXNKHp72hUyDPJpZmA6ItNgDAIoDTIjyjU6Ev3pELHpt3c+GSh4RK1Ob4VoR41a/RP1ogQZtHyAXk5St6/HCnvXY6Qdy+HyzuXg/J2BP6dCl2nLMQKk+PuJmQ3DeBKsJQKOXIm5bORCCQzbBUjGXvn9I5aSmP1ooBEBDV2wxAdpbckOj9CYkEbgRfFSpR6AAofJZFsuoDJwiXCiFRMPkqAwqoX10RH6w150QDQS4QgFvIfIUEWlPY01BFmeQmbLwOMNDxOVLtK+KDMuoOSSFHZu3smkYRqVHmVJK1BU39cNzj1Wiv2DO9l6oFnBdyxuskhq8iriy5NCc8Wyoh+JcpQNsAFceIroKMcVgN23P+oBhNMLlOpOVbwXyXiTN1vjYvGSR6WXr714gr/BUptjQljOAFNo6WIWpKwKbozRsrl0to/rsZANPoJ5HFO79nR0ybX27nV1cSTq14fAYyOcl2laJ/QS4Ho2WWtgjX+gTR98j0H/YoSMgdk6m+QlJqqugBwEigz3sk5hj9hOCDinpx45G30ByjUxa33QbRRAB6x0AKBsn9upiWrfUDhRi0sNIu+wsHISh50oTovjWxh6nSPtZ4fK4HREDkwkacAZokckxK0k9FD0Ekg7VeOvYqaioouJqNCZR0P968jfUVyB7KPvzEKn8DCpZG10KjE0Mj836hAhba1PTVLWzQfbwIGnfaRhk193LijN7qAGy1dumrj5rAgAH+Q5lM55OTHnY/DLwKERAQYfnIcx6dY1ZBDQP2XWxG7qrCwzPiSbNnUpgvR+Ibjmci0Jm9yeocTgz9d9vdePH/8kYuns0vWPGGliAgB0eQQjrKLtCjJzJQ6P8wTyUp+0kn2iCtnuiUy1NnyFShrofQkaJcGqLqx/s6k0w5N71hzTQLc3hZfnJWI5sqlsxhrCyXcsXVlftUm1MNyUBu70Qqb/SUI+T2F/CT4JMqPdbcB1d9p5GdQzWPEOPvOgF0TClIoeWGNtQqQ2cj+xBO4dDwh6rgtdZPaXcMJQnfJFGrVRo6/ap/d9T28eAe3OjlEpdmgmtGyUdMolw37vnboaS5I3pjfk1gaKrdof5GmAi6Y4P1SBK+cG1acBzyG0MbrWeGjBEp1Hx0ND+nMnMUekYgN7pKrByoNH6fOpilYst0MmAVPoiVWHsk5OxxJFgjSd814QQurs0F/lIU/27lX40YgJTT1wP7Kg6HjK+0s/arR1HS7FWlfByNWMo8gBwrlsJVSwNJWGaKaZa/nQi249B3M8ZSGOctxLsUcKssSn/wMapFPhCU919KmvfEThSzYS1KvqaN4MtBCd75Bsn3QPI4ahJzzDvR7faVZC7QjIXB0ouZIVjcwYIW+Sv5U2/VBToR0UWL5EdJygFq+aAcWtX3EGOsEVCnyxolq1rJeTW0TtiZPHTrVVl+MAyUFUF0Ca9IZ+ejISIOoGftDc+PxwhOAAMvDil3LJqMjEQYDbvs8go1740H8ylY1vQhYOh3AifdC0i3CLPdh5D1mmvUJLT0/MI1Gj3+WvzuN46bd0kZQ1XBKL04KsBRkWOLN3s7+HHhcU5is79OhNzjBXvNMTZ8R0CnH1rJtulqgYU4646K6tziJ3xrnnz4e3vNaInJCVj/MdJDwrpIKHaqWJ680TFRzY1excXS6hoiUgOVPrnf8aSSneQSgwnHi1MPa4kBE4GKw4HpddCpoVnE+lKYy9zpqp5VdTjNamrtq0bZ3ByIuhmrn4HS6Q0IOSF7PmVB/a7ILs0ZUC/6XT5x5lMQKETOSjUlXwSBR7zWkKg/nel5N9kSzwOyInCmu08k8iX2xJnaPjtcyB6pVAH1wHOMVHWeWpsx0Wc9By26X8Q94PC4dpUNdIrQtjjukdxDZGB1WF5cjwX1apm7VsqA2qg64najkcupjY1pqiFxlzWuaIAZfwKPQaIpSJnR6UgjNEfvazDLWe2nCVNtFTeAEydeaU9YR4mtH1uDbW4832ofjBHeoajkOk0KsgQSIzn3BDXq6WmCUYT1ZLMHrmSWHkqzJQzBl4HUKOBjtSuLaB8Fd3k7zQLB3ik34ISmmXg5reKWo/rAc2Gl5fZkIx9pMG35a+zT/AQNXz3sGMSSDgOO6unu8yJt8yN9IZXkxyVB4rJ65LtduBSmyYvuqlXe3tMkg1jBJsxjWk3z2J3pCNRyycPXl6/O8dVqNCCQXiRqNO/YsAZGnthwSBZ06gQXaeDi1Guy0ykNljZf4EgvehrNwEvucJ+IhVDPFxad7f2nzSAxzbOLAxN1TY9/5Rt5VwUzn611kQFpIr9B+jbR1T5I5sMFjsYZm5QitlUt2kSzCJ9fAmG/KM4Pk3O/33sLmPY5Sbpzbmkv4JGVa/nl8adSXBUlAwale40kcZUaYb8wKXQhmg2SYG8xnj/K1dEAP55rirCMRFAW6UzSyrQpJWd5UAVF9YpwrpMfaPg0i2ndLOWkANef/uFRPkcAfMqIdwdd2eDHeKZoWoU6avp8q/FnZ0USuNWvMwizpVzWtIfCWPXpgdTfjmBg6toMOsRuj8u0lPgtixLGLyiNIIBDRnUOXAJpMXBUT4MfQnItuwtTYroI01rPBwwB7x8LqGRLrqymoIf1a0UtaT86gmzrZukrS9JGkRs/w0NHiO73YuAXrdaGq10z5tzsAbmqsTo9KWmgejg3WLtX4+ZTWSuiV5jLWkYK7DvgtWfOQbuYJ1sDhUFcc9aTVwcVjeZLJCr4XOAafZZ7ojUKNQgLwDyH+bCVeEDaC3lRzwTnR91wQNyKhK5B4abhcVD1Ou/PthhJrSmOVUxiBkxQhJYSTbo856JDdLV/BhOe+A0lwEiujDGGPp8JISunyWV99I2W7viliE+5NCweNUixUGPhmuJTBzNWdXSLNuX4sYV0l6HGKjcwnu65zfEJ4QOu03GkA1eL8J5IpAjzyJwEJqUGDuShxW6D909sUlEfdjgILEucKu2DeZg9CYRKCujd+KITVW2ll1LWk36SMRC31FEqjiHL61UWVlR8F5OxOQUaxmxNc43E+l6r0CjRCcEZCIhPDIiu1U8UUXrGoRqvlPWwy2Jqs1+DTM+aATfzXZ324IxcuWbL56v1xmmHFAD/8aZz7Bx9x7HGPvbK6thriB7pYDS/vfXROedDOwZ/ao0OW6UyiAa07iubNj+R+BMgIEfqlkRtvdoih/UGzBHDw5psaYrFs8D/s1IRkbO6KxtCmkTzFAzdJFXNFh65PTKvWaikA11FQECeYaE/1qGVQjcEG/3MbRTYxR+8CIOmApwjU0QnwZAJccjp4QkqR5Sgb2rBoIViu7qTi5P7YfMfZ77JU7atIP3rFNB9QW5PTyL3cjKmq6W8taDEE8X5PhDffdvbtHrTR0QCqij8dzs1b8KPWR4WI/sLwAuIXBycv2pJYgIxVudmo4+K9tl8MLNL+aVTqr3OqA70Fii/OKm4FFdnLi4ItF2jT2iVTwEOjdRuqq0T3gAo9Wja1KxQ8a5dqSRY40hCf/LAtTdMlRGIMNYWt3VXJTfa+qRqRpF+Ddbp9HKtNG1mzlsrUlcaTIVY9PWvMpylvBdJwSbKNWSiFYOmbstSEOv+cJJTnUjnPEZMgYnT7UjYwhETzEe8L3xt9vCFkib2KeOJUTRjeOqaImUqum6sols7JmFrsXIkmQr6bmVr+BOoQcLvt03FxdSDd5v9Q7ORz/sYiAEGEjxYUER8S1roOV7NO13ZVNd74zGi0EfNqyr05cpWdXb2vVvpoB2YlXrF+h9QEdgBIY4DCOIAnAjS9CBBm2nahmlip16/KS8CQ1TJjxVh6fULTmTJ9dGpxdPujwzkOvOLzr52dP16CS5VnDLBAetDYos07rDcLrbJZxXGZ4L9U7VwiiAf37QQ3SmqwAkZJfI9pwFzYe9uA3uc6AKU3BLbCsgJoL1KV/QhiHwS6y9gYsedJPsf5+RzGZ78CLv0T3zieXdPy5AJCgbHC5GBZQFOrIn4EeVGq5vzwDtYmdx40wPi4X6gslS335R+5SVnP/UavJwCrpoaeNxXAiUjZLUWWlGoo6ANeRymBloaILA9HSZHk1jGfdXIyTZz7TNvMUlWDxPt5MFLQi29JMBHuJ4PiWiivTcCLRWH8ukgK+uGcAPoVXSI5M7S6aC9bAa161h0QREg/7HTpewXQ0cC1LyVZlYe6OpOTzC7Qd6aMpC/KBplypG6EqpZFQpLv9a3jwEK1WGJKEwDOmgmMnDWQ7egwzyZInFhko5DhVFCqrLAV/feN32QrLtzX83ANhY7RJz3VGfS0qMrBx+X0sZsKVDB4HPI2TgUQWtW+U5k+cVQY/dvHxaO1dOppPYGG1yjuoDRb5rtyKC3TAY0bsa6drv4AyVa2z5NVgXEUW9VoT24RBhjGnA05qq/fUAtS2FjrF8qwTPzECxBK/4eHJIIp/33gQuBpoDn1DhUZAoLtNQvXbsuY5B7NYgRN/XrHfCplcbCBgCK4kJMnEAVEKxm5KRVZeDZUF2x8cTHqAqMjQlQGx+Aw+vKELqcew37XweiQ9A0HC/W9F9DWVe3YRzGXflN4X8l7Z/STmv5K+FpVOufKCKPs8mjsfOxTCLeoL/jlzsy4engNQt5ZecuuFCGbviecpTJD+7kJb9MdpecT8N29PUM4p3gYWDR5TyljOtsSPUL6KJmSbXQnkzfSMmkjJRM3kjJhIyHjNuIydiOUMXWQ9R9SqI/XkAdGQDtDoGwjLa6putglzsYMZvtABa2vIRoAjtl2R5+VttUUfiYFnE7sigTKmZLZMgcEJbZBcvbh2kEm7PXALUwwVfb2IbEeSK8kKyCVNEusnXzhxzR7qk3oykLEMzKioBXuoTLwpOHWB9EqnHwgKGmaViV80GRjuELTnuvOj6gyOsPGYOqTOlGohQWDiOxleFwFTNpPbUOo954DCOLJY2DJ8xZ+uP+gowdgTG9QYya3w5qAMY++ltjuRQNIv1KGIUyynp3s5yQXucUDXuRdvuV/1NIqQIGHSyVKaNZCkgdu3xAbzk8swBjh1UXCSf5kElQUAX+hQFFzWHMTKiOMydgoQ2oixZn8D64rLRhyM4XrKCL0X9RQ/eVepJv+sR+YyKbGc2gC23/hWnpl2VB4MQDCzzQVvys8Li/YKWI0Xe1AhIuikYd9aBE4fnDSpfEE9DtQ5l+fXV0Ux0BnVjpUwLNrD+9vJ8fsPZMX47cV4iiK/3N0+BPzg6gsHQaw0VX+Xo5wgds8zev8jF4AohN8gXdGO0Wx1lRaddpZrVl597udprUb0JADR57GlEeOndXkS73ifA5yxZUWLVt0LoNnu+5j2dhxtyDaAH/yHPxGq8OH5jDxDUxp53M4Y+PQx3ZNXVa/4yzZmxE9UU/CQ9qFUB4LkYZ1VRpzaVGqiyjOCTNUl3SVvthAFti+gJ6N2E/cM33kb4wDUa95Ld3FC6zg5OlLbG0DMU44UtSMI9oqR6/JCIzCY+xNT8ttLCRROerVkkjQWThRDE0zQC1MVwV51AWwdHzqZzqlQwQaqVmIKU/1YiI3nWg+5eG0YV0JH1TxhjHaNFqIYiuUomFErwxRPWkAwqyYIfmFoR5pEdJucboxBvPrWsDp/YNpCjyqbqaBelNDKbOhMdd35zeqhdRC94dylBtDSXRhQdF86hgD2aYBpOoYHZ3Z2FTXaHCCzrtGRbzDuUYa2pQb1WBqAX3vc9eZELoG9RZutP5GPRPVhGmv1gK0gv+zdGMc02zgIJd5yIuCpSJCAosYPTwpulGZFKSQylW7Wt0PU7VOg01lcN0wOvksECvYrdVju96Ge8nqlBm7x9lPVFgySbPZ0iX6EYj2shvcmbnpSnquLXB3bIKLPwycGfyGW3kNpSNEeB+09HlpExIu3EVpKbNJRxrRirmrdTB4W202nMvT2D4mwCOLUheT7e1dlKthhgsk2XZ4FkJIQfZRsWGpYHMQa0FxTN6iNa0A2c2eUXuBjy2rYyfyPTTpPldb/gU5c5XDXICiYNnX1RMqeNkvYjV9jUTnebi9ZaX6Gm1fFGA3x2RpqwmCEfIbz1Qjh3VTzhubON2EDafpGwucX7QHEK9SEfVnuzLDbs7ziI+lLsNFMUJIcHhMHvYdDcbrWBtGs7TiNRGUopuPm9HJlCtJTbnO9bO0EpWdiK0jlcwDjfbQAXaCm+XJ8keUKVWa7RiicqWLMyGP0UfcOIVW57UQG6QJ2vkU+6ewfhj3dApioSzoUy1ld9vv7IyNPM6H/IgDaRUuAQg0xcDgsGylGhmdV9kWa15XV5XyVbtcV79qJuMx+IhjjJg4/uj1LuTw5bZvzeX7Bsr7tUHB6OCzfuA84gV1eQc41Ox9KQTN7sWS/+v//gALtV2kodDpC90DBak9Go7n89sHiwkNOjg0Njy1NDiFefUNwW3cOVSE1l2TtzmIW5PFQffYC3xKuHZHu0G3FgVTNJIrRchwt2S7MLlvU7aSZxrkJstltz1vUsppWmlxfanU5kuFFt7NtQlcq21FarKs1WdHSlM0iOvBx3GnaOSCXtAf49ni3PP3uBtnsKwoJ5pjqT7NktkIAvT9xacQkr7kniyGMVfwSU7J2LcgsNmU1qg8aFwW9he4/hEj4MfBGJ7U/iAS0M8QrbFof8Q8DoD9RuTouluUm1o4b1+JhRKogzsq0UNnn7wu9VEIU9DIBWi1Ulgn8DVDvtXeGIrGor0Isa3pflFE9eSF1Z0t8hATU/X9yrwe7G4x9I6TE8u47YbjkmaNus6iyO59Cu2SNOZ52M77w4nkgHDTTgaaqdlWIbErOWzHo3hzn1VmB5GKkKIBuhJ8D3tNBeKMcFetQklG7a5pfYImwRqHbfsR3uhqoozDs5r8UWcuDEBE12HbTBxuzWRiv1O1C15sa2NyqhKaT0/rZvaHjFqdB30TG5+AZ5ElZqpjftTigbguHLZ5GmNjDstshtgzqU1pbgVJHbb5oRDWpB52+5pvPKu7t09Ca547sIK7oNrdjRPI6l13XpWBGVkrtsOcdCyWfRL9A9L6XjefB7JQ1oq+8j8BrdmD3BGSmES/F+cWU4hA6OdEYZV1mctK+hr3JPNTLyEPhOZIRYMUc8bSvqspUzZK0qqb3QofO4Y63zUBgKYFs2xTjFzliM4+3DetVV5GzYEu6soRjxQav0fNWbr4nvGQlHrj/B6JcwsNTWATvdQ8JEbQypyzMUfW43jSTULoXgAw6ywYkT2LpSGc7SzMYF5UpkJkvNdMn/ScMjg+jEOf25T4d5YaF+Vyrsf+ygrKTLoDa7UGrW7jfKnpyHNTv5K8fWYcnLSaLxngfu05LnCHV/kdJxEQ2eSBl8fLsXnSbUTz0BPID1I6X93p965GW7fxGdAUJz1otEFDx0/kff/GYEeutHjnfy26+pjQHoQC4kgHY5DXzSFH/P5bVYuAqHGFiy+97JobPaqpwRT0wT2kB3ncfFR7jfAwhCoaB6eGexZgh3kcMWOgKeWJtLYj+1gGQ2wkotS1rNiv5cAEO+3nFgOwiU8UZ5bRQaBWaEW8T5SdLDAFfL/wH4lWl2t1y4/V0STRhgvVd4QYcpumaiB1t4eTbI+mjWl+gD7OFBjngHL1VxPtVtUXYlnw07CPenzNDlCTT1CdxnMWh8YoWnIf5B+Kl5q/wuJyuAUjykCYzdb8ckRPxW7GCfpq17Ng6c+XBd4WgoGEEirTq9CB0JTG3GheKuxjdebD86y9NS1zFP+LI7eK2LwnO+nPdR0lNuHBvK+XxWOHE0JpiJE+YjwScBBLT8gTItrMzRcjWK6JuLPUtTnxUpNAl6zyTCBS3LaBZ8K2I2S/Rj8fbTxEuXXmZYYX3DcRj/IGJoOOYdGquKdB8HieNHJMPW5Eu6x7IObqgKxiU6MoP+sOI6msYbd/zdw9jg0gdI/zFczgJnPY75UjaHFPv6j8dEAGKe7bfOKjU/lc1rSGLqak7vOKndCpA5JnfIe41eIcRbDrJpC6yuGqM21Bv+kQPoARPrEh2VX3HRI4JXMq2QFc16DWxOyDa+HvjtiVpBd3tdsQXnu4Ec6gVLtjXoQF3TkrhBUyexjCP6qTBO5VjjJKat6+udhBZDEfrD1D17G3zdWO80f/2HICuQk+UsIdoMuia4VDJd4B/wWt/O9qKNs4ymWe5lsZClDM0peNtBPyE9GxAhRRvj1G3q/UrisbbL0mZvBp7n82NnjycUPHbnQhBzp82xevuWKTa2G7EYTqJ4pHO2tZNO6aJzqYx9Gm5TPPNsIGuUAIn4UNLpNzP7H8LxkcvIi+LRVsAgttfjYs4oIHe6vBOZAdFLtcMLkk0+d/aXRqe7UldwPwCYwH/ENz0dwLZPNLJH5Bn+q5UZaW7snjIZIzmiBK/ATdeCDdWgIzxoY4houvdjJCq3ZuObw44pUR+ppYtmQxBOyXakC4aKqStdTtLmdwE3FUtigxYNWX5v310K/SchOl9Bw8TI4C40NRPyWLhK/OYaCzkawVfs0igPOHC9/ij/5H87xruuOhvjA1tfr1HrjQ3VSlCg/9JHJcHNVcJ4nRgytvP1yI+ZnjfjU8vdhNxF6MgQVDJuE1AEfbpLTEe5yPoFokL/gUkCwoR3xyNJ7IG86bpaciCwKRkJ4zkAfiFUo8J05ae3RucNETZdB9HiXi/XBqVxdomqorTE5rjrPYExyEhxhU/akEgfgm+cHlpoPRwnGsmkzSQlAi20cyZxraX7WVbrO397/XccyxnaOc4Qr3eJ4vpSkNkQwfPWfI56gFNuAT6S6No4mPsqXSoABUwrBTLyEk+NucsL/hUnaQsegtLMSKRmct3rINGIgqiuNFcpP+kUM9RLJNAaEQngnLiIv9jPPAQ9f0pGIiSYknHCI6LvRNf8wAkNnNYFCnN3vxy7Z4p/YJrfXqhaJjKMDODccBBvPfBiTpwKAYqcbEjYjpokA1qlTWtdKqf6Fgs6puN9ny9VdGjDgjw8YP27av6hqbUX//4ZpN1pzHAXEqZ/6kY28BhBgGqEwjAN5h3BRTSygR23bfbV2+BYtvPBHk269ZXgaZOG1sZu7r9nbYWy+enR6uwCuavLCGHuYB3iB9HV+JfM3mFTRs+0ERlHYegZmNzv/ey1Xtcl1JefVsKUW1UB8x66mTeuywSTCrM6sK98OOzSMhmDLghfHq8BqN81zIaSc6q8U+rVdyFAn+XlgNsZiZ9utiP6fwF2rYDMS3xvUS9fUUu6AVxjLBqqEvHyIvqlcr8ErbB0gjzU9MVSrTRZzObdxNqDre9C4ldlw/mL6/IgG6ockGknvJ2O77jqW6rYKGADS4Am4qoXpdzlIyfCgKMFQifu5Ar5mE8N9CwuAeQ0aumLiFnLGwE1ZVhpOkCnQxqXAyaUwojK9wlVmUwj5bbCURS/UzCF0W7KDS6SrkDNMFwVZ29kUgDxAmv4JOvv6YaVoaOmb8+KmqxIadKu2R5PYUzMJwDldOvHJCU39Ppg5hgfGwnhH7fo7nJ4cvhw7uaDE4peNf6l0eHq4o9U8kCHLaS2NwyQCmbhFQ0m8zWBWumStxo2rzP1EQG2ps8OBCspEhOwrk5285KSJCnq4hpsnY/wi/n80JWRINqLayQNv6WbwydaaQHkS03EoVAbvxP4IjfZ7XPuqkZwd5xGajbW5dJZl5GvxBIaJl9jFXipxGy0pVSGdoEmOv2IQ2buzl75RHpsalKkBT7BUht/u+g14m7dijg15KdhN5WPkpm0zNzOIH4X7yHijiLxw5OdieKh6uxzYRUUCu1v000pvP7A8GahUueGLeOKUGFks8R4fhHlbeEWxHT58z2dTNI+jAeyvRerwkOEV2C9qNtsSKBUnPt6Kl4uf6FNqtr1oRmCQkSRRitm0dFI6KPysI91JJ3/EKjYKSODjqmE+YPxSSIyfbHiNux0KtSUF4RX6yxD74jHeL7AkhRjNzIYITYYI2+6ltmKn1+mACmdH2FD4RkQnAyStaWmznUMJL16ug1fZHI50QnMIJoLPDRCvZR5Dn8kFhEchhtz1x/A28jb45aztYrJJ7BHk+fhKLsqjhVjccowdLvvaJaszgkiYI4ER9udXtCPxRi2JO5fBcsOz9WDwN7V03WQqe/1AvP4vx5bP2bWqxGy4kL2ayXnADbUp7kiYaemlNZOde8NGtQTaEYUZbIEKeMzstL/dyQecbheE9di192mYl3lCyDFNJ6iAte9fbYkGIy+pqDm+bKvSLXokH/iLURb/lqZZ+Ium4vZ0E841uKDcTPPljx7UdgPDhCSa/GzBSCW1u1iTszf9XmUFY+VAbEB82M/VCveCBWQPrLPTDk/jydRb6scwRrvBARI7AO8DQZPNkOAT8j/uBczQiGYEdVPFnP9sgyYZAKVqzqFjewX4C6k3MTMJZxm9rBkuFidieQZ1WqZV3NXYTTjdYw2pECkXxGqUxERQGyrebbWVHRpU7i+FZPr8zJjvORe7wNK/yLt/zL2oZLpA0RwuJzVtoykrtVcUWiztUUmjRcrQ1TMBAZjGKZr/OxldlvW59Xdex139nzf20cHRoDeAAPemDjn7o9vjXiQ7oLyMSzW7jlm/qM3U+bmMtD8+vr/sWE+ttsK4TzTUBht11Aiqo2bDRdzhuhAhMDPLaVXKj0TRFKxVQ/bq8iDpCAUhtzZ7jnPmJouXtqN1JPuok5kXzoubIwGoYS6OScZTQE42+CMpxolHVXYdMojyLmL5xFrrVjAVsuHaIk+PHEZ1L/ifhXtkTR7A2U1HrTfsNTH41YMpchnN5CpdIopfwU239pToqMbzgBIdJkGIIqTzP4J2IuX4/wSABFWFea/XK6edhhT6sHY5N47ibiXYMDlKnxnk6lfIFzKC3mHwDR7Reh8oGYlRbr2HzTj/q/26wM5/eDIrOSr5C2Xkcxpe/Ar7Ha67TD3QlwNu1DvWfY+Ziurz+iJiYw/3Us3pjg87RiZ/65nfo0W8L5oGKpFqqC7/b56pgUoe8xeyksAn3CVHnKx0VnsPntl9loG6oIZntFVQ2+TKu1V9v5TlugU6p/ZBGH8qPthTjDi8B3E6eqBp0FXof5jlcIzfH6oGtNGx9LsUCGq4LZtOcTtADpIsnXosQ4OsXYUIEtlORPBmseddRGj4mQlGyBirrVrcDi/OyUSeT3jRG0h678vqwoCmT82okPYWcGcrQzJxiOxsrTEh4L1jSOSi0s7EiF/s9Sn+QWezyD8h1qgmJYltUQIHBvsNJs0JDc//AanSmMmTIwKTJtzvIYRuDnh267XX0Gg0OQDqcfJCs0rJ9YrYM4UEEBD19qWOxNn1SUh1FCZ+ONEZWyfSSm9b/m0vxNct9smtf35ZnRhbNXOzpsHNXP/Ty2c3J2XUrcW7Q5NMAh4gGRpKQwrvFWoX2OGtwm2FWpCsj+dVhy7u1codabbOxfmGXlaxu6XWuMn2wdzPXRLhUuFw4ki/xL8FDORCEaPlP+QTQlK8wAadhWifrjzGFKFlkFZE20s62O/G5xFKd4ghxutgi5sRYzPrn+jecF8+Kf0NUSB7SgziRfcjLyHnEjXyL/IEKUR0agRJptPzfywSLV4LhNFBA9mcKl62ED6vo2bW7OgwfejsYOZcX99hblEzqqbstggv2b26CzPX/wOWfDhVLacjbIlH717ZpJaTJN5DZ7hQX/RKRtkctdhrGLtWibdfqbwVN7UtQyBzRg+YkdsMaFBeDlKf3D/bm46qxMHMxAY0pQpZWdRjDpCX6L9zGNaICVwpPatDUOk/vpko9oyUySCWeWAEeSaCqEd+aqYQGUqwpoRVY6no1iQjeYKtbDTtvNfoFrULxEkTz9lLsNtizF66gV5DKj0TIX8ddpmvEbMGRPIyT7Wm70yjGjWwJ0QZ8l2wZIhBLyUgK7KKb6zgZcy68Qw2JQjlij1IsyHv0eA7W1c7BzSO06NgRegIejKeAfEg/FayoJzbKf/K+RNcr8PNwGGC8JcJb9b+5GFDuHFr5rDjp4DQM8wTykwF/gHemlqrxKaOlluBCM7LUcVIHuWBX9k/FeQBpkzg/XZaa3phyltAhWkB/ymtZiV7FQZTsIb+pefJg2PcQ83omvqeeEELSZKN17myO+x8xfgEPl06o2OMJLlwUogiMI615YoaMAQMks+/apZybj30PGy+iHDww1U15FXcrsiJsB226wi/4um27lcz2MF6a8bBt6uwX+DiDc1lbseClU6i/Jc5mqVVK6TiqLUdZnUXyw0N7JMTe4xMvbSa1KfbjcJu24NyQ8hoPMxo/IKeBkcVMHafItf7sqV6t05iOm5N4D7jvfbP41NZOjqinbFPQLZrAToPZVxWYOo8aGJaGvZjfCo3HPgjumdI5nT9aFbQ3BcYsqJxPfRkiFzA+Y+eLLg7Dl+GTCXibP01YaXVq1fOkcmHsju31tIr6rlmXDrrmX8AUbOJ/dG+NRdxnmBxzdccHBiMnc5yq8jHewJQbElGStVRbR6TDfr7neQkER/SrX3EWxuVVXjNEOg0MRRNjmkho8ds7K2F5VgCt5XoQApxXDUQJzcJHxHoAoO56JJgdLUA0tDq3fGeTnmzcJ5q1z/XTvtcex12iUsi4Gm9OeMaYDD8sIxY054sCB1NxTcw3JLy1jfL3srqYIiRAbHpUKg52O4xIial/20YM3NAvcMqlRU9vYeqoIrXZQkY5u5m++KRHwId4lZnpUMk+y5/diOw+tv4b0DJ1XZXKG2vFNpLd2AnCtES9oNhMzeeJnv/iW2sNt2abacQGsS6VEqrLlKVYou3ye/KGBW5vShEklbu8m0KQdRrInYmBEd1tStrgtoICETSywwSKaUmY0d7PmdDOeKJ5LEu+JIYpS8dTJe9dokm1fGwug4IvchISFBmtnJcn28ADiHIOjpKzfLI0WW3+LoSlk2kqkTqQcl0b/YE/NxlizgLCZk43k+haaHGOoXbzKah2H4+sJKnyhF5aKZJrSUli8VQqBx+0pQ42L0myKkmFEU5IknG2h/CObOrzkUm8QESY5RpoviQJcAo3lAHRjkNPYmRjmOTgwAruuxksRuR7zUhHJsYpJm/5PrjTmCKBdXzbtMTSk0KDmKg/4WXYXiE1mh/ykG/cIFXduOVXXdgAoTUIvLhfoluX2IUXK26azHj2P9Z3rnR5G2DD9d8yP/Y2sH3G+tBTb8GIlWo75fBrTwo6KoGs5zHO+vpK9DvRCE7Ec0RXNNT0DKsPde2mZ0Ij8FgDTMZZrhN60oJeA2m/cySMZ5kTNUyW8g1N86JCcmgf0kQPgzJtUPKxFKiU2m1+lVzY+xBP9fB4v383FLRDtfTerWAMM76vJHGQ7shZ2B9VVUWTnnEq5s3OXrMFl1yeOMfql4xa9GflTZkFDnczr+41U3iD/cxNXalhE78GM53lJg94jHf5cYDc30+riIhWpBghcVm0qzVlUxFFtFiuqKRrzSob+ldaXhhjnZj2+yKnMvrkf7bF2F1dxKHLrbxk8drlZxqgp3aM6Jo/bTrs7G8zDkZwhNziRZdedMUNN9lbotE0d855McQ17X/jOBq6X2qtCUf66BcO6W6mNL2IDgw2U9+RATuKggtA6KSx2UF6KBtYITkuT7c82aWmEj6Y4mtLjk4XjcCl0nSiKMnYeUouY5kRf4fxABnxLRo2mne6WbkAQOXqfPJtuEYjDcgnilNDPL/WUTowxY2LACd1Ep5uavqwjw7gDkEXF+cchaGqShXuwNH7Dg03KJsFAfrtwHRRDAo2fIas0TjssJ82dnziblp109yXPath9vyOauFHTzDvt5Tz01zXpLL+eIDp2rT9Xs1nRus6sUWpTo6zsNJfvUcxeEDHNWymBZ4Rbu46MS4+bpTMyB91RNTDAkYiFuQGzTBupJph/+TdjmGACsA8ShOup7GNtFllc7nWXStDC+fCGuT1FqHsd2HK/iXrb4mk4PYFCla3HpcEIJMK/WAY900TC3Cy8nDpkw1cEW+3Tci9DPs+GgHTOECoqviaLqKWb+VdZBAaPCRbSncqTdAWqW/jdpt72OtrdfK97bIG7493364GtCAw4dunYlB9rv+KjbqFVUOVMKau750IXRyrGptQvgvwqFYJfpPj8icqOebgdjYjjPjQtgmMRj/WgcD6YRreJa1aC0Kno1GvexkDK/3ORjHLDvZzmus8xpt8zO94hCt9zq8mIeGD3wFmTpQvT4YVzx05zLBLc8NMfuE4vj0+0rJJrjPi4VG5hQOo7vagtCrL8B5tmzByq8d7QHtqPozGtBvLzo7h8/U3N/+f5QVr7OIQV7nP83zO/xgFILDw6an5X17lMLyIVpRPYV/t8r6srLbuH9fYigab4qAxh00aNmonl3SLHdruFvEipLVU0WV8JrXdKyFXaEdWMXpvBxE5RSFagwR++jgmVuMchWxqFVn0mPwjWLmgtLUuRV08vH3ESDLOgCoRHWAGN66iYiY3fWWNlq9lOoXpWh5qMIADecovkml4RONMkjzTLai5nQuU7NWlXhNnApN8henaLuN7yPujGs+tjL0PLuerjd+MBopqvmaUqvLuB9OkYASONd6y5NHQx8srLbMENXI7JiwenNtOq7kMExYX2c4ydCTVkyDAumUy+B5YhQQEZNtAIFypTmVhoAMwfDkGmK0z4oJjBbi7OO5cwXJIxjsTsxSvWnsZZRIDcKb2ITNCgiaXydQ4/+0Fx+yNAzxKvY++OmGtNHYtsuOvy46CmEmPti6TqSqxx8ionm5WE4PAjnFe0dAyO+J7Pv10tLBthJftFNIf3YtltnFUtOodxb9opSF8ohg5NFuRgaTwdrVTeRSkKs22uaisqO0WFTXDbdBkpzj81EYdc/gcnh2v831jTYOA3L0KdE6JCQhl1phnmsoZNhNOGwxSPRqE6o9LVwQ2hNcrBdGcuPFV5FRQjJoLIOvuDgqgN7c5oep6ZtfLFIrfvtUz3JoWVFbLB1Xm79AWiDioZqgGvd0IC26rajKaD4BcEDnl8Is1vgdgnhRrH+9a6RJlYfbTAA2lo+2GigqJ+Mg9tyugrDU/xhddVRDMLrDZKCsBO2t18afpZBN8k1hTanJSxHnHspA2vbrtZ4QmZjXVOoSGba3cR5g76jn0LAOjOVMBErgL0lQROGp8gy2kMm4M0BbfjTDYxyZIbTnHJz3YNsW7FDhZxLngpuZrowUASAv6qmOp7i4AKrtGL2sw5prIkmNKqvPga6mSv7sgU+jhvr0gk/iU7y/4OJoA+y7YlZAqMdr5IDB+27T4sSeGOAFTpiK0sOe0rEMyzhqf0RWZD28GAAzDjQtWWSr4yQa6Y2lVyA0sg+wYgWEbI5sNc2AWh2xX0Mj++FKihPOGF1j+UTBEy0eXhLrpDcyY1n/wpK4iTO5f3YL+f4+QOpqRgwOGFB97M2ireGBQgyCghBZWOBBAGHE08cKHfvPcOPOm88sQVk8jrXSVtbGzzRm3YztzMpkkOI+dF7d+ha5sASugTvxOK7VRm7dNe3e3/8K/9e99BY2f978Jo6LqEFjq8lRnuEtksIbN5JGfgAmaUG5wi3s8ZI0NtnjMEyrUqMN4iqRJh2f06LNkx5lbhIsPPPPKJ43OZD1ENyxuGStZzfo2tb297L/My7sqq/b8XtXrv7cedUyqaWaCJRtusDzxbQyNzVk5hf0khhtlvD32OevGgydv9H3w1T9nTtyu1JssNjevR3FUHFDAI4IhoYUedngQQhQJtPGmyWjTMwN1v2RhVRQ010lG//Y4/dbbmZPOWqZZ4Hxx3t3tAkqwxGvML3VnG7VpW8r2dm7VwUuYl5CYVKUpb3WH+8SmMuKHVoXbX2fQNSVGbGzTxWwcsebQZpin135Q/3YLxnUw2LKGG2kDm9nN3ku/PCuxiAcKWteOxw6+10rTq7++WRdj3sZB591tvHpvyLwz4Y0PAq2x8/BcoAD5T/LL8kt0eChL/X/qiTUiDawJ1Ag0fA1Pww15Wf2x+iP1TbVbvUHdoq5TU+pydbE6V52lDlczVKdVp1QnVRtVG1S9KkQlVsEqSCVUBit1X0fr311q00rWxP+3Mrygb9fr2ta1/funR2uo8uqEfyRFQsK5M6dGXnrhxB9+98yeXUODBvTr0a1ThzYtmjVpCFtVKFemWKF8eXJkyZQuTYokiRJEiQyZMClUq4WiuAClls/mRMlUvySSkZdVvfhvQobAYqc8XjKVr/FfI6byCpQq74CByo94vvgOMOUP8VbRV2Cg/AcYgPL7jfoRzC/Hf/Fi1u+xW37lPyVlz+TTLPt6eEn7Tdrvgh9ymTBPMM8dGp6BLq07aVhoUEArsqZ3rirAW8gJYVl7uS5uy1bofcK88ve9mPyEpP6Df4nLxGIkiLlYiM0xiURG0euDjzlihOvxng5PtXqQysNPTTkjBvwQupzpY6cfikHZ/ag3mq1+2DAt24kOAybMWLDhJkyMOChFyjRoM2TOijVb9ty48xEgSIgU6TJkypIjX4FCpcpUaNCkWYtuvZasWrfn0Kkb9xKlSrKg2IlbJRLw0LNNRpFK+ly9oc5LPRMAK9boquLImQMnVKhx4iJCFgQMXKgw4SJEmXPkmpxjy6KJujABaNCAVph+n+x4R8uYTg8mVds37kCXOsPQfXgJ2rQlTYxINviAxIoTb8q0GVfu7BoSTMUsem0UCGGH0M7YJQ9IjZ49bs2ru7wYiiqjdWmMMcRMTMdUBJedYjZaWHwBwwDFTyUDIIcyrEqJ9ShhDByCgsOcFU0MQxxPDKc0lE2xRGkon2KR0lCh49zN1K2coqKCRKJZRUWiqqppWlhedybosbKw4xDiISB+ra8wCqLENSRoamnz0vEOn2CQa4r+CKU3aMhgzLjR5Dx/HneWt+O/yiSSZfjKl++nzJj/TLMqHRYso7dtD7Njp9jjX8SCdrn8ZddJ8Os3zsOkGR9nnndqECv2xUEwPDRC0QFLUeBYq1OPvp2XXxt50ND9UkIIFwDZsucueIFQJy6AoTEnp3gPIi4LhrBTsjpGrPyF6TFvctcSNsh48/ZWt24WxmMiEtpPBw78duTInzmxhIwuYca1p+AM45ZbX0hPoGJLz9QdLxLLMONz0YQGK0vU1a1pfZ12spsBezjZ8k71mh284TvXxg/GTCAeYSao6Fdpzf6S2MLI2crWqyCoCgJQCRJEVahQCNFiqIkTR12SJBpSpdKULp2WTJm0ZcumI1ceXQUK6CtRxkCFCsaqVTNRq5apBg3MNGtlrl07K526WOs1wNZwjEYC/xkjiSOXiRUnTFvibMU6b5t2+DtwIMCxE4HOnQt26VKIGzdC3XsQ5smTCK8+i/TVN9F++inWH3/EzT8eEv+8olw0JMDFIRkXcfUDJsHIVaLJtSRVCJAi5T1p0nTIkOFJNpSCDA+cRC1X5aUHKlSQUqUqDwIC/6iB5xV59JPMluJqsJTDkCGjL5qojanCjwkThEyZYmUW9kHWufFyON4sZ/Dho48vX3b8+Okffx6gPG/0lxlgPHj44sUrm0SoBDmDoHa+7gvE63W4PwSYhN8kSfk30ragQcZ7WCAgCMiRQwMKio48eQxgFDFRooQNHBw3ZcqEqVAhRpUqcQgIUGrUKEJCUqZOncYnBW2aNBnSosWcNh1WdOmypUePPX0G3Bgy5MOIkQDGjAUxYSKEKVMpzFhKZ8VKDmu28tmxV8qBgwqOHDVw4qKJK1fd3Ljp5c7dEg8eVnnytM6Llz3evB3y4eOUL183/Pi5V6WKy9yT4OBttPyCK1fzUqUyU63aiGbNRk2YIG/JEm8rViRatSrVvn01Tp3acOdOknv3Fnz1Fa1ffin2228n/vjj1l9/lfjnn4T5T4ItDKc0AzbsC4T+QCBsgkLZA4NxPxbrMRzOfni84wkEqxCJdiORnEIme4ZCcTCValU63Y8Mhp9ZLHz8HBUqpujRsxQggK5AgaqECIUnXDhUUaKoihHDUaxYzuLEcRAvnpM8+agUKMSpSBERxYrJKlEColQpmDJl4MpVCVWtWpQaNebUqnWkTp1r9erJadDgWKNGy5o0idasmagWLS60ajWhTRugdu0GdegwMJ00R6aLBCaf7ibXPdStT/r0yTVo0I4hQ94ZNkzLiBFjRo3q7DEKD8aNmzRhQnVPUtifKZpcT1PFgRkzusyaVWfevGELFqBbtCjZkiW8li0TtGLVpnXr0mzYEGPTpkjbtinZs8fGvn18jhwBOXYq1pkLUy5dunLt2p0bN3bdujXkyZNAL14Ee/VKxWefzfriC3pffdXmm28UfPedkB9+YJ8/JBY5HjMTNuwMhF5BIByERBqLQjkJjbYrBuM5EslULtd0Hs88gcBsodBckdh8icQ0qdQMmcwiudwshcIcpdLMaen4eLqjiqjXC0ajjMkkPQsblmCzWdoHFUv1iSroM1Usc3Ehrq6yfaMKmycpLGZmvj5Exz/+8WZ83g+y7PE/g+9YPSkY/5dfnlLlNzB/tYImr3I+f/LPwqchFw8MgBVcJyVGF2OeMYzIu4iHJQ4iB39ACJhNMopkPgd3omo9GBzqrYLfuJD5EY4DMZJFf0RxptKUoBcwByroBTFBpOMLEzkDDFm7RNFp3NWgvggSDxEgqGtcg45+hr2RBCDoxTD0jFWkKoA/sGflF9oFaeUwO5bQ6spnaq2RlwEV/w/grWxW+UavbY9Z7PJqwXswSERhBDYMQgNSqgzd5YRBCMHuTsAc4W0hOw0/xDpdKhu5mLbJ35toEQACNnQkiu8W+6fM5yY5MC9tEs/hxR5lkDHAPIJVXGFi9DVKSsb3Co384UQ68OAcBP5EWvwYQtIUA49qYj+BsCbAIHzvDDKWiEvxv+2StLawfladi7fDw0Tkyt4XsLQRAni0a4c0ajF0JRjQQiM4pIjzQnpfoJJjIjvkCwMezrHXL9kMzCTi+BDEkcDFk5vIISR0tKA+Qp2G/aYNr1EkaL3aSOE+2F5E7Q1DAaV4Sj/T4kqpa0+4IjPbaqSIB3XJTbokAwsaSkepPaIK8uDOZgxeJX35kVy0MKFeUkTBXeB5sv15pZzzuowLGUI/ZMjGzabp2+j4WvFOEUuIL6GY9nYXl5cUGNw3g5lk0p+NIPZIdoYCbA1YUQ12P3eJwICe9/MWPIC9SNx6ruATYn3C5KVbk11+0gv5h4bRNOmDO0NikyCWv2FI9DD2koOMiOcZY/yuUA4vkFBc501c8LLvZTgZ4O0pdN5Z+EeodvdFkDdy88LEIYkUb8mEhYPbmPEHgkmRdlyxUyZyj8ZN2jCSRRUdw7HUlbB6xScZ4/eXGkJKtECkupqtywRUdPAaML2MA0sEMcEM4Y/mOnIeYLGLXMb33AOj824UI74u4EpeCBTgbeAhtDF1T8HADMO34nqUGrLfUsyKxo2OnKAVutvrz3ez1EwmBH+OelBfbyYXoKn0O+wuWefvbgIggy+YvIsCnZucue+BdR6Jyez+Vi/oHuMYKYOfKMeolo59e0prUvqLJ2AlSKSkzARHZS8gp5sz2cAqRfvurorBA3sJWTjoFEqicA+KTE/h6iJX3TDV3BJT2lN/1N77/Z/Bfadsao3y++7oTS2A2S0SgBw3sjgbIhGgzyN+ALqraJYIgwdJYRlOzbtO1Ba118qdGo4sVa4PpaBC1ZXWLnFrwE6PEAezFIk2qLZ8gWnD0idOWS031+tzCIDmcGvJwB1hFO60J1xLds67c9sRyp1/6HnDa8rBntuIyn+33I3w688wpNmygiZMVpQER0ZWUxVtNveB64Y4tPtYytQwSgZ5JTfLcPOYlt8tAiGsjR3PNqkSDbCJOL8JKSPpiRgZSchDtQtO80YzaHn12Lmsm3UwCJR9DCnCMqHfbEQmDyi2tZ3CLyNCOt3LR12BLr33FT2lhkWwllTZH4mCNbYSizeRhhiQhDt30jpbNjby52ayo/rlFObSwE2sbGApYUfDXC93r3NJOeyaksvMF25VcNfsc26+byz7GLLXRPw76+0Dwm/BMr6+Hd9QApN0Jc1IkzMxImnFOLZSyEsz8btqG/uhI1NEERPRlproEtolt0iOgmZZwfsR2nE6uQBrUwHAh3qCLu/KMw1QU3gjDOidXkPOUl+WhbKoD/jeCXBZZ8zk7FLNJ7soMzo2dGP8gkC23+h0Q0fIQQWSLhMVJZwJYJj/I/oRZOamZDAQw9KR69HNyNJGvBqMIC0KWWwfE3M/OtvRBwVaI6WjI7bRwVg65GLBLCiEtPi9Kq6d7+GVVcQLGREunHKiNMsrNDs7It2GkpXZ/DdwJ4oQd1yKgy0F53ozVswtNaOhg3cPncqK13lu1vIjgkONJcuekkM3vav7u60HpKxN+AiulpdpZY7VTqrCBG73cKxGgTvG6baJfYEuhWEVpeNUP38EoF3bqFLUO2hNg3AoBmH27lLOSMlw7K6UtWCEkdZKMRb6W2Ya7nUPUbB48iLyIDyGu8UcPSCxBt0v/dhm3kTkAWLJ3a3s3XNkb+lnySJeYOYKijPsAoySHWNEpB7CXaxb13RdxBGlLf6tfvfisodwAA45w0WE5cBcEUPbckhLXyCWWHOUHFknOBJexETTYyfNITEV99LaYePiXeJILnmAetxUyZkT8uRsuUvI6EFuIlJviJzyvQ+reJ5zFDlxHYednIwVcber4NEMtLpaq7mjGj+KEB95oQexNCKU5rLBjapz1rxdyWxXCRBtFvba0Pa4pHHAzJ0CGzihXKw6MwcalO5fnOmT5UmMk9oYu/BjS5cf3NGk8gg+nEiBGY2apo5JyMzXq3iBWlOoEkOwlGWWlJiKG5BLQgdaiU9OwFNrzxFKKuAN9hXYymHywX3bYxfsDuywOStafOQ2PL/os7qrW27P94pnoDYX1so40KMwkqGIlZmEQNfa2ep7bkHtLJWrd/ft4uXgtNtcEw4JHX01bL9lDmg3lci4NeWfDkM3LRHDjCHP0HLNWRj08iE7L7uPNfRqHLr84lUowi9Rly5LSkOnqoFthlrr4BEaByTEYywe3bpXqrz67x5ipsOBZGzu3bhHIoBKxbIB/s6cWau7T2Q04pvDoYKqkBdYEILXnaD7yZ2KUzQdUFV1OzE35dwh9cdRlxw1V+CrwyqakhAMZwRBtklLQOvpVezX79gL5VfGa67H9To9raNGsCJ+Jk4xYIuVl44Nt3I51pzNyD79tNVdfcwjhVkWUZYFGRQzt+ydE+c9eWuiZVFg2mJKQv2dL6b/shqAdHqeS34ENIO2A1noKcNtOEE6hlw+AUmOnCQaX/jTb/EhjGNEgSAZZcHmQT559Tn0fnx3YYyuPXiUaOvuoDG8aCChu8GV6cgLh65+STQ6HrXexJqBulUIbyNGX6zjcMC+Q92/LunHxFPVWnMVG6TlfaHYEaAboZiiobHVgRN7ntNa710PQDPUxYCCmG7wTtYZrrahg5YB7jLJ+8/9fJQ/HWfJptsZcY3uIbzYpzBsV9F9bWI4a3wmJkGNR5et9cz8vdyFn3/Y87nTdD22OeH2OqN/DpqIY00qD0HyVlNjrayslThHVPgdDoqxnbwNdBkxfaEXicFW16seLFEqasic9R6fqVXwmrTO5e2gdYDWIsGzYoigR5VwdGENEq0r/9o6M41tYFZmhQMzQxHyraPV8HbFU7uCJavb++yp5dhRWgqvWXykobZRgKif0m0Rb3N/9iUsdMbAKqyt+TmZb9Wiy8dCOOaq+a4YmtZWMCN7OCK2n9IPok6xC8ssFx6JFjHjEg8B9WnxqfFZxFV0ULmmRXiYrrOqwjMVuFEvBwIGHUsJRe7C93iyMJ2ckLuedCuhqn0+W6PgYrs7xCT5GYu0iZIBuACvpkm3zVxqk1GxsIWlkQnA5RqXqVix3SbPFlE+GG+qZmZXduApVD/yodPq3oZMPnQ4xXHTtVhztENzvhfggZJgCSOTft8FrOmYi1Jxsq1omS2zjhlj9yw5Y5PivNXDi7icufowzxzH7rTUVfrXrFVvI2QETu1ca+ULlqeYTiIT+evng5UyNHROIDfeQb7lz42JN2sYptxfwL6WBqs9Z2jDkMiJcV7vG5xi8XlGMZsy5e4xEobPwG+gf7H/JAUuoiH3+yC9a74Ac3CMSYYwrs7Zv/kqkA4xgmrf0PCTT0qUEe0/hzkGYMWtNtm2Pxdn6KLYFt4ekFG/C9oroaFf+WkcY4RYb5QJ8xLZMv72XWTyvuoLoCKLPNHIQVNMbiF4OuX50ccJ/TQ4fNmkTFrNxaxXXYUfbmv48+LdrBr6Hg0IVOWGNixSXZz5Y7ykvr2qXoe5ZXz6uW0173CqtLE5/tv9U6B0OvXcmj3rNP2qBtvz8ZnU/OUkq1zcIRdFuNwTnC7Pnh5vKf4hvKU0/uq/+ecas/t1+LHFVblky73tKPqBMXEzHHyRwu6eO21Jevg8aR3nYfT30ewZcR3eG98SvlEgf7TvPQ6LZEiXt/YLYNlrjldJ0IJtRMvyFfkoLrL+0eHDz81eXO4lSju47GscRG7xTwqfrdv4cjwG3LHkX/x372jnDTymvnkc9vrRoI/fZQTU3a158kuqfck+3n1I9NTPVNr4Xc7qtQ+r9wPPnqZSyuS2d6QFbTh7i0E3lt5Ku14vEeWrSdbu7iF3Dx/rUg/Ez2jo/gv+7R25iw9tSo7idgqYtYgGgDcB/PT7qcZ7hDinYsgIHdR/0ENNLe2etlxOc3zGgwvc44e7nzdaIvuCD+4/mmMCafj5WLSkaee4ojv/6gPTGc4m5/GOCDP7mJ2F1f3MtYV2ZNo5FbJ+5h4JyY32rLlL4O19dr+P/fmFswyP6PHefMj5C5ueJf3kQ/nxf9Uovuz0x4iZ4U7E7SNT61/m1FPwjB5u3oTxv26y3Pd+ph5wBtLLGCeW0Rg11Qjfd40HkYs8edM0+jai0Xh547uZ3eM5Fjo0XWVd252fg6s6JipFMkqMPBOYEC7rWE3+zJID8/tMGMdzJFAc//cUGuEiK0nayAnzL6dxk4FASDAByRYXHSnnjBm1gUJsYVMmRrNTaAPueChc+vnArAI4EBOlIgM5bjz2O5XcYr7sCTq3Z25k5O11W8lFsoRLXoOvnhSDiRrDx8CQJEO0C1SCNXdwUEZeFQFlCcewAplz4GQob2kftkEqfdRUAZwMOniUaihAbqiglUULFoaxU6GaugdkDt+8I915oN58mzBvd5D/0DC29ezzxi/Nd/GpNReHHRjlCMl232pnd19bh5AUQvD6YsGx1q2di0tJ30aGrzCZCYw8rGiM7o+AmU0UVEF5BF63gseQfBiC4zT8ppjYsinFNhB+HrGQ1GWt3ADaYFjoA8p8NfupgVjmLINyvhgC7WSdi2H1aprGlfYkKusytXLc1DdlZKp6Q9kW5YWWFGQ92gYRHEihHRCXBuDMtmr/672fIPpuu9CBGJ2VTKytIsa3Mu90zV0jKvlncsnkwdD08HhUgSR9z3E4IKpxgMiQlAdtUmnTFlR9f6SZirttW4tc6lHveaPBpEIWNghRMQ6ygEo2ZS8/8IZY9Pd4uBcs4jmGEOWX/lcaeoN8z53KH2SMg9pIRxcwE47iKabEFJmyTQXPaxxkHBBXNiGYAwJhpEUop3P2Z5hf5ZXKEE5PBM1aQ64tRC1zhuYLSV1iSMUnGIVqcyJDBjE8p4uuTTa090DKJBFZFIQtJYp7TygNvGFFSiAX/A2zQRSZB8Dbp27C/beH2/Xv0G9h1jCsW8U03QTb62AgDMYieVgCKj7qwhKdPhgfHZ2MPTDKy09xbemw9gyadi72tq8a/RyAEbL1B1HLOLE7Sy78vCTlNsVZW4yQeEseY47Y+hB4ZYzRguxCa1ADC3TYpJHN4ypTlYqPfZZtH+F0iC4Q53ZGXYzcd6AWBi0i+Op717zE4nENbeKVEUlEAsaTjehBdBGmyoxz7tKDJd3UNywagkL4ZbzObCC5o8HC56JoxcXH6ssbIJ+e1GMI+nnIR3kADa5/GmAERCSlixJZpPFjA8YujmmlXxt9BCboCUZ0zxCiAgMvnvCg2VrbtWvclPKfMeGME+YBUlfU/xooilQ2hhbPcPfjDEumcGfaEqC1MFA2pvA50uAYxUGA1HbBvZzb0JzESO3KbMOOrcpMR/IRVLYg5AvzdLkxB6uSkD3X1HwsmCr23B1BCLD6sTnqCDhlnDElqoM6Mi8boIQ889iQx7kTtO0RSv4RxpLbEoP3Sq3Lr0g3yLmtcWTg6wv1Q4yKOrOUL7HqFp1TkkonhSUApjGXDg6DCLqLE5PpdyX6z0aYH5D86Xdx6166cw2tR9mLV8z3B8QbF3AoO8X1m/SbVJ/9qVwYkpsKnBFONQGqSA9qqq95GSy1QB4Y2IPl40r48RE4hfyeGyNZchznnDi9DdeOHac5PP5Ew9IynpLZfidhxkipxbrPAssHub280XBYvNeDttz1zJuF9dzmPCpytsoBa97gydxa9goG052DsMb5bEFLJIt5cKQcL/rEOVGtqbv+Hau6ujTpZGSelScePzvSxy4pvtyjCbGkfW4xEEoiKHMlmBK4aCmnJUF8ZRdd+LHbs7aq5ayNzNBpJ5UCQ9CF8zYRlv4ZEkEoGIFBQBC8UB4YnoBuC4oKBbtSXbjZ848Q4p3h4hFXjsKxay1cOALR8b5BTjtw5JuMoBulGZyEoJIMOiMYBxCbnf/5UAi0/dMROygGOiMOKovIpdbHqJF4PEYmYRHmTA1BNqWYC5SdSYR+Eo4BoL0BXKIH+k6z/DFi9C5iWFHL6X/L16FkjnNTuok5AEt1jk2LPm93aBxc2C0Vw3KPRFQQ1TWDCin3SoHXj7lCULMNiOBfqOMFed+N2Glv9aFtAjGfKPmkmzgGDlNrYdD6vG04F3uSwcU7OViXkEVrrKmMewoe3dV6dQBoh7Lf0gyAtzKjfYYcFYHZ1P55PeCDJq9u0jp3vkE3GkaqqSHOkMTrQ75/+6Knknyzs3vDfbw3XMgxytnYc/uFYRHG8+ZAfKPZmefw2eez7RM2H3VbJREjg0lVujluTK4wT+L3R3xks/LxhaH5vfs2OZOfFe6Yo3ymHkK0ROFOwK3vrDUKyCf+Pz4ouUpQjtwZs37UARQ1YHBOp4AaUkXF4lWoJ8lvWZ0IgF4tNBQbvYGdbwFmyAgK9CGpDIFLAq/gXsvLRvDaKTwAt04XjUESQW27ZDiYzTcBZFebbbiOExbfnZ3CFqiZi97jK2l21c3QcVZ84t6mYy9bsnO1ruev7pOMZClJJhrIK7h9whKS5VSF282yV4HHJzyvaVLzR3jixzuYHdO0plm/SMxGyS2a5EpAi2wzvIn4KfyEa2p461yjqD2zYD5FN+WeWBlOPsKH7/2e5WEfxSBtYzfYlN/jHcO/IS/UMnkXMN/+pDkwzsXlEGrPvbkyAc8AjPIEEgCv7clS7tYDXTJtPRShAS0p+L2IgSSmIwoDZe91UHaQP6MIXvH5tJAHkFndPlPZ96Q4zxerik9ezRnbCadenI8Z/6ydtDSR0LwTIwxkTS8G1zPnnBMIF3QdWSFQaQ7Bs5kxZ+BZ02M6PD6BbKwx1pgDv6HvJtblPRcWd92LHrJWeJ0sKsUkZN4SgEFM0C/XawF19HnCszCInJvkza2oRx6z1uzfjQU6DBw0cBPFP4wZ15ejX/k5he4j/1lm43CanOva+fCS1PgqEwS3jCxKwmlHr87HhVCen3FMWjOxY46YS11ot6fFwVdwMVQ9+nKcfWKUSzHxKaLQhMud8Vt0JPYXCAMyTQ2W5jF5eTHliASZ+jkga4/Z0kh7zkUrnREafrr2RH3vKfhDvuiWChXG64+Vzu4jE6vPQUsXaGnsPnAVcjfpy7dN00LXTIxgKYJQkWFotT9EZo0z9en8E+bD7uXOHLBkNDUfY20Bx1cvBDUsdfz6B1x73QTR5gg5gJDp5OeaEbUrRZ+8/YLhabfR4GOmbAyBF4ci1rac4akZoSOPMy7GBBn+3gTjCR54UIJAUAok6EA9kNv381nDZYcojk4LZpYcGEddW17pPtI5IKIBsphb1GG7Wa6lIGR4DmGBI3AiX2LWgWuuGpgF4leHsA3iOQE9IIxQ4QNeAOgKg/slmtTOBh2cjmO+hH2SMMbAvJmw7MGClfbJ9mO1GIbq6+9FyNucUH9TcI8W+DAzhGjnWohG0wPtfBj0Ohdr5J7cswq3uNSunak4+OAlVFK9CEoyKaz6gDCjQSpU4WcSOChRzHEgRE7ZunthDEgchIHgzJOEvpSROeWB822kisF8gRxpHyHeq+2jTpTKOx48dtbXXO2BAZPL8GyUAzV9eiQ/TfpdOKjIjIg36zHMNWtZysFSwiE2zLua3V0lZOUGmwxR2X0xGwRc5+RqAX/E0F1pzjkVHdUDq6WuqJ9O1oTR5gmXe0rrEjUIQQz7ADDxuDBRna/2JLvcICfyqPD/dKuLUzXRQ0xsv8eRDdN7JTZMTsDBm7zINvmTGFCyZm2aZ8Jsuc0sC45QPI3hOA9PYW8tqS3YB4Er8YzXKabSisnQaoLeA2j0QP09kMa0xY4Y19Eq/XYKlzuNKs0gCbRyTHgKTPAZZOiuYEaZE/ZhwmcfarwUJhRIXHjW4012b5MTCjv0LVGEzXwKOcgoHUR+LJrwaE1UgxYzFcLZ5G1DLhqOx2CKqEih1vKkHxHiV2jKTpWkWNKz0XeB7yYb3musewGo1NhI+GEWKmDX6UZzXGWyuP6BTpcgEuK2gh0tE3TiNMQE9dNesPOC4vlNASghNXy/WuFo2Wp5Ae1aS0cJpZkeg9eZ1pcsycaCoaWoqmDeBOYyvPxjgCv8pA9KscLIDQdLFXl59UQZ8Ov5NZn2wh1f3/mYUBKFLS8CR8pcx1fTudjbjzVXVlQ5x3tJWcP5PmGC9m8kH1MDYAmPSI0DY5WqUTU0ycOi2MlswGFGemHSwGB+M2EN6N0RiLRhSS4mqUcLMBCB9AnixQzzfsKtNS8NDtD0uIa9fkkGFCpfQwY0F8LcCKtPRCOPOMJEoknascwjSlFdJt8VEByBVbcSYj7QjZTvmI8yfiUsuK4KNQhGd4s9pt60RfQg5SZ+DuWql1Kx9Bh2L5TvQ69GmmDeYrq/4jnuUC8ZDR+myQZWP0YUC//5InC9QAEv+cWUffYyEhYvRDdX8pCwPDDvzUjPQXq7daVlHRvthTXHzO4lGn12gjhSiqs8FfJwYK6P/1cUbdD7aNKbEbuFJa2A3BdBgmOynxgUnFNTfDQoMGbcQi5JE4hRrYNlr/TqYm01AlRydqTLRKJPir19e1Jlkq0S4elyNzgQDDKC0QJ0ZlIZg7BpNoPOKUVNdQ0WZMRU4dU/0OG9mFy/pNXn9cnUUqaN1FnBEZFoCpvhXxxoPsuUrG0cUMF2cjLR+zQ/D5C4RWPBWabylM7ZsDbTbgN7VxHhzcElsoTVgfh4u+fXGwRuBPAyaqylowpFelEQMqaYYaAvOEdLGEo2Z5mVkgrlYNOKtjUpoELGBJ96Dkq4EodjA3hw/a04fgf+O/ybAnFYE0KvG1FU5PXchxFIkHr1YU7kR4t5BZ0J5jMUh3HR5vGA8zq0lVhcJszOSLWRKVnVwY8PvQedrXHgcnhReylrHlgmoOtZTtTceoZGnh1oQIUdwLROxTOixeaBRjV1o52TxBCDet540sz6fHNTv/AGqQgr2CQYVPBAEXhEGGhayh4zusQCTPsvMhAQEGQ3CBMElQ8HFrAltd7Ns8k4ldXkiqtFK778AuPR9Jzvs8NT0/pxJM0ncPmbO+diEHIbWvQJkzna5bNsQnGEePOeTlqtS8u3pYSPfRv6JQEf73nNdohMO0b3s3+Hf8N8tNS8yLsTbxkFR1QAI/oSs9O8kzda984t2fyArzn3Tn9nQ147N5lxQC4mZq6wfa1DBvYkR7LISyVbhM/+9RDryCG9Q305Da6NgAKqkWbOxwfU/+u5YNhDE7rl2Eo9culv4ZB8lo/4FOh9AM5xgFROhOiGDrkip5DfcB0b9DibsR5aA7cYXd1iZB3yWo9A/7+rjIH/51acsxUqWwFWmRKZaEbUwjwuFdf54Td88DUCY9ozFw8TiBzaJ0GUU/Nl9eaOEVBZoi2gtmp+3M3ULiwDREUCqU9xswzvQuQMZskQ4DV0xbqWfYQLhbPMqIdtmodOdywsNZgDKFju64Nl4i1U2GZTrQjVp4k1ILzBngLnhnlA+Q9wGGkRSZ5H+5aei61agv0/JM8k5DMXL1qWM5u6phjfpl0jYdzM4jjwDSLWYawYCT0yXa3KXEk2CIfzGYPxOUMavkhe/RgJFO5MvwOFrLuEX9GBd9XEFt1ymZHbBmwoWArZETiUu1HG7TGyVnqpxgdLTSm6u9ZH/wza4pI2igm4mlQwJT4Pz8PTOMrHjtMR61ORFVXnd0h8EK/zHgc1VmYyDZHbKnjPCtxBO1MXkzfYUY9ql50bp3BuGx0nTTacG+IAbpesp8SQsNOEzr1VDZ131DWoQym3lnC1weKbSdpPkt+ke1YPqTTYvUrjYah70pWqsKQfiE0YGAb8mfmECxGwlzVMQu6aclMe3h0N2jk4cgeuyIJYZ0D4Y5SR/oykz2Kr7+6IkRJdwWdq071IBXov2kfJ/HAZRBisIogWyHz00Ud0XAVx5PueAzVKuJIZyZcXq0MJXc0+vnUV2CHfdmAMxVsPpQsCqhKrbsGb7+ZzoAUxIGqyo3QIMCW7wqTQx0AHgdPlzJJBpWebK9/GH6fCtyVCYygghBsgChgJT13cUOyV9EfOz0R7zaCcJBG227IbHUuTDaq1EfxwLZoVKppB6k6hyhJ96GPFNOCR75qIc9LhcgkI2NlyEQOfk5QJlyNHgQbyECPt4XB+x0MotcKmxPqsxwbJEMNoy+jvpYZqJo6oiEg6vCzIuMH5rf9XVsHg/50zYTNa7P3KNp/QP1c8QszQMdZHnyLepkdYd2qo3zuufXReeriqUAq5XTNT+EljDzPw1evoYdQvFof9bmdTqNgr1xSFGwCNxSI4L20ykaJh5xtt7cWlavJdEh99k5WVWhN2SXBWM3mIVhCxX4q7bW1y8Vp72s6+RSnfcMPTsF6KgJFx95x1a4NfaiJ8zuF6MJAH+gm0aVP6IKEU9rlyD/2jJlwFZhDhNYicAZsB5zPcjVdo9c95qrfPgUmZ7ar3jCDWWasDkYOlcA1TWihuhHaFsSSvw66CiW9mMVbVt8CG/7/9tbkgIT49NxNR3Ygx7pbi0u3cpRE6xHrJf+8yT95c2KBztRA50jSMdKANaJzIz0JfBxs1FOlR6JEE+hV0JTVFh3V65WG5BYOQQP0HErFSQEYATEFmxPi20PJOPw+rfXJ96CJ6Udi+CrQXzZRtOx2Dr7C6kJVHzW47gAYvQ8JNeTT+S3m+VoyGDqIA2rZY/HZMm5SZsSpOKwhN/68MqVqQDPJqsMNEW9rLDSFABIqyqeLtWxXX77XcltBm0Uj6y2hF7HZWilUbA431toyc+OKrt/9R4BOps262Yah+vTHxzp7Zneua5u1qxvJa4ewZmnd16OxnJ9RaQAHudPzb/mNaLiPvLqg/pDmVk/yKQBosH0Za6SKEvHgBKALYJcg/COUOeF1yJc846NVzQ92iLSk5tYBx9XoYqf2/uDlyjFLj9J8EZWzk2rTNhLp/o03rTDggDSddFtLGHjM5D/D25EZMyI3YnOsZFlOXeBq8qJ5tM2N+AxVH4pKjUUiAaLPt+ggtdkWtGsn2/6W9yfIxYQ2TmsL2EtvtpY7IuaY1e9VPkiEH84D2zoPQ3fWqU1oZxujH8Rd5Cm4Lh1tvbiSH1vqbEYgna7z4LFPBzFYKbEYMN4s4b8yLv7+ZgIr5spIf3QKElhdZ6lJZqg4KOMkWop66zDnFvs53hAyIYbvRXl/wl3TCE+h8dihzYPr26K+rGoK1EUHo9axGV7ZN9wZkkEk6QbaIjL6Wa1qHokSeQpr3Tw4lRZL6IK5OOHU2JACUSR9LDaHJAOvNFoDKjCRyo/jeRhDXQo9p3oKl5bLO6Trpx1gOhV3atsjyS4EvSorBm4rr7RNE+6Xok1GzpjXXli17EQgMqpSum1xSSkdpUw3aF9vM9ONlrr7lnLynP4zLaIuV7KQ9SXYTz6UtPgqDCLWVgEb2MyTdGx1R257ZHTXa4lD4D1iyMehT6P312++QvFo0v1SUhBxxo4ZN6Hyu8HAtNHR9BjKG5urrx9G3Jq0vt/3tyyQQbadY3gRNp32Zmcuv2e/1Q7x73eiGUsFdqqxyzI+wxhq7Q3YL9Xza3YNLxAkf7FAgFjpxvo2Ybcok1dtnb7BuKYhnYrgHsnl3a1mOSF3zNGm4t63P0QIAocSe/TO9YNvP6SFG2oaQhSW9OGLIYrxoYHXlLMxWUK6zgyz3XGoMk9IIY/fTFCjuO/BKDf+1OL62CHxV6ngDLUNWndbHXYUEzGHG4DKMtKgxypDNyZNGrNoNd42SGhzW9ESI8NVps/8ltqm/stxE3NQxBJsLNfjiWYGtgN3SwYmSiwhBF9smRWtKQJnrUQJ4njSRxTmqlaPFM8g76X4MOUSIYJW3tqdXtWPTK4NSijYds91kpgxHguCnINAT5fZWTZcwoAIOyoK73JVzpoEuRhgK83+9E80xh6pIVEE9YC6t90aVJqAjuTb6YIgTfcgErJGVg+E5i+AcybzsdYpflJdv8mb/aaY0NoZgm87FZUptZUWl726ZinLURFBNnvU5USqehtiGstshmGTDDlwosBQJqlMI9lEtbFmDoLgAW7GkwHO4IrNag/jTPaRs481ctlF8aDxuvgdemzjH7MeuRL7yYVgASTmz1prlpU1Q0Rok9H/ZPYoWu0avgvbslaO7sMaDAd6U0Pbuif/Kp7GZVrAmJmR3hiAUVw3NPNP7jUdBi/OlF8VUyXKGIJWIbA85navJxqInZE7fOM46DZj5dednc5O2gNNjIGoqCH5my0EazkMg2Y9f/8pk79fuSfQB46tTjS/z/ekTp73s/t0Jma95f38Kat0vfv4xg9MnU/jGqZLhEXaSq6J+lDI+phLmt83QhjaFSsddasDH0WS5rmAzIZTZVJv6X0k2+/o2NCL4Fjlhd7GO3694y9lDlMyBh5q6oYoZn1WKf3sL6TCJbhoXYub6JXFFwSuv115XyeI428hZ21cFXczaa4bgdYH0i8/Hp8hgu1Ph3g3LxasXVADX/E7GcE1xFc7RWTbADT2W2Ax9MKnPQci9ZnwgRfbsP3rDZg+Mrdom4yD0S5gO4kUx4udZjo0LR5ivxR6sx/Np01c3l9xFsp+/K3Bo4yCOUdMOELxl1gpcN4zBD54fQ2uFSlPqT8DOMXFVIr5m34hnZHmsia0zxXqz54Zuz+mCGKTUQyfFEieKqgl2qzwzvKoNdxVSgyHaDWBmkB9UJKmP5YkoAelnTYuiTJqt6rsKWegqABBtO/pC24/pGpg+nDRJQqHWu/rDgb/j8y53skHr5nugLyqWOQQtdYaEtSnHSFf8PLCRxgbwhIFn4g9av1f7vrBxx1ZlCQeZk4DbHWZOD+rOUxAVlJGq2w7cXWJ4CDMG1KsQqNAdqNEEtTvqpOnQJiX7N2EUqTaY1Vjwx/6uN+xbTv2O1VmXnkVyJimVzO3qXnIBKpcZRC5APQaAmTpHoNi03IiqSB3EahZ7dqHY3mleI4Ci5o4141E2acLGaMDHgAOq/zLpGthZ2IInCzNgYBsbMLDZvfx2vk3QsYwWVHHCRR0hjjFSY5HvTI4kYG7xHtUFkoL1q9Q15xDWixFyVncoyG79ZP0uDXt6ae+0H8Owci/lVREpsgEEWZcHA5R+i15fDUMZn0aXXQMrefob+nJL2U4fzPRoB2N8kQuuAxtz3fqvKZ65lcvcuJllp08dunDn5MFqzTl/lI9N/jXFyRMHLt07fZjvu3U7n715I3siA6AguipezwbfdA6od8FC7CMs3Zgn/vVFQ+l/EVZ019uIJvlaX5P1QfykHELy83rJFwM3Aj5tVdKH/ze+OKAxrFpbPLlGftS9sUuyrBSrBJo0L+gWI0qQG+D2IiJdDySYrZstgQ8QrmyPSohiKK3IENE1hsSRY6Smgoy+JHeIU3WX/iECvYK5QH00beEM4XV0ZJEyIz6I6e0hr1mRUeQZWzfF4Y1xDbkRC1zxhWPhWqIo5zrfV9YURtXdfZeg2U1nYltw+NiZ3BGLDtJR1Y2ZElgeOwYhpkLKWhhJHTeu3aBDfLS1wN5AjM0vvceteivXtqY/3gLjE6RTsba3KTT7csKw4OeMmzTrR/ZdcxItCr0t+sRTzJ7xJRGMK9wTZ9Wf2fpXvrTnFxPb2l85LkmEshzFsELtyTnITQDyAPGZaVsjAO93JkAyybWVxa3wYUE8M8NIGu+yKAS2eN+dMfM0DiOBwdnxta3D9zESwNqLMOLnNSsXzVwQo8JD3aKhTdTqqWNwKlrQqeyx/4v9B9og3VMjIRpL6RRrldouXktMvVKP6jO9iqDSmpHkt8RXbMZqkkgCEeD7FCeHPwNxJ7ZcA3HDs/9NFWd/5b0Iy1tL2DLyPTsaOCBcv59fBrI9Lm1OViZn4G9JVW7nD1OsrPergIH+fxuj0XgdbfuhJRlHMwNppTVAo46FcdBTii6nLNZhQjXmg7JHjqehi2Qs9WhdM8+6yAPCu5c7B6J6Yfc88pNibqyo8jwy7bfHYJ/0hV62LWXNtRNC/Nv1NS7eD4l02jvEY2xJ9199cSU3lm9FMPhV2jUvx0kJRVo9wCTgv2Z+keDyRcjQNwocSuA3Lflioh050Tb8FwvoMNJyTtod1ebDNhpk9Ndhwkx2/LhPuFC3NBiyFneA7CrVNrM2hVNZrZdiLKP2Cchyhud/vI00R8SnzFOEjcolIsePrDEaWiMCQzG/+wekOS6GHqlBiDwMdRwdfSOmKSyGd2anlLKXQOwx+T6/PoA4UBlLbB5xe5j7owjzW1WwYtk3cKrefwSDiprw2xEyFxje51NGnbI8dYt5wmC26cILZIY+PO5USNp1VxCLCZAh7lsI4Xy3XjRk41xy7wzpQBa8oJBDuavGZ/fxITQPUgTY5fMNqFpTLZfG2ASxyEN7nguOar/gSvLKuAcog6NftOzjo8ApW2wkw+HUjRBSWBvIZK+gMdArLtbqLwPkf1JuzUrNeZ+1R01yqPac7/7xXRHiSbOcQ7FxvbW+0JBpivqW3HPHQ11bP+oYjZKBuW/NJ5DUIB1NFQAs+o9Ohrsr5IJWYaXwPIuDn5pSypdBo6nrQL/paxbl//7qXoBn79GO54Az9PlT6CPjRL+ZbczIQEO95gxesVp43Tbe2qlu609/AXweG1zwKz7S9F2sGanxlKQugqkEU6PBWFPuc8OBbLy/gBHh/JHTOPj+2udfhBtUI2oo4AnB9G51EABLJpJPxQV/6WZUC5vROU2Qce2icVNJqLvDAiCPNPyhvvWJGYs7bdPqoWZqVyNm86yJL2lOYXGuLCNnk71LVzmmoDAlvejaeAcAZs6qr+aFVQgQXiyN63v+kCu9iCQFAASmUdfoF4yodshe8rJ3BSOnlCz4SJx2KFTVYOFgQDXJyLCYNljrvMgr0MzS2jagPg91q9B9G1NJ+zkKMEfNROfAsxaXpsBLSASp+TN913f5pmmXNgrJSv2B+ldV+7vaGcMvNBdAF0lybmdu7g433xmWdHy3geh9kWceVGlLRAxZS2+0Cx4LbNd7gc2p97IHCq6y3J5HFG7hWXlez5PXQw3uY6MMX7TBtsWvao0UXOwJCyiVcHpzZeB41oDU/RfLrkpndwl5fs4tEm4jpMMmZy7vG55Cdi1xGctO23eO/TKmrS+L7VyFLjlsgLwMQcWD6anmssZuCgV1V5R0lYLOIPGmqq/Lfvvq/NwycyQew6sJQ3NY9vBzuICoe223ETEitJ/2zTjX/hUwX0V4+0u7+uMyheviP/q1FnOxrw41/4KXXyPnCdZTCsRCY5jw50YWrqLvoR48s7PMdudIjLyxA4jDhW0EDDWPAVDglRNdo/NaRQJ9mkceO4Iw89Bp9QpidmTIMA4NNYaSmUNov63kreu5zRTqOq9DWxzStDnt4ZkHoMQOe2DWda0PpIWmhEgHZdjjKmGmE0gZ+Mt38a/pmYdW+axw5DwCI6+/Txuwqdxn9Nw4UrmP7AsPZ2yNgartOHjd83a4/7ycmwefwPWvVXdq3gOsF7bwevCks0H9x74d1TkgfS4yOzzkqfoaWL7F+1Gg1ZZHXd1Pr3zqZb/rKkL/KyhsN/edrBB0nTeZugoWeXhYblrEzv9npkv4E2BCeIo27kQ5nUVSYPjoEfhv6Lu50ZE7CNxxYaxosyN97tRpdF3QgLGmSrqkQHJytKj/Rh3oHBwjij3THSscfh6ahuFDP0h14ONquTYEQxj5w+HFoxssaL50Tk6xV8322vuXmMofPWnPruuGQTLK9aGTyS0IYTBaxNA2cA8DxuraTzN0srPcJ9MmfYiiDdh5B69FJmr8DrC2DmLYEWM7cEi57jBt4zohXTCRagvXJLEQGZmFhHitKBeJxGtMuquDwikZR580P132jiZnYrij505lZ+bdsk36LBX8jGu8KWRtK1RkwXZkZfGDZhCfuuFO7fsm8KsRUkX+BZcj4tRxxF4Fn0LDHPmOoldfYzr0Gb6N/DaRyqm+BmwFFVQw1xxlniOf+RjGw0xyl17FStuTWdewcrY17DCkU+AnR8cf3WNe78zwF0DGgxM6/+4IjZdNUXeF7A8cLg3aNw6jkWhV5p15Nvay+gzLDe9Hk8m/bqJ9NQPfCdidFzC9oyQ978Yr0v6smItqrXix/lOzcbDvdgHHD+svo23TCifxYQWvzJi6Gh7E3jbRXDn99+JeTNhL45XlKLJP1NIV0YXPqkeTBfR/SjwmQd2u3eVq+yrijUzJh+cMDPvGH/QT04Y6SInKG91yy08FetXTU3HpSMxPFlkd2+xd9a/2QAB5DuD10cJnP2k+SL8n9Gre8ZVJW6tr8giUtiwcykSHFLdP6HhA+HUlvPJmixwT3mKxSgy16nPTMmpsVHbzeVOdE8tlHWYASbpXkHfmZEzEoiiho1k2dgnfiOt3u7SqfR/sW7/y4Lc8yAxmd17wZyaqhIJ3WbV6l9InvQyoDBYiGVTZZDWXWll6SOvr26Z591BQDJ6V9fw4XgkjPUsmXzpxLyJ0LbePB64p+LTtcqGn1rcBn9D136aKxakyyFO7hEXtP0BgiWKQ8mZ34CnMmxDeeZU9MlPc7XdHpQeGY0utfZbzLZmRfDHOxGKpDjLDymUNpyqaINM9FTnKGnBIJSpxKN4NHoEJOWM61jxD9Epic7dI0aSJpV5ipRE8mahQkStAcTfvsftSyjBhyeYD0dgB95CMtSpeCriLSKa7DOy3Q37zih+2kzN+k73tdcB5JRPVYyu436nuSmfwZZqepjMzrikbI/Jq7iVBTQrYxwirErf2MBo/cUtQ3o0nX6u+UJEWP9LPoD9YSVNVoVYfK2nrEjkxLXj+qv/3LNR5KuTzGKRVmVHiJzFKaW3dKwYFkOJghhX8raHTVXWtykekP2NGyMsvcziug7H92X5C4b2uUoh7kbBOPDxSfxMo1FmA25DEwVfH3pi5Dt1qlzQOpVzZp6CIkyX9Tgo8m1c1/V6gMTCu0yRc9MqJexFhgyeDzoCLt17JebarC7RgOcMYkfFjFUcMrkfRKJrkXfsoDIdhl7Ddh8rn3W6ZPURRKJVCReeiknm2GGYSgSGV8eBoClUzUb5feDa22kCGxg+6Owyhz+Sg0R8FC5mh47Mv16Jg0WG73o4pPQpuJI8BHQVsZ22VZGhNp0aw3KAwVbO0F0vRz5kykn33vkvInZRL7iLwFo1oQodyM/AsGLAcC0FzfekMDrnadpZxpjvitGjiWI4A/Kb/OhRFwYiZfP0ueoQ1dO8cofwjv8v8s+G537aP5aaC29uIPeD86ds7Ub7AxRfFmy9jevpnvUMJjoLyBr1Pt9ehzvMB5fgokZNQUnL3bUQ/8VHkGKR9Feu8lywa1agLvya5DkO2gmKkjinAOw79OJZGrqvaCZm4/ohSAo1cFiBPzPFcTr1rDhNsT80Hx8AaK7imu5I2EvMqPsu4NC7b7Et92stokjHop21e497IPsAhhyHLLuABRCH3rsxbeznbpN+L5IRx9kZFjkCRQRkyU6UUomBlmuEQtdW1Uc+oWalJ3dPVvzrmNd/bH7xQPETQxZ+33ZryxY/gFS+OW6628bITLxoffvTAjfG3L+Ml46XXOydt6MLz4fVLwBc9tfie5hsXyLJ+8cXI2sXTmFbuvK9+EfVWFP/aLk3GPK5sSLxAoS1aDbuFsVGxAnjxrkLPY7Rs6gWCsDSm1EdvJeW/Fw4XgYuMgJXm64ucBA+4CoecUXOr+xaOWaOWjq0wd7iS5pLOqUCNI70KHMjT8kqhbUyDYIXblAG/yAWG4oInBdxA6+6hjOtZbiKaqIIna+Rb1+56Rr4lMAHjW2/EBqMlNAbS2T7glerDUlyXCVdQgY+KxNGDzMjalmUmSq3RuM6jYqC2+ngfWIxUFYEYf8sZnsRm0EDSrNuCOc6iWUH8HJfCGoh2m0zEJawiyy924ljWzfuO52v19O/XztF4U81PzVshHX3y9ITUnv4J5QhE00gG/mKSTiCRN4jGdUvbrU4awqoCFMtJ3kucmVJTIASNsj7c9TFyD75lkUTdTNuZDZ+wEbu3Sp7k166ZPd1L3RkxmPCC8rinuJjSuxv8mWsXo091k/fXKv9e0ndfWWZrdkmRK4o9rm7FyKOSXJM2FySQWtKl5nTIRZ9f9+lj/s0Yf0Lzq/K7dFkzWqZc/3LOt9nhlOTvr6gYS3LPxXBJjG5CE5rBGb+8GiapQdI6QfUYUZxwqzCqYrghaslC8Agm+iS4kYKpg1ZlbRELF3WbI6sFgDkmIZynKWcdIBKv6e8JOFivr6r59cCMCDPn4f/gAACviSe9o8zxjZiP2/Qv2/RT+mqTiFG+A4mrUaxo9pd45JjBc/mzZ+woyXrG75gBdw+MkKFXsOnL4Sf3R6Ig/a481L3yn1Db+U1Su0ndzbY3jC48euzy9vWZLw9YhMadTgBjg8cYM8HxGdhGZ9nCDkSL8UzOn5ie3eBEtAWSQTs5X27Ys2EKYV6lKzo68ci0BN4N3wcOyaZkbVEPCqXmxaozClvQmDi0HxomPwUI0t/EU8hMRZ5m5FSUhsey4rVahxK7CQrNBgvRFXWajootmCjXzI16kYd+1P4WPqyjvzrCC8A07Cmb0wLqKhaDjxDj2TB9NS5ZlzwmkwjLFDspW8HjIvKXoM41/Ac+/mpF5D33fuG6XZ58Ouh+JKT8ooe8OMTRhD+40sLq+A9WGVcL17kXfedX+sX18JeZd3rAI+QAWftgE2fu1j6sH/fW2A437pUJ2cN2Of3Tt8ja2AezHJrRZRFEal2j5/radfu2r2DoSAnKUtWCwo+Fc5amgmBsms1TaTdOh1E1lTixVy4a3nSzEYS7gPcKR2ejvjTfaPhvKQuCYIFF/h1TAndtKqBHU8xP5iXNdUxeqxvYvvFpG4cJuCQoU2cn/Aq+abbyTKiBXiW8LXwjfedLpPOHjUg4iCnjgIOJoa/DaqI+Ku/fH0r4uLCg3AabIxSHRwfILK+DXzwBL0QiGCtVDR9w2QWa9/f04t0Tm1qT+R8Kp95gvmvQ3SKkzgqu+3jbDNvcRHUgtbe5tQ4usn2cvLaXeyWYR3vROBeBcksNyX2FDb+xfDt73Ege5PiY6rhGZJP9PoLw/2tVcizlUWV87aawJ7hPVyTO5VS2cmc5yXoFcUD7TFZ7m1iKTtGe1mQuHLhpXkSbJpFknw7/S6V0CmEjsOf9aa8MvQKUuE1RmhlkUVlAT2PAec+w6ebdUVRX8kSNoAhIx4IaPKPJliFUJpwoxICrluOd4TWfSCPbd1cKJZoiiNmvndL0axUbxRss7TIPx4ErrbFzdWqdwzcmv6DPfjROpayCrU/sRvPb49Ku2OfIT0flK/cXXHFtcUvb5khI91x7iXMhufINLSWTepmlteUBrtEKMcUBG7qEjM8cfOHM4Rf4cItqhfys7Wmu+GatawAZNXM/g1SxF5txx47qovAhTez1Zrzo++xAiDuIJmh1TnpztB1jsJMkaEytAyUB4at5qbcoDNFtsygl0jrGWDScCE0i+S3ZKyTW/XJzklSZ3RbdAWFxVOcb6nL9Ymr4Gatz7nxSMK5iWAaZ+ROXz4qUQ1nbs/+byJeRWyzNpB6hlcDQ2UsnbjXV5qdNdWBGg7sgH5u/Ar+9IpgpgPSFRD7+s/ZijSxelRF6JU5sXmp3Ozk95OO0xsgUS33dMb8vL6oUBW2C2EWYz7q+s/l1x3e2zLfND+xdZ83NeKEXP5YUSjqffQ4Qti0A2DXhCdOrRK5tJXeR4C1vVxshWzsWeQ0e1VL1X0lw/j1Sl3FvniT5PxIdYmxcaW/l5dEcb53gFZNxn6Q0vRFoux8gpe6n0fJg0V3qH9KxOIJRfvtL9/2v7Q+vSwrDSp4CEqBfOFAPDWbh9diSs2TxIGwP33EbTUYzHHCe943ft15CjsqftY0gGrk0jLwQFrVBGbliC+kv3+Yx6JG2FRMHrY3LDPVIxDHiZTAiVEu6PXL6KHs1dB1ifzgP0hs+0/BhU+PmODkdl5cRRmNQjBmblgrYPRL5BkXcnO5oMRwii1vP2UJiuUAJnWM1QT+YcrPtHz+QaEfBs8U/MGs4VsG1vQJ1PMYhlxRgevUHP5QSt8Nuf/zwCcFMbKPe8mFmAOPuqY6+YK5+6oh5R60w4VWzZSGa0Aalx8GEwGeUutMu9+kLqqAHHn9CBto7rzCx79ISj8gJM3xR0kvKpXnea9LcjYcxhqQoRHHaUTqhhCDpQzk5UvBOjI3BZC+9e5dO4z9zekgYuAzlUL2cd7ZqLyiPC3gSngLzDrYG5o0/kA9z2kcZEjmllvIU5v0ZluThSSoKSCQRCevaPh7fYnvNHUzwx4yYtt1Cnrd9DY86RMW7Wq4Pc4Bc+hrXnqZTL3Ku/5QxGX1gMUQe1zDnOUAUFaPlaYT54DwOOknSbwP81vuxh+Dsv5PWcR5BY4qJ7g/BXaXgCpFzDJlzkQcp3c4iQqphpoergh0zw5lAAFMKwifQVfKNS9oO6owNUlitSCJqWU0xSvIUyyiorPP1oTx4/RxoqMQUhCy5B/Qj7Jw4id46JiyBN287fqoZUwMks9bcHLhmh3IrOEY6iLX4XNOa55NrK11o92JosDXknYUsjOSIFX/DeGColHRci0yEZh1Z1BZhNxkenoQQU+MP8WLwqZtI/aVSRwRiKgjFDRHPqm8ytpqzFjs2n9aRuBJJpM2UB8oca5Go5Tcg0l/OZ6HXerDJldozgqQa7wn+i7Nx52n/fNy2MGR9JkGcQg0Nq7nZ74SYAYjdtrJBHL10xb1hQaxO7cwtRrXrMSEbqhA8VUlSYKAZ7unlZaz9dhntSvQGIMzezGtWWxDW6sMT096XJ6Lh/alxiCv15SEL4a4VyTFgNskDYFZx1eVyVq6wn7i47ooUn5kODTYES7BYnmmBQ3voigKKMAhitFxVhiyc5aFVgBbbZJQmKmC3KsUqnH3WmwJe9V7hOqHPt0Au21/IEHa1fT1xeneWmsM1NWmikAXzdgrCz5ac7UIhjJ6DtjhIEmRnzcYuaTWewgC7IoX6bDYXbpUiEja4Lzn+gFMGE8JGLuVj7m0G6BM+SXbCdTfITlnOaNY7ndcs6kV6xEgFQ2YMlqdf5jsz3shcquXKr8smhC12YdRKlIjluCu4S6zuArGayVbo5ZFOJWRfSok8mbJqT1Uxd1TrC8wpM3C0S2pNL89ulNTaTs3zY6uGy6iWEqaYcvZyLUmHodJYHK6/yefDpsmNJymc772pbUsMd2dY8oSB4+RuwhknsPD8bniIcC66luNyDJYBg6pwLQly0JMH6EADQ/gbE1z0/x0LOrz3U8WZGid24m2QIuEFN5KQq1Pm2QI6EpF5UpDQPF+r3Lk1bOY+0ZuVV1aoMGSW3VolbzChux/ATyiudhS1CjR8gR+hZ7hKwm3mE0sMKsk88mIVuiTmZGvBERlIDxd45wgrtweNN0eGpZyUrgLC2lVuVIXmwpsgs1Yy0GyA3DcTrHztwu3HjnY8nxGgKokGodROtXhMciA1tyV7gCjguKyPTKBhYm1sdAuPDCChooGeyKvF8qAU4sU3t3gYxv30eeWArRo/S/HZM8XRobFYsYFMvC6qup5RqdFp9PlbksJp36i0mdo+HEpxnG0VuK2BzsVLWJ/8QF8NV+B87aTV7tstTrRTFjRry3bk4OVFsjsenvieW+3bokxm6oambj8daedkrHqKrxghTXTmJorw9KzgI/BtEF1rrkE/uEGG3o8gMD6A+y5wTCawZ3dMQ586GLnL1DUToElwLUfGlEQ5founkdOaT0QiaET+7BwA94Z9Ae3QW6q7eAbI5bnnQVJ7he9DAS7T0O0oQzE28g65VRVTlilIOSOJqM+eMW5zr8BIm5pBfm7GvxRtZjbT77xfqelH0whT8W1xgxA4x0q+Z08Cg4YK/vaizZD8gqLYBAL5tQIgnSuLKDDGMC83587y0bLCZLDPYXXh8iRLbZvkMZ8jxlyv8aBebxjQtGA0rZjkRWRkXtCz5xUd8HrdEQ2Hb+zWjWlXHt6AeA3UtoIQtPWkPu0sdz8eRSfsKLsijtGxPKlUfGRxDNIo+SMzv+1VE6eFqHUtciJwsdFgeU84LXrJu9snvqZcJ6uq8SI1SfQmlQ2YXMfgzOuoiWCNUs6tVBRf34xcwKdUxUMT0DIlXzVAke5mHI5OVQsrz50D+b14W3NUC0ZaX86Wk4sxSpZ7aBZpNFpced18yvk1Ze9poL1T62xHlyskXucin1kOP0UMSmqq0pkJoEIGpKKdcJR9QrMkDc3HAAVOGIrOayGN25wR4GXyuseilsiOOG9KlrUIBuK6BL0kKfS1N6MaBe7CBz+N+MOJGzwFtLvc/q0jwC9B3R6GbtiHp+55ASVHqVyTM82iukjNNM0L6jwLkcldLUbyz0GboA4MBf6uvjZfbznzXS0ISlTTQs4vo+Iu0rWo9M/zdySgqjMUElXAGMEXIiGH4605fMP79/AtcvkguXHQE2GbIsIOIciTiNuI+nffIEjLWnoDGonDu7XDwB/l45OduWbXlzKR6s7cde2yQS+TAaySpfZGXe68u5jOuKtdI4uIWvN3kG/NMJlRrGJJHz5uQql1eruFYDu/6c+58+0txXgBlAbdXVIhqpssnA+TaB3xprF3dw/PVNELd28BvSMPhVtvH92WbIl+yxIMT45AyWwwhDZgt4zatO5p37y6QP+sHclGHN6JtYy1L2xdCcwbFwmiFzORSXN1lBByw8T+8LJaUMMdLJqMj+XBLnPKEzYvJBKV4HAMpR5vHfBHUj01xOEvxCYp5GisDJeo3YxYtLLmHMI/apbXMs2OmdKsNCuzonHNlF43GVFnC5xSbQ/RgiQSFUXIfY+p5IJSpEFZQbVBxuAIBkQwO5DuQGl687dkL+Iwr/n8PD/IewiUsEdpGA/XFduAeMJ023YSEuLV7yiiuf7f00Hfy168VKbkqjqBBLe0ExAQ7bTRXNBLoAF4eDLim8UahCl3b8UcL+IiPmj0RENoxQTL6w/tEgp53cq+1y/JRL/PZj2KxlLzy+O9/Qtv/T0JdvWLYTgQQwZcXpk2iphwbWFhSa+93t1TK89G1/7u/5k9qaU8BscYgrfF1oqQfGQ5KP+StA73L/RQk+5hS+s4KrCs9LnDfqxut3679pNHqWO4blbSZo3U679f/7mlQwHVDgVow+USaQdtkSndSMvEawUISTO2MtjRKMsYwTptcMHqA2nYft0jUYsnKjmdCgt4V6nSAlVGWQC2eXCPeAs5WxWMwKotasYE/ELPPGF2nmbv8DV7pjcrOcs25YoUP+hNfTWslvcFMjuKmrUjyWU6m1mGh7/M6i/0SWCJ+6fwxL3dgpHCVxON4hAOq6snO5S81B30a4+0X4ykuyQEdVX7zAGAG1deZyZtKYujuXkaiNNENjj5XLTpkWO4FV523URaekmBPc/Ltaw4gxs3GMhbENBdZdpNpAboGfPY6R+NTf+zqhpfLbeFwwp3s6g3JSwR00rD43m1SXHkTOAXKKQc1D8JDRBTsuxp8fQFXKhPo8mbq+HCSmej/dhfDaxP8K7qbqzYQIQJSCIONctl+JArilr9GypQw/RC+QLkUFTvJ5I8EYJsz2Sa8PEXQ9+NYQ4Rvp57epDoZT+ZzI+v8ZKdbn0m028tiyRuuviaXDqW15a9FRvpyDHS/9WO4zk4rG11V2EX2fTbmsElJzGoyGJuv8pr+shO7XS724YkTUSLaTqEDNMxCTLf1ui0uTzNDb8X2um4Ec+JFby4q1csV+SdA8xm8/6GbX7pM8MZNALnhx+KioGyZm5SNOtTlBfT4IGScnLuYI8LffAjNff5Y2oyW0z5DefybL419fPAyNfisabXqDQanc5OWlaXxnaAGYLs6nFnOGCye00mkT7JpmfsIs81jzj2uSJblgo4evcD9HgIvsZ3mrdyOA6CbL4TF8V3fOK9GF1vQdOTOnyAs5GXX+rlxbBcYaPLJIgNawp+l6mlRBXEcaYu5FsVsPswYrkUc5MdJadeEuSVjsJE0ITILwefmPxx3Y6YcADcETx/FQ8zH0dtd4Kr5hiYH3/t5GW48K37/iNpo8Uf6LvbQn/HvHszsvLSjdCIufetNak3WlQfj+pOoU0/WtFYMb9KWq7i16pNJslFnBsR6ecDpspllsGXJ0zO1hu87d0IgLFurKX4VFzUP/Nn+LS4eDcdQsR78ba6fj3Ix98eP1eCwV2gvQ4jxZqoTTYhS+sSpmQX07f1jD5V6KbbWCTru2OK29Xx7MzTWo265XejSkB8QFlLxpVAfcvJ7WXlvtSI1+zwWxobVY6eITWcgqEQcE/Jh3LNYrwmMePNCmTMNCRkTBc0Yds0pcNIBxVs0sSJOW7H9HQjPsOIAeRNhl5LNhZBf/qNOq1MwHECTl91D3IWormwNoRodUIHVoNbu2uAlhTZHo1wfUENO+l+H8TXoRLYdzmHDrB71AnLt4pM8v6VKV/9gD8HelXkJWGP4t5mpDXkq3ro7NU6QZbr5a2BGRkOpVekg2zILa9r+DiIJlGV8slejgamHb7amEF+t9YGw36ctUj2TXLjsG3kWogSTE4p9oDwx9EGcjZZeGjSHL8MxdRjnu0OgRmWcN1H0zW9rwo4FI5fFbOjjghNxJhGzQaWZn2QqQe2oMHQsBs2wMGhQI82lXex4I9GPkeGUQGgd8jBdn6zpdriOrlAG3mfEzvPlD11LNYHZdpNE1Mr6O7OA3sJZuqTMhvks9ICTpRRDFnXrmTy8XuuVl/XE87YfXP7cLCvocWrsea+D/FW4g33jWtkeyKxBsxUXno84R9OUisVz1Jq7sZhQN7fCXppg3g1hfzhZfcRzo7tDBpo5gHY2ZFdahleOh2PpBO/woygtANj0UxbVjHspC6PJCyA/C6p8CVY8nmQxJsQsqWc8U2QwesM+KCCjdw0l286nizqdBX/r33A4VSU2tYQdUhmjoPixwr14H0qat2gI45B3dyZqBS/rvduhQcytve+ntX+tf1hgFf0HrC7PAVpHvTL1wqY5TOE8d7JWKx0khWiQF5VuGU1Ly/7zFB7/0YANqvoGXsaOdHVizqW6RX8xF7uJWqPTzzfX7vUG8SVx8NheiP3DX119WK3H/Pd/3bOKXsQc1wFToIRxwwFOgm/IlFjcYoFn5XXHcukDsWmwiSxaaHTKljfRaZeJ6X6C4Ebh5YIfZ8A13g6q//1nQ8uRHB8JXt4OU4iy2P8k48cG+VNtzahHadb9TtIIuU4X1vjJWIg/D+KqzV69tkiGSEbHoa8k2rP1Msn5DRwGYugEYLEh9TQ1vDzxjtehTOZKnU6zI9Ghw1qNVPl8x+5j8U3ytreW+MMTSWbaDG3S6VLXu0WF5FGfGt7G9pQtZaBLSl1fjXx2+PrEytSnFwYhhvfc2/9uXI10Uud1qX1hcgV7FCWNipfeIerLaVzCdU2QSyDnKOxKdHme2SeTaWYlmw3eh8L/LDQpnVM554aULjfNO7E8o132nxI/QprX59nijfxs34Mdwo2aO0nXEZeSz016GzDPhFGKta2JWVSSN6jGgvzBI6yPRSOOliQWYzIFBVLeY5FNIkXPrGQuSkyBmptYjx4eMxlb4F7bBEmCAHB5IVmom5KwhcXIBp6xPdkRqiv72AJGMGVlRNPmMnY1IiI+MI9X8WwRA0QKMOPv8S6xdUTiGe/rcxvzgHnt7B2uhAL2xnGOlPkVLpezbJmq6X4FwZd0x4xTZjMJI2xnbZV5jwX5huPiJCwuzO2F+C983QJ16I9jG1UlL//Nr8rvTwxEXukh9gkk5Gpt8eTkVQArNF0NLF065HoRBerxafik1uvp+LJEKLO2Op6wNpPO7y/1qatR74VqyB+3a+ZJnwhdl3VYTbhS498IotElZnp12pg5EEDxoEH0ehw3en+cpnUw4SB9BRbByEYZOSNYHMzMQ9d6kHBVeO0Pp+M9fD69na8i0C59wvF8uDfPjOzmFKdL2WbqQBFaaBolD0JfhlTBkeH1dGjSX/Kl85ye/WDj0FzFY1z1T7W4UPgZPguvLcx97foI3vOKxhu12ReWmXRrs+V3NrSLKp3iviGH7mUTeCliolx2mit4ZmY1a9JcucgxAnvM+1kPVF5Yrq29jRgbwID5V810j6mownFpmxukxONPJgAPm9qSppEXx5Ry42PiNbJ5CMLFbgFEN3H1iwHzNQYhFMSD/KEAvYHC2KdEZngjSdgsJAysJ0Camvy650PYxbjWF7OGh6+XQIu9g6QR++WS1BkbV78bm2G/kgP9JfJmf5nfS53krpHcPGdypYHowZ4aka4R8kabc/zjXPBJqZAdcDq9NZQHfSMOCWeoAGXdfmqdFlxuLUQLI0+B9yPJvXgHmdpodYqdZu2wfeujAB14GScOq3UaFassUMJmYQirRLGQwfQ3wwbaeLzmkM67q1+5Zhrawk4t2kQaPpCdvLQ5TunDl6YgXo/ec/fmQbZJK8L8JtBaVzgYuqxsjQrjWKxgV1ovbKCa+Dmu8Qs5hGHawqFgC8FXfM8Q/ziD67fwvOj35G/XqlNZtm53jri6ipB042NYzQcIQlHoFNI5raVjSS3SoYHnw++H2HogJQYultydp4BVFTAxjAtRYms9R/AG1vWQ0jRSVnxh/4P5sexsPqJIQ3qQR2iu4O1NQ+s+lUzxIp9S/RWfXwzC4Z/E5uR9TalvFGTY9vcC5rTlE4fLFWsbJMAORzEPfh11DXD97hNOmqZtR6kx1hWSVimNb+V7B4/hlg4klXQToV58fqlDIfaBYv6BpgSCwyArXKt0Chm2QsGlRqQAvy2tMM3dfLOrnfpUP3lMa2Sx9KT16fS8OWpK5MJVmi5amKXlrxyFrqkH5fB5HR5Gz4rXxEHGtM4zUBNw2IriWkjraj2lJVRdJqD/LEIq+ts9LNaSssbv4X+Y4WP3TLYo6WWzXGm1IZIZVrfyaCbb1f1CY+3qG5wtc1k1ZzUmdwkfo3qnQfXlC9Qx66JDhYYmnbt86Ijl1rZOqolov6lu8AYBQR87/8zsDcDA8oRyiWLWwLhH14rkc1SoI6CEhzFYcjBWAJ/KnrAtdxwo/M03g5Svc05kJMTpKMBGVWJE/uvP/TE8ZUEt5NQtbnoDiO4vK1rL8Gw+BR3siOnValqpP6CK5JKFp2WF7QdiJ3YFNN6FztlJtOD59I5S1SdyhKVO+FqOP1u+KkTyHmLz0GdM6zOikllMosqohVSx4vTglpfJayOI7rbsHa7rPEDtdAUm8iVkibMIplDdVxFXUsyFMVUaOCqwXo1qoddc4sWzMEkTxz9JaikTlTHMy1e0B4DazU9eCy2bItbK8ACMCnNym0866ICqhZwW5zxClCxAq9oSuxWstvAiNMkpDyu/fnQ+LTBVNGygZYrB3+5WVaOSju85/a48VDfslEtn7GDv1Sr5bFWo1EpjBuqxWaz6C5UCmzHH2KN0kyJSP/sbFmyLUXhJctywmIdp2olsOWTkli++347qjQjrYKLXjB96Vwi6h4zLzo71m4wO3i1N93nFQ0xz+L5mBNMZmdh5yeNoqWwg4ZRjq3C5rNiBj15qFndpBUOE4z3Pxa12DVS4zhy6U50qERIfOWJ+Uk3pGByGrKZqEFQ2rR72mhAjVjrdyA1BHU7n8Cvh14k4Gzu8hYxDyGSuigIWrg1zZOajQKXfA+hliBuUuk+NgK7iwvbqd7BA+D8YUU+QQXBoQvLPhiF8pwaw8jpITblOmS+oQCRZvsdtGJS1jYTpjDhit7Kz2JJEzDeHHHWhd5veqH4EQUhLGg+GNTT8KEZZREyTrZfCWgP9oKwBPCYTAvoQjRFHbWt8JEFg1BCW65bkW9auK6se8JeWVXEdVh2ezvR8ZXgcbuteufzLSZrn5gQQ1peq28bVt7A6JpTg84TY8IG/EZKuK5IThAMTYmjwnyoykEFkkfUAHCT622zMbacS1arq5s+dH2hwfb6uj31IFhcsymF9qrScOFiaVQCaUTSSf8/VRtOTmTnw33qiLTEECBNtNiDo9EK2nm8I9rgmbE4+uCONSW3Lv/yEK6H6jWhwULqEqhNSquTzG1ZnQM6zkc8HKq41b/4LhLKVN7we2y9CEtUvUrCrZBOmCGzIAjYJ0cB5fuDylfEPB3+Gw+qe/wSNp7w4I/zSUQ0p8Kncrfund6N4BWq0w58Eo+w3b3mAVP9oph3G86tndQS9Toy0nUf+dTMJrmgQlFueGHejGuBgXwJeY8AO1Cn16SuIjzA0XflWh0R7rad0de4IAMQtXReghuET2qubDzfn29DDahIpzvsVghu1AxakYXU6jxyQAewyzvWcJhWDPB/iQD/8doFqMujyOMb+n2zsZNalTrrYQ5lEmUaSKero0hcpX9oLvXw+tJHSdS+ESZES4R8HXjzDeyESVKzSBSiCwadAhtCnkSQQgxF1ArUgdweJus9MUvq7BN5ciUvRy3dwhORiTCkizG/RlvgW2xektlLz8TbB28WCfeMD8bls9ZQxmA/LtHwqW3UBh0XqvlmRurqp3UGsRGTncRssSi901bWGFMeLa4FPRs+//h82J+RLz5xOVwa/lgAmzyQyiJpnNu3iO238AujG/XmxfnGxgZdqY5/QmlYJ0B+DPL62P0JMk8+bdGoCJ8SMInzsSuryM1Azke+EvjOeLGfUCrvd1Xd4d+1cVcM8vB42AHaY/V4vJHqeh6PkmvhThMG9LHT5kyCcT4tJ7qY+KqA6TELaYpUth2ElmgrmUXNSplZ+ZbaywIHAWcYryUREULH78DJWaRkA2vMigwQdXDz85pwg8dLjIac7+1cNfoGvRi73Tb3966jNZ0vx2+0tD3da0bPwuuJbF9/ZpTNdB88HVwg4fSycjO8jNnece5WMLVvkLkZak8Ox/I9k0BTyEmUCfIUubzyz3KOE/l0kNm6czQnXcVtT7tkue26QMQQGvRoGiBEOE+EmF3NIqAoKlogMiEni4cyjXcuBKkrafyx2+LYn/mapypumVWTEDWrjRTSZiJlxMxFeGYbVhgaJnFp/bD5JZ9PS3q+SWf2TlqVsgkpp5gwI00acLcGE2NeGtUGvEMSr7a/udEG7OHr1x5/9MrVOu2ClH44wN9PA6mkvgzdQZG42hE5ZaCeFDybF4p4902ECncwxC3k8KAdYon1mvcryS4fFUlnnakK7+cvdkpt8GaB+m3LyhMD46o5Cdjo58u6TGSTDETV3eiJsYUp1v5ZLaxmsBhr1GgrNnwevyPFeOWXuDpWmEcW65G1nfGW1Jl5rmY6rKZYFfkcf4q1+jG/hlV3YxtWTGaDh0M1/IWpkcdIMu1nY/GSWO2QCN9zpol0PaaLZPAhY3zFwO5g/9dITz5w7ljFKjOPSw0I07TrQqm8KpoV/fUij2PM19buLdAgu7toGEdJcUMlZUQufNvKAV9ZjRgVyepIGwauo6mbEXNLgNlgMuWg3Zw7tFe/SjhIME8VyZ9qPLM31zYZ2H9ZzAc+Nkr7fpH+s14cbC78KFsxsrwfyeYv3m6D+7CfeCmRaefsRGLbzWmDLWXOVlNTQB8+IXzeocklsRa8w5EljbJKgENxp0WT+leyBWRG6ExUE74tbW4TcXsBDvUIle0r0tPdsZL6Duo5yq3WOfRZhvMg3cuLOlMT5FRqH6BZIHgYS1fswNUK0tiLo2vLCXDGE51pBCXlB9posn/WnrdDYqp9OWWQ3G0ge1maz0HOIDAqcldu5FFz+qvN3R3piTkkkh85Mi1JR+J1WZ87ID+amteZHdhOZYATpZpGGsd2QDmnWOOVisZcMqzeFHLwtqUxu73A+uT2m1OB6ydjiDDhb4mU2z1MVCALYG9GxQNlnuvsSaOH/FUzIcYYlJ+P3cHiup5j4cKW256qlqyW1IwWtzqrU2r06I+aNJqh4VNYkESuJnjWQi24G8obAeNd4oIOQiZy/Y1dkuL3CulM2XiqQ2oJNVC1lOJ5gh1qb+rV/bU9Xn6JNUn0g0h1AMpSBTKWgC4VynjyTNfZ43kFXyrfmCajV2g5yQtQUREDusyRK0sscZQec3oJylz0fJ13P1SZq+RX9PyPFVI6qKRC3rg3LKCE+SSSbPQGD+46+PDsEt2os9564u2T3x9cWAy950rrSp1tntVsSAOt63IFmgYxQicD15zOmUEWgRi3aLqJQmTcZ2l02fbcYr6D45qThy3u2iAQYC50O2lXocQdztzA3zSN6yehs+DdmrUwiM4+tmQ447TrJw9AbqCfVHNQgzZkjBFuo+JlwGHnt046dz5BTtBQ9HebVizbT1zmjW3H8hGSHXZdg3tlEWL2VDgNOAHVLZNyynWqpk+Y4hTKmQBWOcYaJzxaKuHIxt42vjSwrsrZjeZ5fj4uybxkskHjN6VQUT5b5lMNbLhtyDf6DaCupCwSYMzqIuCappABS7IAozP2ILJExoiCzpE/stXawqk6Ior0mBd+ro4/k0g3bXQQYm2JsbFfHGxmE549PhtqAXykoYThponAGstLTSOgwc7DQnsUFeUcgUuOZcVvpKm6qpTTok6XHIbkwxiwOY1KyjqsyzHCy/qsGOKKshVUl27wGiYJZ9CYdbffubo4dgBR4ojThk6JimoOl1fGjiuyLbofdmOCCFKJxfcklz7GgpMBqm9FDOYUAjUVkK6FoDYC4jyEYNVKzCUmjkH93uQ7XxlI0o/QTsZ9qc2oON3TxBdWG5Vuyq/PbV8id4f2mE7vii7jUYyTg2E04dvSppXEUvyNhWIXbzZ6+CfJEreuxZ1bfrqWNJoVQuHwZ7oUlz8BYhuNyVIinsveLd2SAP28tjZDRZ/v2J39Wz0NCZwdsOHyz4zl9bWVq2uAiiAH1ddXXCSArz152tBX4M5QlB2lvMTKQpqw+/vuTAJU+DbxAnV6s03imZWSuwSGJvf23B0RjHhM7+y9fxUJHSDu7r835X2wo8+t7Flv+pJwM6KelE3cA8/e3kIqkwJ3RuZ5CXA6mkc8eMTJYszK+7Ya9TQoymu+vLUwcDkmUeFg4JyON/HKyyAo4/yHZxUrLTvRQQ8/KtLsrGfVk6dcb6zR4WPFBVSxUZqFekZNLJgA16LJeU9vFPj0af3BEDVZ8N1iRwRK3Iy5pBGGHLjvguI4RsOLxAso8kncnx4WLdGlCqPJXpr7naZHKkyrHxjWIsnoA4zq4I3SQiFqqG0ipgA5JHFVqqlALGDbHReVZ0clHCIsUa1gF1W6YG5MTPfsjLyKhaniaaNC7Th8lc3oVgqapw+Q8C3Dp3CKrdG/GHuDA/wnuZLz7Ok2pdBodJCP6+h3prUHwNregKGeGgSxCgLNuv4GI5LAm4HkhgQWqAwTumGc4AxnxDeElKZXO5FH5rDrxEyYMICdPMEB1MAheNBqqjijqTMvq5Om8sJGMHjK4LPsaR2dY3nw6chp1wKbQzbslwI+X7qGRSCvdwN7oN4P4CYqkidBjcWI8ZbOXcXGB2EGhBisdUYbyCWU38StM89EJECTnnTBAXhTOq6ILiruoH2yjKY5igJhtrQ7YylqZkkIRBjRB+Wp9nKZ5lCtULW8HEmxE5fa9cp8WjYCFt3gB8zWBG+YnsM92L9bsKqQfEUhczv+JM90W/II3P+zVs0bTku01Z7B1XD/NKuYNIuwIUDUcMV2T/gmuX6MOPw7enWsBgMeQb2zQuTKKPBC6dL5f/+Hf8tp10oXH3/IyYcuEkLtj14ZpSHWTjqKNJ1td3aU2jsr87vL3eyQSayOohwtpmcGeDOalMYSoNwhlmuRUcy+QcrZA/hmWNBRhKTkxeysQVhjCvHMNtxNscsFnJ+9rwYhRNk/X9D89vymooLtdcSDFz5Ee4tbsNtOh7e28EhVTaXAjoQYF7qzAs2wSPO5rei8Z5Ng+yiHvefvX/hkGrCAxirkLyWaSsp76fenU1MTieQ8rTeuXwFPAlBpow6P5B4S2JV3+aVcBXCTQ0vLpx+KCRk5Bc143yJPWI2CQEEQILGB78xrvRk0rbUpUUmM23pSjZKQQczpQr8fvOk4n8FMNltX1D3K2+cBsun7TZ2QMVQMf69HJoUf9K1q0qANu9eFhi7fkYZ/FxRaHqOm7DzaovpL5TWX5pUE5JAVGJEawVGtVCdKFAe19Qtjn0I3DgWoYxBCpOZ/E87K5iRkpEWo38/raUQf1iW4i8+nLykIBOdadPfb/7k4Sflenbg4SF1+51RiuR9x42Cnp9x76VyevtBPXtg8U4f6Q/XupZFOSairgmInK2tJ+piPNG1AG9QfCIzyZk0v7v+I3OwtxvZkb6t/mPdu/GatcRBU8ebPScHchElBfAKNX64o4AEnsYThO9/Nr7wuaxkQr4jPOTqBjiAagu08tyAQctTUxyCZhIGdM1h+0doUP7dhhLMUlQG/Xz6Sl/iVO5hj5EFzR8QFQdbbE7uXmtB2M6cS4XAj2QV7WxYTgDvMYhNUStSHBVlTprQiRItQHe9AjIxwF6gqmZjY9kwqBQlpb2ddjzICLtDgNBxfCxd2wtjHD7gSbEGek5Yxz+6lSPTkkhDb4pi916gqX0kzlK/FkTXoKmjl0bzTzACMaXtef44jbj0gLXm9Gh1gtLTq7eUG/SmzuW1QQco/b9+YyEbJsBYpJ4SgDAzABz0hfMNV/xEQSJvvrNfL64rV6j2lxKAPInXEAoLSOw2IWruWzjQFjbepCaux8kWZ6FqLYptVyfM1uKHMBQQbPpFXk/pMfTKrZ3f8ZpNwra4r4uxGEwXNlhE4QOkM74cJp0vQpeX1R0ovlpNy8FTdfggXGdv9gTmUSzlfX4E43S4D13UdXEcsfFX/aGI5JvvNxPFM37dFMWWpI6oBpxqYYCZwxrUI6oM6d1dSeylxuSkYtrj8QDnOi9swQE6I42ZSU6t3MuQPaqTm9W8TPe7WPt1Omx7mVoXY35tMTEyl0oA8pPY9/MmFiWG1pxBeu0Blv61Pj99/9dsSKSBQYK+9lU/4fAODHJMG8q+Gp7hymxh4pbApHUMpELj2oqhr5sYzdanos5bBo7i1N5zpd3WqIojLr9mEVtIoCZVhqvWUYWQziVC0pNVNQo1zTmkmfRWz5CMFCzGBYLgtLsxTc3vN2tsAwx5vJdFAi0HYcGPnnnCj2TcDDLrSVckDd56nE51HhJa0qvzN2W1xHoSf4FzvPF236Pp2XNLW6vIqcUzkacG0xaalEmhimdTIarVMaGcll9aXC8/wUDu2DTU70h3agdBQeQlajUs2kMB8iRvlmYpVFJGWFC3d8i5etPg6TNwFrANP0ncWla/0gqLn9RNyKCg882wl/TWsDd7f0YANyNyphR/HVLBsl4s+CZ/CFbwzMPChxJMHwNgz0EQEkqcPgfRfWoL1OKX+MLvIRYMTE8zIiHAe8YD3swbK8i/lHs/J9893i324UYF4kFqEQrCyyiCOP3IT/ODKy1G0IrhGTc7XwWv9JVs9/DZFpM8KAUfFWNXkI5VUwNGpLDJu0HXNh2EepK5Dbc7pbq7gMPfCiKk61y3ql0AjNNC769VI+YLE4iFrSXt0NJ7rj1EqV01iPRXif4xmAxdKmLvWg6i20HOMqjrDdgQrjJCBUXtK9Ji5IBDckn81eriW7Hpuu0+6v2FEgm6lupyWe7LhYKBIWooY4iWxCcuczAthi2RaGQtQFcyXmMVPImEIcLyNUTHRcrg1i9xws9OQdLX9gKYpXt72dd5GCulx4fgXp2YHSCKaBOUG/9mBDD1IjSsRMmKjd/RuOe0CLbT296bA7w6XgZu9nBPGzgbQvMF7j2BkksLir0ewtLTbEqyZfugmVdY7kg/3sagqH/ohROL+b75bijDUosvy1atCVahRDNo6vysgxEWuUTTf/RYJ632Wi1NpCa6hUygOGD1PaCFAdKfPj3YNTezw+DB3lp/j0JKhRPlfC9saBandCJ2gsdhZ/TVXmb+oZSGRMXqUtXI2TFpWrPApRUwS/Y729yAKQMRJBuL7ivogO94kikjUE4xwlemVgwfvPxe85h7p6alaq7lq8SqRiDAPAR04xOnZFPaZ5lwiFdwJzDBI848/ow41TSeKidMft/tEX62YW//9jBPEXT6/4j696qCBxbuN4LivUtB0d/yN5QxZbB82eINNIRs7xB7Lv7He0wnz4/7flGLBmQ4T0afCy0sHq8P/0+jBM8fqjDQ+t21m9A126eqdZ+Zn1NTZsMJy8vc/pjvi5AdFHeS4ijP5FS36ffegXy4xUQHqJHvyuHD8Oq5IPU8jV+5c/MB9/bdTcrFMzLN8+BeGC8CnYXB/BWfnbP6HH9DYGFIUB5kdgXzvSUhQqHZAo8Nu0PhR425BBOCvg0btcT3I7VP+qSHPhyHS6eTu2snUCqfTe0XEfYOXs3e/PXhmMGA2X9H/4H9zUjGswJmKzIQfj6p7r/6qeCsyceQwISac91F7II+OvZN66OxVxx4ormMw8zPVuFg8TdPx+Us5EC1qjNhIizTG+MM5bj5KggzLsoSx9PRYlr0QnNMardZi9IKNhdkwUmeVWJtqLWWldkwpySqoKt/ghnHxxUQjashjCcUv7sQrR6OC3KhFLsxocF/8pMgN/e4rfE+P1Nq6cdH98vnJ+3W7w+REEy8I8hPC4UWAS7JCNj32YKLsgCw02hwZMBBOUtFqPaJoB31eDaUmkahdjuYgGJo8ZPVImAjp4BMoAQFGdIE8dUyRVmA+h/tcnqCUtu4OjPoGw346FLre1pli9OSq57OlMosLhWFZeSoOh2UWQtmdqWnMolBnVqEwAWavXLq0o+PS2ajlMV5/9ChYDSHvaH5JQ4LSoCBYDhQQ1QemcUhDpvFOmb6e8j0n3cD0G94Aocbl+3rik9PeX3+8bTIB+ZCZCooh85bGTJCUx3HqZgugJd3+JoTT3Xf/5pLHYoPsirBjESNASdo5xHK85MUjjsZf+el5y8/wBiU0G6/kqA50UAmXUF2ZXK4y1PEPXM2ZPqBGasfgt14PmPTI2ukxwgDEIREodwbL+D+uBNCeJrT6WYpIcXWEaru7LbU93Y+4N6rFRquVDxg53mTDXf+oqpocGJgMHvW3umF0tIcb/VRmYMeOANyP9RDkEPIyrFgEQ1J5jcWchrbVWFzS0Gr1HuRaTAu3ifmXV3fuOg3P9Vy2M5vXnr+m3dx8XqwCEEEBhID92MdiDqNJFaMhSxRPOC8J9UY3+kt5ZrVcbYgy1VIdJMUl5kOhOZCHLAESuF0zMzMtlTwtTFeq77fMIpadY1kbZrdMsv1Y3v3BsG+fVF29LgwyJH7UZf466N15smYbAEt4nuN5ojWVtb/Y/i9QWs7O/14NCLdPD1b29EiT0dcijm1ubz9Z4xZ7N2fBPltUunrewMRULDcXtP3D1RDpH1fQ/gWY10+KswszMRHzl2pfHyt+tLz+xWNDm1OBLTBOcVMqntwhc6+1CGqlwBlNa4d4An+K4GCIjNyGWtCH5i9bj+jeSDn0xiHdqynd/7dOvyNG3/HvsLjtRUve0nUrljl6FeB9R8URgYwKXFJhHzC5BHuu8gEw66E8EZI1RX78ZXp4B3vX0Pvx93XzYwe2260Xdny1YoWQmsD8hmPIE2mNOY9w9b6L52SHWmbso5zWZ7cHWyx/4n6B6BzQe3I/Pd5fsNaPqfU1VSpcnAe83NF5b7zCWcS5gT9dJ74McX894Ilj5HDhvJZufPSfGP+rfwS6yorhuuUrsiQGbsJQBj9Bu7BjCTsOyuHEondGhWb++3P90QLi2xBnyQ9NugXD070sdly/1T38ZW/gqGz4fkXT+SyhJ1uIrITlLWOs1S/aWLz7vkwH2gymJq+bwPACuWsnDsl5D+VJL3agkAvaKbXj6Hd293/TXv2VOqF/OvBL0/ywrzNBeXGa9AY/nl81GyktI1SQUUjfR5NpPiooUDF+txTwnPnDJqhYKNbspHGyhjbkvmbMFmiqTlVHZwmVWiZWG2N22mK0qvkRQkRwzrTjmMEqE6Yw5IJD05lkS7EldaY4ImfYhD0x0JpCRJJiTDRiaJKxR9TYjFYNRUussXKsODszQQAxMGGW8WlE12ATIamMSipV54eN9298xPnRiyNq27zli/hpCtS/m9oFDOBem8FSgdSCTYdq1w+4TQ4ZDEst1pkG8m7MSfLmYqIJ/3SnAzMjGrY4qEIiEtjRT0QwPzgWzA/LYx6FU8FB8/CRvyT2PQTGCBzmIJ+QB9ydHTJjQhSSEJlZyKigXMfSjBNWptMvTWKVTqWEJRFKDO5wjzPZZFUE+ZEOpc9u39cUjn9HOkXS+R1uCl/sCpT9gI9WpRjH17x3a/umhkMXNmGUUvXv5zN+KPPkfoOeDQaJQiD0IZ8+xzxZbwh7McVWbE0Juxmtx0RaRNY+PxyOhede7BGqPdGrR+hBnBK9GOKOvm1GZITLLoCmkJfgKSxI8CMU95Rb8HoJmx3o/wHTPgLsOqQwdhy8TUMoXoKO1POcOCeoiloDelnO7iikRbqwo11LUwz9S+nUrKk5cVtGf+UEz/NQXlFZVZK4aVNiyd8Xl0OOSCaBnQ+TbqPV5hwmh7BZh53BMKDHIhexCCMy4fMso+CtR0hGvFyo69lXilCYO8QEVUqxWkn2kST6gF+JnGRbZZTb5xukkq2rM1vyernzwbyOnsJIks0QloHY4ltVceNP+oiSmOHf2q1T4nVRNuNCbJUaPPl82K9+euwR8WehxnWY+janIAT2PDAohOflBsRq4AkIeL0uVxf2eHDXwADwgjYNpVkB6BhGjryAzvX0QKZWFOHbjgIc5AuxgD3YgSFpXKJDBLOs+OEnwF0nK+FOmSAhqU9MrwnnE1lWftdgYmUBEJ1Ul2Z2GUn04b/mTj7W2s/N3wkP5jD4Hs59MbbekdmZoY/TCjG8Wu4MGteuXRfsi/CPKVtyW61WWxkGY0oODrUDsxipJsihdkB9lJeM5N5zzdHk0NW9N0y+5yzzvtrkUedohhRghyWeD4slu+QVP94UJPInts9CsEotHtZouhkLe8cMwNQUOBI9oyPI5fJ+WEcYkG5qWZynuHjDBhKyNYwMf9rR3r4fRyt85Ed5h2v4m6XKHMWwIlspa/zF5cIwUHJ5nEIP58B6DGpItntkj5aL6fZI29eC4aMW6DCEQ8MQ6ZMJ3A/jvGXVv3BW2LOyXLk5j5eNjFCJcaZQQl2m5ibsn9y77wZYJT+b5OC/5ekUlR6tNxUWCaSoF5xIRHh/UZZLvb6bWlWCZ//ni58yVkzv61qYb2jU5PDpyBiUJV2Y4dbQrjt71k5sOMWa32c/6m32++vnF9Kh9LyABa2X1NUt8Q6Pp4VTgTv8pPrpAx7MZ7e1uV55lRYNDUP+0DZxzGHTNKYPTV/gJ0ZFFltN3OSW1IKzk7N0PMurWla1ZUvV0i+f5PDmtLVF39H5s/WQnh1Nc6Nr1Si+T1sDU8Jew97Me2NpGb8z/C4jbexeD0ultWQ1tWSlUaMyPzBmezmJS+Io31PJSKKItFIXf+8v9O+fwfz6+vr4Wzbb9uM84j9tfymBZLD/m8gdv/uo1sLWib7k4+WPl4J1HtFzcCYqwLkBvB/Y3eKUe3M7lkwHek9bEXKjtk0cnpwP1gQ13nn96LBP4VSIilMKPxiRAty6zneYGRZm2fFEtEVmCePOVFgUmW7q8yrc1IzcA1L+3AWgWSH/VovyErx+la/wVEgVlypmccXkyC2x/F93VVe7sHmpeinBoaG5eik+33keL4UZYcodc1h3rizAYULIxiuWpdlGVU3JliGguCAwvqEMxj7OhzGTKDtq0FDN6GDNHax5lSzkNb14iaZgsWZ9lI97ihPIXcf5KV4hTziobsw3qV++WFAMjOP8K+LGgoSNNaHJ4uJlNUmJlZX9jr3kj43XAU/gT8786ZQP+aju7DhND/8DwtKDoc1xSjahcmLvs2WhjIgwRtqzeycqE8xwDvQrYNIZfs26hVFfPJRhjvnzpaUJbIIJgpgOv0yIcHOMQTQIRFRbMfjqH9m9Kk0/mDKW5GgbxpcvstL9hQ+t2kXOW0t+FDc/TtrpKedt1vPnQbTEc0T2Bbtk9+lTYZ0njXwS+Vtj0e0TxL2PYEimlUNKn+Sg+K6XAPK/j1bc3/7Vs89Kow9erHJQ6U/4G7xZmLy+v19L1uTFhwt37lAcHT3Ol4YVYLsEzp4dGJiFnhuXC9zVrQcOtK6qaG2tuDh85cru8t2nT+/+IkHKHc2VElQSHPHTqVli8JWDB1u1tEe7RJBbg792N2hj9JrMGLVF7EtYHMjRau14uZscqc5PaGuOCq42VVe0kWG83SNmmN/TQNd4zGMF89dvLAw63yuDW12gr8+r5T0ricwvcAbRANphCbMTr73LqshUKXJZkaueo+ii743hIcQsPfqyxhA0LGAepKmJOppM98h/4pisgGxKPpZ/TJ6cbnpbgtatYO5Kd+C1wCXd47reUsKycHnW+sJU5nDXcObUc5vsO9GRKo3OM44ozxRIbGkXsSrGTiKIG0URiWLzy0L4eVZ7wERAO2ssoF2UUPljZxAgo1iSZCZOCDF2K3Z435mUCXGufWH+aT6REPg18anG3vUkBBIQvBtmD6vtfgx7gi1i16X/XVPBrqJESj8PAorlCr8/D0BEHFdTmNHW86P2dduscatqVpn45Dh4jHKMkCnau9eKOWxFjz0ZF5V/OCrwhcuZAiEG6AVbAysBrTRQawO3NjYilBij+uSBFzn5iQrEWfMGrv7y7qPLqyQAP77y2kcpjftZIfC6I5DFc3h2mQbGpeKEJqaVbjA3XtozlMstaa1qtfqhi7pTWadgJH97thuKDwtAGJdLsUPFgjm6S/WtQvHtHwsCOQ4xfgZxTwtmdosQDZ1ScCAIrLSt6yfscRe8VtnX2NhHlERBlMLtOz90LLJKQeL7t5/vM6GjC31sIOZLLpvaJZ6FMsCIwFiXiEXXDOYw3IxDvpF9Q27kmAJxnKR+lnIeCa4aEzue8OkP/PLhqf9kPTkoaRPytOlvvvFMU20E+wvymf+mYMcyjMbiUJMnI3UgJaOs1h7OKGyfiUotrVt0a+05pM2Kew36+RvQ4JkWIO/DPs40jGQqAFx1K4eUCIsCgOFW6y0Dp+dS+kr6UmKmXUASCllydXHwkuW10r2zmdve1jA1VdTwU3rcpup/qJHqvaSeAEWEECV+h/1ISYGrsVf5kR+C3RLb5VS0N2w73eKytUB/0gXTIirAwbxsiVyPQovGTB1ax4f4ewRYS3xk/wxLfIiUz7IwQfzeirZTkvw+TJ6kVM9eGAnCbGNavIQwmhnPlBVkeAYh0qlUz755BFRskTr0MhIQKZrwYvDsChPihNvrBUsHLWL6sKgO11nCN23IrjD0zNEuYqj4sK2DnosEm7+HqKu1C6uIFRU/rW6YmjI3fAvdq/O2UbnbqDxhaIdW9Zb88OHv8XVmAzjDkjkRwzF2vveFKSh74nhuG/upzM0n4z+n0qmMEj55xYzV7KyfkOWoiLN2UiUiLblTFbpDaoSOTZEUpxT5OQJlgGsLhZz8QL9dN6pwiEPYIdgsziBGAYYQmJIaQiSVzwqmCiqfZKJn6UZukWACZv8DTU0rbciWO9g0mBuNyTKOTnnZpI/P+M8hkgCD6LA6RLHdI/pG6h1u5JDAbJJ6eiTCZmf/oxSLm9iuSk4ZxvMZ0tsufbcaLRrOIquMyqeD5tcT4SE4vWiIpP24U2pYrXu00a1ukC7tVDU39wVy+379hAn0fwcCdqIRgYE/IsGL3VjMcmM3AgqMbTc7PCphaqlBpojR/i1Odjiwg68DRvr37m0UBJ+AjivIuJ5NLD2ays/3TKxY6bkWnqPN0eXAOY4XfVM4tCcqMbAUNL5FsrkJnZ0kTPbt8h2dy5t+KTCbl3cWFDxF81FwH4wTZeU+5QhegxUYRkDXbqnCzSJMKXwRgl8Upr7aYSiuxESbhaX14Zu3NlQyROgG6KoGh+mnXMj4wPrCxJiodTGJMVcM+C7sfcazctWkJ2JcC5pBUSVh7D+1EjVcZ/OyStJRvarH/4mAgeGYOTcW8Zzc5JSiMsn2T+hsHIo7C/3TllSpOBUP77q1g0bXNLvYpfd6ysSqrmda27yWUDCpzCTqX0vezS6LbF0mVm7uahLuplr/Suha54UE39Wx4ued1WIhxahmgrat2efcOTfGP1R2sdTYKO3f13ietqrqVfR4/0QxOtzIayQeVat++9s1MTqNLgZ4lor7o4MTfD90+DoXDzo4kMNeUqcLLQYNa3mBvBfBxflBw0zEIYiOHccDYq9dt6Pupa4Mid74nyDCSxDtkn45Nc6iokRdvCK2yZCr36xPPPTup4aaI8o2rFpvB+wbos5t3jyXs/EKPEcBGUkay1sf28nPjC0MevLg0NnA8uUB6FHxlPXII96ImhEHVT2AYQJKaFbBl1i+4uvdX6wqX9Sf31pPCKxulXwKbmS92Ytcjn7W7NxdY6kRRbiRVaOv4ffeSlrNIfjGOYmo9pyDS/g9HXQgxUzII8JERax8DYGg9jUt6YNDLqRV0zm7TNw2vXuxdTuGmsKJyvp1Kyo4bu3Wjs3T1hWqdz2pcd2DomkgIpdbCUKmpsBmGwlUVb/xyhK39lKu6IgPms49rgICjZAR3hs5yfax32aL7OmYdZ1NoTt+AjNwpe0iBhKRVdezElcSzSGV1wrLvUZ5okvHhHeByoXfQ9etNSDDju7ub0fHtxN3BIy/Aj7HKXkOVMy7eXNaTNlZIlRlZBiXnzH5fm+yuHrsi7XdMWH5n4/TEb6v0yX57uUpx6t5Jhns+qWQVHVzMCdMSEgS/qyrOiNNTMCLLSBXx6mgLus+xN7XggszLEkkDVGmcH3y8pievL5QMaxw752oRYSl3QoujoygkTk0Irgfg8gtPHGHPnJQVdldMK15wcSCMQnC0P/rIds5uNHTvWZwcM0Y2Cjp1DHgiFEXq6O8aDB+OHJyv8cg5fGs40p1YG474kd+VrZslPmtG5UEXX4fmrw5LQiD8tH13ECTio0fXWWk0egmARId+Bch6reJpqabPU8cD+/SmK12/tFTKaPHjw1rRmV5NGbx8K7jzlRnUeD6hsF//tw7FAyAmo7XUbPq7dfjjvfJMlKePg3qhTa3mOObgppfFNvfd0BszHMyxMT66g5AAhHKNyRBtkfgZcg0AXylbaCgHrpmlYZokhatoRdEpbpKQ9CgIagG3j4hvUZFgvBUuX8e5LHNPdUQXJJQlZSUDINqduyUxg68/fax43t7d77xBkjHn4qHxQVI5VdYN/6YYFn3h40P30/tfD//fXvp/Yc3/mEd+2ZzfN70cAPN+efRcBE3WYTlZjmgkvQs04mB/KAaHjw8qUmurZcslrV9P8VzLo9Nt+85+dnfrkcI4DTCiDVxEse6Tfu45NN7P6eBBcJR8t/8TOXTqQmeXE9C6tOVTP4MDE+nqC7lXFKlPmJgJGwiJoIlBlEQuBE0cnI1y8N2QCXpR4FWBw331Ql2Q8CwytolsztZeie7wtYEW/0rBw+0la7oirZY89fkW7q7H+3uthTkrbY5Dm9jWzPRlPnUUiOZYJUUeGXPySjdKRtySvCrBwYD7u3SgNxuQW8uSoVIIsLUAP6ykZPw2+xrLy83zLfOG5a/LC1fVxasOHzokDR6qqlJMv7JbHVLoPdHn0z3N00e8/XdvHGr+uToqGTW9seH15xttUK0zB1GEBiRkQnDMdAH6R/McA3M5qVRep1Fmfl5N2TN0Ncjr8OTkVnPXYOshp5z+K0Fef37QK09RuUz+WGjwoLyMu1v47LkbnVl2qkfo4iCcHzxktRizTfKNF0AaW87Xe19aX3pQNYBLBKcodwRUcVs/RlUP0Ra5/jFlZuiq5DSSFy5kGaz3QW01SCbbqDUr0ew1MMaCeoyosCwZtOH1I9kPlSAkGxn3bCdNsbPJE5lyCDqTUjzR31y+4sbGy8q0h90+eUxbt+OHSCbBuV6oUUdrtErXzKfirIKdWpFCJTMCYleUAB0Nw563zUN/naQ3PSbJ1k45uJg0evPwg92iUhVK0RnoCrofPuDJ5/eIWOLiS/AteUNlzr46ei00a2XWqdHXexHj5xhbYg2y2htRaqnjuRuSC1kqwuqqithEagtA83wvEIrsbvjnN+clGESatL68X692FbT0YYZMlO2FQw804ekIZTuclheNPHctO3Gw5CjQEvFFs4Lh0706LF5NwKHTuzAoNSwHbC+LjtpT7t2a37j2ia6tYRWRRj3Ii8VOXi4KzgbrkSQXuW6BslXhnvA30DDmioaF8ByWGAmpgmC7f20SKqZjdfd0h3CdwXT8gASTmzRlsx8FTdx0BIX79qwQVNExG06eBCkVVpR4i0FNKzbDfGyX5BH3LS27y0Zlh/u6ZnbVrOup1/A71+5UrcIRixOSnL1lSwZLmSC7lRODvZyYMTpMykF947lbAK7pC92Vx67wyfx1Efe5U59MoO0ghgx4iY9n3N3Y1kyB1HbgjHRMFA8mszFw66BG33hVeEU/a5DME0g97VuCkala8+65pprrzlLsoJ9wRjSkUious545kyBwzIeH3LkYUfCGD90DgFFBGGzmXUmiZ4Ux/vs2/fTKkGTLlNToxPcUyVqT0DWTdM6+jiY69eEYQ8aXJhInMxpYCgVzfqCssZzAgUyIYFSsW0WlLagVaGyMBnMiFpXDNjZEDbAEpmTqEpYlpRwg7RmBckEplt3pVk4vQ1PrAAJJ0L4eRbTlYbj009d/KlpRGc10l/NrNyk0kBZx2+jU0hs6LGnrVoDOs+Lr/eNXuF9VND26aeeLKeNcyB9KOmSYJ4+5HfbV+82QU/nA99QPi+BYIWr3zgiZfyNjTbHiuz6oJbF+Bj9UnN11+SYO1+rO/NeSnGR1+h41boU+2jYhrLiX4fiveeIhICWvPEk8AL132OJ6THBwpsBtIwYv5u4E6+gBYO+zZZlwtky4ZeJ6Sti3+yNjAs+FsIK/Puv9RZaLG2LJOSv7+IT7sC3TsGBITMEFhYdMYLTBs1QMzNrg2aWXgLAvukmcTpokmcQDQOdKoBnHIri0LCqXLgwjdpWsly6S4ctzcy5xcuz/2StX5tUVlf3Op3gC86LD2nBTzN0EpzH3B7EJwL/OpWc0py1zTabWedNr/XqOmYZW98YQv7fZWNL2yH8WBS/PSmoHDMduLk0LzC90C0gBvFeB1QJL/JQUyxwxrNuVNyow8SsQCa46Mf94vYbbKdEJsP4eDXjOP3D035q1eLEVW3qQylxr0nzElHSy2glulqVsqKVeiQjpk1hXI5yq2Kxlf1ba28NjJh+KFFjl4Apxx1GMcr3KoYVjHsOK2ThzgGolXeftc85ADTm9sHW8rqHgt9oZK/nEGIZwY3rc16DObQBR6FqtXHJcPus8VTq+YYDR9OAolEQCImMr8OEe6UcXRJfcrQ2ijRzPuu8m53ZiyiW+AthnuxMnIExROGYw2SN6ymxPjyCoTyWiWWD+nJUPnCT+ZgBG5GIBDutA6h5f511ZZ9nKi2KzIvq+waf+gw+F0lFnoFOqbUpPtdOVlzSXsfXl8L6a448hQxBA8W/lymosn5tsRXklWywQR8m1rBovTIz5QeelkInLNirfKUEeuSRwI7tkwFFfforBSJDOMLw/FMTscFNJrt+5s24OFWXTvz8YCmurc/IQM1rWX4jSMvREA0dHVNdvTT1qfvLC54pYU1ESDT39rNx6SeVWnzMg2HUdsbfLW3MY2UtEUoNnlVp28ULwR4WL6cEBA4N5eSvKBwRRhNluLUKH4AoOeCNM984IHkgHPXiUS/w3rJbEcrDXr5qCc5WFcCd/o3U5p7NzXto+6OPbNDFN5uL6cTfn0iHyCE6OKOu12vRfy/e39iMZeZLdbZ/9uK78p8dLVUxFPgOMKz4jm4eeIN96H3vBpPJ4GqhglpUG2XXgWcfXhMB8s1WRPqWJV28uFhYW2dFi8sKUwypjEc6eYgjrhLGnIpA1NAcBOdjiHWBKoEJcBP2Bpb4txbpz8bXH3l0Y6sloUbon0VY4Wn0ztjaiGL6h/VUKYlG5RRXFyA5GJwKBVJVoNK/YixVNpqPehADgrDhETMWv1dYv3rDQflBPgM+tuHWTa++PGCp8eTrrn9howNAWiNfSjT/D0A6S8h+Q0gLkBzQlOtDykClf8dwMp9s/L6tkL/8+grYaB4MLCL5XvzJpyM6BU+BkfI3Ho6LJl70TyClUoaHfTEjZsoo0Cf6dYAz1g05uDp8ZVZK9sVAprYVs+pwhFVxFVWdKZA7S3MbSrgHvI/xioIi0sJHh3i83rrE5GabGNku4F8+/Ks895ItGIH0Y3qpthYXDpFDDCQAm/17W7qJJRbLU+9J/pBYkAaMgw3SiC4uZ+dPt+SRQMTanufP+uYbYW1TE8IKSbG/zibuSesHlcmSGewCG6AxQgRjFY8jThAE1u4jmBQ+BA9cQpvh26v1i3fEFD4Iu+6pmJZosa/lc5YeO/dtTD/6Yovxbd+BxCUR27Y/0ZzAbDixNtHr8p5r3hqeOfi2q3I/gCeYaHYWgSCC2CzqYxHWdPpUJkgLdH5ED5W0Nch5ZPdTNOFoa/BXOq9WOrOcSnHE16YfFSL4IQAk4sM+pVNJxi8aZMNdQ9Bww6KwKC06S9+87lTWcaXieFZ+dXxffFF8b/zr8SSO5OKBEIxy1lvupwpn8NPWAikhV/uwQ5phRCcARHzGeIIcLBc/VQfVIyAY2IFIodncWWMuvAO+krlGhN1P8GQzrsJie3n2bSEze/Wzqn2S62SWviBc6hQBMHE4Ktk2VirjAEJYc46wBhGhhByz0BV0YPBXD9guuLEbPHqIxz4T8RnLxwqwRNZHsXeaNXC9+SdXvzlnXUrquhySU5WWWpUjCJBOkm7IFDEslVTFL6OyWY8p/88+W0PlmnxEBGE4mcuxH6+LvyAkbfJuVQ61uM6fF0iqSFyo9uJfjaTHhBu+kiQpcCSSId97X5CD2OVfM7lXTFoscypD7fxCU9POXY2/3rq1AlVwLX9FZNjWw2c6HXPKQfiqky1LmNNaC6h2mfY8ilspCmxLGO7dDLkTUtSU7YjNjEXgkr7+ITz5r4Xd9Rwux90PbntBSbpRF8VVEAtSNN7B5X6MXAJPwJOwG/7n5tUFRr7xazRhgMs5eIj8FnWm1cNtHlX3JyHQfqByDy9PyifIbaSR4qJcDsomMw6O73HiTuy3HOkYJxVHvrpliqq0vSe9IAB34n/48uCIcpuSwInY40xQfC/XECfDCNU7l1XSOfgAZQX3aJ3U82lJHN/jmQY32sN01dScLyml0dn/jrkEmy6ckKt5fB2f9/ORiQL/l5cJdunrv+KRxUKXqx8ETsnkVmTHOP1yQ/V7FH79993Q2ExDpmB0ny+SwWczxGc11hwC7Qtol3k00gvzfQEXXTL+kYRBy8d91Su+ZVNMkLzaGW4BtoyjaGyE94e4noyX0ykXbYBKfO/EBx4Vh4D9hopOb0KF+CYnvsV9gDnxZxturIfqt9Tq1H+4yS09LoP7D/CGqsWsKt3tDFF513OCHctuS9l+DywfEzxX+Jxg7Aox/0BqN66ywl5s4GuuZnZ5RGYJIa5AwMIwMSuz4bX1REnefVdyXPwhyPi6mGgyFZnFUYHHRRrYf0PJ6uNVq4/taUzRJr3uurHm2M0qi/a+49KZ5Qk6et1R67IkNcDee759TLUm9O9iRdheFg5SLXM4BJlKs/ZO0qz7YMXg316g8Ln2ufNzrHLkiCxjHvGYme8+IOmGu8CoTlMIg6Ai+Z26U8N0DgqhqVsKRdlWuJr3eR3CevEjDCJEJTGYKlg9wC5IzQ5iyECKUowCAcCOEsyyr3cRgmfQojFD3o2nXOkRgvqghW6Et3++F2bf3L0XDCIQeyleycRCurJizbsaq1auGIIhAlKuNhiccmoYtix/PK6XGnd11xrmDdmxcAz3Vx6FYu2znzksB4/H6fGs+bfXm1nlDATef6e39513A4H+qiqiJEBUSB1+yi5QPCYmMep2E3Oa8MvHQ5OPFMgzCyo91GMblyxtLMjfU9u931n35G/Oq9Pn/gVPGowAYglJ+plHggIBn2i9RlzWChp9F6Ar2P+Ofdx2bq16Tvr8yK8XKl6NKHnbYKliJSgIAgHbvCA1z5CQXjey7I0zrNO3pqQveh6v+cXaM7fS9sRzVQcWcTTC5lXiDwSJLR425k3eyGQlLDGvHCSB+BlB8oYMzg/p/Z9u4ILFixnRyHViGj8fHSqk16tk2kFPXGIuoq9+CG7NDDMaflEwqjAXiLIIWW/RrZ2Kq474GPFoav7S/KxBJ16QVmZVxaZ2GaOsgBFvc5/wOUPKVzEwaLZwvg/2Mc/BOOHLT6mogJREscDXiDqETnODMune6+bbTjBX5Z4xurvV3W7jmdyVKx7tjkbOvDqd6INhe6sqYzoVhpC0w7dnfg//febeJhUz8PIRCZXg0RJenq02B1VxUp4iNpJXh0bkHAA0l8DoYr8YZMdqlm24lvVCgtLv7m37HLWP8+FmiL2GYQ/ySZV02PfLxY9ed9/6/GufC9Ue0OOxfwCK2J1KmniPhi5gX9Adha61wAS4EWC8LW3DYV8cnnvOxCMHQc+rGksklxa/AQ9EjKHgtvEzpi3yx7F+H85AtMo3nD7ZIc2+B+NnxmHYDvQpBdGfiUgD7l0opF+CsjRjgXM3aaWiNPGtsn1A6f+2Kl+GEWUo6wqOWJT33ujeB8nzuVzW80mRihPFZgPui48/JXPtstxsdHBIcbKzvhSxbsaE9IaTJ9wFeDjWfviBQa/sd1KHkFTWlNNCMKHCTHH+49HfpM3nfrObf28N1v9gtJ25mBpXWPvFeNHujtcDec9jAds3W/H2jCH4hiuzs37iUZyvu6ntOB3fvZZ+8OjpDu1NXT5OC9OjqSksM0y5IZ5dYNEkFf/gP+nI8mwQtYjEp8SA9RzL874dv9eQVReVOlIPeyxdc0fOyR86MNqICAb3XJa0ut7s1c+Z5/RmL/axXuKSpzSAiRgfz/eTiT+61jRKLa0NR6aYRjsPFPaUc6/fTFaR1idMSEwzZgQ/8gt5QyNpJ/5gjST9o0DHKOUjeGbc4mw2LJsyB9FLE02M2egSgwwLzg+4OQTvs3lMZEeZzom8wFTgLErYzB5O2rexHXNKjtbEqQPntMHT1kkrpxde/ETgrEPr7uzEEn+O+/C6S7IqnAqrxKaXglC0fc4zpp2qOC4IAjudH+nX9PTyx+K6QRdXnN/cZVw2cPZsXb1sN95fSURPz48DEXJ4jC7GIHdEIJhCIbC+05YvhSTmGGiUBLqOQYy+g1QCdxU4f8Y11v8eU2SzaVPpQ3FHsE0QlIeYkbGoAzIs9AZkdws8zCtuc844px0GoH9X29/QsDptHzngrKMvSPATrWjYFFp9SbeqSU8Ry/bF963fQCI5ZIVdUreSbNjguSW8jkSxqXX9tetYRy1XU7L06vCMUKbsddiCmIs2zxCheyqQg3bo6zNTH/lbHS0+1fy+41d+safG6pA5EStYu8Ms3r5zZxMSlrrERSSuXzM8fME2Iy5wV1YqpHS0uF9sSgq2FXg1rPt/YKeRDlGqqfnXzTiLbuXNlcs5nGZJag4+ugzpzrW0pEsKtBhsDQSSaYtolreOubPhIgD/nWOhWAFEXmIMiydtbsQrx4EbdVNzGvHml4r3t7ScntJK6agCLPlx7XRokXCCQAX8+3SIRLKYdCEvP/khXMS+qrRHXvHkfQFDfg5jj+tbvBrXOxj7StLEqY9tNk9uLg39xVXYZuWq38szgB0tIu/WeCJDFb2p6xW7X3Dogg9xmVoSSnr11mXXOGs863SKF815vZYJl6NHaoE+UeEUIa7SkpRHJqRXVm6hKtOTCkkwTCtgOQxPlWg9wlpBYKQNoVTkDqQQYiy0Cv4NuDwyAg44ugcpDSN3umAwPpkHOGknAVEffMI8QaqA5xfTqLvmpsb4zc13ukhSX1Uu+b5MUj2mgvnue0n5O5LtlbI8RLeJQoZFusZrXd5cPfACj9QF6sqhjF3CXa57591kPZI/Q/IfdD3wQlByeX5MBFD6mLFs3rg4VtI4NmBQ5pNuxoXSHTuZN9hwCIR42BPqWcAHClAEvYdoCKcG02g0aAnBhwUJzgQBbEtE+Tp+QklBYo1T/2HclUTV9P4mF0478rGYjeGToZMQUZw85nBq0MzLjsqgXGaDeH9aYNZcj6Y4U1Ni7SiLOfFkXy37LO7l2JegifTsavvGNdk07XcXijqW7s7esxvoN76jGrfTJt/h/1ua0/CD8b+Kpoampq1rvE2z+O/JzxY+9ozXEw2yVLzuKhjyErMIrx9fg8w2edN+P3gBL2IuXq7MriYVuxH8MJPJvKRwp81mYdBkF6r10S+zuhyzLNWUKRV9+e+qZ0P7Nw9oycxjIRzed6+9SJRzeKsKOn3Z6r55/SViO4vbUJBQ9Me91XFbpoEdL7Bdic7E15q2qVTe3unsBDrEq/IL8gfnW0jUQpo1TwgIxlv7IuVD+9dfUpOWv1gwMuU8lzyvLTt6vzWe30U11FuthesXrwryVURk+WpsZHW1EwXNvkkM3rdKwuleGZ8gldJ9EY7amE05KysMYbUphlXlFauC4lUSQUjZpW0h3bd7CvK5Ie+fhFOYZo5B0CjUSphV5XPhZd1nIiShL52E9NFGOYaI3jud0hbMBPjN30H/xfwgLo2X2FKBfHpdW02HKPk14fymz9+S8euppUHSf4jFwe+9qnwpImRNa8E3Olx84jqElUfJtTLNLjE3NFR+MISNRf5CF4YezLOryMDGGVKTomoBAwib0qAcWenCvx99MIbFYcT6aQ0uLH5mJsaQTlVuIzdtcpHqZO73fNKoS66rVSosH1tosJszexYGMmnSyEveSB9sKj1c9GRJkiJbWgo2BUQ9gK42vhPsQNN/vZc+jQ6okeeNqCP4ndXLxOn38ovKc+UU/3VMRDp+R/kMlLxoJ6ofFCnPdQWhuUwwylpPWXMX5ogRkR7YVpDrxBtsuvQo0nv0shhFiMO6VkyrWFczI2a51KrNLBNrjGkVTBAvpEFGla/oF8+KFVMOfM2sh2dN4VlWCVkBg2H4VIh6G9cJlRaUxcXktrY9o2+dW75EKNul3qV7Oxj40iJJkY7ByaRW3o1+HYyY8hkJqD5kMMM/OKXP5FquNlM/tUWc82IG++4bMdMf1nIBDjqPCR6xOK+ut7MsVe291NLBz0ggzQpMNnncf7rl3z8UyN+/wOJxeBqNFxxjYYLS1o6oG0IpHgESGgJ7sOCrbi82b+/oxb6tfITVWQW4y146SyVnOtGranLtOkZvnKc+/Ucioftgjssgyc4OSbu6ejQsBaVEnkBlUJT4xZkDMRPXCmwL60IKjKnwKj7abeUd3Nfkv/eassrK9C0SqM4MkEBlmIcKDO1DJiRuWnuOts9UmM0GQWY4Liw5tIFbOuEzD+bWW8XDSPl8WN31pqaFXQ0fvrr8lwXAYdXw1LTRtFRR1ToP5Lf4RniDOwbCZl5krk2qLzW9F0vVS5fhzvErBksHVsC+V2b2M8cgm+FvHmK3WtH/MXaLmzenDQfkV2KQnqwwVL34YlkZsnqSJyRpTJoQFRCKkzXOjKPzzHmMW0KbNmNjExNdvL5sm5kONIVImlN4SlN1JPmq24zVGy9MupwjAFGhRsCE8Pq3s7aDV3YX2spwVPn/yrZtTAunIcuuPV3D96ZFiDPWXCnfalgbgcf1je5BttYXh3hkbX24v5SHJ3StHZRcNACkK03s+ATLSNnyAIlhfuBkrhVBsTUEDXcNsuFYamVloKryUU3uliYJl6mx/pUDNOdKqyPMZSfbT3HAFerAMRgwT+wUSRbb0yPt25eN6qWETU1BYJ8eOHvWZgvoeY8xFZX34h4O5lEq8sCdr7em1ejgYGdhDPdMLIFJbf7zhYaxqYpLFZMVnorU4fZA+2ft0+3mjZ8fRby57E21cnLygDrhjlM2YNM07D6/QrJzV9Ka1I8M+4qu/4PuFCE7AlA7oGq/gGEaLymnKed8RS0Zv3X9Rb7UdTlOVgJqSR1UaniUnHUAGmk75j6TkRp+5TSeexkLnnOknhjQ4q2+BxfY5LlkQCSCfrAjWLXkVyGhgm/e6NawupkRGaa7+wVL91F4aX19p6NLCe19y7yje9r3HM1Lyz/Ab+APHCtrneUDD7stu1pBDYbHode1l7RyNDSQ4wLXiwANDzDcTYunCKH341p0Nd9IV/T5OO/xae/rVGbIy5EuifjMtRuzuajiveavqYVH/HdE9eC6TR11BP8YDPFyCDJ1mVOYKV474CnPSJmgs9Zc+NjJXRu8Lq9rw2M7Tz+1H73Ph+DB+OhjUmOTvZnYg4i1NHA5zCvebGxrcz327EvPP3/7t6rgkfkvlEjZQo6pdnNTaoyImjHEqaVG8/Wg8xo1V7OybZFmHR+nfVFOipqzFUo1umEFKfrAmi1XfGykgI1wpS021+Xlxe0KP3mxqHdJicoPhMpXI4a72JDJVGaKmbpMH0DMT+QGOWL4FVN37hXZKUW+zSFEQeGbIkl/b9IcP5Zeo4th/7tx3/cIvwpOVc6qKuFVKcCJg8KSqhnWTpxh1IgKXH7+Z0yMWuI4TlIzDILZLoLURJArZouJJuHev34x5mKNUC2io9q98f74vVoozKpOcFT/nZqMYFCrltVxD+jqbfyuetFfgox/TN5XQ39YpLXnX3RbKKnzK+QEJo5GpikE44zIsgEfbXPy2aXBxOth+x30GT+pPjxuSns6IxuOFTm2Y7E6qM5Km+6FctRJylaJdlhWlkdOskPCbcMXk8d1CxiMxzhQu0XlX2k4i3cgTtN2DBwhT7Xscxh2qfmvKu/qdiMtSoRODAF1lRDVHAjcLFhdNjjkzMpUWKC12znbtKvRumtuZ5ONG+yc2wUHY9W8ATNSo9hwdESzyWCTYS1B6h0CF+G+gNYkZOqE9FGTK3CEV8hvIRgCwMe6Z+9k1qTb7aNe76TSoi2hnITW2ntyv6KGM0m4zSMwIhHB6f9Ryat/DhYS8PqORXEqokys8pDkq/cahurZCcZx7bcNrJfHu8iU7iZhMHoC6QZTqg0edKaFzs4q3qikhNPqNyl3X+g98ETAcfTXOjc+PBPn9NoiPKwGr1N+XtPW5rQ43S7S1hhVDf8Gm140GjtaTsxts06bWEcqamJaqnfe9zrmqoOTNWIRG5Phm4mW+iU4ujJkTUvWYvxYRPahjzWymFKrmnpvgvpaWEFVI1j4xWH45fswLjeTtaY3N33j3Zged7PSVAmLh4InTWwGAVTYdLveDBH0qBaS4rFp7/e/LbxmOeC5dsweu+oH3C91Gp94YnCADPSNNdC1hzGnjhcQF35p67UVXEsFNtwyOCo1nGZWX3ZzBIlITCxlxfFSiH6RwiqeU/EqCXOYxWj5wWY5s47zMPQgzL3UCQXKYr1QqeZ8WPq40uNNL38ziUkvezM5coYtISf5zbJ0JunN8hE3E/UypN78A5IEB/q7UvdvVWW8uKMe9ezIX3jEn/q68ooJ187cuP4qFj+su8buLjN41P1h1k7E2qL39GZJkWIhHQoGSxotlqC7P9qQ2WhPwvszArX88ceXUyAh0VxJeKH4hYRAnS/L+Xd6W6t2GSKK8E39zVvF8TC76+8W/3F+7cBhOlOCJKNUYufdyK27qZgyTilu6hR6OSwIjKwLO9hxfN0KwN9CL4PuYacxaQj27XOnW+/YL+qiMpheaylhMuaEXc5VfxzAnb/OqUyuKDMn5/w6EVqOjwiWy2U6olcCZ+edz6vOS5KS8sbh58+cu+TyOXxPAOPuaaVzsQD2dwhKqZCmoiMO4tCzCBGF+/NWwbAOk9kmxprHCrGZBVNeb6wBCA6/Y0NHpJ8LF2JTr2oW+DhitHm0MArq+bdkXF+dfjauT+OnLfIXOg3BpE6lhrws6a8iPjt544NclLb4u5hnkiNbVuf1oNqOzjr0so5Ul22uyemni9ug9i8meGv5OY+mk7LRNqqTdmQZc0Ya0Y3LuIn/fFyHky7spDsLwdZ5PJ6ZPCf2jRom9nrnoNlJ1e4kGxucozt3pfD867jIhuycbJSV6gv04fXv3SvMc6i9lI6R885FjvBue+maSVTMfmyi2lbR0lKxon5tS1XKuhxsA1RIsPTCZT9Dctal+IpadOsWCRdFMLAVXjlYYciwFD9ssAdjFm8pU3c1qw8uDnzvu1KWZUlNZVR5YpYB1nzW1hawWbcEE7bm7GjdkfOPASj2rtxH9Loekx53bYFCYhBjYiuXVlZujGsIFBHmKlWJcnP4KNqjx4xLg6808y2oxZG5KTUjRgWpmtXxKU5ErtbMKuz21taDB9pa7Pzkr9+wyCrrWsAhB2VrJU4yTb1qDgmuoggT7JldsRPBJARO0tTqY0iL4PQ/IrSakXOBNVq0vjcg6Syy8UUAGT/PgwQm0oIJ4yMlHMGYcCi0aIcQwAu+oPKeq5817XIF+S1ffOHxycBAgNVdijcoQ9ymOwPQ7xE98PkM8TxiGJOPQZPudc9dyzFBcG7JdXq9PjE7uFZCFKxpHOKNPRBPQ9asN94vnd9MZ4V69nF5Y1n1u5swQTEyYjaTYjd7g64YPl1fW2bonJFuSYV+C05Yrbabn1v+7geZ7LXAR+d7bzq4pAzMDwyoTXrXQ7SB5Pt+HE5poQeB/H38LiRr399NBQ4raMFvvkAZM8jN3/TtF9RAOtJAL/TnnX86yCcbwZStn80MuvvG83CrlcEZe6Kna36Rm7apkdo2PS0EeUCGYessgkgFJ60YT1cczvc9FBr6SQaXEEeJFBNyICGc6C0sc2VjMPjbGum1UsrPsTcScG+DIUIEe7q96tkfHD5M0j2HD1dX901v2RIq/hbqKypyLdz7GUxAI0i2h15M5cJ+DNMo3rKzNlRRxWxQVqf9fM1ftXfxQkAqav3wCkJizKk7Pyd3mV2titS/RvFH3+mrUDtl0z17dAnFaIjFpcoDZpiuBV9JxR25g7lhmUpG7XZfXtDpZfbAhIWctnHKgnfCixJ24jeTtqGgxxOE0oTb43FzmnckjVZOVrCiIRpMCLF9eZnpPqus218uWMNRcaSqJo5T454AR1nhv/mwh337ZZ2X0WZVV220KNK2jg5BMLvcTMm32o2p/n3TxCgMZCzOKJI68ogXSNdymVaPxWFoUhuCyE41vPLycWrbZHFmXXWaJa1g7P6iUr1ODvpxQWlNo6Q41G944KjHs4SIIXJ705vddx9lITpiGyR5lQpLmbfxy5OtrpbviyyFEZ7hb6modG/C22Q+PI94IphIeODETnVgnjwUmYH4xRMLnj5qZJ/b7HR2ocZduxol6c4R7PqeKPDIVQeWC2wsFEuATAdpv2Hz2K8yKUeuRw+PnlmAwcNWmz12thrhUswGQgkal1b39ZrUlaTzl9M4TFC95WArOtpaTW1kInmVhiIn2RhZvtgyiPapd82xUa9GxWVn1krVZRAGjrsXetUffnmKgk7TKBJSk3VLUzRCmYRELGFxwo4ocSpTCeuucCsVqh1tSkXlt2/tG2NP9kTBd67HP+3M0EmdBQu/C9gUMXSxckt6Mq/gIaLfj4/wCZ7M0cWO0YtrMutcbg3ReETRFl2fX+vaf+H4wz0mX8cVDbcMUwhyGKNNzYhpGgyDHHp+hl5Y7wiaGKDcBgkmcxdR0ZafE9dEqFgx6n/Q9a70xVFQ7vd3yE2mJ9SrjamHQhg06MC/aE0EDfoQfb8y6qePEa6LxSM5fyL6m0j/YUlFUoXy9bVlOX/5a0sZcPeg1bpNQzQHEqYTQkTyzR2PywsqDjda31UdBfm6mjZBpySTzo7nB5odVwxT7u1KJ7fcHbxxERcrIw/0kpNfLsXP1dQEWasE4/sVnHaxHtnXfMAkaXW9NMSPxfdbrQzPeLU/oRDVzH6CYBTSd8afk6aLXNjOiIzxbNnuGD/+mrRpEBPleJXuN2o93X76DED87O3pMcVs8axCkqySPLdq7kzo7JQa13zuZoJjmi3BzeCj2Ajcd7lGJVYMdAqzbH6YMUCUeyyYzEwK+l2z78iDE30Lx27UYLNfGstfE9P/34HcaaUQzYlThkTuiYOdyuj8VT+FgbdQ01f7nF75ZRilOKV47bO/9XxFRoh9eHB32I96yY5LowBBFuzhwSofQ5g6gQu1zbcMIYkDwTXFmRGGQw98h69lG60T2p5L4+OXPKTaNdZFpFI1P54np62jo/P7a+nWOpjMCJ6DsiHxd1RnFqwZ9P/zWmnTponEbhzql5MrVZu/HRu7di0h0RGRUlFPV8fmov26H34YraqxBCGTEoxNAnqtLGtRnef0uDRmPaVA45xPKiGhwzMzye7CIyPJfyn9U1RI36r7/ru5+n4MP1Bssz56IG9sHdkXll0afzoPnn7u2ENtalGk1g8/pHiOFVWn8IQ3zGZ32MJckh3ZJXsKvSs1ha/a+FsZAF1mPpK9Li85Qjp+Kv9ua/jAgH3t2t8uu+z5vm3FebhJ7hIXvlqrQ/wjMrN+xWBa6kDpQGoahCqmzhqWi6/9FelkiSDI4+MsKa8pNZP6ZwK1pvY+y5ryakiNXbXikYnAhessAbQaoslQOHWTgcBE5uRHvlvnVGTsF3JghBtcJF1d3uXU6f9bdEUkppO+PnOFuc/6jmJUEW1Lr+CjYWczAIu5mFoZt1ZiFI2Gpzt/tyI5WjOwMXHP4Eow1ldTfDQbop3cyVjaokvDkGwfo6o5nIgZrDtZuybnuagjmBkdicp11UHKvfEWIH8MeyLHNZVMfIzMTIg+k6N7W5aVB/SvQuSvH+3WvSM3FecVZpuUe97Bkaj3r/HrxRlqZO6Y1WESFw/40bt/oF3xIeW0Mr915vbZLE7w344Prhy6dqNKkrIpyaGPOz8GTU2WU0GU5GP+gIuNJODG9wPlpkXWWOf7hsbMvciqcqH4+hw+p6y07mX6D5Ues6DY31h0UIRQDoIo4hDnqd00Fn4hacCeMJQdCBmChsigvFXh9cx9o+J8H/e23qgBMm5ZxhhZLfmh0bU+1zmqea9uwf72GU1UTlCWZv2LMRr6Xanr60bJUQfuyNGW+ku1Y3NcEMoAn4YVe/NzMIdzPmqTGnK8GS5crFYdlcv6vuTXLlEXCZtiVsNwjGa27mOpKnhJ8WIroRdLFkkWr7qfM5pzij8wFE/tUFPoy/aOEJLyxDAm5yS5UxfcqDaWkn+xOfYZOHKGjTCnIIKkH1/HmTwiqre3WlmpOpEtfk7wYJd36xgmra5m7uRcTcsDbtMUJ2vMy8yaKP+hSjoZhA9Kpt1ccmjzZk83aam7JTlK9psWLfwCWMKG3/29OmYyIXWg6ti/C1wsgfx3nG3m5gjtOxn0IzrVngqQGnWpB9RLL2TpPonPCsE2WJ0FVywoM7KmzRjDLXKoQ6REEFaXZusuZ7rvFa8T5ZMtuT9qZW/i90oytwPoQ7/TXH257nRiW+QLeF2zFUfeT3qJmqaeV047jw9Pbs8JE3UNKIYViVGjL99BkTPsKfY0G/afjPLNqwKqbxQ5+sq2QF7vLXE18WpnShPZ4Ege8E3HlRafpS01XlAyROuDLH6wUefe03NqcdkjzAZ2ZrNmIC83FqjaLas2huvJTNVntiJvnL+WJVay8wOq2rNefxwuPU8cC+/SWM26Ff+y7/g5ZYnqQELW42pFfqsx36aOCDWTzKVS4dpLqVH5tdV3kpM+TUyJXuSllmjj1fFpyzTJiamK4P9Q6e8Tk2v9rgXBjCGNwiISMVPeuGkv+quC0/GGDbuxnmVGGL2dcTNz8uwR3N4HDxPfcxdbJJ9kpTxNQ0J6qKlJldAPSBQ0hGF499wZf3GFIWibwfDtILN1iUGa8tqIY2HVv+mN/xqrP5AYNkOWlaV1mQ2uEevx7eZml0zR8NYKG43JZEZ+tfAXr7Xj6UN33F3e2vFr6fP5H3d8/LHMSKXZFv+wz2ZLSjRLwR218lS7tJT6bBjz3QPTr+btkP5AS8pQ3EFsZcywJK5P8X4FcHD40+X2PkGoriaxR75typZDR2ixpKbUHP6BPFFeUwltv+6YHPzFKj+t1QTkcYEtzCmNLmZQkKHJiB3kc/CHBHvRBw4JtrDtk2HGAgXde4HGCA4lItFLbpfiOXIoKzDIwWOvye8Tw0BkHWzrhnHkR25yaexBfLcaQnQkyR99CCG9KZVr0velZP8WxhlSrKT6abM73gHdJEzhIs4T4u7lreGZCN8BPYINwLXZlz0zkT6MAklS4JGH+wLO/0UWV7ywjCmBGH/kiHmIpM/vqg5m0neCy8fslGYayPrBvKaon7hMjMjDewak/BSfz8evaG4Ec5iwiEO+CZSXFoHwW+qGK/Nw+eKjOecWl+OhNdwR8aOFyv++SxeY/1L15bxHJ/4Z/gXdC9aD+vqLF9Ob4gNyY52wcb28fn1zS4zPs3/TV4DCsCxheTGvOhF8uUql/aW/S/+OgfAMcQcuOv5ZPOyeUj7DTk8ANUV5ryNDETGwzNweokdOgKFKi4Ntt27o2tzFfycPM40dXZTdJR96RuRENZO6oL1H9MChHT5RnDokiQKp5+1qZXn2mgkUKPQKbs6o6XazdCrrawisv7NeBcyw4Eg39copAm+zOahiMvJZsZNh6lsg3MYk+M1ffCk6iH0EF3zhGRcNtKXNyzI4MN/aeuy9Y8cDbsxu0UbXdKKFmxcho4ZMx+KOW+iPWzzApwJerv+o9K23SlEVZkno6A1C7P6ZC3IafRLxBNXo6e1appEKo4VSTdnsNbRoGqU0TBxhs0bVly8qB+20BEW1xQ9oSop1DF5OxWlhl6NPUB8bqwXTlQC5/YeS/6dmUcqYzKzY1Lq61Ni4+piwuIqbpmWgYO3fn9cWoZpRjWxsXfi6MRnloEW1n/8NNRaFcSupjxb4f8ndQ9Oym3EVYVuXzB80/D5AknalsKwsSO0/FRqPuevd+oYTgqsGjus+BP3ftHZ4O6xbommyzyBCMSmKJJiX9eoD2PyrZ7M+7sL96+pXrNoClpw6bpP1oKihP+2QPZhE2QmtcCHWJIS/NYQZz5NblU6Fq8otQNajqVvwWWPW58BV9d2DCVuQvgTBgZwTXjEL+6UqjpRE780a7jc5rbNTugHK3alVZvfVsaigRUsax6V6kMtrV8yRoZ587muVHTmqjl7QPM6iII5utxrutZesnDm9a+cZ17gxYw3dd2tyVUULd7DFx9KiDCHKtWuZJgiO9zAzXm7OeHiRA0lIi02yTXhZdhqZESAz0sQfswCGeb2TY8n0puBCr+RqWtJzTCCs/Y8eLIlIbBodxWVG+9xjwXv1Sq0dRGb8yM+wrIBdEsw7G8hhTjSInNeVlUkvY1sbMRbbbC/+B3q2u7s2zZJqnqyKoSEYP30Dhy6r544mdU4Njmy8OPScnze/fAU9Q9369PV1WYbcE8c3+E2/SWpePFk9WN3Sw29dX3t9he9S/PFtH0gfbDte9JtGfYua1Uqlt1a/x0Hs33A8/tKLLUtnebcbN9y86XH+Qrt+KXejb1ymoab8XB5nqxNZ36vjl7RHhs4PtSLrtCtba/oQzrZ9CBGnfw9i3eX/9o94Td7mqBOD8qnfS9TT73p7+4vRqlU+Rtd9EjTwix7iDCSC8PJIJmJNSv/bRgxLGBGJzGwZ1MCYs8uHfOAUwWRsMRsxnmEALhs6KBPIb0NeE7foY1VrdoVBkwOB7+W5g6q7sz165B+kfnSYN9z3kFR6BRY0kfh+5F9kTn7mSN7vV1kKcAttblywhH6fNxInyOyLcS+IDdi1sdKZyAI9XwTA4d5YLvAqWuebNhSy5TdHR/y3F+hYy0eMDU8ezM0Q36ABlhzq59KUHPZVzCS6OtJVnrapc4PVbjOiCNHksNErZOd/IfVt19xN8/AuSXBmv1Mpn+Brbn2cGAgp5LG1o9EZt3lIKhRPOCiaUNJDqOh+GGIQP+UWZcnY2qQJ0UFadebWlaRSOQfb5ZVyaX8bhjyEFSURiDtYIfrBy7uejyGzgTB5afXK1qYeDYzMWkLeDHRpq9PulLTlHbq5rsGHfBp1Y2zNY2wtUs78leU/eR907ffN3XoR+A2upziI26/nmrj9+/Eoude3i8SOOLTHZZCxZ/HUQtKUQ8o5EC7ABdwra0bZyy875NhX7fBlPU0Yt6pijGpuY00Q48mgshO7vdxul92JKkhTcWlT496mpr2NTaVU8Q0Ch3MVk8O/489GXmmrQ/UlR8xrvhlMiqBY0oJ6w3im7YlYfl7bJaaCueHMwYgfxfTHWW8/DS4rvj5OIYylTEblH7uKMxI+AWczhI/pFw8kZPiD8eESsh6H3GJl+HYMwEHjWd8jhsygLLTUUxOTc8RYAh+xW0ofpoc7xt23eksDSnuX1dbU1m7cqMjQ1ruZvDHJzsUdKOkYWHnuYAiRgmqX3FxcEMRqVs65w2WUZeNkr7d1KeiGSdk9Q2fUhtX6jqN2E8DxhmeafHQJiloXhI3ReFcsDqzIlh+31NUFZAkEw/I/57D0gbcuTDbopcB2V82EnHbcfppwBIuiiIkjKiptnDmQwx2IlAwNnrzTqjdHkFSRVbTqYysIevgdNL2nTGYGITWaZDPnKdEkMBKjQyLF+K0MwT2eoeIsb+2i6tlPHdj0byq9Mjz8ZDxxJtIw6U8aZSwhgF+c27cpBfYcc0713HOswFmLkdUqXHyOQy/3keoAcb7EE/lALRn7I1/NL2rlZA7ORs1fB67mKQPdPd102rDWh3y+LLFNuqvMgT5NLBzhnDg4GZi+GHx5OvWM/jHiCSEGGJGBAUCmxNrsYPLq5lBBIcpn1Yseupr0SPGF44HHcX6cVfGYw2MiBV22UCLuPIK9+L3sXpMDEyw55jA0AD6Yt7fm0GsS7hCnIVvuQH0ZWrXqpGjauGNjX+QM+zdhJ6sepdX0o8arIYvyd/a+uodbF/PzFNstDnuwZPIOXnO45SfnJ4Vbyl3xrgsCN27YtwVWrxl5qxX3s7cfGxLytqly3WJQjol9jeatEeEduZ1bw2c2O6DN2JIOwkpXsUXOcRHxOHN+uj4J+RH4qL6K8wvdxsTuyrghxNqo1RTOFWriOyp82l9KHHAUAm9bJ9Z9wqubZbdt3jfra8N4Qaj2RoKJ0WodvzadYqgn3knOLx8RfwDbkVMvHzKpWJaZQlMMePB5iK9x9ppgDeK+9YrRXi7mDbh13mDc9fYRiVo1R4vZTD/8SJ8zJly8+fjljIwIY9yjaRDNW+Wpzc+F77/AKPUjEwvH7BU4O7fC4ETkM2E72RPslIWSwl0PqhrZaeWT/0k+50WUHIt4uHwRfjIOO22Z1boywV7eJr7L0EZ+1ak7GplVK29XiDvAypWBttb7ra3UKw34RPPeLEunBbdmnAioAJIC62it0nvqD68lLdVihm5Uo9vQ2iFjoVfAHXKDIzojg7cHtvEH9uLadz/ZdBePIIRGsGk8r8muIQCH8OYhzK+6wm3E/M53QkYFKYyAcyd+72GPWB+SvXQzN40DM0vmv9Vwg4vTELBnr5CC3WLYbkqKpXWBUbVqMtFKk7ulAkRZS/31NbrXwFSoAhSpCoDq9+rKof4pKQLuHnskFDbG+KXGuUN4jeAJ0iqmQWRmwIAVI9I9s2tqzGEVz5z2TEoi6wmyH/wQSNG1G6Lii7kohGUqBbRokdimSX9L9Kq2tbRpHMzul+dC4nfrLyeaRS/HvploCBoSD0APTZqESUZGpH37JHN14OyyxnBLTASbJj0fl6iWBHbkNx7exPZHerpV9Z806CFlQ9wAnE8jKuWg8s69n2f1jPgt1Dtb+CbCWNhqzK8j401YxDw6MGLgvSoC7A6AVV1ok5OrPU4GAwS7/S7Px5P/ZVG5X95PEhFjttYWtpk1rmxuXlnahBs6bieEfjWTagnKzbbS2KgkjY5ZGluDrXylwyHIgNn1xiceM3yDFkVmyn8SscTCw8ZY6JTulAJlxv7JckCfVsAJSyISwTcVJvR5W9jtYjCmnvZtA4/EvuPIOihHDBklLZ+gMPWsOx0OQ3FMi3u1dCetjM6RoJqNcqjTBxD9jg78JjtjO+0mJs0kiZzIFJofTvNZs/qANnnHd0vtKa+cKPXuSnvvg4jy9Ve2VI3gETh4b9vV9zbTzdSutGRH4brvxu0pLadOPOHcnfbe8S/WBjFg0CN1zST6PdZra68hCVTEUO5rqNgp0hgcXUAgmsYZneGU/qJGK87UlGekES/46/Dco4Fdu5wB3wvYj5VsTbW9eE519eanN90eWVl5sIUcJAe0OEzkrjTm1/86fKzJLQ1LzY2LjglNXluZnjuOwgwl+3MydObUnPCQesUo3GMWfg4ubcOkaNiSHo3ieM5vFOXthMm35ee8lZk7ozfm/geHzTdwxuL6vq4aTXfJOo3iVI5TsWJwckxMSSx/MrtXv+Bj9hG65uyrsblP6EqjuZzkc4t00i8dxQWW6h5LdUEUGhU2snZkSWHGBy8ODOqGQ6I5fqJffpJtSA7/UbQg4PfE2y6/Az7/6Ewm3rkIPH135cYqd1h45iu1lfLROiOUdhwcmmyZjH9dtq1tMi9v0pkMSaUuPH47Ce3OTTybl7qUW7r1P5j4dNb8t2FnASmdIkHFz4ESkSg+wT9tRs2qEsuqmspGL6scVDqQQ4RCnDPpQRi7GSsD5ii9wojDsTaZArQQnDgs3ZmrWhTYxVUt3s93dtphcIKmCckTfCPRGuaEFq7pWuop9vMbNI41tcx2nqApGro+zjIEDFlpOYacXZSGaKjwhJtJT24IobWQ75YsQyuS6n1z7taPeIdiVPEX9SqvjY3b9yO3QJj3HYHcHXRH7xENIaAC9a6G2vRRVJeyh67nCIGu+jGvdicTXtKxtTHM8WBnXe2PGY3hHR3hJQ8OJ64m21aTxEoFKHZcCdDvPX/jVHjDcseasLBZ75un1oQt7PrdPFgStj/88bBTUfgShmYwTFUEj8e4RPpMmGXNEuRAPyAbuCiE5jtykXCrRbUAIpPQmwx3ecDZ2xZNaVZdWKIO5v3q+G8mBXhnXcDwu1G6PWNY+Zbcl0ZX402EhAAMcFu6bWPbBvKfXIXFRcyW1FIWZ7w2LbXG3P9IjHpPMV30UdY7oY+FMpkTWzrXRioWKdZGNnVMZK4MIUfCx5DspQtLuTzCPyabOpG4oDTi7gkYh/dypo3T2qlx2Lg8BVzkGJ51NEIJPOOwpwfGso5nDWYNZcGOsIrxBDJee9Cb9a9WueiTvE/enFBB8ZGHreiGeMdjXkM04U5DUGA0jPMa2yrA8MR69JCnN562lWV6ZKQzJurkkPKm3DxEdF11QFNksy0tnH7oIbqJpyHqdYSSuLVeBjFI1/lDAjSnOhUK3YeEIn3NUWVNnWu4DZ9SRGiG9QtY0DBrMv1ITTxts64+h5Bga+oNs+0l/Izhcc/kfni96VPLnir6y7JZhbQKNw4MnhkceHqwbw1d5S6p70ko3xzdkbKTFqt0A1NjtiQzI/PJR6PnpYmQXk5AeW3RffPqRkPI9VhFqiJ2Bnlj3jyNDgQrBFtdtJk0WSu6apfbcGWoKjQDty1//xyuSLelriObizJr8f0DeDgtVjmMD9zPxZBPoUg3+3JQkzYlJWVzqv6L5TU1yyorqUr3icjpd/8WkvjF7z/pzh/ODwnuyTYp5bTULalpW1JT4PXv9YZbrQFO1u8MTTV63vrdHTADiabtnx/1erXEVfwDtE1a1ifMr0vTl7Uve9Spn40TWn/r5Xdt3CiBoCHYOm8o/YeHEIWwJUF7qWJSu6Qj++0qmzxO7o7EkhOAyI5d5gns0R86lOsl4Hsfsyf03Dlp374pluUlSLFV4hcW3w5MM6fnuOm5ebP8ovTPaIL07KDhZCltdZg2BTt995TSatUfq4aLKAn1uCQ7H7DZ9j/XsOCTfDOz+w1v3tfWdhp4/fU2e7R/nvrnz9Wtnw0OxgU25NUJCJem1F/dlht1x+w3rA336CySpb6Ma+PTKfUGFiCug2wn53AknnLlcAU3H/ZNpO1M+o0rc3WiVRRGsGKTYfpJt5ICe9L3EcEGWAziaoywH4KdURCJjscYpKDPxV1z6suyjC4woQXvPsJJc5cWUqhM0tQLB+KMXHnBp8VpTJKqpWlJS2b/0biUauXnvkPrVUoUQJ+habRpoW7wr9F/2jE2bntdIeAmGUdhUTj/VBO/ZHyux/AV/5VhZEtXVdRv6Wlawt9rmzVU8BUGcZZwg20PvP32/OTbAd2lwfwtj2/6/2CU1fbV3R/r4eXbfzOBlN9lDm4pSNrm3P3KbFslb5w/UPFvuD6H1z3FXzXXOwdPsP9w4+lo+8vVpe3puNphSWvpD9vNjGFuYso9mrA984IbmaC6Q8vq+NH4VbyvwObCFxqiUWM85XY4zjypVrN65Ry6IK0rHUfI00miocUg6z/LReHwYDwXP6g0cPFDquXS5Bo09u0WO+wCRtw2XUZz3JyAqJJJXm/5VXOyrKckNxAhDxkFcWMsSxetPN/qYj3Pu6APEUj7IGhyLMR5xDFC3BNo7pDYYvC60bdkT48UVOlDca6Z8NVFH/3m9xO/Xb++ISlTkZn0/uDM1tbWRcs28BFBK0ZUmBklE3OSn1KaoMQYlp6H2IRDhYnIijE2Fo/wdlv9xmMNWUy0udM+Ak44zDla12p1RxnE2Nvbh7ehUDGbC2e72QiRUGrTUyShitNAsfcUaCvGvvCKZQqPwxAwNEZYadC6m5rEwBAHGckiOxwk+4/2OaeZTqjpoAztVaGlxNhtLzZexomeaCdB050q2buY7TjN0JAKNE2reAMfUmLw4yihe/aEj9D9bprFTAIm0+rEoyK1c1Ec8r1SMzRDdV6w4rvpAqtHziHrdmH9luzAIqCrD7ewMn2SbdMwatwgtbPy3rda6ZIbKZHKoDDF0LbXeyicqu//Nzjfbic62FIunnptDEUeNo2/UFht1cZbjaQEC2DkW/bbIVFo1dcKNP1bVBSp6kavRNs0K1I7agTWPGjcd3xP39OKJxPU7sEiGq32HgMCtVJ4TdFqBOOlLNXM0IfEFOsiiKrtqrVRb9xlRXc4E4aa02gpSNH0yt9enNN8VQ6WE+b7YnR29J94CB3pZlt1m36oZUY1HmnEdu9Fw1kZlH1ogLrWx+leBasR3fsnrmlo+Qkyr0GRM3vs5OItP47dpTfP1yAK/S55JfflQFM/rbbWb/dxFd78f4jkeaj4MwfN/ejaGeqqyZqz1+ARK4w458T1bLNVLPxzxb9Ake7rFwwIt8WVEdaPdqq4h0LtHzNrdv/Rf0ceax09PRXFdcR0D9XFF0W03JyBQOOfdBIfopv7zI1uqi5CPyq6KLC6w3HXVl36PoiaHY9ARREJQzA8PBn5oZvLDiLpZEGH6jXTNc+vyziha76W2pCX+XOT29hDVVaP90Cluwnl8GCk8PF/OZnerb8cbdWsSOu4K4vde6iCiIN9qBzqpqeinrbdbVOtUUzgZd/kohNdObPoS14f05KFKQOJ5Ze1YSuBASAfid8hDwF4MY8JHUcoEjT5C9uR5149CNrNYAkzbg8YjNcTnyKaOQGBx3yLkHNCUpm0bGIU8ZmwT5n5SAmSXNiNu/YzduP47eRMyGQC0H+pJD6HaSVPIRwVUzY8rbOMMBuNMEeMGdn0APZlggjpnwojMG6Cgh4QbVqek2BvnTfwbpEJirmXxXm3YU6NIak2/d/z2eHRD38HszYid9q//eT7TueTSZUYe1RS38gxiKgp6EyD6z7iBw3aMjqLNARMsKSJqLhHj4kjU6+iVKckracgWCNFfOscWM06mPkzdJNrWWWsJ7WZmCUbnL3TXxOQzQzRm6gITwDDqkrHeIKfwMTXJ6O2uwPAFLsic+G3HG2KFxcZ9a1/5gqW2mBMVcYQ/KQGI0BN98myoqYPl4YLWVrwodaEo6+TOcVaRs2Ac72la6ItbW4qzTDMG7Taaa3uJKfJU034BeDZRltBep6AO14zAyOT3xDl+QcJt4z07sHynz755P//FzV2Ov9985OD8i9V1T3dkwYpz6AkBkq2fxcVq+P/fplXtOLWkA+ONBV7BtGxROTK0Ftvjb75liZtoVs3/a9fqJa7Qd1sAp+gYd8vAUUbalN8YdveDrz4Mz4kjmBhVrv5p2O7sH89u//AuvUHhDYIEs0uhrqRrwDaoOsL5Xa0u2+C6Xa5lweONK5aBfayRKKoxdW0GTEZv/BN2Pwe+rkTGS57YS/92Zf9N3A5P394woRZjNTktv+/A8vPy9/9OHd97feSAUcq1Mv+/DU/XkFdWzj5pw1qWVBeeQDb2PXa5s2/GXj66TG9jDCCtyMv8rtJm9i2TeJJexdrJ20OpFKPrIxtTNIrlh178tgypFge25gMBIQQlOg+QX7bCzux2BgcMyJjOpuOzQ4df+6k7h8p+hRFY9HGD95Py7saVRCFPciy8T7oIF4SoGqF6HaACkHEGBO/A3lED2Z8HoIgiGVnZ13qy8sqebs0pqBaPa3oNuZlvCWdeCSwbftkYM6b8GBwhDzYA7jYmfL6htfTbvX9Bf1lS4sAfQizepeyLqmtD4ZsQRMzXSlIAT4v64SF4R4648ZMqWqoU2nI/6jkfaYTdR/1R3ueCn+t9EzT38G/0N2bHFtDkUBhim8835bHFIaU0DQt0xgpLgHNVeMCWBZ9XhVp/WAwMuxQbAUexONLNiT4fBqxV1Z1lV/HHuBn3FYYjcDajn+RukePcbt5xwaSSkvSa4XMhljpwZTXedvU11MO9i03Od057js4U0PGZfaF7Q5LyCyv15eE1u4J2xUWn6l4qul6w38bmmgxtKLQdYUlSjBuee0FzrnfTMuMvo/PtO1/V4BGpOvuNqUX4yecbrG8MHVhwmKhqMzgZE7Lq4RDyzrydPKrvql27LhQeDCvOMSO/STLLowGb0f9556n73igyDOf+4dgf/nruRsOOAxGt56Xqxt5q54NBWDkSZ+hW9K3RJeuo+bVOlniiUccd3jVMME4kGMqyAWGF+z2ipOLeln1Au4TIiYXP76VF9ipXyRNSL67BOdl71SKOUUE808/rDSGCdNQhWR1Nr71SuppWFHwYc6B5colzUpa+gvMmmH4d3z2wJPPFUQFHZEoo5WzLNdO+pJl4z/Qnxe1ZKxbpDyj+EvyqtZWwLPgyEl2FzeXwSYXN9uPxWyDncpB5S7D1oRP/fA7Ng1Wrw64E138hvfEcOD3qQG7PbB1i3NgINDaKv3/Ko9Xjw+e2vDWWx0de4u8RFyleV6qq1tRFVfR5khSUX//rx4ZbOxXiJkHQcb4hBlCma1cnPzlBLQXqWfVNOlHkbdgnvcIozL/gVkeT3mEZaZDlp8SYq8hxF13YTAjQqbKMxK6nVnurrAQjfS4qIpKQKK2ltRbO6ZZSy8oVhdumbcH7vpHoAei3vro3od2Tqz/g7pD1Pyv+1yQ6BOuUTl9YXDowuSZp30J1Zqyd68m5SZdfbfMQcuaNFUJG/ctzbX0dNfm9+X25esTIIrph1DMCGN3M3jyc8zMjiH9fgbCasFn0zdD56HKqsHAln1qRVXl4PxzbPfujRvnjd3u9EAvywW4kIpan4ZcbfTKleChd195kuSgzs6dgzw/Gdqs3OpuHTph4wcPbMjY1ShOvyhwdVAcbz9vVN0pysgL/EYeBWLbek0FucBMp7ZkNnWbU2lV9GmaGh20UxP8uzEEo9siryFgbyZ8MLojk6lxeagV8ieQ/hUt3Avh4W8mHkocSHi5Gpzd4WsbGWZM8qerU0e7oYD9Lz7467Lbm4xZpxtsCs6t0PrB/pBpMx46UbqZWWf9dYrWo128/uKqZwpp5ZrrtSmV40i96uSneXkXmCNih49szCYyL+UtRStqjX+j4Y7D4eYkQfSFwcklsjaUBs9qQbbZJ7w59Wti102ksMvFzGwhu1Tp1LE5m7ul95R9ynt7q+9NhF9nsCd+D5BhOMbkZQhDeIZh5gRmhDGRvQk+eH/6MfMJAvZ7/Ej2adkkTUjGvf3QuSww7VkjIgZ5vRInrRX+t9VSvslSHp3lDrhB8GpRJJLzE1nyojKFumV/vPMbh4d4HJ/D1bbHb+cdzxsWRNgvV7S07G9tXVW0pvzGWKXbi7zIXXytRv75ZxKiVVMuW/yhLMWy1jpxlh1bBPo4K8GMXY9V/BSdmoOz4qTLNmHdup91Qc9oT2CsDvdZYKw+txAFZxzaKMs9rA6qwzL2UU70cJQ6aq+u3KU5JB8oKfN49bxdbYMo783oDccyjI+SD2r0hoHfmh+d3EwBtbB1PdPOqI5RHcrNn8VghswYFFrzVxoD606L762gqBW3VTdzGB2fcweARUQZKlGcIqgiLkgVJ1IyxIWRezr5SDLPiuW1cL/g8n7itmHcRvvE27g/cfh+Cm5jBcg0tLIXOLyNeyJ3d+eTrNi1DZD53Uh95BpBuKB52eaAkNbEq2cb1esenFQt4Sg5lZuYxoSAkAIOKnZ2Rme7yXFio3lC6Gs7rV8tVXDtSet1p62GBuAuvI7qt+Uz+8uMsyG3MqChlamjl7NeHTVk4dHXlPTZyqryOgUxlwTDLIvBG6w8PR5U79ZcoIt15cVZTS0nS2bujtXwZ+5e8frzktFvkLrUN7XVE6lfX0kSy6mV6QF8R2S1QW7Gf+5TcatiGbVcLsnl1jJiV8WBsz/4j2rCLQw8p6AJSaBoNN7BxFZp5v63erodD2VZ0tDW7nfq6q7pUGWY6+yFcJ5oLXKUq5NvD9T16z58wOkpTRbxtn++t19PK6UpkfWIUPLCLde04vKNdZxAXrtA0M4L5NTtxvDWFrGkIjsqu0IKg27dGho9FRYfzrxw0GJacSwe4YlIkrdH1gxnBKAP3epPj2t/QrJJV/m30xm6jt7e9bqcsAWPtbF6o7w+g89q9amXcJP/0Vt9gZ0XQOE7TSRGNKFfefAztmxVY4+pqo8d87i6x7+gCGwMIyjb1FgMAQDtPr3T1/mSx4Vh2JTF7rTbQSFis485m13wpLz1NTx4kul25AbXM/ulrGbQWJDktUcQCNaJWwcD8mP4mH1S7fO2DXvg6hFssIJRBrOZDPCKuZHdD6H8r99ZIvJd+JrfqwjjcQx6aLkmyS3DNOe9MeAe8aWMLD6R9/LV+avqY/NTIg54GLV3KFb+si+2HEMjDSsUG1YVXtvz9OW2TAqfx6nMtstP99SGoyUV69YIErz2Rzrdo/Zk1IdSKGlAKASzOBBckqZQEBmfi2Kg3RBzPHacGc9pjOjz7wQVm4mMM1+Ct2PE1JeFwdb7seb3dB0ZE3vXB2e6ozsK1t8py4w136eO7/xlijT8rkhFdC/IHxl7JzKAtffRcYmx9JTznprqxYLLJExOwnNqXKtuEQL9otibfDrM/BnFMQTDX35UuPZkugz1+URZzvdKaAwaejT9L80l9d9FKL5Ld7dmdGJ4caK319Pb2/zt0aOBVR0dVYFj79Y/1gl9lEQ4/Ywxy2dJuB5cgN3X3NPMHaGFdaPRptpaFTHtbfs8g0sWlZkk1QouqlFmrFwXSk3mRa8Sb6cFW/JyfJGTbAKjfpBq86U1/FZQ9iPHox6FAYsRY8QXRjhrlsb0E1qihQuQRmjxY24m3wADHMCeYxHzhN/poeBgE6a5bbV5/uR8Q6K0t9Qeddyn7RiV559LX13lzhPECY7D2c+uj8kF5ZsJwr9TV+hwFOqW+635fS7xKKV82Dd6gW4pOL89tN8z3B2abDLNVgAyS14x+vnTSj5T1fGeUZCFCbyIgmxrfnRt5fyj/uSdy3ilv0mfldD8/eRSwr8Y2tU598wHllQVprKkfvDMXOeu3TFoLfkBPyM9AFMnqxszbzoS/w1FcW6yQ+N7/7fTb58fs+s0mAGyNQz0UYQXhcdL2sDBVu+vLMoPsC9XhU5ZGaHGypURBKZXEu35T4vpt/dXbCs81Pfb/qKfzfD5SQYWaQM9fU13KyhYb2UGT7jdEBwW0bvvDEEWXlYWzwJV8PV/BUe35Ju56739hCLIX8pt+jTkb6t+9WJNtfXt8nQmV+ovVxD9XtD4rT/JBJBApH7Odijx6Tlhlyv7tkJT/PR3JuMq0x/xKy74c5v3ZpOIhbylNHy8XSn9GJzqAK7F0z/3wm4nvuqNJTXVnqLCVTSxvx/kzclxQ4G7yIpf6gGhy8v8/YR+ND+Cxuml3SS7aZl+GgatVJSP5zU8I2qVbGxuUOO4wtEPRW99YpVWlhvT7RE3KhDfcwcy3QGZwQEWqxh/FTStF+zYltXTuCJpyRro5diXNg8OK3z/+7FC6d5U6hJbxIB+6kr+OwG44D1SxrSI9qKeJkFexHxtXlpUcRSVPupY5RR3KmUloE8uG4tYjJBv8c4jekiPYbZlw6PHIj0cNpmwiEK5WP1YtvTcPit4SfLpM6N3tGRlbdLzUWZ9/c7czKXVxZvG8gEhRHFAN2yDllL8l9UyuNyZ5bsQM32jhG9iQ47hJDkPOGKWnMAf5sDNHxwnIRS5RWLfNHfmwFsBI6wAk0viz7W7TnMZZS5wc6zMaAzurWiGrWShpU9Y19WKld/5nSXTK513StfBrsdypxEMmZ4daHj0hwPHo59XMxgsTXrPpcSnwwoQseW9zRH6OhzfTkT7YXBvUlGEfzozQBbsSXW5XCAYqmIOTtH+Mj3HTjt0G0thMgHQEHZiAtiWxa74DP1MIKjCjntJqLVOjPthgr/7ExD5oLY8sCfpa3d9XJKO81lMUPohfpIknMbwo3o3k37fJoBe6mabzTfMInlf+NVEm2veb1NgDD609jxf4jzkcUt0Rh6fFbuTz9JbnGuW5EVi4bElxveSPiYIBWEgVrQ8Rr4WPG70gZJgU8HOHsTsys+Qz849AFIqs0pWZ4hocT5Bmzdt+op3A3ycVMpLDbc/ssLibe+3PqFiN8iRbUBN9/4/7NMFahosAowvKq3mwO3jtz9NqlhutE/fsndkU7soe2yWRXgTuYFi350RKL3lesM/D+YeFVnUPZoeNU6Vc2XjWxuuty9tPxz+14b3g6mwoqu/rLHVXDoQUxT+4VnvNw/+2fhLNCIDlbU1H9i/tu+QOveXINVo+qhKxlDntE2s7dH0ojC55sDJUx+dPxV7auSnjpMHKic/DP9wze/KrPFA4bgy63eQPVUYeIB9gZXDusDeKuf3hkZQB+Rwj2wfR/7XTlh+FKdCZxWZAbzafHOVPE+aly1QrFhxe/JzOgu2dvwAgE2RFOnyZXbci5xmLQdOH47JfeAJbkukTPkWru3btepBJGu/UXjNv7/RkxNjLtT713hdMG328TuCBl3nAQ4H5XA085rAzd2/0YsDjUVOZIjAiCF5boi2fS8k5Agh2bzsZmV7FeQyWrwfiB/U+J9+ZxPtJiuI/G1vfPT0AY2TcVMrcDWnEBSx8LIm50/RwjlkTrjnJ/4hvyY/ePnNvzWtQ6/KL4DWYdwzgrz+Xfa7CI68jaO4pdWDFnyyiEXsGqHA/MBmB2Q8b+RVZTA5bC06j1b9qKOhd1Gtc96pjcekaX/cm86v3JAUe14/O6fC+/zA0SA18NE5xNawmbLhY7iPcBL2xmHPkgDs8CHcrR3S5iA7REyUGfUWth1F0cKm62q3fT6HwW719YSvhbXK+/a1gm+S/QmMsDjGnGMd4+BRvx1NoQ5gXK72vFJzGrn/rZNh/ghWSiUr/8sLwn2YtC7B2tE3qVSBK2ur+ay7CwDZiH6qN+ZvP7jyzuCd5RbcMUUqT58bEoCwjiHSQg6UOwwHZwdmQfC+NB0oBFXqzpyGXSGukF3jkqnEWKfBslSKyIvtiB2zmfMqdqgzlKBkuhXtgKyQnwT1Kyyk1sdqUIkGJrW9XE4vZO3ea5VaZ6wTG9InrXZ7e2/v+HiTlLsX7d1GdBqmpsD87uqARj0XoYoYGh/uTcFjBb2YbuN/GfElP4FYb4HdIQgvN3qalh9GbPzHe6MBMB8CsE5DaixGf1zkFTtmF08JBe+GvysQFjnw7lo3tvSDEtlhRAu2CNBPTz09N2h7SGC2wblluXYbUUyApXFbvEzyVetSQ83fNpe2fiWRBW9bOWf6wrQjawe4X71730XQDAJMcD/kMg4aprGRZCbE8XKXFrVOGUM69cp+Jfri4TNnDr+IKg8rdeFT1EMK3P7s2ZTUM7ecbdvqTVFLci/4hwQLys4FSc+7t4mR7cJ/sBZjPp2zQc97SH1w3obbNFNHw9k4W/xBXmNvU1l8chihh/Sk3VxQl0JgBNQPiDEKa+JwES4HfWleQOtfiBHfcM7E/DJCYRiZUDYdmI4ZjLZOmx1krtDjind5hPQijWDKD8xqrNpkWmhpxfHWlsUZVX/sXGxtWdfSulD6hpHAiu0kiYTdowQGG9dTJFW0W+RG1etdtSYNDoct8cTxuheKT3ttTek5uikzq8HNF00TfbV44p9wcumw2sGUs0w7Dlb+CcRfeU5we87vmPuh7erZ37sBYOJEcGN3Y5F6O+T2nvTSaNnZzQGljN7DZ7z9uxeBsWEUhrGqBMBKVHX+8yjMFiQxF+nCHFMUMkhyKPvJQVBqs40BfTy9qaio4PLa4khlrOXC9GQsEasptBan3aJF0+pNLiq31gXvF7MVJEiyGxrSU3K7Hn6RHTY05EkSwrIZ/2080Pjj/XtLqhz1oY/YUxPYqz4RAr1c3Br/gadnKTTt0qpdasVAamTqToX6vDqiPaJ+CV6QmiQtHBQLCOuNgZffW3AveBbdi1hvNp8+za4KfhpT42D1/u7IuPjxGSFUILW5EZDsnxDcObU9DlYMbt8e2MAy4AUmqJ5lUBToKdCkXhoeNHmEUHQoDFaUUaz0c+t213e9Eo8QQvEMghQq7mllr5qAxZ5jOUNgs2I7qJK5y+BdERyOilwnLvmAsaY4kX0W3n9jx02mbJ8xwRDb299XoQGJSqy6jMJGycrjAv7xldKgHP+QP6i0d4kSrdx8wbg3CT5QpTEbhFb/SDb5E07HdsEfk9gNGP0Qsn1uCXMB2OE1JtQoR1KHlRrIIoQ6hVrVaYVOEyTeBaWVefKDIzQRwX7BGqEV0qhCL2oITQlWtXt2xGZzDQzOEYMgfYX6mFKvUVRP0B7CXRd/84300ZroYBrbx4Qa5bCyv/oKIEgrBnTJgvZg7NDPL5mheMNEZqbUalqd4eeHOiZLw9NmOWNnZi6x85MHhjM+fgqxcUdOEUYbiuHIhx+e79/d+AOO/0ZqjYHQQwWPjcj95DQSOfVLhQgU/P+BFn6cfqe/LXrI/+VP14XvWocqSlYu3UfT7G+vOkbtssN1GQHX33FSGGrbUWiL2EnlaJ+/U7bPDon+pJfvPyitOtXesTSSl6Q/xsevOmO3dxS7HAnThIhkfTWejy2Iv7GIYFrQoIwS8MTjmixVbXo/2kxGAdq/cwI2y0mgAJZIW4t/VLSylPjWi+NCijJq0k6vYgqwyDvmhvF85Jhx/dH8wq9u1/riw/e24Tl1kfaE5f8+8Akwbnlci9QSLslQJWgQJx9xaOSeyu35TSFo72+KQRt+K1JyvHM/SMqL38/59UKj27quHADTj0BfyUUgkJMeimFlZLRHDOrE/cOGHSQ2VlRdOhuAZrxKdLySzY+zM1iS9cYJyTbG2rdOiYNl+1OkVqtUMlYl6dleyfFw2F42Z4HN8bDXBD6zOUBuYQW0cQI2bwE1EvfG8Xa4H+qjPF5zgS9AYvRJ8e7LZQwOPStscNdOhE4gCEFHgu1iG1/Hf/JXTtkbEawMwcq/o+20piR9oe0+uD1MzPJraTm4VgWHpYR3i1ma0NCVC5hrmPTXVjcRqX8EDcG+pouuZW0jRbPadv8eCsdEAyAO4mrvfDD/v3luvY/PdZOew19sFAMFgM3YgALosN/50Y7YS3SCmivgS2T4txhuxy02jGC4U3KQ0fTDAx480fWrte8TOCEL7RLDvGkYnubB4q5QGYET76/9qugIcJZQh/RnolptJtofoiam2iBQWKqTPcmaB9hZPAfXD1XFjodRt/dT6o/DX3YoedNK+J0XV89ynXtvEC3LQeF70hyQ/Us1OAoFpmGYbtJtabEJS3m5O3v62uxPIozy/yxfCKCTUu1SaTslDY5aaEG9KP9Ubm4t3zTk8ruJC+ZBbSFGlg/hFsT9iJsSahuFiX+uPl48+/EnU/rhAJ3OAmtZs/tDQ559zqQ/QlEU2HcqbFot7PHDJjETtbiOBnQdTn8aq1/QsmwhDbOvK8HCc1x1fr9ZpOjuIyxMYGIPpLAGBGUovgGTUOlMrIC7NDDz7E1bcESLIvOgVs0ZXqxJPWLxDfjY8cQ0DcnfIF0hS7AlIV0IxLsIwxd5v14FzL4iHxgIZ97xvqnXv+m9o3/LNkQUalyawjNgMECRRAXDXI9qhiBTR/bDryBXJK+lHUAjeORnV6EcR8Ab8XLiyiEFyBW94zhxKlDtC2byEzxGEgEwa1J2LqLoDIrehHeq76vRoLozqbfc0t4ku2ZNau03/FBNcT9zWisO3ONjpfYxKQiWoANr2RBnk0SPh8zHBfFz1AMm9OplPfL7LZyw+LDPN+qZdI9ioirS0mCDrcZRY0NYxauN60EdPcv76ZYYeUsi1JX969Tqdf2UyqfkTamU73CVqn+mlaoP/4WnNPZCnPUpqJlbNDtbBCP+5PR60NW6I80KWR8/9gQMUtppSI8njjHYNJbNG2KsL5rLrJ89WOyOo3XJRWmnDVZwf5Nn/fv3uxzdu3cJB3bi539OAYqdth86Gx17UujtdD8hrXqLVL3YbG5eVGs8F2I0Q/knNYhJDJ94+TZ+++UTcFCKXehPt9IBZsGIw0EVV8k5bUuZSlIZmjcI+eHlKP4OYEBdIV8ryLERgXarhYijGK71RXG4DrJgOGAHFANOEXQoPBFJYBRGRJ4oxLrxOQwnAevPIEcoBoKJCqes4wa7zT7QabOZOKhSyuZSa9Tk/FjZmAZDMN7j6j9CcYY/gvhniIX/Vn+3XQMl0PKn+VpByfwbM7QAYM5+XKI5ry2KzKcnEJ4tdyz5qZg7KUNeFlAJnHY6s6SG0ZftTqfBgH0NXTdwYTGYz4cF0w8vCEEYfjKqW1CBVyh3AgUwhqUScyLSduu8A1IUL20QWHCLY+NwVedA0NKR2MSru5wf4V+oVFNc5SggPYnAsyeStiYDRYOmFTCsmP5YPFChCzY6eDkW1gybPcNi30UBAVR0IhCFETkTQn4IIcUHz0B0hv9Mtlb5Dk+JcVXoAsrb0XcxsfWp1sSUV3Nbt7YmiUmBln9l9gOCFFZ5LqNmwOt9Pb14b08nl6UXDNIB4f1hBv1aWDkjq5NdJGu+YgMmXBvS5jx37wIJcOBvh6UqTxxLn5ebcOiRsPR1yAMVMCDv7ta8PB0LLqTso6weD+X85F3PNPCtQLvGvvr5TuVYcUyDYXtSdyqCxQFqjMQcz/yt/LtquPIf8x3FqMeVw1Wz+3eH6P3BXLQ0prJQEiPRTiNwH6RXjIj47nN3xWDT3sDA8vI3/5wLnvFMBk/WVE/qd9o9NdUe4L2z6y7I2ZSicDLRSSNBpEMGpvp3WomX89Mk8fBsZbzSFI7H6C2E0RL8ugTLDy8IxwrCBnuMPHSdERSLjQ4Ko5rr5hCvJ2gDtUq5J5/pF4uH34uLf2Tq2TcowYF9TMCaYIeW66/zu8CZnuZc8LOwHY0M8buxSDOsKVpXNFyUN9cPnEbGYpCOIcAnd8g5pJz+RzuLiSUUVk8JnKXC9jAVcpNOYFB2URd8kQeB8HJXV0722Fh2TnSvHAUr2NSekx0BS9m/+NkW+OXlPs1BNVy6VabO0slys1j00Pex1m9ajEaDPX19t1+EBFycoLvaCPW7D7gC1UVIvhyfz9AHJSUuG7hi4qbDcTPai4hixBwisd/n7KZzxKIYhJfWQXo9NycCAibcAH/MeQvM3mWLXvKll9y//NwGBMg5pwv3+eikQXOGsbkxVxccgF8r1n9eCEdGdMJVkBFVzawex8aoMQyfcmFODJRfCJWUQJVVE8ET/ciqrGxrLy1tb99EhltyyNFS3ubKyl6ggGuup9/r+HClemfTFTIHVsPr4Tjrp/OAT8ZYAbPVY+3ZyD3msK6K8ZSUfoyXlnguSy9/DUIyzOTb4unwrb7aYbNZQXPbIVCm1nVYNpQ5+MzYNzt3sTe+WJrG/En4LS/g0OVBJqOqikPGvatpmbgx29O69OWt+HuX3TuPHjjK/WT2buALzwj/7VT6jwHhzn91JTXNlVn6Cz3gW3Rz3Wa/MPaXlVU1EUMGIApnLTH+tweJWcOspzMA7Y+deMNq1dMZ+rilNR4dW6dj0HWuDX3nl9OJpTPgu8drto//TpQ8k5Pz1FZPX18xvNDQsHgXqzk01NgAdKiph+vjch9xB++FbJ65rslm5Dj3CZfr5Y46HPfgxe8NL8eUXxDqQ/lnQl+fp7eXVnvqORnodh6rjWY9bAVLVuuDton+pvWBfi09NdVuT4F5jKvpKwY2EIbNYYBxzIVNSBeszMmV7fMxFEkRnZ2g6yooOD8oXFoHCaNDYQn53hRIi99dx1qxJgMr5PZPvOF9MYptkOIitCtkM/fR6ZqbkWPD2BPuMXTiAbddE8i+M4h8av9+P4aXoMDgedZ66W/gND40aLBhA5ahH6B+mID7ob81Gs3z8qHpj9nADniam6uxar+kZ2AAIOOjLmwrTuEO0hNsksZ9acNheX+xZIUyPKWtRdG8fLISWIm+1dwoSMKT1JaOHilux0NDbOsXfPVp58cR91WqKZ5KfupaXs24n+toeYzUKMeCkcctkthFg+rvDLvB9qZaqPON5KeMArDeMmGjHEYcNmB5SPmH+5D7T0oLY5/uZqEst/C2xRVvqQjZ0KHzSFVjTGJTNz50jXwUhTuMFA6IEAB7IrmgzmwnpZRVSmCxC6Uzu6lxg+9XEW4WjszYhzbcZrDVauIGHuxyArGfVPZEbPtbwDfRBL5Q0XZLTGt3wI7e8hhBHre0lQcgW1080pUfH79MLCFJ0ujAghewYQROkHaXA7uF+6uVkS08jCCVcwrDTqa3BNbqy3St27BTP6GL++VrsTggszbG7Jp52lpLje17j0zE5XxguxFkYYwj3LkY4Yg1bJESXNm4PUw4lmuhwqmgKVUhRRWqKOtJadakJRoaFSd2Z7BIjen9pzFn8j+GE5KaMIQqUXQ86Ydd0dFvJA5ebyTFBLxXfBBo3a/AHFbs7w58ECv2MYPqtuhL43vrWS2rZbsVvdPQyodXJI/4XXe0KA625nM+sFTrW3THOg+zpBw3dGvmlzmWwbc+C3nvH7S5T48TKgta1dA7ihnQ5zCtJPNPtAyoPfEGrTQX6zOUVvQMtEL36tDc58ATrf0i7U0iYOYx9qQ8mazv157GuLIWpDgbR2NxRmT7XU9Td0zsZzL3E8p7qxizSpl0suJ9IRssqsv21+2XE2NAmiOkcyo3N8pvEZANEMuiFfJ+/EqubS7Q8gWQMXMZm50bUbijGudiVqtxagbtnoS6JX8cIm2oma8Dm+Ivqnw/B1KfiqpYtNDHOY1Ge+VfnMukIZb1UdNM+Pbz579NPidFiOaUvrBj1WD9niz7X2htTUxEH2BPoXcEoso9NpyFl/msrknPQlaWBzi+kHgvJ0iudCYkHClqFPgJGotQVjniarzH114SA3v7ehB0/pKjaZZBDXKDobPrvk8iZIuK47gkSgIvGCd8Y5JrxMd4RaEUBoNOCDeutEThUf153JkNUVjUhoBncOzvTuSblon+qNYayXywAKiY8+Ff7NfowGlIf1JOcM5r7L8+pNQT7HD2hBoQzKP40Erxefv83Kwziw/A7qtrPqkJ332itg5GnfWpT2Bu0jrob1ikwT2yzRWb7Ac3W90mqqAAN6Oo3udqkE62c7QP38jXP1w3+l+Kv4/KFUwCeoxapOKJCz060XtOEmd+RF2XJGYvMJs6J9hS+69UPF6L1Xj+6SpYo5bCIlpIoBQX5xDD0GqdMIG5VOAaZkjafBI76UkHlzmdwijnED40OfNfSRY8k83Ni4k+nMKBuDl8E9jZKePGgUxu8B3Pm7PO6ef6kqJIMr89KB7FnfjmyYyM6pB3rn0Cnb/5oh4o8L0xfZIQtVKWF05n+Ld+YIm9BHZP7upCSKSri4RJdzvc8o7nprvL4S4o8Czzms3eZR7KXVWp8k2YCFVlJRQZxQ2RYkWRK1cCCZn7DcF6wwG9/oBBH2xYUSnFTQMmC0CqKOU3iGULhI1nPHyI9xAt7GS/3RPZVoR1wemR9LzfURx94FHMM44VhN47rTm/kcAIkHkj6EWphuqzqrHlNAAefSav4X6YBCdO37q1uFKeb4gJtvgCo8rscE3q2PPMUiOBMtEnt5P7+jzuwrpAnMmhP/PMZy+s76WfquCwRQG81xP9UVP2Ygl0nUajnRSBHt7jxmKOeJX/ByHupXeJmmB5YBhLalt5YEuHhAZBYiyqrUQp8dtiTwBAvaIdmWT1WJNwp8Xhg8k3OtvxE7kn0wCX4MUzgR3f6HI4LCaGO+/WAjyrXEpFO5mTI5HejRJGEwRHAuqzY6OS/qjo9lBxxBNPHGgVuPA0YhvGnUlhmxOeUakcDr4LdOpv8urw7rl9uM5r2vnd9bnbb75WUmyTN56ebT5pikSTliTCijPCdk7rZXRbxtNMn9ynma6Y5+Jj/j2TedykfPZrrKemxlNcDJBxxIXgCGl32UlsgMBZzxQnsGtC+p10k1TWs8WJO/uu2w3QOLWTdOAfx+LNLrM+Q2aXeXK40AyUU7Ue3Up4LlwoNNOAQ7U2ebaGlR925SqAYTu5BOExFzRP/e85d1bYfNiRyFXNts+xeQwIx98Y7Rq9sfvgU8I3hGVdc+uqDM/N/px5IJ0h+0I2FoP5wYFrx8fBvn1AvoXW1/Yy8OjI/MbG530H/LOyPp0B8C6Q6E81m48eJfENhDCQ2IA3NZ3cIA48CQdm9KWt2OJ17WkL9qeZC97rFQuh/v3ijqbo96m+qQwYdQHy1TjLSegzot/dTSaGsTF2WKi1/91oxpF1TywuFNp5FhazXg/YzQzoD6Awczf+8waWHg848kBezAt5LKRr6gdsmMc64xjxA/g+sDnUKhjXgP9VZRu1meNtBbYCR1HsU4Xgjj7Td2UIdg7yF4ZUp0duVMrpKX50UHdZBl3JYAguRB85v3+i99E+sTVjvGjogT6HXvqtn78fe+3a9TDt16PJjGPxKSo1WxfecXLJ3yarYuiOZVfQBM2P5nwzYbctQ3aEojwuOzt6926wkU1ZEc49R0RleRhvKir7owJbU3SeKpjAF6CI6m9XK7769nvz95uP14U0jLAAa6QhpO745u8Kv//2KwXvN6L6EH3E94OnH4JHoPijT7mFUXbNK6T+Je9neu3Tre3wBnga7gOJoJFzctnQkPRdrv6O90XLGnug+AfkXn3slWDDfn3wflmwXnYA6QcMIIKnw/pqhKfTRg6M4WOVLzeP4mhcouzNnYbnnjPk3ZHdV4c/UBhdGxjlzGbP3erqxYfJ6vb+SjX8eP750iAQvo/mBOcy6GAWpuF+f7+R23QOg2FzIMztbkAd02FoWgEEzpXfnYHFKqXvzSCXqucVyU6FwVzwXXeCYKdD8PvMYErCnxQBfu/elb0dZsOc7gZcWjo4eFrN8kBdGcnFx8BWXv4ogS9uerQ7K7MlQHwsNuKAlsys/vubXgxcAJK6HZpWjt25W9lqmgcwUj2H479d/IDAeQVxAVGMoFTbij8qeP+uvj5PSakMmUU4twNkmzZ5LmchKLsj3N44bGLVqGlN0Dt6ZeUTIf04pbmVlX+L4GkeuAKNvU/zEivKe7q/mQNMcB6xCj7skErSEWTWKZ5FkCRhxQrAkvae7KMwP7xtiME8B3uoAylmaM1q9dcrgZr8WzPbwvnwAdIDX2cwhtoMov1oztZzCVOksxAMHoXmQW0lJuOAn+UNmAbuaY5l0YvH58d5x6riM/KYq4Q5X38NZdWsZORLHSNlhYVXMXDd55+EZH0XYM3vTMEO3z/t7FWKwfy02TeKy0JgzTf/2CRGV6WjfOJTmHG02N+cFsPfGKvT+GcsX7c9nE4PpfsbiB7dxmhYC+uMHac31cEaUcyGdod/NoOeHLM2doNWRNsbQEPAe2oCCuqE/1T/4GG8P6t/C0VZKtCnnTCAaTCHK0j9lSfv+eCI4II73h2fG3dZSAIoEG5S8vDssODn8zxfqYMyay9QgEBhmYcDCRMOkiZiLikUUw8pDuZ2iMCIFCrQIZHSkRUtva9l4+ztndBFBbRhI8S7CJHS72YgOGfbMjYZu6bfIudovjqhofrbNUexEg2aEGFCwrOVJoUpPFyRbcpWjFyaW5kdzpHTBv5drwXAm/dt3DjWJ1uqQwgzdJf39b4agdGnyzzvPXEKENYJCwn/d+0R7tlfs++ybpCpW8r2vueR4f/2Mg/+2T7h1hVjY9f+J4Qoh7m7EmTbiUreLnrluZ9A1+mBrXvNWTWPEGwXvymABEh8YYbtEYkqSp8dU+39c6hc3zMSRL7Gp6iawwn48L+uPUcGFLgT20g6cQDcBNmBFE4NJWt5gxTPw5074S3cpjNoXL87WBJrsJryAIQMnokElBPAgWfEpFpRsj57R77CgOnveF/S61/y3tFjhlzZUUDJyOwn/glTOTlTy4/19Xns9o5Q1VBa2pAqtMNuB3zlJ1TxxE58/6BU9vpcTrY9zaBSBojGMPXqr1LEaeBYD/pJJjqINhWYzXZ2P2SSo8jU8g4Szv3+GbFx9/0exZUoLGkZpcFLJJYjjowdkbqgjBDwBhYXmxv5c6BpHywbeSu1WZcBE/eHTlZY6Onrc2S99eZbYLI34TsOdnY+X33/yXptOsDtto/eetsTt2OHLSorMwtcPf/bb4ddXfGJgjjBB7CBeEp8YdFsdmHy50yx6LFOuJ1dnzVFUR/KShbGPaUbtB4+HHIhcpPYtTrjWgqxkeC/+VgFNMF2uzLkTJQl/nlEtsLHjnk2bAiI/kTzaESGEf6NDcCFMQH/+iq02hWlvrqwULaPOY1PA0S/CXzbeLLFcL/MMoBHuSxpg9iODoQNvu+Wl/+Lp+JmBf4rVmA85TRXyZ3KBjdAve4Luywjd9JeavZV7meWw0UZcrHv+1TVhzxl4CGACF9L5OizpzbCz4k6Cirttjf2POLyaHwuxhVrf73w2okTntHR5ExKPcZ+rv/kNwL1OAQNaX2e9vb0X0x5ZEsl6FteUzsM77gsOHs4lDXJZjIh19iK0GhAx1gvJDnG35QIx5IFvo0gtpiY4fq5+0cUc0wD3fFUtR2AZav5PP0/kT0YmwPgP3xkjSgvb+x+13vl2oq49+9/I8UUh9K3nIfl3SzP6TNRm7So3vUerxQO5SBdoRE9K2vlMr53hPzUfG5ES0DmtCcMuAsXj7psFpBKiKMm8iRUiyccxDTY1ECEpsht7QvKDAmBv8/qzny78TdnxOqlDcFSuc8a365Wt8dbfZqFbd4l/WwCr7n1PXNz2TkBaGqLeIwXh298PlKEn6Tm8AcOknNeCwmymsGogqyQHMydv0t8WGsTaH4vjjP0ly5h7CjeVcyMMpR5oGq8GvKQlnOl9lk+u8BmUwiNmzEDSEhq6IQT8uplwAfsr73fAsW0Nei0/h2oKzifAt64WwC16NfoGuBZ/P1Ug7owEDaUBZzRlUOZloW/XHcmqxyx0AsGwlgCaGMqMbWmQrmL1N+BNL+z2nH7D2CG4at+J6igu2NJxjeG+Kse+o2HWl6vBVDewGetA0DS//1QpZz+B8fkvqNUTfEYL3pMSFqvoshEShAojydru/ZEG4ETLryHFeX5G+H+/RjMDmqLlFaXafbyIbaSPvctj/Aj3Hb3GgRdu9u26VVkZf0bKPpG0VbEZlQEKnKAqqP4uho212A2YCyaN7FfcCc+1XA1+eV0ugXLJZE+gnXLhXP/37kXwmcS3HTqUsVJyfbpEh61s93j4IpKPMAzBN7jGQpstSki5yWSzqTLZ2M9Jtn99a0mT+ysfDNi8tkI02pMkREhhdei1bbwXu2rcOzqVP3ZfrQgMi88F+XsWvOvWHh+2Ov+Uhw0UbdZJjs1W5mwH6kJuwKlgTSLpmE322joJJ1B0mmIRl/gRclqV/bQVNM81YXAksP5ZRK+1d6YZDfHyjKAMnWIjMWjRhIox79tqTfBa52wxPzdY8UlQIF3MfpMlxNzr44u4SJjCFd97iowRcJJRX/8lf/vNfgdWhwaXFyo7lyAGo/15k+vW9dzqicf8piIPeAl9AcD5eZtK1YcOCDCO6e28125a7vB2pXuyeDJGc/NArO7yxH79mF5v8mdtVu+e8bh9WZlDcuH+4P6o413V99M8Myk3n3ZboxTOgC9C4ILRttxKN0yj88bHZTHGNVFx4r+p0CV/wG/qCD/uIuNobHGjesZdDLFQ9GdMNBSl2j09b/r0izhijCI5oYsGDA9Dtl6HXWuMoQ8fJgKUUYHdSAZothDr5ZCDrN7Xexgp+Y6MRCcfMuL5Amf1MJk5/PZyV07Kd8OrQnYwgHKQYhBq7LDtyMl0D9HNOPNw9qR+BGWVs2zyLtsGL2GPnhS250tUT5Zj9IPQ9W0o0fEI+uuaud22tB7HX3g8PY7R6I88p5IPxbBw9Kmxn1HtHDHVitnPzoyIw2+8OSeXWUcfDvK49sQ8Xlezvry5dvbpTpORFhu1uyeUoKzRbJ7MxMG9gFNisxuA0msLufKE5CZfqsb2x7EwGHaUTNF95lv18VJtxq+OPnfq4ChP1+tu3X51G+efv79ps3NLldvG5BhUnBG84E+N22wr+V6OAMCk+eraZtqT3fK0Ww9FjCBllqPoZnz6P/aaU233cAiN/zBqBKAq7gsZHFW3+W2q1WXevLlcfvKyRtf0PqeHUF6FAEnSiOQQyYmChXR/GtR7/Fe1HfnOXRC9VDn+CnFwzTQEOIKaRiYTniIGrBGFTnWHaOOHc3kcjNvnFgSVWQdANkIjcbKIpY2MKokGbgus1PEwC5+DglgArsIEnPakGkSx69dvCMNXbIbx/C/cr1MDw5dWY2tvnLogalHofx4kauuHH5oWhfALVmi6tkF+moux+o8i7ppfHq23mNd71nf6+nt8HS8JN4VFb8+unh2wuMpMC+0hEMe6GkMlWA11yKxO9fLHNyYr7ovkGhh54nSwPgJUQdXj8Rmy7q/TdyWidLBsXGbXq8+mV0b8FRAbfbJ6tWylGqCjbEnVGDXGmuY1cM7BvlY7CESB0kXNkfN0b7fphYcWC+E3H+grYP5plcRkeoLkeiLKJEo6kuRmGPKygQKgIIAkT8LZMoMMkQRcrFahEuOPy2TnxsJkr9wTu5P+OMMpuy+moQ6O33oTVP8E5Ocr9j0KMsEAhLr5+fhJSz4gv3iLTEqvvXu5gvBWIRZPx1dpnntMldICZdwUS9KMMAMYnC+PdyRAz/38mtX7ujpFhNgiE3wGXaf/pzfEIEjQ6ZAmeiZFE8Pftg575z3cSjhnHcS4J25T4nUDlG+U+MUWwbXi8EzHQqy67Mr4P+rQivC/QDwDy8PVf0rKs/W24MAIq44WQGvX7yTlpQBh0VcUaHgszoWgK3L/z/gZGm3nT2RH4jjCJ84u02bJ2g0MGvFNzkojjoppxPPJFc5F33l2+7QsO6WsNDCxJtu900Paob1LewOuhXBeYT9lfiU5JOj3TX2F5ex1mQec4L3SHQ/bEhr5jAz4DCTvXrnavGE+MncGMioDCiD/lJuxJBmwiYjRVJc9eCSZsewMQsGkkANeLLkK5P9StbGFSPaKpKHsqyBc/QGT6QKPNfHVaiWnjn7A45dKgdF/VoS5v9+heFY9S7EDzfxAMIv2ILmchEoXLCGFP+8pyYpXFvFQ23xuUlsNODBVhDubCZY7oPAXppUfRSIgVuwuFEOH9lRH0tKqzgg5sj2egojAs85HNejmW0em7eRdtyOvbJuHeA/07Y4tERDwFiksXBfPSfUKaSOULIRiYlJ68TnlHCMc6RIzpOUJ7BxWeoPMvX2NR0dICCGECjV2YcelEAI1YRYc0reWj9365TWkbrnUzJsfieYUvs8YJaNulbjpuXL8zkA7j9GOxzkQDLv4qFD62daSMuxyubxKyDAYdwU/kDsA2OC1L7vTBNyQEcn0zkGIPZo3YdOrAN/m6TQ7X+ZQBCD+YoX9lQK179xUWe8dd4+JJu4ZdxUCRZOCPhfBVS5FlNQaGhWUKsqL3LgNZ4uG97OZOM55fohV78cR8p3vsuL4b6r5KCMUPyqHawPZ2GJLYqu1Uh/qYCou5IyNn1+5wTIPbwvN7cDsfKPHw362SD1yid/gMDq68ScHQKCRqilwYBDKBK7+lr49z/EVk/eLTWIOElV2CUVqRgVBNOG9ogLP/6CeVxEL3z8wvRD2uhrPpIMCbbUEo5kkJryh+X+P19Yp6ruiRGOfkZ/N1+CZghSHqViCRyuIcLhYoAYoPDtUl5vjsrbo2kO3/WmWge/fAFW/gsQ3RsTkxFDENGf1dUWMxgcf6Y/XezXF/Pk4M6tK2Sqiy/lvYfbcQTHcfBCF6t1Ar5cdLiEMNVlfc413BdsvrDrU+wGkAvVNpKtoIK9aDfN2DoGjNhkOyJOn80XPWRpVFQp2SsPLGaeY1Wx3exIGdm+NyfnLGmRf+F0s6sDXmgqDjwiJhvJ38kOwQBQbP8znKMmYeFhPHQRdfU54X8WkNmz0dNM5mlU1m2L4wUbntpLCLmWvRE81w104ca5MWxMW5tG+b+Xvh1UIW1BgbLj3gsvLXtC27LVB9bhtON2EaAKcK6m8Wmf4LC49HGygGadiBQRrrYdS+mMCDp9acZa3fkYESXStrDL7bH6jdGunrWFNzPohTFzR4tIfWw7nf41gXO2IkGSpP6qGK0KZB36COX/VPQZwK/1z3zUZxHX/hYWsP+dEaBGntsfwNeVWDS4NJ9Y0RwjZgHTi2HP8C4DdbnhDBvqBx7Ad3Ufpq1oz/bSyauOPCo+WOAXR6qw8KyxlWqI6bDif1F/HQ/mqrH6wkwrG5uduGtdfHEoCCtoCu8LMtkLXgWtZ7CgIDztym7vtby8Ff7j2lmT87PcrKXBKzJqajJWBC9d/Q0ws2fAgEWAd4CjV+TgyJCzMqIkOlyTfT92SM6uGw7QOzXjv55XOlR9CpVf/zqucWI2F+ECzWViYLDnSUT3wf+nwk13HhzAZb5hxPRit/7n8dzkt4vb477ruUv8pBls8/g8iSS/JKDGrgNj1zrWO9cPz66bW8cX/ARqzePPjl+dFwrnr47P8nkul2VNPeMgjuE+HzYAE1LtSAK9nUHglnScTtI1jKfILeEi3OtqhEAIFCE6xVwEQTQomYiAdpZz7zN75KaYVCSt6P2xemy+VW4ykQOYYaDNlPXOBru1gEq6675Jv22619CjMPiB9zoHZiu1ChgYTdSpY3vJTZECec3KxULqITXoGJpqLVrKXeQCiDmSjE0PGkaNoGl/INQPs2f+vQTUpYYzbLgfxICHK7OzTMpwTF3Z3z6gwZW8D1U4zJu+Pah99MIbmFhipJQQwPxEZ8wHztce1nunRyekUQnYEucH0c4EPgw+KGIeDtCmvWulxfNNJklJ0+vArN7/r9bQAJEw1ozYCn9Z17/6tll3i++99weiDTg8cBw+GGF+2HuQz0Wz/tulruXeKcxhJtBmfLk7RKU1XhBvzRJk9WEw+B2m5MpcFSerj8crwVUrOrkW1ZGdvP8Jig/lH4Yvz1xlMC13MD8TFO3Jf9teHLyP3keJ/wVEkMB1tZcK6wLpTPoLe3E76V9gJGWJ+d1wRcbPhfzt3QSgFoPHasHZB/fI+FqQzyZ19bK87uCl5F3QY6p/dbIK47K6R4tNHR0sxTV46aDFEWrWmHs78JtRLDalaKp9L+clUJr2UCyN6I1y2tzO6uvzFBbS/GhSLuG0NeeZ/BhyBqx/N+f49oVYfMUhgkVQg8jn4vgv+RIJ/8t40V0E9aLc6OeHfnb7WUshrv2r62Xo+Wgu6rUZuQzSYyLj+x4fwJ2hYbDn8b5xhFj2twT3I3Ur6z20+3TmTgmPJ+kpP92NvsdqW3uKxTqlEPaC21OvzguRhNggno3Hn80JLWIhVwJSuh5s4wUZq7Y4yLHSxSt+m8V7wRqscYyt+/r1uY5rc1WWtUkKA9ipY+7aic+9/jU/+CSmwaJ9PkhVB2ybfgGove/sDwDlH0nLCdSLCn4qegXwS9GzAPX2iLs//jfUaJkDhy4e95+NxDEnzLm+0ENnY5Gzv44DIAuoO/ugOTCKVDosFheZYQJ4exmBWW4umPY3cLNWR6/DuBeeOpMtC4so0oxgYptHivCyf/HYQBrjZCD+GVus1narewdqvem2tg++crRSKq08Gtyn++rPPzurau1Pn+p5imXGNytCRqdBWPLLq4H+4sTs7OlqoduRyDxLvfs/1aF/70ERscL3U7vNZj9ZiN1WzdeQhWuxulw8/tz41TmhcO7q+BwfGKfI99o8T3/zCksCm4dJTd/NJe7HCCwByfBHEgi1VvBx9zdwYEe2PlmRB+SU5o5EYoT14FKRUSK5s4+Obr/gPuKdAOHVufE5vrZJJy+IuqoBaz6/dLmuCgPeIa7JdU0+GXedAQBLBBABhYuBsGh9x0dPa0xT3uIjEoj4+OmTVJwpyI+iLrGcSad+f4kBKOQ0bhopwo4fBwXA5OhtV10eNYHGxrL9jvQF9RAJxFdjd7ZGj6+grXRCIeZ/o7PT43LcGzaACcVDQxmNtWv5m8IoD1ERu7ROp/fH4NIoTJGXnK3fgf9al7Dcobkv14YpeFsLLHepEs59mgkM2SUqv8PhdG4LuSiejWo5FBK01xcp155wmvbIPt97wqk9YpxHWX3oEKdw8TVr+heBQAxoG92YQhd25oJqMgoOF7Z3M9HiJOoIHB5bhpYIw84wuE5ddX0aOr3dlofx4X1R6ML4ZvVQOKDq0f+RTPe6k//mDK0XeFMI2WFOaM6vvSzDtJhq2znNvSt/u3HxowCybuJAudTB9IrnSao5YiSwm0o0Ft3nLTLqMwKvTvfWGxUR5uCXPE6nLvmCXpRs1mCykX0sX4lXkFjWFK74dc4XON63hbh+/fkdIvKNrRnRtx5P+e+TI7cvxDJOzBPGG/N//LMI/iCZNPb9/0b57q9NZy/MySi93fjHIytJYOPfh9mWvpsV6l1paGQvQxT5pZLcqrYnyepPmhv1f1DMcQqDxB+FC+OTMs49ODgwwpF2TByUZ9EYURF3Qtw0i6t3B/fY5/6rQyF/kzqYkT5LHMhwQFErjIndfXwDcRuM5ii8IslAjihyfHsWjeNPrndiI2p1QST2LMTwEgBEGiSFGbszFmgAN62DJ0RcV92xDGQFlCfk1qXfrhSC/2j/0i2HMYorj8/X14CyFEUwRJdHCOG5FO0R9kD3PkD4wc2MNiePPxH7Z3u61RFBdxHtEBG103GYaFi4HxsMuBi7s0WO2RaPoanjMl4K7HoBALHiDMFWomY/iDzeZJZIKEdld3CcONoF4DCRIxSLRY7WMgrcRNWx2ByNnZLwnijN8GjgRvzbx19ArcYUBEBL5cld56DSfSnEiXuGrZUUxbkujoTCxKTRU2mGH3NSYF8CtFl272sDzZGZDesMvFcsUFMkD4R76K+N0MIAX1DmLIqv0nh4UafeVhoVqgjlo9FyIzHDSInvEOMuNveO1evitkps8VvCJ3P6wrcQ/jRoavEOwKEcHDWeQXWBy4NHqyLvUGxJ/4itiI4pamBETFh3n0UwlpmkbME9OPBt1HPFYHZjni0W8gdZpEzwD5xrTEFcFEvzMuNgv1cvsjXiwvvj6PBOqccx9BJPY7rghPYJx50S/SKuy6k8ktt8KFCgKBXWdgKjpIoDMDNQkh8FzBeqcO5v5BtrVAkXBgLVPxktIVbO7Zl8APQKPmMnbB7zsS8p3Q1ChMEshvhmmxNVH8vVd3InUF74Mij7aTE73YZ23TRYC3psJmFxEJg5mvJKr/BSEWZFG2YP4sBJ+/exk3FRNiiPng8EkSOTPMP7JobxmKbNv2lZ+WWJASE8JBi9iKwvTVb5VbhYzrlK3pmqm4G9cIwHNJJEDrmn5KEC8+HgfPWkGNQq4seEt5BGVlWOyy0d5TXwMyn9C4sSXJS7EHHGaIhuP7Bv1zLcD1HcGFWcoDvBZsKgiMtI4tvKbV+5vWX59rRFuko9hv0MDU7p+UaynmGunDFyMNZSc1390M9A+oUFoHCskYqQHKwEYMGeDlHbMBmKNkWOKaVZA54dadjrA5JoCYOsoQkX77jiOqz0bV5iIP3dQ0DYu2a1/aICh3x4LQOvyx4vxfs6iGcYOJQ5PwdhZQOfg2L5wWcy3nVCOeCfAym8wy8UzlN+g5Xsajf728P2trWdvSxkEAMYyAiQ6Xa1q23tZGtLg8y0iy0tCzLRTnaykEU1eM+LurXHt7aHfVTdKpWca3N73YLb25xsnWlLu9pFYQ1sa++FunnBqxa811vdo+OtRaj96jpkVKlNRDQ2lL4MNlF6cCTLiOqeFQjqwhn3AysObzN6rbSMIL3jyx9q2Y0VAQ==) - format("woff2"); -} - -.material-icons { - font-family: "Material Icons"; /* stylelint-disable font-family-no-missing-generic-family-keyword */ - font-weight: normal; - font-style: normal; - font-size: 20px; /* Default icon size */ - display: inline-block; - line-height: 1; - text-transform: none; - letter-spacing: normal; - word-wrap: normal; - white-space: nowrap; - direction: ltr; - position: relative; - top: -0.05em; - vertical-align: middle; - - /* Support for all WebKit browsers. */ - -webkit-font-smoothing: antialiased; - - /* Support for Safari and Chrome. */ - text-rendering: optimizeLegibility; - - /* Support for Firefox. */ - -moz-osx-font-smoothing: grayscale; - - /* Support for IE. */ - font-feature-settings: "liga"; -} - -.material-icons.md-18 { - font-size: 18px; -} - -.material-icons.md-24 { - font-size: 24px; -} - -.material-icons.md-36 { - font-size: 36px; -} - -.material-icons.md-48 { - font-size: 48px; -} - -.dropdown-menu > li > a > .material-icons, -.dropdown-menu > li > form > button > .material-icons { - margin-right: 6px; -} - -.custom-icon { - position: relative; - top: -0.1em; - vertical-align: middle; - fill: currentColor; - stroke-width: 0; -} diff --git a/airflow/www/static/css/switch.css b/airflow/www/static/css/switch.css deleted file mode 100644 index 39d5db446127b..0000000000000 --- a/airflow/www/static/css/switch.css +++ /dev/null @@ -1,103 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -.switch-label { - display: inline-block; - margin: 0; - cursor: pointer; - font-weight: normal; -} - -.switch-label.disabled { - cursor: not-allowed; -} - -.switch-input { - /* Visually hidden input but still accessible */ - position: absolute; - overflow: hidden; - clip: rect(1px, 1px, 1px, 1px); - border: 0; - width: 1px; - height: 1px; - padding: 0; - white-space: nowrap; - clip-path: inset(50%); -} - -.switch { - box-sizing: content-box; - display: inline-flex; - align-items: center; - vertical-align: middle; - border-radius: 999px; - width: 2.5rem; - padding: 2px; - background-color: #c4c2c1; - cursor: pointer; -} - -.switch::before { - border-radius: 50%; - width: 1.5rem; - height: 1.5rem; - content: ""; - background-color: #edecec; - transition-timing-function: ease-in-out; - transition-duration: 0.25s; - transition-property: transform, background-color; -} - -.switch-input:disabled + .switch { - opacity: 0.4; - cursor: not-allowed; -} - -.switch-input:checked + .switch { - background-color: #017cee; -} - -.switch-input:checked + .switch::before { - background-color: #fffefd; - transform: translateX(1rem); -} - -.switch-input:focus + .switch { - box-shadow: 0 0 0 3px rgba(1, 124, 238, 0.4); -} - -.switch-input:not(:checked) + .switch:hover { - background-color: #9e9e9c; -} - -.switch-input:checked.switch-input--error + .switch { - background-color: #e43921; -} - -.switch-input:not(:checked).switch-input--error + .switch { - background-color: #824840; -} - -.switch-input:focus + .switch::before { - background-color: #fff; -} - -.switch-input:not(:checked) + .switch:hover::before { - background-color: #f5f5f5; -} diff --git a/airflow/www/static/declarations.d.ts b/airflow/www/static/declarations.d.ts deleted file mode 100644 index 9e9f4ea9f9f9a..0000000000000 --- a/airflow/www/static/declarations.d.ts +++ /dev/null @@ -1,20 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -declare module "*.css"; diff --git a/airflow/www/static/js/App.tsx b/airflow/www/static/js/App.tsx deleted file mode 100644 index 6bfc0b23f4630..0000000000000 --- a/airflow/www/static/js/App.tsx +++ /dev/null @@ -1,93 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/* - Base setup for anywhere we add react to the UI -*/ - -import React, { PropsWithChildren, useEffect } from "react"; -import { BrowserRouter } from "react-router-dom"; -import { ChakraProvider } from "@chakra-ui/react"; -import { CacheProvider } from "@emotion/react"; -import type { EmotionCache } from "@emotion/cache"; -import { QueryClient, QueryClientProvider } from "react-query"; - -import theme from "./theme"; -import { ContainerRefProvider, useContainerRef } from "./context/containerRef"; -import { TimezoneProvider } from "./context/timezone"; -import { AutoRefreshProvider } from "./context/autorefresh"; -import { isInputInFocus } from "./utils/useKeysPress"; - -const queryClient = new QueryClient({ - defaultOptions: { - queries: { - notifyOnChangeProps: "tracked", - refetchOnWindowFocus: false, - retry: 1, - retryDelay: 500, - refetchOnMount: true, // Refetches stale queries, not "always" - staleTime: 5 * 60 * 1000, // 5 minutes - initialDataUpdatedAt: new Date().setMinutes(-6), // make sure initial data is already expired - }, - mutations: { - retry: 1, - retryDelay: 500, - }, - }, -}); - -interface AppProps extends PropsWithChildren { - cache: EmotionCache; -} - -// Chakra needs to access the containerRef provider so our tooltips pick up the correct styles -const ChakraApp = ({ children }: PropsWithChildren) => { - const containerRef = useContainerRef(); - return ( - - - - - {children} - - - - - ); -}; - -function App({ children, cache }: AppProps) { - useEffect(() => { - localStorage.setItem(isInputInFocus, "false"); - }, []); - return ( - - - - {children} - - - - ); -} - -export default App; diff --git a/airflow/www/static/js/README.md b/airflow/www/static/js/README.md deleted file mode 100644 index 1b4c6607a6626..0000000000000 --- a/airflow/www/static/js/README.md +++ /dev/null @@ -1,43 +0,0 @@ - - -# Grid View - -In 2.3.0 the Tree view was completely rebuilt using React and renamed to Grid. Here is a primer on the new technologies used: - -## [React](https://reactjs.org/) - -The most popular javascript framework for building user interfaces with reusable components. -Written as javascript and html together in `.jsx` files. -In-component state can be managed via `useState()`, application state that spans many components can be managed via a context provider (see `/context` for examples), API state can be managed by React Query (see below) - -## [Chakra UI](https://chakra-ui.com/) - -A good component and helper function library. Tooltips, modals, toasts, switches, etc are all out of the box -Styles are applied via global theme when initializing the app or inline with individual components like `` - -## [React Query](https://react-query.tanstack.com/) - -A powerful async data handler that makes it easy to manage loading/error states as well as caching, refetching, background updates, etc. -This is our state management for any data that comes from an API. -Each API request is its own hook. Ie `useTasks` will get all the tasks for a DAG - -## [React Testing Library](https://testing-library.com/docs/react-testing-library/intro/) - -Easily write tests for react components and hooks diff --git a/airflow/www/static/js/api/index.ts b/airflow/www/static/js/api/index.ts deleted file mode 100644 index c334b68dd2ac7..0000000000000 --- a/airflow/www/static/js/api/index.ts +++ /dev/null @@ -1,119 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import camelcaseKeys from "camelcase-keys"; - -import useClearRun from "./useClearRun"; -import useQueueRun from "./useQueueRun"; -import useMarkFailedRun from "./useMarkFailedRun"; -import useMarkSuccessRun from "./useMarkSuccessRun"; -import useClearTask from "./useClearTask"; -import useMarkFailedTask from "./useMarkFailedTask"; -import useMarkSuccessTask from "./useMarkSuccessTask"; -import useExtraLinks from "./useExtraLinks"; -import useMarkTaskDryRun from "./useMarkTaskDryRun"; -import useGraphData from "./useGraphData"; -import useGridData from "./useGridData"; -import useMappedInstances from "./useMappedInstances"; -import useAssets from "./useAssets"; -import useAssetsSummary from "./useAssetsSummary"; -import useAsset from "./useAsset"; -import useAssetDependencies from "./useAssetDependencies"; -import useAssetEvents from "./useAssetEvents"; -import useSetDagRunNote from "./useSetDagRunNote"; -import useSetTaskInstanceNote from "./useSetTaskInstanceNote"; -import useUpstreamAssetEvents from "./useUpstreamAssetEvents"; -import useTaskInstance from "./useTaskInstance"; -import useTaskFailedDependency from "./useTaskFailedDependency"; -import useDag from "./useDag"; -import useDagCode from "./useDagCode"; -import useDagDetails from "./useDagDetails"; -import useHealth from "./useHealth"; -import usePools from "./usePools"; -import useDags from "./useDags"; -import useDagRuns from "./useDagRuns"; -import useHistoricalMetricsData from "./useHistoricalMetricsData"; -import { useTaskXcomEntry, useTaskXcomCollection } from "./useTaskXcom"; -import useEventLogs from "./useEventLogs"; -import useCalendarData from "./useCalendarData"; -import useCreateAssetEvent from "./useCreateAssetEvent"; -import useRenderedK8s from "./useRenderedK8s"; -import useTaskDetail from "./useTaskDetail"; -import useTIHistory from "./useTIHistory"; - -axios.interceptors.request.use((config) => { - config.paramsSerializer = { - indexes: null, - }; - return config; -}); - -axios.interceptors.response.use((res: AxiosResponse) => { - // Do not camelCase rendered_fields or extra - const stopPaths = ["rendered_fields", "extra", "asset_events.extra"]; - // Do not camelCase xCom entry results - if (res.config.url?.includes("/xcomEntries/")) { - stopPaths.push("value"); - } - return res.data ? camelcaseKeys(res.data, { deep: true, stopPaths }) : res; -}); - -axios.defaults.headers.common.Accept = "application/json"; - -export { - useClearRun, - useClearTask, - useDag, - useDagCode, - useDagDetails, - useDagRuns, - useDags, - useAsset, - useAssets, - useAssetDependencies, - useAssetEvents, - useAssetsSummary, - useExtraLinks, - useGraphData, - useGridData, - useHealth, - useMappedInstances, - useMarkFailedRun, - useMarkFailedTask, - useMarkSuccessRun, - useMarkSuccessTask, - useMarkTaskDryRun, - usePools, - useQueueRun, - useSetDagRunNote, - useSetTaskInstanceNote, - useTaskInstance, - useUpstreamAssetEvents, - useHistoricalMetricsData, - useTaskXcomEntry, - useTaskXcomCollection, - useTaskFailedDependency, - useEventLogs, - useCalendarData, - useCreateAssetEvent, - useRenderedK8s, - useTaskDetail, - useTIHistory, -}; diff --git a/airflow/www/static/js/api/useAsset.ts b/airflow/www/static/js/api/useAsset.ts deleted file mode 100644 index 769bcc44f4b2e..0000000000000 --- a/airflow/www/static/js/api/useAsset.ts +++ /dev/null @@ -1,38 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; - -import { getMetaValue } from "src/utils"; -import type { API } from "src/types"; - -interface Props { - uri: string; -} - -export default function useAsset({ uri }: Props) { - return useQuery(["asset", uri], () => { - const assetUrl = getMetaValue("asset_api").replace( - "__URI__", - encodeURIComponent(uri) - ); - return axios.get(assetUrl); - }); -} diff --git a/airflow/www/static/js/api/useAssetDependencies.ts b/airflow/www/static/js/api/useAssetDependencies.ts deleted file mode 100644 index 3e69de2c23ee7..0000000000000 --- a/airflow/www/static/js/api/useAssetDependencies.ts +++ /dev/null @@ -1,100 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; -import ELK, { ElkShape, ElkExtendedEdge } from "elkjs"; - -import { getMetaValue } from "src/utils"; -import { getTextWidth } from "src/utils/graph"; - -import type { NodeType, DepEdge, DepNode } from "src/types"; - -export interface AssetDependencies { - edges: DepEdge[]; - nodes: DepNode[]; -} -interface GenerateProps { - nodes: DepNode[]; - edges: DepEdge[]; - font: string; -} - -export interface AssetGraph extends ElkShape { - children: NodeType[]; - edges: ElkExtendedEdge[]; -} - -const generateGraph = ({ nodes, edges, font }: GenerateProps) => ({ - id: "root", - layoutOptions: { - "spacing.nodeNodeBetweenLayers": "40.0", - "spacing.edgeNodeBetweenLayers": "10.0", - "layering.strategy": "INTERACTIVE", - algorithm: "layered", - "crossingMinimization.semiInteractive": "true", - "spacing.edgeEdgeBetweenLayers": "10.0", - "spacing.edgeNode": "10.0", - "spacing.edgeEdge": "10.0", - "spacing.nodeNode": "20.0", - "elk.direction": "DOWN", - }, - children: nodes.map(({ id, value }) => ({ - id, - // calculate text width and add space for padding/icon - width: getTextWidth(value.label, font) + 36, - height: 40, - value, - })), - edges: edges.map((e) => ({ - id: `${e.source}-${e.target}`, - sources: [e.source], - targets: [e.target], - })), -}); - -const formatDependencies = async ({ edges, nodes }: AssetDependencies) => { - const elk = new ELK(); - - // get computed style to calculate how large each node should be - const font = `bold ${16}px ${ - window.getComputedStyle(document.body).fontFamily - }`; - - const graph = await elk.layout(generateGraph({ nodes, edges, font })); - - return graph as AssetGraph; -}; - -export default function useAssetDependencies() { - return useQuery("assetDependencies", async () => { - const assetDepsUrl = getMetaValue("asset_dependencies_url"); - return axios.get(assetDepsUrl); - }); -} - -export const useAssetGraphs = () => { - const { data: assetDependencies } = useAssetDependencies(); - return useQuery(["assetGraphs", assetDependencies], () => { - if (assetDependencies) { - return formatDependencies(assetDependencies); - } - return undefined; - }); -}; diff --git a/airflow/www/static/js/api/useAssetEvents.ts b/airflow/www/static/js/api/useAssetEvents.ts deleted file mode 100644 index a3762efb63e22..0000000000000 --- a/airflow/www/static/js/api/useAssetEvents.ts +++ /dev/null @@ -1,87 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios from "axios"; -import { useQuery, UseQueryOptions } from "react-query"; - -import { getMetaValue } from "src/utils"; -import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; -import type { - AssetEventCollection, - GetAssetEventsVariables, -} from "src/types/api-generated"; - -interface Props extends GetAssetEventsVariables { - options?: UseQueryOptions; -} - -const useAssetEvents = ({ - assetId, - sourceDagId, - sourceRunId, - sourceTaskId, - sourceMapIndex, - limit, - offset, - orderBy, - options, -}: Props) => { - const query = useQuery( - [ - "assets-events", - assetId, - sourceDagId, - sourceRunId, - sourceTaskId, - sourceMapIndex, - limit, - offset, - orderBy, - ], - () => { - const assetsUrl = getMetaValue("asset_events_api"); - - const params = new URLSearchParamsWrapper(); - - if (limit) params.set("limit", limit.toString()); - if (offset) params.set("offset", offset.toString()); - if (orderBy) params.set("order_by", orderBy); - if (assetId) params.set("asset_id", assetId.toString()); - if (sourceDagId) params.set("source_dag_id", sourceDagId); - if (sourceRunId) params.set("source_run_id", sourceRunId); - if (sourceTaskId) params.set("source_task_id", sourceTaskId); - if (sourceMapIndex) - params.set("source_map_index", sourceMapIndex.toString()); - - return axios.get(assetsUrl, { - params, - }); - }, - { - keepPreviousData: true, - ...options, - } - ); - return { - ...query, - data: query.data ?? { assetEvents: [], totalEntries: 0 }, - }; -}; - -export default useAssetEvents; diff --git a/airflow/www/static/js/api/useAssets.ts b/airflow/www/static/js/api/useAssets.ts deleted file mode 100644 index 6a8570891185d..0000000000000 --- a/airflow/www/static/js/api/useAssets.ts +++ /dev/null @@ -1,49 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; - -import { getMetaValue } from "src/utils"; -import type { API } from "src/types"; - -interface Props { - dagIds?: string[]; - enabled?: boolean; -} - -export default function useAssets({ dagIds, enabled = true }: Props) { - return useQuery( - ["assets", dagIds], - () => { - const assetsUrl = getMetaValue("assets_api"); - const dagIdsParam = - dagIds && dagIds.length ? { dag_ids: dagIds.join(",") } : {}; - - return axios.get(assetsUrl, { - params: { - ...dagIdsParam, - }, - }); - }, - { - enabled, - } - ); -} diff --git a/airflow/www/static/js/api/useAssetsSummary.ts b/airflow/www/static/js/api/useAssetsSummary.ts deleted file mode 100644 index 1910771e3f228..0000000000000 --- a/airflow/www/static/js/api/useAssetsSummary.ts +++ /dev/null @@ -1,85 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; - -import { getMetaValue } from "src/utils"; -import type { AssetListItem } from "src/types"; -import type { unitOfTime } from "moment"; - -export interface AssetsData { - assets: AssetListItem[]; - totalEntries: number; -} - -export interface DateOption { - count: number; - unit: unitOfTime.DurationConstructor; -} - -interface Props { - limit?: number; - offset?: number; - order?: string; - uri?: string; - updatedAfter?: DateOption; -} - -export default function useAssetsSummary({ - limit, - offset, - order, - uri, - updatedAfter, -}: Props) { - const query = useQuery( - ["assets_summary", limit, offset, order, uri, updatedAfter], - () => { - const assetsUrl = getMetaValue("assets_summary"); - const orderParam = order ? { order_by: order } : {}; - const uriParam = uri ? { uri_pattern: uri } : {}; - const updatedAfterParam = - updatedAfter && updatedAfter.count && updatedAfter.unit - ? { - // @ts-ignore - updated_after: moment() - .subtract(updatedAfter.count, updatedAfter.unit) - .toISOString(), - } - : {}; - return axios.get(assetsUrl, { - params: { - offset, - limit, - ...orderParam, - ...uriParam, - ...updatedAfterParam, - }, - }); - }, - { - keepPreviousData: true, - } - ); - return { - ...query, - data: query.data ?? { assets: [], totalEntries: 0 }, - }; -} diff --git a/airflow/www/static/js/api/useCalendarData.ts b/airflow/www/static/js/api/useCalendarData.ts deleted file mode 100644 index e2a7171e8abb4..0000000000000 --- a/airflow/www/static/js/api/useCalendarData.ts +++ /dev/null @@ -1,48 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import { useQuery } from "react-query"; -import axios, { AxiosResponse } from "axios"; - -import { getMetaValue } from "src/utils"; - -const DAG_ID_PARAM = "dag_id"; - -const dagId = getMetaValue(DAG_ID_PARAM); -const calendarDataUrl = getMetaValue("calendar_data_url"); - -interface DagState { - count: number; - date: string; - state: string; -} - -interface CalendarData { - dagStates: DagState[]; -} - -const useCalendarData = () => - useQuery(["calendarData"], async () => { - const params = { - [DAG_ID_PARAM]: dagId, - }; - return axios.get(calendarDataUrl, { params }); - }); - -export default useCalendarData; diff --git a/airflow/www/static/js/api/useClearRun.ts b/airflow/www/static/js/api/useClearRun.ts deleted file mode 100644 index 8835b3caec30f..0000000000000 --- a/airflow/www/static/js/api/useClearRun.ts +++ /dev/null @@ -1,68 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useMutation, useQueryClient } from "react-query"; - -import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; -import { getMetaValue } from "src/utils"; -import { useAutoRefresh } from "src/context/autorefresh"; -import useErrorToast from "src/utils/useErrorToast"; - -const csrfToken = getMetaValue("csrf_token"); -const clearRunUrl = getMetaValue("dagrun_clear_url"); - -export default function useClearRun(dagId: string, runId: string) { - const queryClient = useQueryClient(); - const errorToast = useErrorToast(); - const { startRefresh } = useAutoRefresh(); - return useMutation( - ["dagRunClear", dagId, runId], - ({ - confirmed = false, - only_failed = false, - }: { - confirmed: boolean; - only_failed?: boolean; - }) => { - const params = new URLSearchParamsWrapper({ - csrf_token: csrfToken, - confirmed, - only_failed, - dag_id: dagId, - dag_run_id: runId, - }).toString(); - - return axios.post(clearRunUrl, params, { - headers: { - "Content-Type": "application/x-www-form-urlencoded", - }, - }); - }, - { - onSuccess: (_, { confirmed }) => { - if (confirmed) { - queryClient.invalidateQueries("gridData"); - startRefresh(); - } - }, - onError: (error: Error) => errorToast({ error }), - } - ); -} diff --git a/airflow/www/static/js/api/useClearTask.ts b/airflow/www/static/js/api/useClearTask.ts deleted file mode 100644 index 56f7d7b8601f5..0000000000000 --- a/airflow/www/static/js/api/useClearTask.ts +++ /dev/null @@ -1,117 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useMutation, useQueryClient } from "react-query"; -import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; -import { getMetaValue } from "../utils"; -import { useAutoRefresh } from "../context/autorefresh"; -import useErrorToast from "../utils/useErrorToast"; - -const csrfToken = getMetaValue("csrf_token"); -const clearUrl = getMetaValue("clear_url"); - -export default function useClearTask({ - dagId, - runId, - taskId, - logicalDate, - isGroup, -}: { - dagId: string; - runId: string; - taskId: string; - logicalDate: string; - isGroup: boolean; -}) { - const queryClient = useQueryClient(); - const errorToast = useErrorToast(); - const { startRefresh } = useAutoRefresh(); - - return useMutation( - ["clearTask", dagId, runId, taskId], - ({ - past, - future, - upstream, - downstream, - recursive, - failed, - confirmed, - mapIndexes = [], - }: { - past: boolean; - future: boolean; - upstream: boolean; - downstream: boolean; - recursive: boolean; - failed: boolean; - confirmed: boolean; - mapIndexes?: number[]; - }) => { - const params = new URLSearchParamsWrapper({ - csrf_token: csrfToken, - dag_id: dagId, - dag_run_id: runId, - confirmed, - logical_date: logicalDate, - past, - future, - upstream, - downstream, - recursive, - only_failed: failed, - }); - - if (isGroup) { - params.append("group_id", taskId); - } else { - params.append("task_id", taskId); - } - - mapIndexes.forEach((mi: number) => { - params.append("map_index", mi.toString()); - }); - - return axios.post(clearUrl, params.toString(), { - headers: { - "Content-Type": "application/x-www-form-urlencoded", - }, - }); - }, - { - onSuccess: (_, { confirmed }) => { - if (confirmed) { - queryClient.invalidateQueries("gridData"); - queryClient.invalidateQueries([ - "mappedInstances", - dagId, - runId, - taskId, - ]); - queryClient.invalidateQueries(["clearTask", dagId, runId, taskId]); - startRefresh(); - } - }, - onError: (error: Error, { confirmed }) => { - if (confirmed) errorToast({ error }); - }, - } - ); -} diff --git a/airflow/www/static/js/api/useClearTaskDryRun.ts b/airflow/www/static/js/api/useClearTaskDryRun.ts deleted file mode 100644 index ba0cbf6bb1638..0000000000000 --- a/airflow/www/static/js/api/useClearTaskDryRun.ts +++ /dev/null @@ -1,110 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; -import type { MinimalTaskInstance } from "src/types"; -import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; -import { getMetaValue } from "../utils"; - -const csrfToken = getMetaValue("csrf_token"); -const clearUrl = getMetaValue("clear_url"); - -const useClearTaskDryRun = ({ - dagId, - runId, - taskId, - logicalDate, - isGroup, - past, - future, - upstream, - downstream, - recursive, - failed, - mapIndexes = [], - enabled = false, -}: { - dagId: string; - runId: string; - taskId: string; - logicalDate: string; - isGroup: boolean; - past: boolean; - future: boolean; - upstream: boolean; - downstream: boolean; - recursive: boolean; - failed: boolean; - mapIndexes?: number[]; - enabled?: boolean; -}) => - useQuery( - [ - "clearTask", - dagId, - runId, - taskId, - mapIndexes, - past, - future, - upstream, - downstream, - recursive, - failed, - ], - () => { - const params = new URLSearchParamsWrapper({ - csrf_token: csrfToken, - dag_id: dagId, - dag_run_id: runId, - confirmed: false, - logical_date: logicalDate, - past, - future, - upstream, - downstream, - recursive, - only_failed: failed, - }); - - if (isGroup) { - params.append("group_id", taskId); - } else { - params.append("task_id", taskId); - } - - mapIndexes.forEach((mi: number) => { - params.append("map_index", mi.toString()); - }); - - return axios.post( - clearUrl, - params.toString(), - { - headers: { - "Content-Type": "application/x-www-form-urlencoded", - }, - } - ); - }, - { enabled } - ); - -export default useClearTaskDryRun; diff --git a/airflow/www/static/js/api/useCreateAssetEvent.ts b/airflow/www/static/js/api/useCreateAssetEvent.ts deleted file mode 100644 index f2435a0c43ccd..0000000000000 --- a/airflow/www/static/js/api/useCreateAssetEvent.ts +++ /dev/null @@ -1,52 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useMutation, useQueryClient } from "react-query"; - -import { getMetaValue } from "src/utils"; -import type { API } from "src/types"; -import useErrorToast from "src/utils/useErrorToast"; - -interface Props { - assetId?: number; - uri?: string; -} - -const createAssetUrl = getMetaValue("create_asset_event_api"); - -export default function useCreateAssetEvent({ assetId, uri }: Props) { - const queryClient = useQueryClient(); - const errorToast = useErrorToast(); - - return useMutation( - ["createAssetEvent", uri], - (extra?: API.AssetEvent["extra"]) => - axios.post(createAssetUrl, { - asset_uri: uri, - extra: extra || {}, - }), - { - onSuccess: () => { - queryClient.invalidateQueries(["assets-events", assetId]); - }, - onError: (error: Error) => errorToast({ error }), - } - ); -} diff --git a/airflow/www/static/js/api/useDag.ts b/airflow/www/static/js/api/useDag.ts deleted file mode 100644 index 798d014a357d2..0000000000000 --- a/airflow/www/static/js/api/useDag.ts +++ /dev/null @@ -1,31 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; - -import { getMetaValue } from "src/utils"; -import type { API } from "src/types"; - -const dagApiUrl = getMetaValue("dag_api"); - -const useDag = () => - useQuery(["dagQuery"], () => axios.get(dagApiUrl)); - -export default useDag; diff --git a/airflow/www/static/js/api/useDagCode.ts b/airflow/www/static/js/api/useDagCode.ts deleted file mode 100644 index f6a3e8b407991..0000000000000 --- a/airflow/www/static/js/api/useDagCode.ts +++ /dev/null @@ -1,41 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; - -import { getMetaValue } from "src/utils"; -import useDag from "./useDag"; - -export default function useDagCode() { - const { data: dagData } = useDag(); - - return useQuery( - ["dagSourceQuery"], - () => { - const dagSourceApiUrl = getMetaValue("dag_source_api"); - return axios.get(dagSourceApiUrl, { - headers: { Accept: "text/plain" }, - }); - }, - { - enabled: !!dagData?.dagId, - } - ); -} diff --git a/airflow/www/static/js/api/useDagDetails.ts b/airflow/www/static/js/api/useDagDetails.ts deleted file mode 100644 index 16c8e4397ce0e..0000000000000 --- a/airflow/www/static/js/api/useDagDetails.ts +++ /dev/null @@ -1,53 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; - -import { getMetaValue } from "src/utils"; -import type { API } from "src/types"; -import type { DAG, DAGDetail } from "src/types/api-generated"; -import useDag from "./useDag"; - -const dagDetailsApiUrl = getMetaValue("dag_details_api"); - -const combineResults = ( - dagData: DAG, - dagDetailsData: DAGDetail -): Omit => ({ ...dagData, ...dagDetailsData }); - -const useDagDetails = () => { - const { data: dagData } = useDag(); - const dagDetailsResult = useQuery( - ["dagDetailsQuery"], - () => axios.get(dagDetailsApiUrl), - { - enabled: !!dagData, - } - ); - return { - ...dagDetailsResult, - data: combineResults( - dagData || {}, - dagDetailsResult.data ? dagDetailsResult.data : {} - ), - }; -}; - -export default useDagDetails; diff --git a/airflow/www/static/js/api/useDagRuns.tsx b/airflow/www/static/js/api/useDagRuns.tsx deleted file mode 100644 index 5bc1a43748862..0000000000000 --- a/airflow/www/static/js/api/useDagRuns.tsx +++ /dev/null @@ -1,50 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; -import type { API } from "src/types"; - -import { getMetaValue } from "src/utils"; - -const useDagRuns = ({ - dagId, - state, - limit, - orderBy, -}: API.GetDagRunsVariables) => { - const dagRunsUrl = getMetaValue("dag_runs_url").replace("__DAG_ID__", dagId); - - return useQuery( - ["dagRuns", state, dagId, limit], - async () => - axios.get(dagRunsUrl, { - params: { - state: state ? state.join(",") : state, - limit, - order_by: orderBy, - }, - }), - { - refetchInterval: (autoRefreshInterval || 1) * 1000, - } - ); -}; - -export default useDagRuns; diff --git a/airflow/www/static/js/api/useDags.tsx b/airflow/www/static/js/api/useDags.tsx deleted file mode 100644 index 8939da21d96c7..0000000000000 --- a/airflow/www/static/js/api/useDags.tsx +++ /dev/null @@ -1,40 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; -import type { API } from "src/types"; - -import { getMetaValue } from "src/utils"; - -const dagsUrl = getMetaValue("dags_url"); - -const useDags = ({ paused }: API.GetDagsVariables) => - useQuery( - ["dags", paused], - async () => - axios.get(dagsUrl, { - params: { paused }, - }), - { - refetchInterval: (autoRefreshInterval || 1) * 1000, - } - ); - -export default useDags; diff --git a/airflow/www/static/js/api/useEventLogs.tsx b/airflow/www/static/js/api/useEventLogs.tsx deleted file mode 100644 index 4b5cc92ce59a4..0000000000000 --- a/airflow/www/static/js/api/useEventLogs.tsx +++ /dev/null @@ -1,85 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; - -import { getMetaValue } from "src/utils"; -import type { API } from "src/types"; -import { useAutoRefresh } from "src/context/autorefresh"; - -export default function useEventLogs({ - dagId, - taskId, - runId, - limit, - offset, - orderBy, - after, - before, - owner, - includedEvents, - excludedEvents, -}: API.GetEventLogsVariables) { - const { isRefreshOn } = useAutoRefresh(); - return useQuery( - [ - "eventLogs", - dagId, - taskId, - runId, - limit, - offset, - orderBy, - after, - before, - owner, - excludedEvents, - includedEvents, - ], - () => { - const eventsLogUrl = getMetaValue("event_logs_api"); - const orderParam = orderBy ? { order_by: orderBy } : {}; - const excludedParam = excludedEvents - ? { excluded_events: excludedEvents } - : {}; - const includedParam = includedEvents - ? { included_events: includedEvents } - : {}; - return axios.get(eventsLogUrl, { - params: { - offset, - limit, - ...{ dag_id: dagId }, - ...{ task_id: taskId }, - ...{ run_id: runId }, - ...orderParam, - ...excludedParam, - ...includedParam, - after, - before, - }, - }); - }, - { - refetchInterval: isRefreshOn && (autoRefreshInterval || 1) * 1000, - keepPreviousData: true, - } - ); -} diff --git a/airflow/www/static/js/api/useExtraLinks.ts b/airflow/www/static/js/api/useExtraLinks.ts deleted file mode 100644 index a61f0f3aa69d0..0000000000000 --- a/airflow/www/static/js/api/useExtraLinks.ts +++ /dev/null @@ -1,82 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; -import { getMetaValue } from "../utils"; - -const extraLinksUrl = getMetaValue("extra_links_url"); - -interface LinkData { - url: string | null; - error: string | null; -} - -export default function useExtraLinks({ - dagId, - taskId, - logicalDate, - mapIndex, - extraLinks, - tryNumber, -}: { - dagId: string; - taskId: string; - logicalDate: string; - mapIndex?: number | undefined; - extraLinks: string[]; - tryNumber?: number | undefined; -}) { - return useQuery( - ["extraLinks", dagId, taskId, logicalDate, mapIndex, tryNumber], - async () => { - const data = await Promise.all( - extraLinks.map(async (link) => { - const definedMapIndex = mapIndex ?? -1; - const tryNumberParam = - tryNumber !== undefined ? `&try_number=${tryNumber}` : ""; - const url = `${extraLinksUrl}?task_id=${encodeURIComponent( - taskId - )}&dag_id=${encodeURIComponent( - dagId - )}&logical_date=${encodeURIComponent( - logicalDate - )}&link_name=${encodeURIComponent( - link - )}&map_index=${definedMapIndex}${tryNumberParam}`; - try { - const datum = await axios.get(url); - return { - name: link, - url: datum.url, - }; - } catch (e) { - // eslint-disable-next-line no-console - console.error(e); - return { - name: link, - url: "", - }; - } - }) - ); - return data; - } - ); -} diff --git a/airflow/www/static/js/api/useGraphData.ts b/airflow/www/static/js/api/useGraphData.ts deleted file mode 100644 index e7a94156ef265..0000000000000 --- a/airflow/www/static/js/api/useGraphData.ts +++ /dev/null @@ -1,61 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import { useQuery } from "react-query"; -import axios, { AxiosResponse } from "axios"; - -import { getMetaValue } from "src/utils"; -import useFilters, { - FILTER_DOWNSTREAM_PARAM, - FILTER_UPSTREAM_PARAM, - ROOT_PARAM, -} from "src/dag/useFilters"; -import type { WebserverEdge, DepNode } from "src/types"; - -const DAG_ID_PARAM = "dag_id"; - -const dagId = getMetaValue(DAG_ID_PARAM); -const graphDataUrl = getMetaValue("graph_data_url"); - -interface GraphData { - edges: WebserverEdge[]; - nodes: DepNode; - arrange: string; -} - -const useGraphData = () => { - const { - filters: { root, filterDownstream, filterUpstream }, - } = useFilters(); - - return useQuery( - ["graphData", root, filterUpstream, filterDownstream], - async () => { - const params = { - [DAG_ID_PARAM]: dagId, - [ROOT_PARAM]: root, - [FILTER_UPSTREAM_PARAM]: filterUpstream, - [FILTER_DOWNSTREAM_PARAM]: filterDownstream, - }; - return axios.get(graphDataUrl, { params }); - } - ); -}; - -export default useGraphData; diff --git a/airflow/www/static/js/api/useGridData.test.ts b/airflow/www/static/js/api/useGridData.test.ts deleted file mode 100644 index efbfe5d9820b2..0000000000000 --- a/airflow/www/static/js/api/useGridData.test.ts +++ /dev/null @@ -1,63 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/* global describe, test, expect */ - -import type { DagRun } from "src/types"; -import { areActiveRuns } from "./useGridData"; - -const commonDagRunParams = { - runId: "runId", - logicalDate: "2022-01-01T10:00+00:00", - dataIntervalStart: "2022-01-01T05:00+00:00", - dataIntervalEnd: "2022-01-01T10:00+00:00", - runType: "scheduled" as DagRun["runType"], - queuedAt: null, - startDate: null, - endDate: null, - lastSchedulingDecision: null, - externalTrigger: false, - conf: null, - note: "", -}; - -describe("Test areActiveRuns()", () => { - test("Correctly detects active runs", () => { - const runs: DagRun[] = [ - { state: "success", ...commonDagRunParams }, - { state: "queued", ...commonDagRunParams }, - ]; - expect(areActiveRuns(runs)).toBe(true); - }); - - test("Returns false when all runs are resolved", () => { - const runs: DagRun[] = [ - { state: "success", ...commonDagRunParams }, - { state: "failed", ...commonDagRunParams }, - { state: "failed", ...commonDagRunParams }, - ]; - const result = areActiveRuns(runs); - expect(result).toBe(false); - }); - - test("Returns false when there are no runs", () => { - const result = areActiveRuns(); - expect(result).toBe(false); - }); -}); diff --git a/airflow/www/static/js/api/useGridData.ts b/airflow/www/static/js/api/useGridData.ts deleted file mode 100644 index 795a5db1d5b81..0000000000000 --- a/airflow/www/static/js/api/useGridData.ts +++ /dev/null @@ -1,171 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import { useQuery } from "react-query"; -import axios, { AxiosResponse } from "axios"; - -import { getMetaValue } from "src/utils"; -import { useAutoRefresh } from "src/context/autorefresh"; -import useErrorToast from "src/utils/useErrorToast"; -import useFilters, { - BASE_DATE_PARAM, - NUM_RUNS_PARAM, - RUN_STATE_PARAM, - RUN_TYPE_PARAM, - now, - FILTER_DOWNSTREAM_PARAM, - FILTER_UPSTREAM_PARAM, - ROOT_PARAM, -} from "src/dag/useFilters"; -import type { Task, DagRun, RunOrdering, API } from "src/types"; -import { camelCase } from "lodash"; -import useSelection from "src/dag/useSelection"; - -const DAG_ID_PARAM = "dag_id"; - -// dagId comes from dag.html -const dagId = getMetaValue(DAG_ID_PARAM); -const gridDataUrl = getMetaValue("grid_data_url"); - -export interface GridData { - dagRuns: DagRun[]; - groups: Task; - ordering: RunOrdering; - errors: string[]; -} - -export const emptyGridData: GridData = { - dagRuns: [], - groups: { - id: null, - label: null, - instances: [], - }, - ordering: [], - errors: [], -}; - -const formatOrdering = (data: GridData) => ({ - ...data, - ordering: data.ordering.map((o: string) => camelCase(o)) as RunOrdering, -}); - -export const areActiveRuns = (runs: DagRun[] = []) => - runs.filter((run) => ["queued", "running"].includes(run.state)).length > 0; - -const useGridData = () => { - const { isRefreshOn, stopRefresh } = useAutoRefresh(); - const errorToast = useErrorToast(); - const { - filters: { - baseDate, - numRuns, - runType, - runState, - root, - filterDownstream, - filterUpstream, - }, - onBaseDateChange, - } = useFilters(); - const { - onSelect, - selected: { taskId, runId }, - } = useSelection(); - const query = useQuery( - [ - "gridData", - baseDate, - numRuns, - runType, - runState, - root, - filterUpstream, - filterDownstream, - runId, - ], - async () => { - const params = { - [ROOT_PARAM]: root, - [FILTER_UPSTREAM_PARAM]: filterUpstream, - [FILTER_DOWNSTREAM_PARAM]: filterDownstream, - [DAG_ID_PARAM]: dagId, - [BASE_DATE_PARAM]: baseDate === now ? undefined : baseDate, - [NUM_RUNS_PARAM]: numRuns, - [RUN_TYPE_PARAM]: runType, - [RUN_STATE_PARAM]: runState, - }; - const response = await axios.get(gridDataUrl, { - params, - }); - if (runId && !response.dagRuns.find((dr) => dr.runId === runId)) { - const dagRunUrl = getMetaValue("dag_run_url") - .replace("__DAG_ID__", dagId) - .replace("__DAG_RUN_ID__", runId); - - // If the run id cannot be found in the response, try fetching it to see if its real and then adjust the base date filter - try { - const selectedRun = await axios.get( - dagRunUrl - ); - if (selectedRun?.logicalDate) { - onBaseDateChange(selectedRun.logicalDate); - } - // otherwise the run_id isn't valid and we should unselect it - } catch (e) { - onSelect({ taskId }); - } - } - // turn off auto refresh if there are no active runs - if (!areActiveRuns(response.dagRuns)) stopRefresh(); - // if any errors returned then show as toast message - if (response.errors.length > 0) { - response.errors.forEach((errorMsg) => { - const error = Error(errorMsg); - errorToast({ - title: "Error", - error, - }); - }); - } - - return response; - }, - { - // only refetch if the refresh switch is on - refetchInterval: isRefreshOn && (autoRefreshInterval || 1) * 1000, - keepPreviousData: true, - onError: (error: Error) => { - stopRefresh(); - errorToast({ - title: "Auto-refresh Error", - error, - }); - throw error; - }, - select: formatOrdering, - } - ); - return { - ...query, - data: query.data ?? emptyGridData, - }; -}; - -export default useGridData; diff --git a/airflow/www/static/js/api/useHealth.ts b/airflow/www/static/js/api/useHealth.ts deleted file mode 100644 index ab86698507cb2..0000000000000 --- a/airflow/www/static/js/api/useHealth.ts +++ /dev/null @@ -1,37 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; -import type { API } from "src/types"; - -import { getMetaValue } from "src/utils"; - -const healthUrl = getMetaValue("health_url"); - -const useHealth = () => - useQuery( - ["health"], - async () => axios.get(healthUrl), - { - refetchInterval: (autoRefreshInterval || 1) * 1000, - } - ); - -export default useHealth; diff --git a/airflow/www/static/js/api/useHistoricalMetricsData.ts b/airflow/www/static/js/api/useHistoricalMetricsData.ts deleted file mode 100644 index d14ce13fb6937..0000000000000 --- a/airflow/www/static/js/api/useHistoricalMetricsData.ts +++ /dev/null @@ -1,40 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; -import type { HistoricalMetricsData } from "src/types"; - -import { getMetaValue } from "src/utils"; - -const url = getMetaValue("historical_metrics_data_url"); - -const useHistoricalMetricsData = (startDate: string, endDate: string) => - useQuery( - ["historical_metrics_data", startDate, endDate], - async () => - axios.get(url, { - params: { start_date: startDate, end_date: endDate }, - }), - { - refetchInterval: (autoRefreshInterval || 1) * 1000, - } - ); - -export default useHistoricalMetricsData; diff --git a/airflow/www/static/js/api/useMappedInstances.ts b/airflow/www/static/js/api/useMappedInstances.ts deleted file mode 100644 index 0d76e70305cd3..0000000000000 --- a/airflow/www/static/js/api/useMappedInstances.ts +++ /dev/null @@ -1,56 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; - -import { getMetaValue } from "src/utils"; -import { useAutoRefresh } from "src/context/autorefresh"; -import type { API } from "src/types"; - -const mappedInstancesUrl = getMetaValue("mapped_instances_api"); - -export default function useMappedInstances({ - dagId, - dagRunId, - taskId, - limit, - offset, - orderBy, -}: API.GetMappedTaskInstancesVariables) { - const url = mappedInstancesUrl - .replace("_DAG_RUN_ID_", dagRunId) - .replace("_TASK_ID_", taskId); - const orderParam = - orderBy && orderBy !== "map_index" ? { order_by: orderBy } : {}; - const { isRefreshOn } = useAutoRefresh(); - return useQuery( - ["mappedInstances", dagId, dagRunId, taskId, offset, orderBy], - () => - axios.get(url, { - params: { offset, limit, ...orderParam }, - }), - { - keepPreviousData: true, - refetchInterval: isRefreshOn && (autoRefreshInterval || 1) * 1000, - // staleTime should be similar to the refresh interval - staleTime: (autoRefreshInterval || 1) * 1000, - } - ); -} diff --git a/airflow/www/static/js/api/useMarkFailedRun.ts b/airflow/www/static/js/api/useMarkFailedRun.ts deleted file mode 100644 index 11a94e4264da4..0000000000000 --- a/airflow/www/static/js/api/useMarkFailedRun.ts +++ /dev/null @@ -1,61 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useMutation, useQueryClient } from "react-query"; - -import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; -import { getMetaValue } from "src/utils"; -import { useAutoRefresh } from "src/context/autorefresh"; -import useErrorToast from "src/utils/useErrorToast"; - -const csrfToken = getMetaValue("csrf_token"); -const markFailedUrl = getMetaValue("dagrun_failed_url"); - -export default function useMarkFailedRun(dagId: string, runId: string) { - const queryClient = useQueryClient(); - const errorToast = useErrorToast(); - const { startRefresh } = useAutoRefresh(); - return useMutation( - ["dagRunFailed", dagId, runId], - ({ confirmed = false }: { confirmed: boolean }) => { - const params = new URLSearchParamsWrapper({ - csrf_token: csrfToken, - confirmed, - dag_id: dagId, - dag_run_id: runId, - }).toString(); - - return axios.post(markFailedUrl, params, { - headers: { - "Content-Type": "application/x-www-form-urlencoded", - }, - }); - }, - { - onSuccess: (_, { confirmed }) => { - if (confirmed) { - queryClient.invalidateQueries("gridData"); - startRefresh(); - } - }, - onError: (error: Error) => errorToast({ error }), - } - ); -} diff --git a/airflow/www/static/js/api/useMarkFailedTask.ts b/airflow/www/static/js/api/useMarkFailedTask.ts deleted file mode 100644 index a3fbe554016b9..0000000000000 --- a/airflow/www/static/js/api/useMarkFailedTask.ts +++ /dev/null @@ -1,106 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios from "axios"; -import { useMutation, useQueryClient } from "react-query"; -import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; -import { getMetaValue } from "../utils"; -import { useAutoRefresh } from "../context/autorefresh"; -import useErrorToast from "../utils/useErrorToast"; - -const failedUrl = getMetaValue("failed_url"); -const csrfToken = getMetaValue("csrf_token"); - -export default function useMarkFailedTask({ - dagId, - runId, - taskId, - isGroup, -}: { - dagId: string; - runId: string; - taskId: string; - isGroup: boolean; -}) { - const queryClient = useQueryClient(); - const errorToast = useErrorToast(); - const { startRefresh } = useAutoRefresh(); - return useMutation( - ["markFailed", dagId, runId, taskId], - ({ - past, - future, - upstream, - downstream, - mapIndexes = [], - }: { - past: boolean; - future: boolean; - upstream: boolean; - downstream: boolean; - mapIndexes?: number[]; - }) => { - const params = new URLSearchParamsWrapper({ - csrf_token: csrfToken, - dag_id: dagId, - dag_run_id: runId, - confirmed: true, - past, - future, - upstream, - downstream, - }); - - if (isGroup) { - params.append("group_id", taskId); - } else { - params.append("task_id", taskId); - } - - mapIndexes.forEach((mi: number) => { - params.append("map_index", mi.toString()); - }); - - return axios.post(failedUrl, params.toString(), { - headers: { - "Content-Type": "application/x-www-form-urlencoded", - }, - }); - }, - { - onSuccess: () => { - queryClient.invalidateQueries("gridData"); - queryClient.invalidateQueries([ - "mappedInstances", - dagId, - runId, - taskId, - ]); - queryClient.invalidateQueries([ - "confirmStateChange", - dagId, - runId, - taskId, - ]); - startRefresh(); - }, - onError: (error: Error) => errorToast({ error }), - } - ); -} diff --git a/airflow/www/static/js/api/useMarkSuccessRun.ts b/airflow/www/static/js/api/useMarkSuccessRun.ts deleted file mode 100644 index 119fb29839d62..0000000000000 --- a/airflow/www/static/js/api/useMarkSuccessRun.ts +++ /dev/null @@ -1,60 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useMutation, useQueryClient } from "react-query"; -import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; -import { getMetaValue } from "../utils"; -import { useAutoRefresh } from "../context/autorefresh"; -import useErrorToast from "../utils/useErrorToast"; - -const markSuccessUrl = getMetaValue("dagrun_success_url"); -const csrfToken = getMetaValue("csrf_token"); - -export default function useMarkSuccessRun(dagId: string, runId: string) { - const queryClient = useQueryClient(); - const errorToast = useErrorToast(); - const { startRefresh } = useAutoRefresh(); - return useMutation( - ["dagRunSuccess", dagId, runId], - ({ confirmed = false }: { confirmed: boolean }) => { - const params = new URLSearchParamsWrapper({ - csrf_token: csrfToken, - confirmed, - dag_id: dagId, - dag_run_id: runId, - }).toString(); - - return axios.post(markSuccessUrl, params, { - headers: { - "Content-Type": "application/x-www-form-urlencoded", - }, - }); - }, - { - onSuccess: (_, { confirmed }) => { - if (confirmed) { - queryClient.invalidateQueries("gridData"); - startRefresh(); - } - }, - onError: (error: Error) => errorToast({ error }), - } - ); -} diff --git a/airflow/www/static/js/api/useMarkSuccessTask.ts b/airflow/www/static/js/api/useMarkSuccessTask.ts deleted file mode 100644 index 3676684b6a8ff..0000000000000 --- a/airflow/www/static/js/api/useMarkSuccessTask.ts +++ /dev/null @@ -1,106 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios from "axios"; -import { useMutation, useQueryClient } from "react-query"; -import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; -import { getMetaValue } from "../utils"; -import { useAutoRefresh } from "../context/autorefresh"; -import useErrorToast from "../utils/useErrorToast"; - -const csrfToken = getMetaValue("csrf_token"); -const successUrl = getMetaValue("success_url"); - -export default function useMarkSuccessTask({ - dagId, - runId, - taskId, - isGroup, -}: { - dagId: string; - runId: string; - taskId: string; - isGroup: boolean; -}) { - const queryClient = useQueryClient(); - const errorToast = useErrorToast(); - const { startRefresh } = useAutoRefresh(); - return useMutation( - ["markSuccess", dagId, runId, taskId], - ({ - past, - future, - upstream, - downstream, - mapIndexes = [], - }: { - past: boolean; - future: boolean; - upstream: boolean; - downstream: boolean; - mapIndexes?: number[]; - }) => { - const params = new URLSearchParamsWrapper({ - csrf_token: csrfToken, - dag_id: dagId, - dag_run_id: runId, - confirmed: true, - past, - future, - upstream, - downstream, - }); - - if (isGroup) { - params.append("group_id", taskId); - } else { - params.append("task_id", taskId); - } - - mapIndexes.forEach((mi: number) => { - params.append("map_index", mi.toString()); - }); - - return axios.post(successUrl, params.toString(), { - headers: { - "Content-Type": "application/x-www-form-urlencoded", - }, - }); - }, - { - onSuccess: () => { - queryClient.invalidateQueries("gridData"); - queryClient.invalidateQueries([ - "mappedInstances", - dagId, - runId, - taskId, - ]); - queryClient.invalidateQueries([ - "confirmStateChange", - dagId, - runId, - taskId, - ]); - startRefresh(); - }, - onError: (error: Error) => errorToast({ error }), - } - ); -} diff --git a/airflow/www/static/js/api/useMarkTaskDryRun.ts b/airflow/www/static/js/api/useMarkTaskDryRun.ts deleted file mode 100644 index 51478299410a5..0000000000000 --- a/airflow/www/static/js/api/useMarkTaskDryRun.ts +++ /dev/null @@ -1,93 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; -import type { TaskState, MinimalTaskInstance } from "src/types"; -import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; -import { getMetaValue } from "../utils"; - -const confirmUrl = getMetaValue("confirm_url"); - -const useMarkTaskDryRun = ({ - dagId, - runId, - taskId, - state, - isGroup, - past, - future, - upstream, - downstream, - mapIndexes = [], - enabled = false, -}: { - dagId: string; - runId: string; - taskId: string; - state: TaskState; - isGroup: boolean; - past: boolean; - future: boolean; - upstream: boolean; - downstream: boolean; - mapIndexes?: number[]; - enabled?: boolean; -}) => - useQuery( - [ - "confirmStateChange", - dagId, - runId, - taskId, - state, - past, - future, - upstream, - downstream, - mapIndexes, - ], - () => { - const params = new URLSearchParamsWrapper({ - dag_id: dagId, - dag_run_id: runId, - past, - future, - upstream, - downstream, - state, - }); - - if (isGroup) { - params.append("group_id", taskId); - } else { - params.append("task_id", taskId); - } - - mapIndexes.forEach((mi: number) => { - params.append("map_index", mi.toString()); - }); - return axios.get(confirmUrl, { - params, - }); - }, - { enabled } - ); - -export default useMarkTaskDryRun; diff --git a/airflow/www/static/js/api/usePools.ts b/airflow/www/static/js/api/usePools.ts deleted file mode 100644 index 664626a2ab4be..0000000000000 --- a/airflow/www/static/js/api/usePools.ts +++ /dev/null @@ -1,41 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; -import { useAutoRefresh } from "src/context/autorefresh"; -import type { API } from "src/types"; - -import { getMetaValue } from "src/utils"; - -const poolsUrl = getMetaValue("pools_url"); - -const usePools = () => { - const { isRefreshOn } = useAutoRefresh(); - - return useQuery( - ["pools"], - async () => axios.get(poolsUrl), - { - refetchInterval: isRefreshOn && (autoRefreshInterval || 1) * 1000, - } - ); -}; - -export default usePools; diff --git a/airflow/www/static/js/api/useQueueRun.ts b/airflow/www/static/js/api/useQueueRun.ts deleted file mode 100644 index 28157a1879ea2..0000000000000 --- a/airflow/www/static/js/api/useQueueRun.ts +++ /dev/null @@ -1,59 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useMutation, useQueryClient } from "react-query"; -import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; -import { getMetaValue } from "../utils"; -import { useAutoRefresh } from "../context/autorefresh"; -import useErrorToast from "../utils/useErrorToast"; - -const csrfToken = getMetaValue("csrf_token"); -const queuedUrl = getMetaValue("dagrun_queued_url"); - -export default function useQueueRun(dagId: string, runId: string) { - const queryClient = useQueryClient(); - const errorToast = useErrorToast(); - const { startRefresh } = useAutoRefresh(); - return useMutation( - ["dagRunQueue", dagId, runId], - ({ confirmed = false }: { confirmed: boolean }) => { - const params = new URLSearchParamsWrapper({ - csrf_token: csrfToken, - confirmed, - dag_id: dagId, - dag_run_id: runId, - }).toString(); - return axios.post(queuedUrl, params, { - headers: { - "Content-Type": "application/x-www-form-urlencoded", - }, - }); - }, - { - onSuccess: (_, { confirmed }) => { - if (confirmed) { - queryClient.invalidateQueries("gridData"); - startRefresh(); - } - }, - onError: (error: Error) => errorToast({ error }), - } - ); -} diff --git a/airflow/www/static/js/api/useRenderedK8s.ts b/airflow/www/static/js/api/useRenderedK8s.ts deleted file mode 100644 index 1b1828e569a09..0000000000000 --- a/airflow/www/static/js/api/useRenderedK8s.ts +++ /dev/null @@ -1,43 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; - -import { getMetaValue } from "src/utils"; - -const url = getMetaValue("rendered_k8s_data_url"); - -const useRenderedK8s = ( - runId: string | null, - taskId: string | null, - mapIndex?: number -) => - useQuery( - ["rendered_k8s", runId, taskId, mapIndex], - async () => - axios.get(url, { - params: { run_id: runId, task_id: taskId, map_index: mapIndex }, - }), - { - enabled: !!runId && !!taskId, - } - ); - -export default useRenderedK8s; diff --git a/airflow/www/static/js/api/useSetDagRunNote.ts b/airflow/www/static/js/api/useSetDagRunNote.ts deleted file mode 100644 index 7f561d47feede..0000000000000 --- a/airflow/www/static/js/api/useSetDagRunNote.ts +++ /dev/null @@ -1,66 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useMutation, useQueryClient } from "react-query"; - -import { getMetaValue } from "src/utils"; -import type { API } from "src/types"; -import useErrorToast from "src/utils/useErrorToast"; - -import { emptyGridData } from "./useGridData"; -import type { GridData } from "./useGridData"; - -const setDagRunNoteURI = getMetaValue("set_dag_run_note"); - -interface Props { - dagId: string; - runId: string; -} - -export default function useSetDagRunNote({ dagId, runId }: Props) { - const queryClient = useQueryClient(); - const errorToast = useErrorToast(); - const setDagRunNote = setDagRunNoteURI.replace("_DAG_RUN_ID_", runId); - - return useMutation( - ["setDagRunNote", dagId, runId], - (note: string | null) => - axios.patch(setDagRunNote, { note }), - { - onSuccess: async (data) => { - const note = data.note ?? null; - - const updateGridData = (oldGridData: GridData | undefined) => - !oldGridData - ? emptyGridData - : { - ...oldGridData, - dagRuns: oldGridData.dagRuns.map((dr) => - dr.runId === runId ? { ...dr, note } : dr - ), - }; - - await queryClient.cancelQueries("gridData"); - queryClient.setQueriesData("gridData", updateGridData); - }, - onError: (error: Error) => errorToast({ error }), - } - ); -} diff --git a/airflow/www/static/js/api/useSetTaskInstanceNote.ts b/airflow/www/static/js/api/useSetTaskInstanceNote.ts deleted file mode 100644 index caca6c28065ab..0000000000000 --- a/airflow/www/static/js/api/useSetTaskInstanceNote.ts +++ /dev/null @@ -1,129 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useMutation, useQueryClient } from "react-query"; - -import { getMetaValue } from "src/utils"; -import useErrorToast from "src/utils/useErrorToast"; - -import type { API } from "src/types"; - -const setTaskInstancesNoteURI = getMetaValue("set_task_instance_note"); -const setMappedTaskInstancesNoteURI = getMetaValue( - "set_mapped_task_instance_note" -); - -interface Props { - dagId: string; - runId: string; - taskId: string; - mapIndex?: number; -} - -export default function useSetTaskInstanceNote({ - dagId, - runId, - taskId, - mapIndex = -1, -}: Props) { - const queryClient = useQueryClient(); - const errorToast = useErrorToast(); - // Note: Werkzeug does not like the META URL on dag.html with an integer. It can not put - // _MAP_INDEX_ there as it interprets that as the integer. Hence, we pass 0 as the integer. - // To avoid we replace other stuff, we add the surrounding strings to the replacement query. - const url = ( - mapIndex >= 0 ? setMappedTaskInstancesNoteURI : setTaskInstancesNoteURI - ) - .replace("_DAG_RUN_ID_", runId) - .replace("_TASK_ID_/0/setNote", `_TASK_ID_/${mapIndex}/setNote`) - .replace("_TASK_ID_", taskId); - - return useMutation( - ["setTaskInstanceNotes", dagId, runId, taskId, mapIndex], - (note: string | null) => - axios.patch(url, { note }), - { - onSuccess: async (data) => { - const note = data.note ?? null; - - const updateMappedInstancesResult = ( - oldMappedInstances?: API.TaskInstanceCollection - ) => { - if (!oldMappedInstances) { - return { - taskInstances: [], - totalEntries: 0, - }; - } - if (mapIndex === undefined || mapIndex < 0) return oldMappedInstances; - return { - ...oldMappedInstances, - taskInstances: oldMappedInstances.taskInstances?.map((ti) => - ti.dagRunId === runId && - ti.taskId === taskId && - ti.mapIndex === mapIndex - ? { ...ti, note } - : ti - ), - }; - }; - - const updateTaskInstanceResult = ( - oldTaskInstance?: API.TaskInstance - ) => { - if (!oldTaskInstance) throw new Error("Unknown value..."); - if ( - oldTaskInstance.dagRunId === runId && - oldTaskInstance.taskId === taskId && - ((oldTaskInstance.mapIndex == null && mapIndex < 0) || - oldTaskInstance.mapIndex === mapIndex) - ) { - return { - ...oldTaskInstance, - note, - }; - } - return oldTaskInstance; - }; - /* - This will force a refetch of gridData. - Mutating the nested object is quite complicated, - we should simplify the gridData API object first - */ - await queryClient.invalidateQueries("gridData"); - - if (mapIndex >= 0) { - await queryClient.cancelQueries("mappedInstances"); - queryClient.setQueriesData( - "mappedInstances", - updateMappedInstancesResult - ); - } - - await queryClient.cancelQueries("taskInstance"); - queryClient.setQueriesData( - ["taskInstance", dagId, runId, taskId, mapIndex], - updateTaskInstanceResult - ); - }, - onError: (error: Error) => errorToast({ error }), - } - ); -} diff --git a/airflow/www/static/js/api/useTIHistory.ts b/airflow/www/static/js/api/useTIHistory.ts deleted file mode 100644 index 1d1ee1d40f586..0000000000000 --- a/airflow/www/static/js/api/useTIHistory.ts +++ /dev/null @@ -1,62 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios from "axios"; -import { useQuery, UseQueryOptions } from "react-query"; -import { useAutoRefresh } from "src/context/autorefresh"; -import type { - GetTaskInstanceTriesVariables, - TaskInstanceCollection, -} from "src/types/api-generated"; - -import { getMetaValue } from "src/utils"; - -interface Props extends GetTaskInstanceTriesVariables { - mapIndex?: number; - options?: UseQueryOptions; -} - -export default function useTIHistory({ - dagId, - dagRunId, - taskId, - mapIndex, - options, -}: Props) { - const { isRefreshOn } = useAutoRefresh(); - return useQuery( - ["tiHistory", dagId, dagRunId, taskId, mapIndex], - () => { - let tiHistoryUrl = getMetaValue("task_tries_api") - .replace("_DAG_ID_", dagId) - .replace("_DAG_RUN_ID_", dagRunId) - .replace("_TASK_ID_", taskId); - - if (mapIndex !== undefined && mapIndex > -1) { - tiHistoryUrl = tiHistoryUrl.replace("/tries", `/${mapIndex}/tries`); - } - - return axios.get(tiHistoryUrl); - }, - { - refetchInterval: isRefreshOn && (autoRefreshInterval || 1) * 1000, - ...options, - } - ); -} diff --git a/airflow/www/static/js/api/useTaskDetail.tsx b/airflow/www/static/js/api/useTaskDetail.tsx deleted file mode 100644 index d7e0ce81c1c43..0000000000000 --- a/airflow/www/static/js/api/useTaskDetail.tsx +++ /dev/null @@ -1,33 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import type { API } from "src/types"; -import { useQuery } from "react-query"; -import { getMetaValue } from "../utils"; - -const taskDetailURI = getMetaValue("task_detail_api"); - -export default function useTaskDetail({ taskId }: { taskId: string }) { - return useQuery(["taskDetails", taskId], async () => { - const url = taskDetailURI.replace("_TASK_ID_", taskId); - - return axios.get(url); - }); -} diff --git a/airflow/www/static/js/api/useTaskFailedDependency.ts b/airflow/www/static/js/api/useTaskFailedDependency.ts deleted file mode 100644 index 08168deebffaf..0000000000000 --- a/airflow/www/static/js/api/useTaskFailedDependency.ts +++ /dev/null @@ -1,63 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios, { AxiosResponse } from "axios"; -import { useAutoRefresh } from "src/context/autorefresh"; -import type { API } from "src/types"; -import { useQuery } from "react-query"; -import { getMetaValue } from "../utils"; - -const taskDependencyURI = getMetaValue("task_dependency_api"); -const mappedTaskDependencyURI = getMetaValue("mapped_task_dependency_api"); - -export default function useTaskFailedDependency({ - dagId, - taskId, - runId, - mapIndex, -}: { - dagId: string; - taskId: string; - runId: string; - mapIndex?: number | undefined; -}) { - const { isRefreshOn } = useAutoRefresh(); - return useQuery( - ["taskFailedDependencies", dagId, taskId, runId, mapIndex], - async () => { - const definedMapIndex = mapIndex ?? -1; - const url = ( - definedMapIndex >= 0 ? mappedTaskDependencyURI : taskDependencyURI - ) - .replace("_DAG_RUN_ID_", runId) - .replace( - "_TASK_ID_/0/dependencies", - `_TASK_ID_/${mapIndex}/dependencies` - ) - .replace("_TASK_ID_", taskId); - - const datum = await axios.get< - AxiosResponse, - API.TaskInstanceDependencyCollection - >(url); - return datum; - }, - { refetchInterval: isRefreshOn && (autoRefreshInterval || 1) * 1000 } - ); -} diff --git a/airflow/www/static/js/api/useTaskInstance.ts b/airflow/www/static/js/api/useTaskInstance.ts deleted file mode 100644 index c5a96f3e44047..0000000000000 --- a/airflow/www/static/js/api/useTaskInstance.ts +++ /dev/null @@ -1,66 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios from "axios"; -import type { API } from "src/types"; -import { useQuery, UseQueryOptions } from "react-query"; -import { useAutoRefresh } from "src/context/autorefresh"; - -import { getMetaValue } from "src/utils"; -import type { SetOptional } from "type-fest"; - -const taskInstanceApi = getMetaValue("task_instance_api"); - -interface Props - extends SetOptional { - options?: UseQueryOptions; -} - -const useTaskInstance = ({ - dagId, - dagRunId, - taskId, - mapIndex, - options, -}: Props) => { - let url: string = ""; - if (taskInstanceApi) { - url = taskInstanceApi - .replace("_DAG_ID_", dagId) - .replace("_DAG_RUN_ID_", dagRunId) - .replace("_TASK_ID_", taskId); - } - - if (mapIndex !== undefined && mapIndex >= 0) { - url += `/${mapIndex.toString()}`; - } - - const { isRefreshOn } = useAutoRefresh(); - - return useQuery( - ["taskInstance", dagId, dagRunId, taskId, mapIndex], - () => axios.get(url), - { - refetchInterval: isRefreshOn && (autoRefreshInterval || 1) * 1000, - ...options, - } - ); -}; - -export default useTaskInstance; diff --git a/airflow/www/static/js/api/useTaskLog.ts b/airflow/www/static/js/api/useTaskLog.ts deleted file mode 100644 index a5e9bab69f82d..0000000000000 --- a/airflow/www/static/js/api/useTaskLog.ts +++ /dev/null @@ -1,83 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import { useState } from "react"; -import axios, { AxiosResponse } from "axios"; -import { useQuery } from "react-query"; -import { useAutoRefresh } from "src/context/autorefresh"; -import type { API, TaskInstance } from "src/types"; - -import { getMetaValue } from "src/utils"; - -const taskLogApi = getMetaValue("task_log_api"); - -interface Props extends API.GetLogVariables { - state?: TaskInstance["state"]; -} - -const useTaskLog = ({ - dagId, - dagRunId, - taskId, - taskTryNumber, - mapIndex, - fullContent = false, - state, -}: Props) => { - let url: string = ""; - const [isPreviousStatePending, setPrevState] = useState(true); - if (taskLogApi) { - url = taskLogApi - .replace("_DAG_RUN_ID_", dagRunId) - .replace("_TASK_ID_", taskId) - .replace(/-1$/, taskTryNumber.toString()); - } - - const { isRefreshOn } = useAutoRefresh(); - - // Only refresh is the state is pending - const isStatePending = - state === "deferred" || - state === "scheduled" || - state === "running" || - state === "up_for_reschedule" || - state === "up_for_retry" || - state === "queued" || - state === "restarting"; - - // We also want to get the last log when the task was finished - const expectingLogs = isStatePending || isPreviousStatePending; - - return useQuery( - ["taskLogs", dagId, dagRunId, taskId, mapIndex, taskTryNumber, fullContent], - () => { - setPrevState(isStatePending); - return axios.get(url, { - headers: { Accept: "text/plain" }, - params: { map_index: mapIndex, full_content: fullContent }, - }); - }, - { - refetchInterval: - expectingLogs && isRefreshOn && (autoRefreshInterval || 1) * 1000, - } - ); -}; - -export default useTaskLog; diff --git a/airflow/www/static/js/api/useTaskXcom.ts b/airflow/www/static/js/api/useTaskXcom.ts deleted file mode 100644 index d8758e60b9056..0000000000000 --- a/airflow/www/static/js/api/useTaskXcom.ts +++ /dev/null @@ -1,73 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import type { API } from "src/types"; -import { getMetaValue } from "src/utils"; -import { useQuery } from "react-query"; -import axios, { AxiosResponse } from "axios"; - -// tryNumber is not required to get XCom keys or values but is used -// in query key so refetch will occur if new tries are available -interface TaskXcomCollectionProps extends API.GetXcomEntriesVariables { - tryNumber: number; -} -interface TaskXcomProps extends API.GetXcomEntryVariables { - tryNumber: number; -} - -export const useTaskXcomCollection = ({ - dagId, - dagRunId, - taskId, - mapIndex, - tryNumber, -}: TaskXcomCollectionProps) => - useQuery(["taskXcoms", dagId, dagRunId, taskId, mapIndex, tryNumber], () => - axios.get( - getMetaValue("task_xcom_entries_api") - .replace("_DAG_RUN_ID_", dagRunId) - .replace("_TASK_ID_", taskId), - { params: { map_index: mapIndex } } - ) - ); - -export const useTaskXcomEntry = ({ - dagId, - dagRunId, - taskId, - mapIndex, - xcomKey, - tryNumber, -}: TaskXcomProps) => - useQuery( - ["taskXcom", dagId, dagRunId, taskId, mapIndex, xcomKey, tryNumber], - () => { - const taskXcomEntryApiUrl = getMetaValue("task_xcom_entry_api") - .replace("_DAG_RUN_ID_", dagRunId) - .replace("_TASK_ID_", taskId) - .replace("_XCOM_KEY_", encodeURIComponent(xcomKey)); - - return axios.get(taskXcomEntryApiUrl, { - params: { map_index: mapIndex, stringify: false }, - }); - }, - { - enabled: !!xcomKey, - } - ); diff --git a/airflow/www/static/js/api/useUpstreamAssetEvents.ts b/airflow/www/static/js/api/useUpstreamAssetEvents.ts deleted file mode 100644 index 437205501d6c5..0000000000000 --- a/airflow/www/static/js/api/useUpstreamAssetEvents.ts +++ /dev/null @@ -1,51 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import axios from "axios"; -import { useQuery, UseQueryOptions } from "react-query"; - -import { getMetaValue } from "src/utils"; -import type { - AssetEventCollection, - GetUpstreamAssetEventsVariables, -} from "src/types/api-generated"; - -interface Props extends GetUpstreamAssetEventsVariables { - options?: UseQueryOptions; -} - -const useUpstreamAssetEvents = ({ dagId, dagRunId, options }: Props) => { - const upstreamEventsUrl = ( - getMetaValue("upstream_asset_events_api") || - `api/v1/dags/${dagId}/dagRuns/_DAG_RUN_ID_/upstreamAssetEvents` - ).replace("_DAG_RUN_ID_", encodeURIComponent(dagRunId)); - - const query = useQuery( - ["upstreamAssetEvents", dagRunId], - () => axios.get(upstreamEventsUrl), - options - ); - - return { - ...query, - data: query.data ?? { assetEvents: [], totalEntries: 0 }, - }; -}; - -export default useUpstreamAssetEvents; diff --git a/airflow/www/static/js/assetUtils.js b/airflow/www/static/js/assetUtils.js deleted file mode 100644 index d2d556c4a2a38..0000000000000 --- a/airflow/www/static/js/assetUtils.js +++ /dev/null @@ -1,90 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/* global $, isoDateToTimeEl, document */ - -import { getMetaValue } from "./utils"; - -export function openAssetModal(dagId, summary, nextAssets, error) { - const assetEvents = nextAssets.events || []; - const expression = nextAssets.asset_expression; - const assetsUrl = getMetaValue("assets_url"); - $("#asset_expression").empty(); - $("#assets_tbody").empty(); - $("#assets_error").hide(); - $("#dag_id").text(dagId); - $("#asset_expression").text(JSON.stringify(expression, null, 2)); - $("#assetNextRunModal").modal({}); - if (summary) $("#next_run_summary").text(summary); - assetEvents.forEach((d) => { - const row = document.createElement("tr"); - - const uriCell = document.createElement("td"); - const assetLink = document.createElement("a"); - assetLink.href = `${assetsUrl}?uri=${encodeURIComponent(d.uri)}`; - assetLink.innerText = d.uri; - uriCell.append(assetLink); - - const timeCell = document.createElement("td"); - if (d.lastUpdate) timeCell.append(isoDateToTimeEl(d.lastUpdate)); - - row.append(uriCell); - row.append(timeCell); - $("#assets_tbody").append(row); - }); - - if (error) { - $("#assets_error_msg").text(error); - $("#assets_error").show(); - } -} - -export function getAssetTooltipInfo(dagId, run, setNextAssets) { - let nextRunUrl = getMetaValue("next_run_assets_url"); - if (dagId) { - if (nextRunUrl.includes("__DAG_ID__")) { - nextRunUrl = nextRunUrl.replace("__DAG_ID__", dagId); - } - $.get(nextRunUrl) - .done((nextAssets) => { - const assetEvents = nextAssets.events; - let count = 0; - let title = "Pending assets:
"; - setNextAssets(nextAssets); - assetEvents.forEach((d) => { - if (!d.created_at) { - if (count < 4) title += `${d.uri}
`; - count += 1; - } - }); - if (count > 4) { - title += `
And ${count - 4} more.`; - } - title += "
Click to see more details."; - $(run).attr("data-original-title", () => title); - }) - .fail((response, textStatus, err) => { - const description = - (response.responseJSON && response.responseJSON.error) || - "Something went wrong."; - const error = `${textStatus}: ${err} ${description}`; - setNextAssets([], error); - }); - } -} diff --git a/airflow/www/static/js/assets/AssetDetails.tsx b/airflow/www/static/js/assets/AssetDetails.tsx deleted file mode 100644 index 105b60aa75caf..0000000000000 --- a/airflow/www/static/js/assets/AssetDetails.tsx +++ /dev/null @@ -1,111 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import React from "react"; -import { - Spinner, - Flex, - Grid, - GridItem, - Heading, - Link, - Box, -} from "@chakra-ui/react"; -import { isEmpty } from "lodash"; - -import { useAsset } from "src/api"; -import { getMetaValue } from "src/utils"; -import RenderedJsonField from "src/components/RenderedJsonField"; - -import Events from "./AssetEvents"; - -const gridUrl = getMetaValue("grid_url"); - -interface Props { - uri: string; -} - -const AssetDetails = ({ uri }: Props) => { - const { data: asset, isLoading } = useAsset({ uri }); - - const hasProducingTasks = !!asset?.producingTasks?.length; - const hasConsumingDags = !!asset?.consumingDags?.length; - - return ( - - {isLoading && } - - {hasProducingTasks && ( - - Tasks that update this Asset - {asset?.producingTasks?.map((task) => { - if (!task.taskId || !task.dagId) return null; - const url = `${gridUrl?.replace( - "__DAG_ID__", - task.dagId - )}?&task_id=${encodeURIComponent(task.taskId)}`; - return ( - - {task.dagId}.{task.taskId} - - ); - })} - - )} - {hasConsumingDags && ( - - DAGs that consume this Asset - {asset?.consumingDags?.map((dag) => { - if (!dag.dagId) return null; - const url = gridUrl?.replace("__DAG_ID__", dag.dagId); - return ( - - {dag.dagId} - - ); - })} - - )} - - {asset?.extra && !isEmpty(asset?.extra) && ( - - )} - - {asset && asset.id && } - - - ); -}; - -export default AssetDetails; diff --git a/airflow/www/static/js/assets/AssetEvents.tsx b/airflow/www/static/js/assets/AssetEvents.tsx deleted file mode 100644 index 318b6819264cc..0000000000000 --- a/airflow/www/static/js/assets/AssetEvents.tsx +++ /dev/null @@ -1,129 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import React, { useMemo, useState } from "react"; -import { snakeCase } from "lodash"; -import type { SortingRule } from "react-table"; -import { Box, Flex, Heading, Select } from "@chakra-ui/react"; - -import { useAssetEvents } from "src/api"; - -import { CardList, type CardDef } from "src/components/Table"; -import type { AssetEvent } from "src/types/api-generated"; -import AssetEventCard from "src/components/AssetEventCard"; - -type Props = { - assetId?: number; - showLabel?: boolean; -}; - -const cardDef: CardDef = { - card: ({ row }) => , -}; - -const Events = ({ assetId, showLabel }: Props) => { - const limit = 25; - const [offset, setOffset] = useState(0); - const [sortBy, setSortBy] = useState[]>([ - { id: "timestamp", desc: true }, - ]); - - const sort = sortBy[0]; - const orderBy = sort ? `${sort.desc ? "-" : ""}${snakeCase(sort.id)}` : ""; - - const { - data: { assetEvents = [], totalEntries = 0 }, - isLoading: isEventsLoading, - } = useAssetEvents({ - assetId, - limit, - offset, - orderBy, - }); - - const columns = useMemo( - () => [ - { - Header: "When", - accessor: "timestamp", - }, - { - Header: "Asset", - accessor: "assetUri", - }, - { - Header: "Source Task Instance", - accessor: "sourceTaskId", - }, - { - Header: "Triggered Runs", - accessor: "createdDagruns", - }, - { - Header: "Extra", - accessor: "extra", - }, - ], - [] - ); - - const data = useMemo(() => assetEvents, [assetEvents]); - - return ( - - - {showLabel && "Events"} - - Sort: - - - - - - ); -}; - -export default Events; diff --git a/airflow/www/static/js/assets/AssetList.test.tsx b/airflow/www/static/js/assets/AssetList.test.tsx deleted file mode 100644 index d09a6a433b59a..0000000000000 --- a/airflow/www/static/js/assets/AssetList.test.tsx +++ /dev/null @@ -1,125 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/* global describe, test, expect */ - -import React from "react"; -import { render } from "@testing-library/react"; - -import * as useAssetsModule from "src/api/useAssetsSummary"; -import { Wrapper } from "src/utils/testUtils"; - -import type { UseQueryResult } from "react-query"; -import type { AssetListItem } from "src/types"; -import AssetsList from "./AssetsList"; - -const assets = [ - { - id: 0, - uri: "this_asset", - extra: null, - lastAssetUpdate: null, - totalUpdates: 0, - createdAt: new Date().toISOString(), - updatedAt: new Date().toISOString(), - }, - { - id: 1, - uri: "that_asset", - extra: null, - lastAssetUpdate: new Date().toISOString(), - totalUpdates: 10, - createdAt: new Date().toISOString(), - updatedAt: new Date().toISOString(), - }, - { - id: 1, - uri: "extra_asset", - extra: null, - lastAssetUpdate: new Date().toISOString(), - totalUpdates: 1, - createdAt: new Date().toISOString(), - updatedAt: new Date().toISOString(), - }, -]; - -type UseAssetsReturn = UseQueryResult & { - data: useAssetsModule.AssetsData; -}; - -const returnValue = { - data: { - assets, - totalEntries: assets.length, - }, - isSuccess: true, -} as UseAssetsReturn; - -const emptyReturnValue = { - data: { - assets: [] as AssetListItem[], - totalEntries: 0, - }, - isSuccess: true, - isLoading: false, -} as UseAssetsReturn; - -describe("Test Assets List", () => { - test("Displays a list of assets", () => { - jest - .spyOn(useAssetsModule, "default") - .mockImplementation(() => returnValue); - - const { getByText, queryAllByTestId } = render( - {}} />, - { wrapper: Wrapper } - ); - - const listItems = queryAllByTestId("asset-list-item"); - - expect(listItems).toHaveLength(3); - - expect(getByText(assets[0].uri)).toBeDefined(); - expect(getByText("Total Updates: 0")).toBeDefined(); - - expect(getByText(assets[1].uri)).toBeDefined(); - expect(getByText("Total Updates: 10")).toBeDefined(); - - expect(getByText(assets[2].uri)).toBeDefined(); - expect(getByText("Total Updates: 1")).toBeDefined(); - }); - - test("Empty state displays when there are no assets", () => { - jest - .spyOn(useAssetsModule, "default") - .mockImplementation(() => emptyReturnValue); - - const { getByText, queryAllByTestId, getByTestId } = render( - {}} />, - { wrapper: Wrapper } - ); - - const listItems = queryAllByTestId("asset-list-item"); - - expect(listItems).toHaveLength(0); - - expect(getByTestId("no-assets-msg")).toBeInTheDocument(); - expect(getByText("No Data found.")).toBeInTheDocument(); - }); -}); diff --git a/airflow/www/static/js/assets/AssetsList.tsx b/airflow/www/static/js/assets/AssetsList.tsx deleted file mode 100644 index 4be107adca3ed..0000000000000 --- a/airflow/www/static/js/assets/AssetsList.tsx +++ /dev/null @@ -1,178 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import React, { useMemo, useState } from "react"; -import { Box, Flex, Text, Link, ButtonGroup, Button } from "@chakra-ui/react"; -import { snakeCase } from "lodash"; -import type { Row, SortingRule } from "react-table"; -import { useSearchParams } from "react-router-dom"; - -import { useAssetsSummary } from "src/api"; -import { CellProps, Table, TimeCell } from "src/components/Table"; -import type { API } from "src/types"; -import { getMetaValue } from "src/utils"; -import type { DateOption } from "src/api/useAssetsSummary"; - -import type { OnSelectProps } from "./types"; - -interface Props { - onSelect: (props: OnSelectProps) => void; -} - -const DetailCell = ({ cell: { row } }: CellProps) => { - const { totalUpdates, uri } = row.original; - return ( - - {uri} - - Total Updates: {totalUpdates} - - - ); -}; - -const DATE_FILTER_PARAM = "updated_within"; - -const dateOptions: Record = { - month: { count: 30, unit: "days" }, - week: { count: 7, unit: "days" }, - day: { count: 24, unit: "hours" }, - hour: { count: 1, unit: "hour" }, -}; - -const AssetsList = ({ onSelect }: Props) => { - const limit = 25; - const [offset, setOffset] = useState(0); - - const [searchParams, setSearchParams] = useSearchParams(); - - const dateFilter = searchParams.get(DATE_FILTER_PARAM) || undefined; - - const [sortBy, setSortBy] = useState[]>([ - { id: "lastAssetUpdate", desc: true }, - ]); - const sort = sortBy[0]; - const order = sort ? `${sort.desc ? "-" : ""}${snakeCase(sort.id)}` : ""; - - const { - data: { assets, totalEntries }, - isLoading, - } = useAssetsSummary({ - limit, - offset, - order, - updatedAfter: dateFilter ? dateOptions[dateFilter] : undefined, - }); - - const columns = useMemo( - () => [ - { - Header: "URI", - accessor: "uri", - Cell: DetailCell, - }, - { - Header: "Last Update", - accessor: "lastAssetUpdate", - Cell: TimeCell, - }, - ], - [] - ); - - const data = useMemo(() => assets, [assets]); - const memoSort = useMemo(() => sortBy, [sortBy]); - - const onAssetSelect = (row: Row) => { - if (row.original.uri) onSelect({ uri: row.original.uri }); - }; - - const docsUrl = getMetaValue("assets_docs"); - - return ( - <> - {!assets.length && !isLoading && !dateFilter && ( - - Looks like you do not have any assets yet. Check out the{" "} - - docs - {" "} - to learn how to create an asset. - - )} - - Filter assets with updates in the past: - - - {Object.keys(dateOptions).map((option) => { - const filter = dateOptions[option]; - const isSelected = option === dateFilter; - return ( - - ); - })} - - - - - - - ); -}; - -export default AssetsList; diff --git a/airflow/www/static/js/assets/CreateAssetEvent.tsx b/airflow/www/static/js/assets/CreateAssetEvent.tsx deleted file mode 100644 index f51b069b78ef6..0000000000000 --- a/airflow/www/static/js/assets/CreateAssetEvent.tsx +++ /dev/null @@ -1,113 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import React, { useState } from "react"; -import { - Button, - FormControl, - FormErrorMessage, - FormLabel, - Modal, - ModalBody, - ModalCloseButton, - ModalContent, - ModalFooter, - ModalHeader, - ModalOverlay, - Textarea, -} from "@chakra-ui/react"; - -import { useContainerRef } from "src/context/containerRef"; -import { useCreateAssetEvent, useAsset } from "src/api"; - -interface Props { - isOpen: boolean; - onClose: () => void; - uri: string; -} - -function checkJsonString(str: string) { - try { - JSON.parse(str); - } catch (e) { - return false; - } - return true; -} - -const CreateAssetEventModal = ({ uri, isOpen, onClose }: Props) => { - const containerRef = useContainerRef(); - const [extra, setExtra] = useState(""); - const { data: asset } = useAsset({ uri }); - - const isJson = checkJsonString(extra); - const isDisabled = !!extra && !isJson; - - const { mutate: createAssetEvent, isLoading } = useCreateAssetEvent({ - assetId: asset?.id, - uri: asset?.uri, - }); - - const onSubmit = () => { - createAssetEvent(extra ? JSON.parse(extra) : undefined); - onClose(); - }; - - if (!asset) return null; - - return ( - - - - Manually create event for {asset?.uri} - - - - Extra (optional) - - {% elif "examples" in form_details.schema and form_details.schema.examples %} - - {% else %} - - {% endif %} - {% elif form_details.schema and "object" in form_details.schema.type %} - - {% elif form_details.schema and ("integer" in form_details.schema.type or "number" in form_details.schema.type) %} - - {% elif form_details.schema and "string" in form_details.schema.type and "format" in form_details.schema and form_details.schema.format == "multiline" %} - - {% else %} - - {% if "examples" in form_details.schema and form_details.schema.examples %} - - {% for option in form_details.schema.examples -%} - - {% endfor -%} - - {% endif %} - {% endif %} - {% if form_details.description -%} - {{ form_details.description }} - {%- endif %} - - -{% endmacro %} - -{% block content %} - {{ super() }} -

- Trigger DAG: {{ dag.dag_display_name }} - {{ dag.description[0:150] + '…' if dag.description and dag.description|length > 150 else dag.description|default('', true) }} -

- {{ dag_docs(doc_md, False) }} -
- - - - {% if recent_confs|length > 0 %} -
-
- - -
-
- {% endif %} - {%- if form_fields %} -
-
-

DAG conf Parameters

-
- - -
- -
- - {% for form_key, form_details in form_fields.items() %} - {% if ("section" not in form_details.schema or not form_details.schema.section) and ("const" not in form_details.schema or not form_details.schema.const) %} - {{ form_element(form_key, form_details) }} - {% endif %} - {% endfor %} - -
- - {% for form_section, form_items in form_fields.values() | groupby(attribute="schema.section", default="") %} - {% if form_section %} -
-
- -
-
- - - {% for form_key, form_details in form_fields.items() %} - {% if form_details.schema.section == form_section and not form_details.schema.const %} - {{ form_element(form_key, form_details) }} - {% endif %} - {% endfor %} - -
-
-
-
-
- {% endif %} - {% endfor %} - -
-
- -
-
- - - - - - - - - - - - - - -
- - - {{ form.logical_date(class_="form-control", disabled=False) }} -
- - - -
- -
-
-
-
-
- - - {%- else %} -
- -
- {{ form.logical_date(class_="form-control", disabled=False) }} -
-
-
-
- - -
-
- - - {%- endif %} - {% if dag.get_is_paused() %} -
- -
- {% endif %} - - Cancel - -{% endblock %} - -{% block tail_js %} - - - - - - - {{ super() }} -{% endblock %} diff --git a/airflow/www/templates/airflow/variable_edit.html b/airflow/www/templates/airflow/variable_edit.html deleted file mode 100644 index 93a8d0f9ad25d..0000000000000 --- a/airflow/www/templates/airflow/variable_edit.html +++ /dev/null @@ -1,28 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% extends 'appbuilder/general/model/edit.html' %} - -{% block tail %} - {{ super() }} - - -{% endblock %} diff --git a/airflow/www/templates/airflow/variable_list.html b/airflow/www/templates/airflow/variable_list.html deleted file mode 100644 index 41ca6fa6cbcd5..0000000000000 --- a/airflow/www/templates/airflow/variable_list.html +++ /dev/null @@ -1,57 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% extends 'appbuilder/general/model/list.html' %} - -{% block content %} - {% if can_create_variable() %} -
-
- {% if csrf_token %} - - {% endif %} -
- -
-
- - -
-
- - -
-
- - -
- -
-
-
- {% endif %} - {{ super() }} - -{% endblock %} diff --git a/airflow/www/templates/airflow/variable_show.html b/airflow/www/templates/airflow/variable_show.html deleted file mode 100644 index 87fda206ceddb..0000000000000 --- a/airflow/www/templates/airflow/variable_show.html +++ /dev/null @@ -1,28 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% extends 'appbuilder/general/model/show.html' %} - -{% block tail %} - {{ super() }} - -{% endblock %} diff --git a/airflow/www/templates/airflow/variable_show_widget.html b/airflow/www/templates/airflow/variable_show_widget.html deleted file mode 100644 index e459bfa930311..0000000000000 --- a/airflow/www/templates/airflow/variable_show_widget.html +++ /dev/null @@ -1,70 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% import 'appbuilder/general/lib.html' as lib %} -{% include 'appbuilder/general/confirm.html' %} -{% include 'appbuilder/general/alert.html' %} - -{% block columns %} - -{% if fieldsets %} - - {% for fieldset_item in fieldsets %} - {% if fieldset_item[1].get('expanded') == None %} - {% set expanded = True %} - {% else %} - {% set expanded = fieldset_item[1].get('expanded') %} - {% endif %} - {% call lib.accordion_tag(loop.index,fieldset_item[0], expanded) %} -
- - {% for item in fieldset_item[1].get('fields') %} - - - - - {% endfor %} -
{{label_columns.get(item)}}{{value_columns[include_columns.index(item)]}}
- {% endcall %} - {% endfor %} - -{% else %} -
- - - {% for item in include_columns %} - - - - - {% endfor %} -
{{label_columns.get(item)}} - {{value_columns[loop.index-1]}} -
-{% endif %} - -{% endblock columns %} - - -{% block actions %} -
- {{ lib.render_action_links(actions, pk, modelview_name) }} - {{ lib.lnk_back() }} -
-{% endblock actions %} diff --git a/airflow/www/templates/airflow/xcom.html b/airflow/www/templates/airflow/xcom.html deleted file mode 100644 index 68bbd72248bfa..0000000000000 --- a/airflow/www/templates/airflow/xcom.html +++ /dev/null @@ -1,38 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% extends "airflow/task_instance.html" %} -{% block title %}DAGs - {{ appbuilder.app_name }}{% endblock %} - -{% block content %} - {{ super() }} -

{{ title }}

- - - - - - {% for attr, value in attributes %} - - - - - {% endfor %} -
KeyValue
{{ attr }}{{ value }}
-{% endblock %} diff --git a/airflow/www/templates/analytics/google_analytics.html b/airflow/www/templates/analytics/google_analytics.html deleted file mode 100644 index 379f32f930242..0000000000000 --- a/airflow/www/templates/analytics/google_analytics.html +++ /dev/null @@ -1,28 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - - - - diff --git a/airflow/www/templates/analytics/matomo.html b/airflow/www/templates/analytics/matomo.html deleted file mode 100644 index d45154e2162f7..0000000000000 --- a/airflow/www/templates/analytics/matomo.html +++ /dev/null @@ -1,34 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - - - - diff --git a/airflow/www/templates/analytics/metarouter.html b/airflow/www/templates/analytics/metarouter.html deleted file mode 100644 index 5c60093d0cde7..0000000000000 --- a/airflow/www/templates/analytics/metarouter.html +++ /dev/null @@ -1,25 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - - diff --git a/airflow/www/templates/analytics/segment.html b/airflow/www/templates/analytics/segment.html deleted file mode 100644 index 9799eb4b434ad..0000000000000 --- a/airflow/www/templates/analytics/segment.html +++ /dev/null @@ -1,25 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - - diff --git a/airflow/www/templates/appbuilder/custom_icons.html b/airflow/www/templates/appbuilder/custom_icons.html deleted file mode 100644 index 49bf637fe3c9f..0000000000000 --- a/airflow/www/templates/appbuilder/custom_icons.html +++ /dev/null @@ -1,34 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% macro icon_path(name) %} - {# - All icons added should adhere to Material Design standards, e.g. 24dp - https://material.io/design/iconography/system-icons.html#design-principles - #} - {% if name == 'kubernetes' %} - - {% endif %} -{% endmacro %} - -{% macro icon(name, size=20, attrs='') %} - - {{ icon_path(name) }} - -{% endmacro %} diff --git a/airflow/www/templates/appbuilder/dag_docs.html b/airflow/www/templates/appbuilder/dag_docs.html deleted file mode 100644 index 23fc6852ce685..0000000000000 --- a/airflow/www/templates/appbuilder/dag_docs.html +++ /dev/null @@ -1,41 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% macro dag_docs(doc_md, folded=True) %} - {% if doc_md is defined and doc_md is not none %} - - {% endif %} -{% endmacro %} diff --git a/airflow/www/templates/appbuilder/flash.html b/airflow/www/templates/appbuilder/flash.html deleted file mode 100644 index 37df620085178..0000000000000 --- a/airflow/www/templates/appbuilder/flash.html +++ /dev/null @@ -1,74 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{#- - Adapted from: https://github.com/dpgaspar/Flask-AppBuilder/blob/master/flask_appbuilder/templates/appbuilder/flash.html --#} - - -{# - Split messages into two arrays: one for regular alerts, another for DAG import errors -#} -{% with messages = get_flashed_messages(with_categories=true) %} - {% set dag_import_errors = [] %} - {% set regular_alerts = [] %} - - {% if messages %} - {% for category, m in messages %} - {% if category == 'dag_import_error' %} - {{ dag_import_errors.append((category, m)) if dag_import_errors.append((category, m)) != None else '' }} - {% elif not (request.path == auth_manager.get_url_login() and 'access is denied' in m.lower()) %} - {# Don't show 'Access is Denied' alert if user is logged out and on the login page. #} - {{ show_message(m, category) }} - {% endif %} - {% endfor %} - {% endif %} - - {% if dag_import_errors %} -
-
- -
-
- {% for category, m in dag_import_errors %} -
-
{{ m }}
-
- {% endfor %} -
-
-
-
- {% endif %} -{% endwith %} - -{% block tail %} - -{% endblock %} diff --git a/airflow/www/templates/appbuilder/index.html b/airflow/www/templates/appbuilder/index.html deleted file mode 100644 index 19c6f60db9090..0000000000000 --- a/airflow/www/templates/appbuilder/index.html +++ /dev/null @@ -1,20 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% extends "airflow/dag.html" %} diff --git a/airflow/www/templates/appbuilder/loading_dots.html b/airflow/www/templates/appbuilder/loading_dots.html deleted file mode 100644 index a1bcb359c1649..0000000000000 --- a/airflow/www/templates/appbuilder/loading_dots.html +++ /dev/null @@ -1,24 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% macro loading_dots(id=None, classes=None) %} - - - -{% endmacro %} diff --git a/airflow/www/templates/appbuilder/navbar.html b/airflow/www/templates/appbuilder/navbar.html deleted file mode 100644 index 6f22d65343fa1..0000000000000 --- a/airflow/www/templates/appbuilder/navbar.html +++ /dev/null @@ -1,69 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% set menu = appbuilder.menu %} -{% set languages = appbuilder.languages %} - - diff --git a/airflow/www/templates/appbuilder/navbar_menu.html b/airflow/www/templates/appbuilder/navbar_menu.html deleted file mode 100644 index db2de352d3141..0000000000000 --- a/airflow/www/templates/appbuilder/navbar_menu.html +++ /dev/null @@ -1,60 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% set external_menu_url_prefixes = [ - 'https://github.com', - 'https://airflow.apache.org', - 'http://apache-airflow-docs.s3-website.eu-central-1.amazonaws.com'] -%} - -{% macro menu_item(item) %} - {% set url = item.get_url() %} - {% set prefix = "/".join(url.split("/")[:3]) %} - {% if prefix in external_menu_url_prefixes %} - {{ item.label }} - {% else %} - {{ item.label }} - {% endif %} -{% endmacro %} - -{% for item1 in auth_manager.filter_permitted_menu_items(menu.get_list()) %} - {% if item1 %} - {% if item1.childs %} - - {% else %} -
  • {{ menu_item(item1) }}
  • - {% endif %} - {% endif %} -{% endfor %} diff --git a/airflow/www/templates/appbuilder/navbar_right.html b/airflow/www/templates/appbuilder/navbar_right.html deleted file mode 100644 index fb82a15191511..0000000000000 --- a/airflow/www/templates/appbuilder/navbar_right.html +++ /dev/null @@ -1,98 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - -{% macro locale_menu(languages) %} - {% set locale = session['locale'] %} - {% if not locale %} - {% set locale = 'en' %} - {% endif %} - -{% endmacro %} - -{# clock and timezone menu #} - - -{% if auth_manager.is_logged_in() %} - -{% else %} -
  • - login{{_("Log In")}} -
  • -{% endif %} diff --git a/airflow/www/templates/swagger-ui/index.j2 b/airflow/www/templates/swagger-ui/index.j2 deleted file mode 100644 index 62661a369aac6..0000000000000 --- a/airflow/www/templates/swagger-ui/index.j2 +++ /dev/null @@ -1,87 +0,0 @@ -{# - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. -#} - - - - - - - {{ title | default('Swagger UI') }} - - - - - - - -
    - - - - - diff --git a/airflow/www/tsconfig.json b/airflow/www/tsconfig.json deleted file mode 100644 index 137a4cef52b77..0000000000000 --- a/airflow/www/tsconfig.json +++ /dev/null @@ -1,46 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/* - * TypeScript config - */ -{ - "compilerOptions": { - "strict": true, - "allowJs": true, - "importsNotUsedAsValues": "error", - "target": "ES6", - "module": "ES6", - "moduleResolution": "node", - "isolatedModules": true, - "esModuleInterop": true, - "resolveJsonModule": true, - "skipLibCheck": true, - "jsx": "preserve", - "types": ["node", "jest"], - "baseUrl": ".", - "paths": { - // Be sure to update aliases in webpack.config.js and jest.config.js - "src/*": ["static/js/*"] - }, - "noEmit": true - }, - "include": ["static"], - "exclude": ["node_modules", "static/dist"] -} diff --git a/airflow/www/utils.py b/airflow/www/utils.py index e64d53f51fe40..ca4e98645145f 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -17,429 +17,9 @@ # under the License. from __future__ import annotations -import json -import logging -import textwrap -import time -from collections.abc import Sequence -from typing import TYPE_CHECKING, Any, Callable -from urllib.parse import urlencode - -from flask import request, url_for -from flask.helpers import flash -from flask_appbuilder.forms import FieldConverter -from flask_appbuilder.models.filters import BaseFilter -from flask_appbuilder.models.sqla import filters as fab_sqlafilters -from flask_appbuilder.models.sqla.filters import get_field_setup_query, set_value_to_type -from flask_appbuilder.models.sqla.interface import SQLAInterface -from flask_babel import lazy_gettext -from markdown_it import MarkdownIt from markupsafe import Markup -from pygments import highlight, lexers -from pygments.formatters import HtmlFormatter -from sqlalchemy import delete, func, select, tuple_, types -from sqlalchemy.ext.associationproxy import AssociationProxy - -from airflow.api_fastapi.app import get_auth_manager -from airflow.models.dagrun import DagRun -from airflow.models.dagwarning import DagWarning -from airflow.models.errors import ParseImportError -from airflow.models.taskinstance import TaskInstance -from airflow.utils import timezone -from airflow.utils.code_utils import get_python_source -from airflow.utils.helpers import alchemy_to_dict -from airflow.utils.json import WebEncoder -from airflow.utils.state import State, TaskInstanceState -from airflow.www.forms import DateTimeWithTimezoneField -from airflow.www.widgets import AirflowDateTimePickerWidget - -if TYPE_CHECKING: - from flask_appbuilder.models.sqla import Model - from pendulum.datetime import DateTime - from pygments.lexer import Lexer - from sqlalchemy.orm.session import Session - from sqlalchemy.sql import Select - from sqlalchemy.sql.operators import ColumnOperators - - -TI = TaskInstance - -logger = logging.getLogger(__name__) - - -def datetime_to_string(value: DateTime | None) -> str | None: - if value is None: - return None - return value.isoformat() - - -def get_mapped_instances(task_instance, session): - return session.scalars( - select(TaskInstance) - .where( - TaskInstance.dag_id == task_instance.dag_id, - TaskInstance.run_id == task_instance.run_id, - TaskInstance.task_id == task_instance.task_id, - ) - .order_by(TaskInstance.map_index) - ).all() - - -def get_instance_with_map(task_instance, session): - if task_instance.map_index == -1: - data = alchemy_to_dict(task_instance) - # Fetch logical_date explicitly since it's not a column and a proxy - data["logical_date"] = task_instance.logical_date - return data - mapped_instances = get_mapped_instances(task_instance, session) - return get_mapped_summary(task_instance, mapped_instances) - - -priority: list[None | TaskInstanceState] = [ - TaskInstanceState.FAILED, - TaskInstanceState.UPSTREAM_FAILED, - TaskInstanceState.UP_FOR_RETRY, - TaskInstanceState.UP_FOR_RESCHEDULE, - TaskInstanceState.QUEUED, - TaskInstanceState.SCHEDULED, - TaskInstanceState.DEFERRED, - TaskInstanceState.RUNNING, - TaskInstanceState.RESTARTING, - None, - TaskInstanceState.SUCCESS, - TaskInstanceState.SKIPPED, - TaskInstanceState.REMOVED, -] - - -def get_mapped_summary(parent_instance, task_instances): - mapped_states = [ti.state for ti in task_instances] - - group_state = None - for state in priority: - if state in mapped_states: - group_state = state - break - - group_queued_dttm = datetime_to_string( - min((ti.queued_dttm for ti in task_instances if ti.queued_dttm), default=None) - ) - - group_start_date = datetime_to_string( - min((ti.start_date for ti in task_instances if ti.start_date), default=None) - ) - group_end_date = datetime_to_string( - max((ti.end_date for ti in task_instances if ti.end_date), default=None) - ) - - return { - "task_id": parent_instance.task_id, - "run_id": parent_instance.run_id, - "state": group_state, - "queued_dttm": group_queued_dttm, - "start_date": group_start_date, - "end_date": group_end_date, - "mapped_states": mapped_states, - "try_number": parent_instance.try_number, - "logical_date": parent_instance.logical_date, - } - - -def get_dag_run_conf( - dag_run_conf: Any, *, json_encoder: type[json.JSONEncoder] = json.JSONEncoder -) -> tuple[str | None, bool]: - result: str | None = None - - conf_is_json: bool = False - if isinstance(dag_run_conf, str): - result = dag_run_conf - elif isinstance(dag_run_conf, (dict, list)) and any(dag_run_conf): - result = json.dumps(dag_run_conf, sort_keys=True, cls=json_encoder, ensure_ascii=False) - conf_is_json = True - - return result, conf_is_json - - -def encode_dag_run( - dag_run: DagRun | None, *, json_encoder: type[json.JSONEncoder] = json.JSONEncoder -) -> tuple[dict[str, Any] | None, None | str]: - if not dag_run: - return None, None - - try: - dag_run_conf, conf_is_json = get_dag_run_conf(dag_run.conf, json_encoder=json_encoder) - encoded_dag_run = { - "run_id": dag_run.run_id, - "queued_at": datetime_to_string(dag_run.queued_at), - "start_date": datetime_to_string(dag_run.start_date), - "end_date": datetime_to_string(dag_run.end_date), - "state": dag_run.state, - "logical_date": datetime_to_string(dag_run.logical_date), - "data_interval_start": datetime_to_string(dag_run.data_interval_start), - "data_interval_end": datetime_to_string(dag_run.data_interval_end), - "run_type": dag_run.run_type, - "last_scheduling_decision": datetime_to_string(dag_run.last_scheduling_decision), - "conf": dag_run_conf, - "conf_is_json": conf_is_json, - "note": dag_run.note, - "triggered_by": dag_run.triggered_by.value, - } - except ValueError as e: - logger.error("Error while encoding the DAG Run!", exc_info=e) - if str(e) == "Circular reference detected": - return None, ( - f"Circular reference detected in the DAG Run config (#{dag_run.run_id}). " - f"You should check your webserver logs for more details." - ) - else: - raise e - - return encoded_dag_run, None - - -def check_import_errors(fileloc, bundle_name, session): - # Check dag import errors - import_errors = session.scalars( - select(ParseImportError).where( - ParseImportError.filename == fileloc, ParseImportError.bundle_name == bundle_name - ) - ).all() - if import_errors: - for import_error in import_errors: - flash( - f"Broken DAG: [{import_error.filename}, Bundle name: {bundle_name}] {import_error.stacktrace}", - "dag_import_error", - ) - - -def check_dag_warnings(dag_id, session): - dag_warnings = session.scalars(select(DagWarning).where(DagWarning.dag_id == dag_id)).all() - if dag_warnings: - for dag_warning in dag_warnings: - flash(dag_warning.message, "warning") - - -def get_params(**kwargs): - """Return URL-encoded params.""" - return urlencode({d: v for d, v in kwargs.items() if v is not None}, True) - - -def generate_pages( - current_page, - num_of_pages, - search=None, - status=None, - tags=None, - window=7, - sorting_key=None, - sorting_direction=None, -): - """ - Generate the HTML for a paging component. - - Uses a similar logic to the paging auto-generated by Flask managed views. The paging - component defines a number of pages visible in the pager (window) and once the user - goes to a page beyond the largest visible, it would scroll to the right the page numbers - and keeps the current one in the middle of the pager component. When in the last pages, - the pages won't scroll and just keep moving until the last page. Pager also contains - pages. - This component takes into account custom parameters such as search, status, and tags - which could be added to the pages link in order to maintain the state between - client and server. It also allows to make a bookmark on a specific paging state. - - :param current_page: the current page number, 0-indexed - :param num_of_pages: the total number of pages - :param search: the search query string, if any - :param status: 'all', 'active', or 'paused' - :param tags: array of strings of the current filtered tags - :param window: the number of pages to be shown in the paging component (7 default) - :param sorting_key: the sorting key selected for dags, None indicates that sorting is not needed/provided - :param sorting_direction: direction of sorting, 'asc' or 'desc', - None indicates that sorting is not needed/provided - :return: the HTML string of the paging component - """ - void_link = "javascript:void(0)" - first_node = Markup( - """
  • - « -
  • """ - ) - - previous_node = Markup( - """""" - ) - - next_node = Markup( - """""" - ) - - last_node = Markup( - """
  • - » -
  • """ - ) - - page_node = Markup( - """
  • - {page_num} -
  • """ - ) - - output = [Markup('
      ')] - - is_disabled = "disabled" if current_page <= 0 else "" - - qs = get_params( - page=0, - search=search, - status=status, - tags=tags, - sorting_key=sorting_key, - sorting_direction=sorting_direction, - ) - first_node_link = void_link if is_disabled else f"?{qs}" - output.append( - first_node.format( - href_link=first_node_link, - disabled=is_disabled, - ) - ) - - page_link = void_link - if current_page > 0: - qs = get_params( - page=current_page - 1, - search=search, - status=status, - tags=tags, - sorting_key=sorting_key, - sorting_direction=sorting_direction, - ) - page_link = f"?{qs}" - - output.append(previous_node.format(href_link=page_link, disabled=is_disabled)) - - mid = window // 2 - last_page = num_of_pages - 1 - - if current_page <= mid or num_of_pages < window: - pages = list(range(0, min(num_of_pages, window))) - elif mid < current_page < last_page - mid: - pages = list(range(current_page - mid, current_page + mid + 1)) - else: - pages = list(range(num_of_pages - window, last_page + 1)) - def is_current(current, page): - return page == current - - for page in pages: - qs = get_params( - page=page, - search=search, - status=status, - tags=tags, - sorting_key=sorting_key, - sorting_direction=sorting_direction, - ) - vals = { - "is_active": "active" if is_current(current_page, page) else "", - "href_link": void_link if is_current(current_page, page) else f"?{qs}", - "page_num": page + 1, - } - output.append(page_node.format(**vals)) - - is_disabled = "disabled" if current_page >= num_of_pages - 1 else "" - - qs = get_params( - page=current_page + 1, - search=search, - status=status, - tags=tags, - sorting_key=sorting_key, - sorting_direction=sorting_direction, - ) - page_link = void_link if current_page >= num_of_pages - 1 else f"?{qs}" - - output.append(next_node.format(href_link=page_link, disabled=is_disabled)) - - qs = get_params( - page=last_page, - search=search, - status=status, - tags=tags, - sorting_key=sorting_key, - sorting_direction=sorting_direction, - ) - last_node_link = void_link if is_disabled else f"?{qs}" - output.append( - last_node.format( - href_link=last_node_link, - disabled=is_disabled, - ) - ) - - output.append(Markup("
    ")) - - return Markup("\n".join(output)) - - -def epoch(dttm): - """Return an epoch-type date (tuple with no timezone).""" - return (int(time.mktime(dttm.timetuple())) * 1000,) - - -def make_cache_key(*args, **kwargs): - """Get a unique key per URL; used by cache.""" - path = request.path - args = str(hash(frozenset(request.args.items()))) - return (path + args).encode("ascii", "ignore") - - -def task_instance_link(attr): - """Generate a URL to the Graph view for a TaskInstance.""" - dag_id = attr.get("dag_id") - task_id = attr.get("task_id") - run_id = attr.get("run_id") - map_index = attr.get("map_index", None) - logical_date = attr.get("logical_date") or attr.get("dag_run.logical_date") - - if map_index == -1: - map_index = None - - url = url_for( - "Airflow.grid", - dag_id=dag_id, - task_id=task_id, - dag_run_id=run_id, - map_index=map_index, - logical_date=logical_date, - tab="graph", - ) - url_root = url_for( - "Airflow.grid", - dag_id=dag_id, - task_id=task_id, - root=task_id, - dag_run_id=run_id, - map_index=map_index, - logical_date=logical_date, - tab="graph", - ) - return Markup( - """ - - {task_id} - - - - - """ - ).format(url=url, task_id=task_id, url_root=url_root) +from airflow.utils.state import State def state_token(state): @@ -452,498 +32,3 @@ def state_token(state): title="Current State: {state}">{state} """ ).format(color=color, state=state, fg_color=fg_color) - - -def state_f(attr): - """Get 'state' & return a formatted string with HTML for a given State.""" - state = attr.get("state") - return state_token(state) - - -def nobr_f(attr_name): - """Return a formatted string with HTML with a Non-breaking Text element.""" - - def nobr(attr): - f = attr.get(attr_name) - return Markup("{}").format(f) - - return nobr - - -def datetime_f(attr_name): - """Return a formatted string with HTML for given DataTime.""" - - def dt(attr): - f = attr.get(attr_name) - return datetime_html(f) - - return dt - - -def datetime_html(dttm: DateTime | None) -> str: - """Return an HTML formatted string with time element to support timezone changes in UI.""" - as_iso = dttm.isoformat() if dttm else "" - if not as_iso: - return Markup("") - as_iso_short = as_iso - if timezone.utcnow().isoformat()[:4] == as_iso[:4]: - as_iso_short = as_iso[5:] - # The empty title will be replaced in JS code when non-UTC dates are displayed - return Markup('').format(as_iso, as_iso_short) - - -def json_f(attr_name): - """Return a formatted string with HTML for given JSON serializable.""" - - def json_(attr): - f = attr.get(attr_name) - serialized = json.dumps(f, cls=WebEncoder) - return Markup("{}").format(serialized) - - return json_ - - -def dag_link(attr): - """Generate a URL to the Graph view for a Dag.""" - dag_id = attr.get("dag_id") - logical_date = attr.get("logical_date") or attr.get("dag_run.logical_date") - if not dag_id: - return Markup("None") - url = url_for("Airflow.grid", dag_id=dag_id, logical_date=logical_date) - return Markup('{}').format(url, dag_id) - - -def dag_run_link(attr): - """Generate a URL to the Graph view for a DagRun.""" - dag_id = attr.get("dag_id") - run_id = attr.get("run_id") - logical_date = attr.get("logical_date") or attr.get("dag_run.logical_date") - - if not dag_id: - return Markup("None") - - url = url_for( - "Airflow.grid", - dag_id=dag_id, - logical_date=logical_date, - dag_run_id=run_id, - tab="graph", - ) - return Markup('{run_id}').format(url=url, run_id=run_id) - - -def _get_run_ordering_expr(name: str) -> ColumnOperators: - expr = DagRun.__mapper__.columns[name] - # Data interval columns are NULL for runs created before 2.3, but SQL's - # NULL-sorting logic would make those old runs always appear first. In a - # perfect world we'd want to sort by ``get_run_data_interval()``, but that's - # not efficient, so instead the columns are coalesced into logical_date, - # which is good enough in most cases. - if name in ("data_interval_start", "data_interval_end"): - expr = func.coalesce(expr, DagRun.logical_date) - return expr.desc() - - -def sorted_dag_runs( - query: Select, *, ordering: Sequence[str], limit: int, session: Session -) -> Sequence[DagRun]: - """ - Produce DAG runs sorted by specified columns. - - :param query: An ORM select object against *DagRun*. - :param ordering: Column names to sort the runs. should generally come from a - timetable's ``run_ordering``. - :param limit: Number of runs to limit to. - :param session: SQLAlchemy ORM session object - :return: A list of DagRun objects ordered by the specified columns. The list - contains only the *last* objects, but in *ascending* order. - """ - ordering_exprs = (_get_run_ordering_expr(name) for name in ordering) - runs = session.scalars(query.order_by(*ordering_exprs, DagRun.id.desc()).limit(limit)).all() - runs.reverse() - return runs - - -def format_map_index(attr: dict) -> str: - """Format map index for list columns in model view.""" - value = attr["map_index"] - if value < 0: - return Markup(" ") - return str(value) - - -def pygment_html_render(s, lexer=lexers.TextLexer): - """Highlight text using a given Lexer.""" - return highlight(s, lexer(), HtmlFormatter(linenos=True)) - - -def render(obj: Any, lexer: Lexer, handler: Callable[[Any], str] | None = None): - """Render a given Python object with a given Pygments lexer.""" - if isinstance(obj, str): - return Markup(pygment_html_render(obj, lexer)) - - elif isinstance(obj, (tuple, list)): - out = "" - for i, text_to_render in enumerate(obj): - if lexer is lexers.PythonLexer: - text_to_render = repr(text_to_render) - out += Markup("
    List item #{}
    ").format(i) - out += Markup("
    " + pygment_html_render(text_to_render, lexer) + "
    ") - return out - - elif isinstance(obj, dict): - out = "" - for k, v in obj.items(): - if lexer is lexers.PythonLexer: - v = repr(v) - out += Markup('
    Dict item "{}"
    ').format(k) - out += Markup("
    " + pygment_html_render(v, lexer) + "
    ") - return out - - elif handler is not None and obj is not None: - return Markup(pygment_html_render(handler(obj), lexer)) - - else: - # Return empty string otherwise - return "" - - -def json_render(obj, lexer): - """Render a given Python object with json lexer.""" - out = "" - if isinstance(obj, str): - out = Markup(pygment_html_render(obj, lexer)) - elif isinstance(obj, (dict, list)): - content = json.dumps(obj, sort_keys=True, indent=4) - out = Markup(pygment_html_render(content, lexer)) - return out - - -def wrapped_markdown(s, css_class="rich_doc"): - """Convert a Markdown string to HTML.""" - md = MarkdownIt("gfm-like", {"html": False}) - if s is None: - return None - s = textwrap.dedent(s) - return Markup(f'
    {md.render(s)}
    ') - - -def get_attr_renderer(): - """Return Dictionary containing different Pygments Lexers for Rendering & Highlighting.""" - return { - "bash": lambda x: render(x, lexers.BashLexer), - "bash_command": lambda x: render(x, lexers.BashLexer), - "doc": lambda x: render(x, lexers.TextLexer), - "doc_json": lambda x: render(x, lexers.JsonLexer), - "doc_md": wrapped_markdown, - "doc_rst": lambda x: render(x, lexers.RstLexer), - "doc_yaml": lambda x: render(x, lexers.YamlLexer), - "hql": lambda x: render(x, lexers.SqlLexer), - "html": lambda x: render(x, lexers.HtmlLexer), - "jinja": lambda x: render(x, lexers.DjangoLexer), - "json": lambda x: json_render(x, lexers.JsonLexer), - "md": wrapped_markdown, - "mysql": lambda x: render(x, lexers.MySqlLexer), - "postgresql": lambda x: render(x, lexers.PostgresLexer), - "powershell": lambda x: render(x, lexers.PowerShellLexer), - "py": lambda x: render(x, lexers.PythonLexer, get_python_source), - "python_callable": lambda x: render(x, lexers.PythonLexer, get_python_source), - "rst": lambda x: render(x, lexers.RstLexer), - "sql": lambda x: render(x, lexers.SqlLexer), - "tsql": lambda x: render(x, lexers.TransactSqlLexer), - "yaml": lambda x: render(x, lexers.YamlLexer), - } - - -class UtcAwareFilterMixin: - """Mixin for filter for UTC time.""" - - def apply(self, query, value): - """Apply the filter.""" - if isinstance(value, str) and not value.strip(): - value = None - else: - value = timezone.parse(value, timezone=timezone.utc) - - return super().apply(query, value) - - -class FilterIsNull(BaseFilter): - """Is null filter.""" - - name = lazy_gettext("Is Null") - arg_name = "emp" - - def apply(self, query, value): - query, field = get_field_setup_query(query, self.model, self.column_name) - value = set_value_to_type(self.datamodel, self.column_name, None) - return query.filter(field == value) - - -class FilterIsNotNull(BaseFilter): - """Is not null filter.""" - - name = lazy_gettext("Is not Null") - arg_name = "nemp" - - def apply(self, query, value): - query, field = get_field_setup_query(query, self.model, self.column_name) - value = set_value_to_type(self.datamodel, self.column_name, None) - return query.filter(field != value) - - -class FilterGreaterOrEqual(BaseFilter): - """Greater than or Equal filter.""" - - name = lazy_gettext("Greater than or Equal") - arg_name = "gte" - - def apply(self, query, value): - query, field = get_field_setup_query(query, self.model, self.column_name) - value = set_value_to_type(self.datamodel, self.column_name, value) - - if value is None: - return query - - return query.filter(field >= value) - - -class FilterSmallerOrEqual(BaseFilter): - """Smaller than or Equal filter.""" - - name = lazy_gettext("Smaller than or Equal") - arg_name = "lte" - - def apply(self, query, value): - query, field = get_field_setup_query(query, self.model, self.column_name) - value = set_value_to_type(self.datamodel, self.column_name, value) - - if value is None: - return query - - return query.filter(field <= value) - - -class UtcAwareFilterSmallerOrEqual(UtcAwareFilterMixin, FilterSmallerOrEqual): - """Smaller than or Equal filter for UTC time.""" - - -class UtcAwareFilterGreaterOrEqual(UtcAwareFilterMixin, FilterGreaterOrEqual): - """Greater than or Equal filter for UTC time.""" - - -class UtcAwareFilterEqual(UtcAwareFilterMixin, fab_sqlafilters.FilterEqual): - """Equality filter for UTC time.""" - - -class UtcAwareFilterGreater(UtcAwareFilterMixin, fab_sqlafilters.FilterGreater): - """Greater Than filter for UTC time.""" - - -class UtcAwareFilterSmaller(UtcAwareFilterMixin, fab_sqlafilters.FilterSmaller): - """Smaller Than filter for UTC time.""" - - -class UtcAwareFilterNotEqual(UtcAwareFilterMixin, fab_sqlafilters.FilterNotEqual): - """Not Equal To filter for UTC time.""" - - -class UtcAwareFilterConverter(fab_sqlafilters.SQLAFilterConverter): - """Retrieve conversion tables for UTC-Aware filters.""" - - -class AirflowFilterConverter(fab_sqlafilters.SQLAFilterConverter): - """Retrieve conversion tables for Airflow-specific filters.""" - - conversion_table = ( - ( - "is_utcdatetime", - [ - UtcAwareFilterEqual, - UtcAwareFilterGreater, - UtcAwareFilterSmaller, - UtcAwareFilterNotEqual, - UtcAwareFilterSmallerOrEqual, - UtcAwareFilterGreaterOrEqual, - ], - ), - # FAB will try to create filters for extendedjson fields even though we - # exclude them from all UI, so we add this here to make it ignore them. - ("is_extendedjson", []), - ("is_json", []), - *fab_sqlafilters.SQLAFilterConverter.conversion_table, - ) - - def __init__(self, datamodel): - super().__init__(datamodel) - - for _, filters in self.conversion_table: - if FilterIsNull not in filters: - filters.append(FilterIsNull) - if FilterIsNotNull not in filters: - filters.append(FilterIsNotNull) - - -class CustomSQLAInterface(SQLAInterface): - """ - FAB does not know how to handle columns with leading underscores because they are not supported by WTForm. - - This hack will remove the leading '_' from the key to lookup the column names. - """ - - def __init__(self, obj, session: Session | None = None): - super().__init__(obj, session=session) - - def clean_column_names(): - if self.list_properties: - self.list_properties = {k.lstrip("_"): v for k, v in self.list_properties.items()} - if self.list_columns: - self.list_columns = {k.lstrip("_"): v for k, v in self.list_columns.items()} - - clean_column_names() - # Support for AssociationProxy in search and list columns - for obj_attr, desc in self.obj.__mapper__.all_orm_descriptors.items(): - if isinstance(desc, AssociationProxy): - proxy_instance = getattr(self.obj, obj_attr) - if hasattr(proxy_instance.remote_attr.prop, "columns"): - self.list_columns[obj_attr] = proxy_instance.remote_attr.prop.columns[0] - self.list_properties[obj_attr] = proxy_instance.remote_attr.prop - - def is_utcdatetime(self, col_name): - """Check if the datetime is a UTC one.""" - from airflow.utils.sqlalchemy import UtcDateTime - - if col_name in self.list_columns: - obj = self.list_columns[col_name].type - return ( - isinstance(obj, UtcDateTime) - or isinstance(obj, types.TypeDecorator) - and isinstance(obj.impl, UtcDateTime) - ) - return False - - def is_extendedjson(self, col_name): - """Check if it is a special extended JSON type.""" - from airflow.utils.sqlalchemy import ExtendedJSON - - if col_name in self.list_columns: - obj = self.list_columns[col_name].type - return ( - isinstance(obj, ExtendedJSON) - or isinstance(obj, types.TypeDecorator) - and isinstance(obj.impl, ExtendedJSON) - ) - return False - - def is_json(self, col_name): - """Check if it is a JSON type.""" - from sqlalchemy import JSON - - if col_name in self.list_columns: - obj = self.list_columns[col_name].type - return ( - isinstance(obj, JSON) or isinstance(obj, types.TypeDecorator) and isinstance(obj.impl, JSON) - ) - return False - - def get_col_default(self, col_name: str) -> Any: - if col_name not in self.list_columns: - # Handle AssociationProxy etc, or anything that isn't a "real" column - return None - return super().get_col_default(col_name) - - filter_converter_class = AirflowFilterConverter - - -class DagRunCustomSQLAInterface(CustomSQLAInterface): - """ - Custom interface to allow faster deletion. - - The ``delete`` and ``delete_all`` methods are overridden to speed up - deletion when a DAG run has a lot of related task instances. Relying on - SQLAlchemy's cascading deletion is comparatively slow in this situation. - """ - - def delete(self, item: Model, raise_exception: bool = False) -> bool: - self.session.execute(delete(TI).where(TI.dag_id == item.dag_id, TI.run_id == item.run_id)) - return super().delete(item, raise_exception=raise_exception) - - def delete_all(self, items: list[Model]) -> bool: - self.session.execute( - delete(TI).where(tuple_(TI.dag_id, TI.run_id).in_((x.dag_id, x.run_id) for x in items)) - ) - return super().delete_all(items) - - -# This class is used directly (i.e. we can't tell Fab to use a different -# subclass) so we have no other option than to edit the conversion table in -# place -FieldConverter.conversion_table = ( - ("is_utcdatetime", DateTimeWithTimezoneField, AirflowDateTimePickerWidget), - *FieldConverter.conversion_table, -) - - -class UIAlert: - """ - Helper for alerts messages shown on the UI. - - :param message: The message to display, either a string or Markup - :param category: The category of the message, one of "info", "warning", "error", or any custom category. - Defaults to "info". - :param roles: List of roles that should be shown the message. If ``None``, show to all users. - :param html: Whether the message has safe html markup in it. Defaults to False. - - - For example, show a message to all users: - - .. code-block:: python - - UIAlert("Welcome to Airflow") - - Or only for users with the User role: - - .. code-block:: python - - UIAlert("Airflow update happening next week", roles=["User"]) - - You can also pass html in the message: - - .. code-block:: python - - UIAlert('Visit airflow.apache.org', html=True) - - # or safely escape part of the message - # (more details: https://markupsafe.palletsprojects.com/en/2.0.x/formatting/) - UIAlert(Markup("Welcome %s") % ("John & Jane Doe",)) - """ - - def __init__( - self, - message: str | Markup, - category: str = "info", - roles: list[str] | None = None, - html: bool = False, - ): - self.category = category - self.roles = roles - self.html = html - self.message = Markup(message) if html else message - - def should_show(self) -> bool: - """ - Determine if the user should see the message. - - The decision is based on the user's role. - """ - if self.roles: - current_user = get_auth_manager().get_user() - if current_user is not None: - user_roles = {r.name for r in getattr(current_user, "roles", [])} - else: - # Unable to obtain user role - default to not showing - return False - - if user_roles.isdisjoint(self.roles): - return False - return True diff --git a/airflow/www/views.py b/airflow/www/views.py deleted file mode 100644 index 927a54fe0971e..0000000000000 --- a/airflow/www/views.py +++ /dev/null @@ -1,5702 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import collections.abc -import contextlib -import copy -import datetime -import itertools -import json -import logging -import math -import operator -import os -import sys -import traceback -import warnings -from bisect import insort_left -from collections import defaultdict -from collections.abc import Collection, Iterator, Mapping, MutableMapping, Sequence -from functools import cache, cached_property -from json import JSONDecodeError -from pathlib import Path -from typing import TYPE_CHECKING, Any -from urllib.parse import unquote, urlencode, urljoin, urlparse, urlsplit - -import configupdater -import flask.json -import lazy_object_proxy -import re2 -import sqlalchemy as sqla -from croniter import croniter -from flask import ( - Response, - abort, - before_render_template, - current_app, - flash, - g, - has_request_context, - make_response, - redirect, - render_template, - request, - send_from_directory, - session as flask_session, - url_for, -) -from flask_appbuilder import BaseView, ModelView, expose -from flask_appbuilder._compat import as_unicode -from flask_appbuilder.actions import action -from flask_appbuilder.const import FLAMSG_ERR_SEC_ACCESS_DENIED -from flask_appbuilder.models.sqla.filters import BaseFilter -from flask_appbuilder.urltools import get_order_args, get_page_args, get_page_size_args -from flask_appbuilder.widgets import FormWidget -from flask_babel import lazy_gettext -from itsdangerous import URLSafeSerializer -from jinja2.utils import htmlsafe_json_dumps, pformat # type: ignore -from markupsafe import Markup, escape -from pendulum.datetime import DateTime -from pendulum.parsing.exceptions import ParserError -from sqlalchemy import and_, case, desc, func, inspect, or_, select, tuple_, union_all -from sqlalchemy.exc import IntegrityError -from sqlalchemy.orm import joinedload -from wtforms import BooleanField, validators - -import airflow -from airflow import models, plugins_manager, settings -from airflow.api.common.airflow_health import get_airflow_health -from airflow.api.common.mark_tasks import ( - set_dag_run_state_to_failed, - set_dag_run_state_to_queued, - set_dag_run_state_to_success, - set_state, -) -from airflow.api_fastapi.app import get_auth_manager -from airflow.auth.managers.models.resource_details import AccessView, DagAccessEntity, DagDetails -from airflow.configuration import AIRFLOW_CONFIG, conf -from airflow.exceptions import ( - AirflowConfigException, - AirflowException, - AirflowNotFoundException, - ParamValidationError, - RemovedInAirflow3Warning, -) -from airflow.executors.executor_loader import ExecutorLoader -from airflow.hooks.base import BaseHook -from airflow.jobs.job import Job -from airflow.jobs.scheduler_job_runner import SchedulerJobRunner -from airflow.jobs.triggerer_job_runner import TriggererJobRunner -from airflow.models import Connection, DagModel, DagTag, Log, Trigger, XCom -from airflow.models.asset import AssetDagRunQueue, AssetEvent, AssetModel, DagScheduleAssetReference -from airflow.models.dag import get_asset_triggered_next_run_info -from airflow.models.dag_version import DagVersion -from airflow.models.dagrun import RUN_ID_REGEX, DagRun, DagRunType -from airflow.models.errors import ParseImportError -from airflow.models.serialized_dag import SerializedDagModel -from airflow.models.taskinstance import TaskInstance, TaskInstanceNote -from airflow.plugins_manager import PLUGINS_ATTRIBUTES_TO_DUMP -from airflow.providers_manager import ProvidersManager -from airflow.sdk.definitions.asset import Asset, AssetAlias -from airflow.sdk.execution_time import secrets_masker -from airflow.security import permissions -from airflow.ti_deps.dep_context import DepContext -from airflow.ti_deps.dependencies_deps import SCHEDULER_QUEUED_DEPS -from airflow.timetables._cron import CronMixin -from airflow.timetables.base import DataInterval, TimeRestriction -from airflow.timetables.simple import ContinuousTimetable -from airflow.utils import json as utils_json, timezone, yaml -from airflow.utils.airflow_flask_app import get_airflow_app -from airflow.utils.api_migration import mark_fastapi_migration_done -from airflow.utils.dag_edges import dag_edges -from airflow.utils.db import get_query_count -from airflow.utils.docs import get_doc_url_for_provider, get_docs_url -from airflow.utils.helpers import exactly_one -from airflow.utils.log.log_reader import TaskLogReader -from airflow.utils.net import get_hostname -from airflow.utils.session import NEW_SESSION, create_session, provide_session -from airflow.utils.state import DagRunState, State, TaskInstanceState -from airflow.utils.strings import to_boolean -from airflow.utils.task_group import TaskGroup, task_group_to_dict_legacy -from airflow.utils.timezone import td_format, utcnow -from airflow.utils.types import NOTSET, DagRunTriggeredByType -from airflow.version import version -from airflow.www import auth, utils as wwwutils -from airflow.www.decorators import action_logging, gzipped -from airflow.www.forms import ( - DagRunEditForm, - DateTimeForm, - TaskInstanceEditForm, - create_connection_form_class, -) -from airflow.www.widgets import AirflowModelListWidget, AirflowVariableShowWidget - -if TYPE_CHECKING: - from sqlalchemy.orm import Session - - from airflow.auth.managers.models.batch_apis import IsAuthorizedDagRequest - from airflow.models.dag import DAG - from airflow.models.operator import Operator - -PAGE_SIZE = conf.getint("webserver", "page_size") -FILTER_TAGS_COOKIE = "tags_filter" -FILTER_STATUS_COOKIE = "dag_status_filter" -FILTER_LASTRUN_COOKIE = "last_run_filter" -LINECHART_X_AXIS_TICKFORMAT = ( - "function (d, i) { let xLabel;" - "if (i === undefined) {xLabel = d3.time.format('%H:%M, %d %b %Y')(new Date(parseInt(d)));" - "} else {xLabel = d3.time.format('%H:%M, %d %b')(new Date(parseInt(d)));} return xLabel;}" -) - -SENSITIVE_FIELD_PLACEHOLDER = "RATHER_LONG_SENSITIVE_FIELD_PLACEHOLDER" - -logger = logging.getLogger(__name__) - - -def sanitize_args(args: dict[str, Any]) -> dict[str, Any]: - """ - Remove all parameters starting with `_`. - - :param args: arguments of request - :return: copy of the dictionary passed as input with args starting with `_` removed. - """ - return {key: value for key, value in args.items() if not key.startswith("_")} - - -# Following the release of https://github.com/python/cpython/issues/102153 in Python 3.9.17 on -# June 6, 2023, we are adding extra sanitization of the urls passed to get_safe_url method to make it works -# the same way regardless if the user uses latest Python patchlevel versions or not. This also follows -# a recommended solution by the Python core team. -# -# From: https://github.com/python/cpython/commit/d28bafa2d3e424b6fdcfd7ae7cde8e71d7177369 -# -# We recommend that users of these APIs where the values may be used anywhere -# with security implications code defensively. Do some verification within your -# code before trusting a returned component part. Does that ``scheme`` make -# sense? Is that a sensible ``path``? Is there anything strange about that -# ``hostname``? etc. -# -# C0 control and space to be stripped per WHATWG spec. -# == "".join([chr(i) for i in range(0, 0x20 + 1)]) -_WHATWG_C0_CONTROL_OR_SPACE = ( - "\x00\x01\x02\x03\x04\x05\x06\x07\x08\t\n\x0b\x0c" - "\r\x0e\x0f\x10\x11\x12\x13\x14\x15\x16\x17\x18\x19\x1a\x1b\x1c\x1d\x1e\x1f " -) - - -def get_safe_url(url): - """Given a user-supplied URL, ensure it points to our web server.""" - if not url: - return url_for("Airflow.index") - - # If the url contains semicolon, redirect it to homepage to avoid - # potential XSS. (Similar to https://github.com/python/cpython/pull/24297/files (bpo-42967)) - if ";" in unquote(url): - return url_for("Airflow.index") - - url = url.lstrip(_WHATWG_C0_CONTROL_OR_SPACE) - - host_url = urlsplit(request.host_url) - redirect_url = urlsplit(urljoin(request.host_url, url)) - if not (redirect_url.scheme in ("http", "https") and host_url.netloc == redirect_url.netloc): - return url_for("Airflow.index") - - # This will ensure we only redirect to the right scheme/netloc - return redirect_url.geturl() - - -def get_date_time_num_runs_dag_runs_form_data(www_request, session, dag): - """Get Execution Data, Base Date & Number of runs from a Request.""" - date_time = www_request.args.get("logical_date") - run_id = www_request.args.get("run_id") - # First check run id, then check logical_date, if not fall back on the latest dagrun - if run_id: - dagrun = dag.get_dagrun(run_id=run_id, session=session) - date_time = dagrun.logical_date - elif date_time: - date_time = _safe_parse_datetime(date_time) - else: - date_time = dag.get_latest_logical_date(session=session) or timezone.utcnow() - - base_date = www_request.args.get("base_date") - if base_date: - base_date = _safe_parse_datetime(base_date) - else: - # The DateTimeField widget truncates milliseconds and would loose - # the first dag run. Round to next second. - base_date = (date_time + datetime.timedelta(seconds=1)).replace(microsecond=0) - - default_dag_run = conf.getint("webserver", "default_dag_run_display_number") - num_runs = www_request.args.get("num_runs", default=default_dag_run, type=int) - - # When base_date has been rounded up because of the DateTimeField widget, we want - # to use the logical_date as the starting point for our query just to ensure a - # link targeting a specific dag run actually loads that dag run. If there are - # more than num_runs dag runs in the "rounded period" then those dagruns would get - # loaded and the actual requested run would be excluded by the limit(). Once - # the user has changed base date to be anything else we want to use that instead. - query_date = base_date - if date_time < base_date <= date_time + datetime.timedelta(seconds=1): - query_date = date_time - - drs = session.scalars( - select(DagRun) - .where(DagRun.dag_id == dag.dag_id, DagRun.logical_date <= query_date) - .order_by(desc(DagRun.logical_date)) - .limit(num_runs) - ).all() - dr_choices = [] - dr_state = None - for dr in drs: - dr_choices.append((dr.logical_date.isoformat(), dr.run_id)) - if date_time == dr.logical_date: - dr_state = dr.state - - # Happens if base_date was changed and the selected dag run is not in result - if not dr_state and drs: - dr = drs[0] - date_time = dr.logical_date - dr_state = dr.state - - return { - "dttm": date_time, - "base_date": base_date, - "num_runs": num_runs, - "logical_date": date_time.isoformat(), - "dr_choices": dr_choices, - "dr_state": dr_state, - } - - -def _safe_parse_datetime(v, *, allow_empty=False, strict=True) -> datetime.datetime | None: - """ - Parse datetime and return error message for invalid dates. - - :param v: the string value to be parsed - :param allow_empty: Set True to return none if empty str or None - :param strict: if False, it will fall back on the dateutil parser if unable to parse with pendulum - """ - if allow_empty is True and not v: - return None - try: - return timezone.parse(v, strict=strict) - except (TypeError, ParserError): - abort(400, f"Invalid datetime: {v!r}") - - -def node_dict(node_id, label, node_class): - return { - "id": node_id, - "value": {"label": label, "rx": 5, "ry": 5, "class": node_class}, - } - - -def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session) -> dict[str, Any]: - """ - Create a nested dict representation of the DAG's TaskGroup and its children. - - Used to construct the Graph and Grid views. - """ - query = session.execute( - select( - TaskInstance.task_id, - TaskInstance.run_id, - TaskInstance.state, - case( - (TaskInstance.map_index == -1, TaskInstance.try_number), - else_=None, - ).label("try_number"), - func.min(TaskInstanceNote.content).label("note"), - func.count(func.coalesce(TaskInstance.state, sqla.literal("no_status"))).label("state_count"), - func.min(TaskInstance.queued_dttm).label("queued_dttm"), - func.min(TaskInstance.start_date).label("start_date"), - func.max(TaskInstance.end_date).label("end_date"), - ) - .join(TaskInstance.task_instance_note, isouter=True) - .where( - TaskInstance.dag_id == dag.dag_id, - TaskInstance.run_id.in_([dag_run.run_id for dag_run in dag_runs]), - ) - .group_by( - TaskInstance.task_id, - TaskInstance.run_id, - TaskInstance.state, - case( - (TaskInstance.map_index == -1, TaskInstance.try_number), - else_=None, - ), - ) - .order_by(TaskInstance.task_id, TaskInstance.run_id) - ) - - grouped_tis: dict[str, list[TaskInstance]] = collections.defaultdict( - list, - ((task_id, list(tis)) for task_id, tis in itertools.groupby(query, key=lambda ti: ti.task_id)), - ) - - @cache - def get_task_group_children_getter() -> operator.methodcaller: - sort_order = conf.get("webserver", "grid_view_sorting_order", fallback="topological") - if sort_order == "topological": - return operator.methodcaller("topological_sort") - if sort_order == "hierarchical_alphabetical": - return operator.methodcaller("hierarchical_alphabetical_sort") - raise AirflowConfigException(f"Unsupported grid_view_sorting_order: {sort_order}") - - def task_group_to_grid(item: Operator | TaskGroup) -> dict[str, Any]: - if not isinstance(item, TaskGroup): - - def _mapped_summary(ti_summaries: list[TaskInstance]) -> Iterator[dict[str, Any]]: - run_id = "" - record: dict[str, Any] = {} - - def set_overall_state(record): - for state in wwwutils.priority: - if state in record["mapped_states"]: - record["state"] = state - break - # When turning the dict into JSON we can't have None as a key, - # so use the string that the UI does. - with contextlib.suppress(KeyError): - record["mapped_states"]["no_status"] = record["mapped_states"].pop(None) - - for ti_summary in ti_summaries: - if run_id != ti_summary.run_id: - run_id = ti_summary.run_id - if record: - set_overall_state(record) - yield record - record = { - "task_id": ti_summary.task_id, - "run_id": run_id, - "queued_dttm": ti_summary.queued_dttm, - "start_date": ti_summary.start_date, - "end_date": ti_summary.end_date, - "mapped_states": {ti_summary.state: ti_summary.state_count}, - "state": None, # We change this before yielding - } - continue - record["queued_dttm"] = min( - filter(None, [record["queued_dttm"], ti_summary.queued_dttm]), default=None - ) - record["start_date"] = min( - filter(None, [record["start_date"], ti_summary.start_date]), default=None - ) - # Sometimes the start date of a group might be before the queued date of the group - if ( - record["queued_dttm"] - and record["start_date"] - and record["queued_dttm"] > record["start_date"] - ): - record["queued_dttm"] = None - record["end_date"] = max( - filter(None, [record["end_date"], ti_summary.end_date]), default=None - ) - record["mapped_states"][ti_summary.state] = ti_summary.state_count - if record: - set_overall_state(record) - yield record - - if item_is_mapped := item.get_needs_expansion(): - instances = list(_mapped_summary(grouped_tis[item.task_id])) - else: - instances = [ - { - "task_id": task_instance.task_id, - "run_id": task_instance.run_id, - "state": task_instance.state, - "queued_dttm": task_instance.queued_dttm, - "start_date": task_instance.start_date, - "end_date": task_instance.end_date, - "try_number": task_instance.try_number, - "note": task_instance.note, - } - for task_instance in grouped_tis[item.task_id] - ] - - setup_teardown_type = {} - if item.is_setup is True: - setup_teardown_type["setupTeardownType"] = "setup" - elif item.is_teardown is True: - setup_teardown_type["setupTeardownType"] = "teardown" - - return { - "id": item.task_id, - "instances": instances, - "label": item.label, - # TODO: Task-SDK: MappedOperator doesn't support extra links right now - "extra_links": getattr(item, "extra_links", []), - "is_mapped": item_is_mapped, - "has_outlet_assets": any(isinstance(i, (Asset, AssetAlias)) for i in (item.outlets or [])), - "operator": item.operator_name, - "trigger_rule": item.trigger_rule, - **setup_teardown_type, - } - - # Task Group - task_group = item - children = [task_group_to_grid(child) for child in get_task_group_children_getter()(item)] - - def get_summary(dag_run: DagRun): - child_instances = [ - item - for sublist in (child["instances"] for child in children if "instances" in child) - for item in sublist - if item["run_id"] == dag_run.run_id - if item - ] - - children_queued_dttms = (item["queued_dttm"] for item in child_instances) - children_start_dates = (item["start_date"] for item in child_instances) - children_end_dates = (item["end_date"] for item in child_instances) - children_states = {item["state"] for item in child_instances} - - group_state = next((state for state in wwwutils.priority if state in children_states), None) - group_queued_dttm = min(filter(None, children_queued_dttms), default=None) - group_start_date = min(filter(None, children_start_dates), default=None) - group_end_date = max(filter(None, children_end_dates), default=None) - # Sometimes the start date of a group might be before the queued date of the group - if group_queued_dttm and group_start_date and group_queued_dttm > group_start_date: - group_queued_dttm = None - - return { - "task_id": task_group.group_id, - "run_id": dag_run.run_id, - "state": group_state, - "queued_dttm": group_queued_dttm, - "start_date": group_start_date, - "end_date": group_end_date, - } - - def get_mapped_group_summaries(): - mapped_ti_query = session.execute( - select(TaskInstance.task_id, TaskInstance.state, TaskInstance.run_id, TaskInstance.map_index) - .where( - TaskInstance.dag_id == dag.dag_id, - TaskInstance.task_id.in_(child["id"] for child in children), - TaskInstance.run_id.in_(r.run_id for r in dag_runs), - ) - .order_by(TaskInstance.task_id, TaskInstance.run_id) - ) - # Group tis by run_id, and then map_index. - mapped_tis: Mapping[str, Mapping[int, list[TaskInstance]]] = defaultdict( - lambda: defaultdict(list) - ) - for ti in mapped_ti_query: - mapped_tis[ti.run_id][ti.map_index].append(ti) - - def get_mapped_group_summary(run_id: str, mapped_instances: Mapping[int, list[TaskInstance]]): - child_instances = [ - item - for sublist in (child["instances"] for child in children if "instances" in child) - for item in sublist - if item and item["run_id"] == run_id - ] - - children_queued_dttms = (item["queued_dttm"] for item in child_instances) - children_start_dates = (item["start_date"] for item in child_instances) - children_end_dates = (item["end_date"] for item in child_instances) - children_states = {item["state"] for item in child_instances} - - # TODO: This assumes TI map index has a one-to-one mapping to - # its parent mapped task group, which will not be true when we - # allow nested mapping in the future. - mapped_states: MutableMapping[str, int] = defaultdict(int) - for mi_values in mapped_instances.values(): - child_states = {mi.state for mi in mi_values} - state = next(s for s in wwwutils.priority if s in child_states) - value = state.value if state is not None else "no_status" - mapped_states[value] += 1 - - group_state = next((state for state in wwwutils.priority if state in children_states), None) - group_queued_dttm = min(filter(None, children_queued_dttms), default=None) - group_start_date = min(filter(None, children_start_dates), default=None) - group_end_date = max(filter(None, children_end_dates), default=None) - - return { - "task_id": task_group.group_id, - "run_id": run_id, - "state": group_state, - "queued_dttm": group_queued_dttm, - "start_date": group_start_date, - "end_date": group_end_date, - "mapped_states": mapped_states, - } - - return [get_mapped_group_summary(run_id, tis) for run_id, tis in mapped_tis.items()] - - # We don't need to calculate summaries for the root - if task_group.group_id is None: - return { - "id": task_group.group_id, - "label": task_group.label, - "children": children, - "instances": [], - } - - if next(task_group.iter_mapped_task_groups(), None) is not None: - return { - "id": task_group.group_id, - "label": task_group.label, - "children": children, - "tooltip": task_group.tooltip, - "instances": get_mapped_group_summaries(), - "is_mapped": True, - } - - group_summaries = [get_summary(dr) for dr in dag_runs] - - return { - "id": task_group.group_id, - "label": task_group.label, - "children": children, - "tooltip": task_group.tooltip, - "instances": group_summaries, - } - - return task_group_to_grid(dag.task_group) - - -def get_key_paths(input_dict): - """Return a list of dot-separated dictionary paths.""" - for key, value in input_dict.items(): - if isinstance(value, dict): - for sub_key in get_key_paths(value): - yield f"{key}.{sub_key}" - else: - yield key - - -def get_value_from_path(key_path, content): - """Return the value from a dictionary based on dot-separated path of keys.""" - elem = content - for x in key_path.strip(".").split("."): - try: - x = int(x) - elem = elem[x] - except ValueError: - elem = elem.get(x) - - return elem - - -def get_task_stats_from_query(qry): - """ - Return a dict of the task quantity, grouped by dag id and task status. - - :param qry: The data in the format (, , , ), - ordered by and - """ - data = {} - last_dag_id = None - has_running_dags = False - for dag_id, state, is_dag_running, count in qry: - if last_dag_id != dag_id: - last_dag_id = dag_id - has_running_dags = False - elif not is_dag_running and has_running_dags: - continue - - if is_dag_running: - has_running_dags = True - if dag_id not in data: - data[dag_id] = {} - data[dag_id][state] = count - return data - - -def redirect_or_json(origin, msg, status="", status_code=200): - """ - Return json which allows us to more elegantly handle side effects in-page. - - This is useful because some endpoints are called by javascript. - """ - if request.headers.get("Accept") == "application/json": - if status == "error" and status_code == 200: - status_code = 500 - return Response(response=msg, status=status_code, mimetype="application/json") - else: - if status: - flash(msg, status) - else: - flash(msg) - return redirect(origin) - - -###################################################################################### -# Error handlers -###################################################################################### - - -def not_found(error): - """Show Not Found on screen for any error in the Webserver.""" - return ( - render_template( - "airflow/error.html", - hostname=get_hostname() if conf.getboolean("webserver", "EXPOSE_HOSTNAME") else "", - status_code=404, - error_message="Page cannot be found.", - ), - 404, - ) - - -def method_not_allowed(error): - """Show Method Not Allowed on screen for any error in the Webserver.""" - return ( - render_template( - "airflow/error.html", - hostname=get_hostname() if conf.getboolean("webserver", "EXPOSE_HOSTNAME") else "", - status_code=405, - error_message="Received an invalid request.", - ), - 405, - ) - - -def show_traceback(error): - """Show Traceback for a given error.""" - is_logged_in = get_auth_manager().is_logged_in() - return ( - render_template( - "airflow/traceback.html", - python_version=sys.version.split(" ")[0] if is_logged_in else "redacted", - airflow_version=version if is_logged_in else "redacted", - hostname=( - get_hostname() - if conf.getboolean("webserver", "EXPOSE_HOSTNAME") and is_logged_in - else "redacted" - ), - info=( - traceback.format_exc() - if conf.getboolean("webserver", "EXPOSE_STACKTRACE") and is_logged_in - else "Error! Please contact server admin." - ), - ), - 500, - ) - - -###################################################################################### -# BaseViews -###################################################################################### - - -class AirflowBaseView(BaseView): - """Base View to set Airflow related properties.""" - - from airflow import macros - - route_base = "" - - extra_args = { - # Make our macros available to our UI templates too. - "macros": macros, - "get_docs_url": get_docs_url, - } - - if not conf.getboolean("core", "unit_test_mode"): - executor, _ = ExecutorLoader.import_default_executor_cls() - extra_args["sqlite_warning"] = settings.engine and (settings.engine.dialect.name == "sqlite") - if not executor.is_production: - extra_args["production_executor_warning"] = executor.__name__ - extra_args["otel_metrics_on"] = conf.getboolean("metrics", "otel_on") - extra_args["otel_traces_on"] = conf.getboolean("traces", "otel_on") - - line_chart_attr = { - "legend.maxKeyLength": 200, - } - - def render_template(self, *args, **kwargs): - # Add triggerer_job only if we need it - if TriggererJobRunner.is_needed(): - kwargs["triggerer_job"] = lazy_object_proxy.Proxy(TriggererJobRunner.most_recent_job) - - if "dag" in kwargs: - kwargs["can_edit_dag"] = get_auth_manager().is_authorized_dag( - method="PUT", details=DagDetails(id=kwargs["dag"].dag_id), user=g.user - ) - url_serializer = URLSafeSerializer(current_app.config["SECRET_KEY"]) - kwargs["dag_file_token"] = url_serializer.dumps(kwargs["dag"].fileloc) - - return super().render_template( - *args, - # Cache this at most once per request, not for the lifetime of the view instance - scheduler_job=lazy_object_proxy.Proxy(SchedulerJobRunner.most_recent_job), - **kwargs, - ) - - -class Airflow(AirflowBaseView): - """Main Airflow application.""" - - @expose("/health") - def health(self): - """ - Check the health status of the Airflow instance. - - Includes metadatabase, scheduler and triggerer. - """ - airflow_health_status = get_airflow_health() - - return flask.json.jsonify(airflow_health_status) - - @expose("/home") - @auth.has_access_view() - def index(self): - """Home view.""" - from airflow.models.dag import DagOwnerAttributes - - hide_paused_dags_by_default = conf.getboolean("webserver", "hide_paused_dags_by_default") - default_dag_run = conf.getint("webserver", "default_dag_run_display_number") - - num_runs = request.args.get("num_runs", default=default_dag_run, type=int) - current_page = request.args.get("page", default=0, type=int) - arg_search_query = request.args.get("search") - arg_tags_filter = request.args.getlist("tags") - arg_status_filter = request.args.get("status") - arg_lastrun_filter = request.args.get("lastrun") - arg_sorting_key = request.args.get("sorting_key", "dag_id") - arg_sorting_direction = request.args.get("sorting_direction", default="asc") - - if request.args.get("reset_tags") is not None: - flask_session[FILTER_TAGS_COOKIE] = None - # Remove the reset_tags=reset from the URL - return redirect(url_for("Airflow.index")) - - if arg_lastrun_filter == "reset_filter": - flask_session[FILTER_LASTRUN_COOKIE] = None - return redirect(url_for("Airflow.index")) - - filter_tags_cookie_val = flask_session.get(FILTER_TAGS_COOKIE) - filter_lastrun_cookie_val = flask_session.get(FILTER_LASTRUN_COOKIE) - - # update filter args in url from session values if needed - if (not arg_tags_filter and filter_tags_cookie_val) or ( - not arg_lastrun_filter and filter_lastrun_cookie_val - ): - tags = arg_tags_filter or (filter_tags_cookie_val and filter_tags_cookie_val.split(",")) - lastrun = arg_lastrun_filter or filter_lastrun_cookie_val - return redirect(url_for("Airflow.index", tags=tags, lastrun=lastrun)) - - if arg_tags_filter: - flask_session[FILTER_TAGS_COOKIE] = ",".join(arg_tags_filter) - - if arg_lastrun_filter: - arg_lastrun_filter = arg_lastrun_filter.strip().lower() - flask_session[FILTER_LASTRUN_COOKIE] = arg_lastrun_filter - - if arg_status_filter is None: - filter_status_cookie_val = flask_session.get(FILTER_STATUS_COOKIE) - if filter_status_cookie_val: - arg_status_filter = filter_status_cookie_val - else: - arg_status_filter = "active" if hide_paused_dags_by_default else "all" - flask_session[FILTER_STATUS_COOKIE] = arg_status_filter - else: - status = arg_status_filter.strip().lower() - flask_session[FILTER_STATUS_COOKIE] = status - arg_status_filter = status - - dags_per_page = PAGE_SIZE - - start = current_page * dags_per_page - end = start + dags_per_page - - # Get all the dag id the user could access - filter_dag_ids = get_auth_manager().get_permitted_dag_ids(user=g.user) - - with create_session() as session: - # read orm_dags from the db - dags_query = select(DagModel).where(DagModel.is_active) - - if arg_search_query: - escaped_arg_search_query = arg_search_query.replace("_", r"\_") - dags_query = dags_query.where( - DagModel.dag_id.ilike("%" + escaped_arg_search_query + "%", escape="\\") - | DagModel._dag_display_property_value.ilike( - "%" + escaped_arg_search_query + "%", escape="\\" - ) - | DagModel.owners.ilike("%" + escaped_arg_search_query + "%", escape="\\") - ) - - if arg_tags_filter: - dags_query = dags_query.where(DagModel.tags.any(DagTag.name.in_(arg_tags_filter))) - - dags_query = dags_query.where(DagModel.dag_id.in_(filter_dag_ids)) - filtered_dag_count = get_query_count(dags_query, session=session) - if filtered_dag_count == 0 and len(arg_tags_filter): - flash( - "No matching DAG tags found.", - "warning", - ) - flask_session[FILTER_TAGS_COOKIE] = None - return redirect(url_for("Airflow.index")) - - # find DAGs which have a RUNNING DagRun - running_dags = dags_query.join(DagRun, DagModel.dag_id == DagRun.dag_id).where( - (DagRun.state == DagRunState.RUNNING) | (DagRun.state == DagRunState.QUEUED) - ) - - lastrun_running_is_paused = session.execute( - running_dags.with_only_columns(DagModel.dag_id, DagModel.is_paused).distinct(DagModel.dag_id) - ).all() - - lastrun_running_count_active = len( - list(filter(lambda x: not x.is_paused, lastrun_running_is_paused)) - ) - lastrun_running_count_paused = len(list(filter(lambda x: x.is_paused, lastrun_running_is_paused))) - - # find DAGs for which the latest DagRun is FAILED - subq_all = ( - select(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) - .group_by(DagRun.dag_id) - .subquery() - ) - subq_failed = ( - select(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) - .where(DagRun.state == DagRunState.FAILED) - .group_by(DagRun.dag_id) - .subquery() - ) - subq_join = ( - select(subq_all.c.dag_id, subq_all.c.start_date) - .join( - subq_failed, - and_( - subq_all.c.dag_id == subq_failed.c.dag_id, - subq_all.c.start_date == subq_failed.c.start_date, - ), - ) - .subquery() - ) - failed_dags = dags_query.join(subq_join, DagModel.dag_id == subq_join.c.dag_id) - - lastrun_failed_is_paused_count = dict( - session.execute( - failed_dags.with_only_columns(DagModel.is_paused, func.count()).group_by( - DagModel.is_paused - ) - ).all() - ) - - lastrun_failed_count_active = lastrun_failed_is_paused_count.get(False, 0) - lastrun_failed_count_paused = lastrun_failed_is_paused_count.get(True, 0) - - if arg_lastrun_filter == "running": - dags_query = running_dags - elif arg_lastrun_filter == "failed": - dags_query = failed_dags - - all_dags = dags_query - active_dags = dags_query.where(~DagModel.is_paused) - paused_dags = dags_query.where(DagModel.is_paused) - - status_is_paused = session.execute( - all_dags.with_only_columns(DagModel.dag_id, DagModel.is_paused).distinct(DagModel.dag_id) - ).all() - - status_count_active = len(list(filter(lambda x: not x.is_paused, status_is_paused))) - status_count_paused = len(list(filter(lambda x: x.is_paused, status_is_paused))) - all_dags_count = status_count_active + status_count_paused - - if arg_status_filter == "active": - current_dags = active_dags - num_of_all_dags = status_count_active - lastrun_count_running = lastrun_running_count_active - lastrun_count_failed = lastrun_failed_count_active - elif arg_status_filter == "paused": - current_dags = paused_dags - num_of_all_dags = status_count_paused - lastrun_count_running = lastrun_running_count_paused - lastrun_count_failed = lastrun_failed_count_paused - else: - current_dags = all_dags - num_of_all_dags = all_dags_count - lastrun_count_running = lastrun_running_count_active + lastrun_running_count_paused - lastrun_count_failed = lastrun_failed_count_active + lastrun_failed_count_paused - - if arg_sorting_key == "dag_id": - if arg_sorting_direction == "desc": - current_dags = current_dags.order_by( - func.coalesce(DagModel.dag_display_name, DagModel.dag_id).desc() - ) - else: - current_dags = current_dags.order_by( - func.coalesce(DagModel.dag_display_name, DagModel.dag_id) - ) - elif arg_sorting_key == "last_dagrun": - dag_run_subquery = ( - select( - DagRun.dag_id, - sqla.func.max(DagRun.logical_date).label("max_logical_date"), - ) - .group_by(DagRun.dag_id) - .subquery() - ) - current_dags = current_dags.outerjoin( - dag_run_subquery, and_(dag_run_subquery.c.dag_id == DagModel.dag_id) - ) - null_case = case((dag_run_subquery.c.max_logical_date.is_(None), 1), else_=0) - if arg_sorting_direction == "desc": - current_dags = current_dags.order_by( - null_case, dag_run_subquery.c.max_logical_date.desc() - ) - else: - current_dags = current_dags.order_by(null_case, dag_run_subquery.c.max_logical_date) - else: - sort_column = DagModel.__table__.c.get(arg_sorting_key) - if sort_column is not None: - null_case = case((sort_column.is_(None), 1), else_=0) - if arg_sorting_direction == "desc": - current_dags = current_dags.order_by(null_case, sort_column.desc()) - else: - current_dags = current_dags.order_by(null_case, sort_column) - - dags: list[DagModel] = ( - session.scalars( - current_dags.options(joinedload(DagModel.tags)).offset(start).limit(dags_per_page) - ) - .unique() - .all() - ) - - asset_triggered_dag_ids = {dag.dag_id for dag in dags if dag.asset_expression is not None} - if asset_triggered_dag_ids: - asset_triggered_next_run_info = get_asset_triggered_next_run_info( - asset_triggered_dag_ids, session=session - ) - else: - asset_triggered_next_run_info = {} - - file_tokens = {} - for dag in dags: - dag.can_edit = get_auth_manager().is_authorized_dag( - method="PUT", details=DagDetails(id=dag.dag_id), user=g.user - ) - can_create_dag_run = get_auth_manager().is_authorized_dag( - method="POST", - access_entity=DagAccessEntity.RUN, - details=DagDetails(id=dag.dag_id), - user=g.user, - ) - dag.can_trigger = dag.can_edit and can_create_dag_run - dag.can_delete = get_auth_manager().is_authorized_dag( - method="DELETE", details=DagDetails(id=dag.dag_id), user=g.user - ) - url_serializer = URLSafeSerializer(current_app.config["SECRET_KEY"]) - file_tokens[dag.dag_id] = url_serializer.dumps(dag.fileloc) - - dagtags = session.execute(select(func.distinct(DagTag.name)).order_by(DagTag.name)).all() - tags = [ - {"name": name, "selected": bool(arg_tags_filter and name in arg_tags_filter)} - for (name,) in dagtags - ] - - owner_links_dict = DagOwnerAttributes.get_all(session) - - if get_auth_manager().is_authorized_view(access_view=AccessView.IMPORT_ERRORS, user=g.user): - import_errors = select(ParseImportError).order_by(ParseImportError.id) - - can_read_all_dags = get_auth_manager().is_authorized_dag(method="GET", user=g.user) - if not can_read_all_dags: - # if the user doesn't have access to all DAGs, only display errors from visible DAGs - import_errors = import_errors.where( - tuple_(ParseImportError.filename, ParseImportError.bundle_name).in_( - select(DagModel.fileloc, DagModel.bundle_name) - .distinct() - .where(DagModel.dag_id.in_(filter_dag_ids)) - ), - ) - - import_errors = session.scalars(import_errors) - for import_error in import_errors: - stacktrace = import_error.stacktrace - if not can_read_all_dags: - # Check if user has read access to all the DAGs defined in the file - file_dag_ids = ( - session.query(DagModel.dag_id) - .filter(DagModel.fileloc == import_error.filename) - .all() - ) - requests: Sequence[IsAuthorizedDagRequest] = [ - { - "method": "GET", - "details": DagDetails(id=dag_id[0]), - } - for dag_id in file_dag_ids - ] - if not get_auth_manager().batch_is_authorized_dag( - requests, user=get_auth_manager().get_user() - ): - stacktrace = "REDACTED - you do not have read permission on all DAGs in the file" - flash( - f"Broken DAG: [{import_error.filename}]\r{stacktrace}", - "dag_import_error", - ) - - from airflow.plugins_manager import import_errors as plugin_import_errors - - for filename, stacktrace in plugin_import_errors.items(): - flash( - f"Broken plugin: [{filename}] {stacktrace}", - "error", - ) - - num_of_pages = math.ceil(num_of_all_dags / dags_per_page) - - state_color_mapping = State.state_color.copy() - state_color_mapping["null"] = state_color_mapping.pop(None) - - page_title = conf.get(section="webserver", key="instance_name", fallback="DAGs") - page_title_has_markup = conf.getboolean( - section="webserver", key="instance_name_has_markup", fallback=False - ) - - dashboard_alerts = [fm for fm in settings.DASHBOARD_UIALERTS if fm.should_show()] - - def _iter_parsed_moved_data_table_names(): - for table_name in inspect(session.get_bind()).get_table_names(): - segments = table_name.split("__", 3) - if len(segments) >= 3: - if segments[0] == settings.AIRFLOW_MOVED_TABLE_PREFIX: - # Second segment is a version marker that we don't need to show. - yield segments[-1], table_name - - if get_auth_manager().is_authorized_configuration(method="GET", user=g.user) and conf.getboolean( - "webserver", "warn_deployment_exposure" - ): - robots_file_access_count = ( - select(Log) - .where(Log.event == "robots") - .where(Log.dttm > (utcnow() - datetime.timedelta(days=7))) - ) - robots_file_access_count = get_query_count(robots_file_access_count, session=session) - if robots_file_access_count > 0: - flash( - Markup( - "Recent requests have been made to /robots.txt. " - "This indicates that this deployment may be accessible to the public internet. " - "This warning can be disabled by setting webserver.warn_deployment_exposure=False in " - "airflow.cfg. Read more about web deployment security ' - "here" - ), - "warning", - ) - - return self.render_template( - "airflow/dags.html", - dags=dags, - show_trigger_form_if_no_params=conf.getboolean("webserver", "show_trigger_form_if_no_params"), - dashboard_alerts=dashboard_alerts, - migration_moved_data_alerts=sorted(set(_iter_parsed_moved_data_table_names())), - current_page=current_page, - search_query=arg_search_query or "", - page_title=Markup(page_title) if page_title_has_markup else page_title, - page_size=dags_per_page, - num_of_pages=num_of_pages, - num_dag_from=min(start + 1, num_of_all_dags), - num_dag_to=min(end, num_of_all_dags), - num_of_all_dags=num_of_all_dags, - paging=wwwutils.generate_pages( - current_page, - num_of_pages, - search=escape(arg_search_query) if arg_search_query else None, - status=arg_status_filter or None, - tags=arg_tags_filter or None, - sorting_key=arg_sorting_key or None, - sorting_direction=arg_sorting_direction or None, - ), - num_runs=num_runs, - tags=tags, - owner_links=owner_links_dict, - state_color=state_color_mapping, - status_filter=arg_status_filter, - status_count_all=all_dags_count, - status_count_active=status_count_active, - status_count_paused=status_count_paused, - lastrun_filter=arg_lastrun_filter, - lastrun_count_running=lastrun_count_running, - lastrun_count_failed=lastrun_count_failed, - tags_filter=arg_tags_filter, - sorting_key=arg_sorting_key, - sorting_direction=arg_sorting_direction, - auto_refresh_interval=conf.getint("webserver", "auto_refresh_interval"), - asset_triggered_next_run_info=asset_triggered_next_run_info, - file_tokens=file_tokens, - ) - - @expose("/assets") - @auth.has_access_asset("GET") - def assets(self): - """Assets view.""" - state_color_mapping = State.state_color.copy() - state_color_mapping["null"] = state_color_mapping.pop(None) - return self.render_template( - "airflow/assets.html", - auto_refresh_interval=conf.getint("webserver", "auto_refresh_interval"), - state_color_mapping=state_color_mapping, - ) - - @expose("/cluster_activity") - @auth.has_access_view(AccessView.CLUSTER_ACTIVITY) - def cluster_activity(self): - """Cluster Activity view.""" - state_color_mapping = State.state_color.copy() - state_color_mapping["no_status"] = state_color_mapping.pop(None) - return self.render_template( - "airflow/cluster_activity.html", - auto_refresh_interval=conf.getint("webserver", "auto_refresh_interval"), - state_color_mapping=state_color_mapping, - ) - - @expose("/next_run_assets_summary", methods=["POST"]) - @provide_session - def next_run_assets_summary(self, session: Session = NEW_SESSION): - """Next run info for asset triggered DAGs.""" - allowed_dag_ids = get_auth_manager().get_permitted_dag_ids(user=g.user) - - if not allowed_dag_ids: - return flask.json.jsonify({}) - - # Filter by post parameters - selected_dag_ids = {unquote(dag_id) for dag_id in request.form.getlist("dag_ids") if dag_id} - - if selected_dag_ids: - filter_dag_ids = selected_dag_ids.intersection(allowed_dag_ids) - else: - filter_dag_ids = allowed_dag_ids - - asset_triggered_dag_ids = session.scalars( - select(DagModel.dag_id) - .where(DagModel.dag_id.in_(filter_dag_ids)) - .where(DagModel.asset_expression.is_not(None)) - ).all() - - asset_triggered_next_run_info = get_asset_triggered_next_run_info( - asset_triggered_dag_ids, session=session - ) - - return flask.json.jsonify(asset_triggered_next_run_info) - - @expose("/dag_stats", methods=["POST"]) - @auth.has_access_dag("GET", DagAccessEntity.RUN) - @provide_session - def dag_stats(self, session: Session = NEW_SESSION): - """Dag statistics.""" - allowed_dag_ids = get_auth_manager().get_permitted_dag_ids(user=g.user) - - # Filter by post parameters - selected_dag_ids = {unquote(dag_id) for dag_id in request.form.getlist("dag_ids") if dag_id} - if selected_dag_ids: - filter_dag_ids = selected_dag_ids.intersection(allowed_dag_ids) - else: - filter_dag_ids = allowed_dag_ids - if not filter_dag_ids: - return flask.json.jsonify({}) - - dag_state_stats = session.execute( - select(DagRun.dag_id, DagRun.state, sqla.func.count(DagRun.state)) - .group_by(DagRun.dag_id, DagRun.state) - .where(DagRun.dag_id.in_(filter_dag_ids)) - ) - dag_state_data = {(dag_id, state): count for dag_id, state, count in dag_state_stats} - - payload = { - dag_id: [ - {"state": state, "count": dag_state_data.get((dag_id, state), 0)} - for state in State.dag_states - ] - for dag_id in filter_dag_ids - } - return flask.json.jsonify(payload) - - @expose("/task_stats", methods=["POST"]) - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - @provide_session - def task_stats(self, session: Session = NEW_SESSION): - """Task Statistics.""" - allowed_dag_ids = get_auth_manager().get_permitted_dag_ids(user=g.user) - - if not allowed_dag_ids: - return flask.json.jsonify({}) - - # Filter by post parameters - selected_dag_ids = {unquote(dag_id) for dag_id in request.form.getlist("dag_ids") if dag_id} - - if selected_dag_ids: - filter_dag_ids = selected_dag_ids.intersection(allowed_dag_ids) - else: - filter_dag_ids = allowed_dag_ids - - running_dag_run_query_result = ( - select(DagRun.dag_id, DagRun.run_id) - .join(DagModel, DagModel.dag_id == DagRun.dag_id) - .where(DagRun.state == DagRunState.RUNNING, DagModel.is_active) - ) - - running_dag_run_query_result = running_dag_run_query_result.where(DagRun.dag_id.in_(filter_dag_ids)) - - running_dag_run_query_result = running_dag_run_query_result.subquery("running_dag_run") - - # Select all task_instances from active dag_runs. - running_task_instance_query_result = select( - TaskInstance.dag_id.label("dag_id"), - TaskInstance.state.label("state"), - sqla.literal(True).label("is_dag_running"), - ).join( - running_dag_run_query_result, - and_( - running_dag_run_query_result.c.dag_id == TaskInstance.dag_id, - running_dag_run_query_result.c.run_id == TaskInstance.run_id, - ), - ) - - if conf.getboolean("webserver", "SHOW_RECENT_STATS_FOR_COMPLETED_RUNS", fallback=True): - last_dag_run = ( - select(DagRun.dag_id, sqla.func.max(DagRun.logical_date).label("logical_date")) - .join(DagModel, DagModel.dag_id == DagRun.dag_id) - .where(DagRun.state != DagRunState.RUNNING, DagModel.is_active) - .group_by(DagRun.dag_id) - ) - - last_dag_run = last_dag_run.where(DagRun.dag_id.in_(filter_dag_ids)) - last_dag_run = last_dag_run.subquery("last_dag_run") - - # Select all task_instances from active dag_runs. - # If no dag_run is active, return task instances from most recent dag_run. - last_task_instance_query_result = ( - select( - TaskInstance.dag_id.label("dag_id"), - TaskInstance.state.label("state"), - sqla.literal(False).label("is_dag_running"), - ) - .join(TaskInstance.dag_run) - .join( - last_dag_run, - and_( - last_dag_run.c.dag_id == TaskInstance.dag_id, - last_dag_run.c.logical_date == DagRun.logical_date, - ), - ) - ) - - final_task_instance_query_result = union_all( - last_task_instance_query_result, running_task_instance_query_result - ).alias("final_ti") - else: - final_task_instance_query_result = running_task_instance_query_result.subquery("final_ti") - - qry = session.execute( - select( - final_task_instance_query_result.c.dag_id, - final_task_instance_query_result.c.state, - final_task_instance_query_result.c.is_dag_running, - sqla.func.count(), - ) - .group_by( - final_task_instance_query_result.c.dag_id, - final_task_instance_query_result.c.state, - final_task_instance_query_result.c.is_dag_running, - ) - .order_by( - final_task_instance_query_result.c.dag_id, - final_task_instance_query_result.c.is_dag_running.desc(), - ) - ) - data = get_task_stats_from_query(qry) - payload: dict[str, list[dict[str, Any]]] = defaultdict(list) - for dag_id, state in itertools.product(filter_dag_ids, State.task_states): - payload[dag_id].append({"state": state, "count": data.get(dag_id, {}).get(state, 0)}) - return flask.json.jsonify(payload) - - @expose("/last_dagruns", methods=["POST"]) - @auth.has_access_dag("GET", DagAccessEntity.RUN) - @provide_session - def last_dagruns(self, session: Session = NEW_SESSION): - """Last DAG runs.""" - allowed_dag_ids = get_auth_manager().get_permitted_dag_ids(user=g.user) - - # Filter by post parameters - selected_dag_ids = {unquote(dag_id) for dag_id in request.form.getlist("dag_ids") if dag_id} - - if selected_dag_ids: - filter_dag_ids = selected_dag_ids.intersection(allowed_dag_ids) - else: - filter_dag_ids = allowed_dag_ids - - if not filter_dag_ids: - return flask.json.jsonify({}) - - last_runs_subquery = ( - select( - DagRun.dag_id, - sqla.func.max(DagRun.logical_date).label("max_logical_date"), - ) - .group_by(DagRun.dag_id) - .where(DagRun.dag_id.in_(filter_dag_ids)) # Only include accessible/selected DAGs. - .subquery("last_runs") - ) - - query = session.execute( - select( - DagRun.dag_id, - DagRun.start_date, - DagRun.end_date, - DagRun.state, - DagRun.logical_date, - DagRun.data_interval_start, - DagRun.data_interval_end, - ).join( - last_runs_subquery, - and_( - last_runs_subquery.c.dag_id == DagRun.dag_id, - last_runs_subquery.c.max_logical_date == DagRun.logical_date, - ), - ) - ) - - resp = { - r.dag_id.replace(".", "__dot__"): { - "dag_id": r.dag_id, - "state": r.state, - "logical_date": wwwutils.datetime_to_string(r.logical_date), - "start_date": wwwutils.datetime_to_string(r.start_date), - "end_date": wwwutils.datetime_to_string(r.end_date), - "data_interval_start": wwwutils.datetime_to_string(r.data_interval_start), - "data_interval_end": wwwutils.datetime_to_string(r.data_interval_end), - } - for r in query - } - return flask.json.jsonify(resp) - - @expose("/code") - def legacy_code(self): - """Redirect from url param.""" - return redirect(url_for("Airflow.code", **sanitize_args(request.args))) - - @expose("/dags//code") - @auth.has_access_dag("GET", DagAccessEntity.CODE) - def code(self, dag_id): - """Dag Code.""" - kwargs = { - **sanitize_args(request.args), - "dag_id": dag_id, - "tab": "code", - } - - return redirect(url_for("Airflow.grid", **kwargs)) - - @expose("/dag_details") - def legacy_dag_details(self): - """Redirect from url param.""" - return redirect(url_for("Airflow.dag_details", **sanitize_args(request.args))) - - @expose("/dags//details") - def dag_details(self, dag_id): - """Get Dag details.""" - return redirect(url_for("Airflow.grid", dag_id=dag_id)) - - @expose("/rendered-templates") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - @provide_session - def rendered_templates(self, session): - """Get rendered Dag.""" - dag_id = request.args.get("dag_id") - task_id = request.args.get("task_id") - map_index = request.args.get("map_index", -1, type=int) - logical_date = request.args.get("logical_date") - dttm = _safe_parse_datetime(logical_date) - form = DateTimeForm(data={"logical_date": dttm}) - root = request.args.get("root", "") - - logger.info("Retrieving rendered templates.") - dag: DAG = get_airflow_app().dag_bag.get_dag(dag_id) - dag_run = dag.get_dagrun( - select(DagRun.run_id).where(DagRun.logical_date == dttm).order_by(DagRun.id.desc()).limit(1), - session=session, - ) - raw_task = dag.get_task(task_id).prepare_for_execution() - - no_dagrun = False - url_serializer = URLSafeSerializer(current_app.config["SECRET_KEY"]) - - title = "Rendered Template" - html_dict = {} - - ti: TaskInstance - if dag_run is None: - # No DAG run matching given logical date. This usually means this - # DAG has never been run. Task instance rendering does not really - # make sense in this situation, but "works" prior to AIP-39. This - # "fakes" a temporary DagRun-TaskInstance association (not saved to - # database) for presentation only. - ti = TaskInstance(raw_task, map_index=map_index) - ti.dag_run = DagRun(dag_id=dag_id, logical_date=dttm) - no_dagrun = True - else: - ti = dag_run.get_task_instance(task_id=task_id, map_index=map_index, session=session) - if ti: - ti.refresh_from_task(raw_task) - else: - flash(f"there is no task instance with the provided map_index {map_index}", "error") - return self.render_template( - "airflow/ti_code.html", - show_trigger_form_if_no_params=conf.getboolean( - "webserver", "show_trigger_form_if_no_params" - ), - dag_run_id=dag_run.run_id if dag_run else "", - html_dict=html_dict, - dag=dag, - task_id=task_id, - logical_date=logical_date, - map_index=map_index, - form=form, - root=root, - title=title, - ) - - try: - ti.get_rendered_template_fields(session=session) - except AirflowException as e: - if not e.__cause__: - flash(f"Error rendering template: {e}", "error") - else: - msg = Markup("Error rendering template: {0}

    OriginalError: {0.__cause__}").format(e) - flash(msg, "error") - except Exception as e: - flash(f"Error rendering template: {e}", "error") - - # Ensure we are rendering the unmapped operator. Unmapping should be - # done automatically if template fields are rendered successfully; this - # only matters if get_rendered_template_fields() raised an exception. - # The following rendering won't show useful values in this case anyway, - # but we'll display some quasi-meaingful field names. - task = ti.task.unmap(None) - - renderers = wwwutils.get_attr_renderer() - - for template_field in task.template_fields: - content = getattr(task, template_field) - renderer = task.template_fields_renderers.get(template_field, template_field) - if renderer in renderers: - html_dict[template_field] = renderers[renderer](content) if not no_dagrun else "" - else: - html_dict[template_field] = Markup("
    {}
    ").format( - pformat(content) if not no_dagrun else "" - ) - - if isinstance(content, dict): - if template_field == "op_kwargs": - for key, value in content.items(): - renderer = task.template_fields_renderers.get(key, key) - if renderer in renderers: - html_dict[f"{template_field}.{key}"] = ( - renderers[renderer](value) if not no_dagrun else "" - ) - else: - html_dict[f"{template_field}.{key}"] = Markup( - "
    {}
    " - ).format(pformat(value) if not no_dagrun else "") - else: - for dict_keys in get_key_paths(content): - template_path = f"{template_field}.{dict_keys}" - renderer = task.template_fields_renderers.get(template_path, template_path) - if renderer in renderers: - content_value = get_value_from_path(dict_keys, content) - html_dict[template_path] = ( - renderers[renderer](content_value) if not no_dagrun else "" - ) - return self.render_template( - "airflow/ti_code.html", - show_trigger_form_if_no_params=conf.getboolean("webserver", "show_trigger_form_if_no_params"), - html_dict=html_dict, - dag_run_id=dag_run.run_id if dag_run else "", - dag=dag, - task_id=task_id, - task_display_name=task.task_display_name, - logical_date=logical_date, - map_index=map_index, - form=form, - root=root, - title=title, - dag_file_token=url_serializer.dumps(dag.fileloc), - ) - - @expose("/rendered-k8s") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - @provide_session - def rendered_k8s(self, *, session: Session = NEW_SESSION): - """Get rendered k8s yaml.""" - if not settings.IS_K8S_OR_K8SCELERY_EXECUTOR: - abort(404) - # This part is only used for k8s executor so providers.cncf.kubernetes must be installed - # with the get_rendered_k8s_spec method - from airflow.providers.cncf.kubernetes.template_rendering import get_rendered_k8s_spec - - dag_id = request.args.get("dag_id") - task_id = request.args.get("task_id") - if task_id is None: - logger.warning("Task id not passed in the request") - abort(400) - logical_date = request.args.get("logical_date") - dttm = _safe_parse_datetime(logical_date) - form = DateTimeForm(data={"logical_date": dttm}) - root = request.args.get("root", "") - map_index = request.args.get("map_index", -1, type=int) - logger.info("Retrieving rendered k8s.") - - dag: DAG = get_airflow_app().dag_bag.get_dag(dag_id) - task = dag.get_task(task_id) - run_id = session.scalar( - select(DagRun.run_id).where(DagRun.logical_date == dttm).order_by(DagRun.id.desc()).limit(1) - ) - dag_run = dag.get_dagrun( - run_id=run_id, - session=session, - ) - ti = dag_run.get_task_instance(task_id=task.task_id, map_index=map_index, session=session) - - if not ti: - raise AirflowException(f"Task instance {task.task_id} not found.") - - pod_spec = None - if not isinstance(ti, TaskInstance): - raise ValueError("not a TaskInstance") - try: - pod_spec = get_rendered_k8s_spec(ti, session=session) - except AirflowException as e: - if not e.__cause__: - flash(f"Error rendering Kubernetes POD Spec: {e}", "error") - else: - tmp = Markup("Error rendering Kubernetes POD Spec: {0}

    Original error: {0.__cause__}") - flash(tmp.format(e), "error") - except Exception as e: - flash(f"Error rendering Kubernetes Pod Spec: {e}", "error") - title = "Rendered K8s Pod Spec" - - if pod_spec: - content = wwwutils.get_attr_renderer()["yaml"](yaml.dump(pod_spec)) - else: - content = Markup("
    Error rendering Kubernetes POD Spec
    ") - - return self.render_template( - "airflow/ti_code.html", - show_trigger_form_if_no_params=conf.getboolean("webserver", "show_trigger_form_if_no_params"), - dag_run_id=dag_run.run_id if dag_run else "", - html_dict={"k8s": content}, - dag=dag, - task_id=task_id, - task_display_name=task.task_display_name, - logical_date=logical_date, - map_index=map_index, - form=form, - root=root, - title=title, - ) - - @expose("/object/rendered-k8s") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - @provide_session - def rendered_k8s_data(self, *, session: Session = NEW_SESSION): - """Get rendered k8s yaml.""" - if not settings.IS_K8S_OR_K8SCELERY_EXECUTOR: - return {"error": "Not a k8s or k8s_celery executor"}, 404 - # This part is only used for k8s executor so providers.cncf.kubernetes must be installed - # with the get_rendered_k8s_spec method - from airflow.providers.cncf.kubernetes.template_rendering import get_rendered_k8s_spec - - dag_id = request.args.get("dag_id") - task_id = request.args.get("task_id") - if task_id is None: - return {"error": "Task id not passed in the request"}, 404 - run_id = request.args.get("run_id") - if run_id is None: - return {"error": "Run id not passed in the request"}, 404 - map_index = request.args.get("map_index", -1, type=int) - logger.info("Retrieving rendered k8s data.") - - dag: DAG = get_airflow_app().dag_bag.get_dag(dag_id) - task = dag.get_task(task_id) - dag_run = dag.get_dagrun(run_id=run_id, session=session) - ti = dag_run.get_task_instance(task_id=task.task_id, map_index=map_index, session=session) - - if not ti: - return {"error": f"can't find task instance {task.task_id}"}, 404 - pod_spec = None - if not isinstance(ti, TaskInstance): - return {"error": f"{task.task_id} is not a task instance"}, 500 - try: - pod_spec = get_rendered_k8s_spec(ti, session=session) - except AirflowException as e: - if not e.__cause__: - return {"error": f"Error rendering Kubernetes POD Spec: {e}"}, 500 - else: - tmp = Markup("Error rendering Kubernetes POD Spec: {0}

    Original error: {0.__cause__}") - return {"error": tmp.format(e)}, 500 - except Exception as e: - return {"error": f"Error rendering Kubernetes Pod Spec: {e}"}, 500 - - return pod_spec - - @expose("/get_logs_with_metadata") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - @auth.has_access_dag("GET", DagAccessEntity.TASK_LOGS) - @provide_session - def get_logs_with_metadata(self, session: Session = NEW_SESSION): - """Retrieve logs including metadata.""" - dag_id = request.args.get("dag_id") - task_id = request.args.get("task_id") - logical_date_str = request.args["logical_date"] - map_index = request.args.get("map_index", -1, type=int) - try_number = request.args.get("try_number", type=int) - metadata_str = request.args.get("metadata", "{}") - response_format = request.args.get("format", "json") - - # Validate JSON metadata - try: - metadata: dict = json.loads(metadata_str) or {} - except json.decoder.JSONDecodeError: - return {"error": "Invalid JSON metadata"}, 400 - - # Convert string datetime into actual datetime - try: - logical_date = timezone.parse(logical_date_str, strict=True) - except ValueError: - error_message = ( - f"Given logical date {logical_date_str!r} could not be identified as a date. " - "Example date format: 2015-11-16T14:34:15+00:00" - ) - return {"error": error_message}, 400 - - task_log_reader = TaskLogReader() - if not task_log_reader.supports_read: - return { - "message": "Task log handler does not support read logs.", - "error": True, - "metadata": {"end_of_log": True}, - } - - ti = session.scalar( - select(models.TaskInstance) - .where( - TaskInstance.task_id == task_id, - TaskInstance.dag_id == dag_id, - TaskInstance.logical_date == logical_date, - TaskInstance.map_index == map_index, - ) - .join(TaskInstance.dag_run) - .options(joinedload(TaskInstance.trigger).joinedload(Trigger.triggerer_job)) - .limit(1) - ) - - if ti is None: - return { - "message": "*** Task instance did not exist in the DB\n", - "error": True, - "metadata": {"end_of_log": True}, - } - - try: - dag = get_airflow_app().dag_bag.get_dag(dag_id) - if dag: - ti.task = dag.get_task(ti.task_id) - - if response_format == "json": - logs, metadata = task_log_reader.read_log_chunks(ti, try_number, metadata) - message = logs[0] if try_number is not None else logs - return {"message": message, "metadata": metadata} - - metadata["download_logs"] = True - attachment_filename = task_log_reader.render_log_filename(ti, try_number, session=session) - log_stream = task_log_reader.read_log_stream(ti, try_number, metadata) - return Response( - response=log_stream, - mimetype="text/plain", - headers={"Content-Disposition": f"attachment; filename={attachment_filename}"}, - ) - except AttributeError as e: - error_messages = [f"Task log handler does not support read logs.\n{e}\n"] - metadata["end_of_log"] = True - return {"message": error_messages, "error": True, "metadata": metadata} - - @expose("/log") - @auth.has_access_dag("GET", DagAccessEntity.TASK_LOGS) - @provide_session - def log(self, session: Session = NEW_SESSION): - """Retrieve log.""" - dag_id = request.args["dag_id"] - task_id = request.args.get("task_id") - map_index = request.args.get("map_index", -1, type=int) - logical_date = request.args.get("logical_date") - - if logical_date: - dttm = _safe_parse_datetime(logical_date) - else: - dttm = None - - form = DateTimeForm(data={"logical_date": dttm}) - dag_model = DagModel.get_dagmodel(dag_id) - - ti: TaskInstance = session.scalar( - select(models.TaskInstance) - .filter_by(dag_id=dag_id, task_id=task_id, logical_date=dttm, map_index=map_index) - .limit(1) - ) - - num_logs = 0 - if ti is not None: - num_logs = ti.try_number - logs = [""] * num_logs - root = request.args.get("root", "") - return self.render_template( - "airflow/ti_log.html", - show_trigger_form_if_no_params=conf.getboolean("webserver", "show_trigger_form_if_no_params"), - logs=logs, - dag=dag_model, - dag_run_id=ti.run_id if ti else "", - title="Log by attempts", - dag_id=dag_id, - task_id=task_id, - task_display_name=ti.task_display_name if ti else "", - logical_date=logical_date, - map_index=map_index, - form=form, - root=root, - wrapped=conf.getboolean("webserver", "default_wrap"), - ) - - @expose("/redirect_to_external_log") - @auth.has_access_dag("GET", DagAccessEntity.TASK_LOGS) - @provide_session - def redirect_to_external_log(self, session: Session = NEW_SESSION): - """Redirects to external log.""" - dag_id = request.args.get("dag_id") - task_id = request.args.get("task_id") - logical_date = request.args.get("logical_date") - dttm = _safe_parse_datetime(logical_date) - map_index = request.args.get("map_index", -1, type=int) - try_number = request.args.get("try_number", 1) - - ti = session.scalar( - select(models.TaskInstance) - .filter_by(dag_id=dag_id, task_id=task_id, logical_date=dttm, map_index=map_index) - .limit(1) - ) - - if not ti: - flash(f"Task [{dag_id}.{task_id}] does not exist", "error") - return redirect(url_for("Airflow.index")) - - task_log_reader = TaskLogReader() - if not task_log_reader.supports_external_link: - flash("Task log handler does not support external links", "error") - return redirect(url_for("Airflow.index")) - - handler = task_log_reader.log_handler - url = handler.get_external_log_url(ti, try_number) - return redirect(url) - - @expose("/task") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - @provide_session - def task(self, session: Session = NEW_SESSION): - """Retrieve task.""" - dag_id = request.args.get("dag_id") - task_id = request.args.get("task_id") - logical_date = request.args.get("logical_date") - dttm = _safe_parse_datetime(logical_date) - map_index = request.args.get("map_index", -1, type=int) - form = DateTimeForm(data={"logical_date": dttm}) - root = request.args.get("root", "") - dag = get_airflow_app().dag_bag.get_dag(dag_id) - - if not dag or task_id not in dag.task_ids: - flash(f"Task [{dag_id}.{task_id}] doesn't seem to exist at the moment", "error") - return redirect(url_for("Airflow.index")) - task = copy.copy(dag.get_task(task_id)) - task.resolve_template_files() - - ti: TaskInstance | None = session.scalar( - select(TaskInstance) - .options( - # HACK: Eager-load relationships. This is needed because - # multiple properties mis-use provide_session() that destroys - # the session object ti is bounded to. - joinedload(TaskInstance.queued_by_job, innerjoin=False), - joinedload(TaskInstance.trigger, innerjoin=False), - ) - .filter_by(logical_date=dttm, dag_id=dag_id, task_id=task_id, map_index=map_index) - ) - if ti is None: - ti_attrs: list[tuple[str, Any]] | None = None - else: - ti.refresh_from_task(task) - ti_attrs_to_skip = [ - "dag_id", - "key", - "mark_success_url", - "log", - "log_url", - "task", - "trigger", - "triggerer_job", - ] - # Some fields on TI are deprecated, but we don't want those warnings here. - with warnings.catch_warnings(): - warnings.simplefilter("ignore", RemovedInAirflow3Warning) - all_ti_attrs = ( - # fetching the value of _try_number to be shown under name try_number in UI - (name, getattr(ti, name)) - for name in dir(ti) - if not name.startswith("_") and name not in ti_attrs_to_skip - ) - ti_attrs = sorted((name, attr) for name, attr in all_ti_attrs if not callable(attr)) - - attr_renderers = wwwutils.get_attr_renderer() - - attrs_to_skip: frozenset[str] = getattr(task, "HIDE_ATTRS_FROM_UI", frozenset()) - - def include_task_attrs(attr_name): - return not ( - attr_name == "HIDE_ATTRS_FROM_UI" - or attr_name.startswith("_") - or attr_name in attr_renderers - or attr_name in attrs_to_skip - ) - - task_attrs = [ - (attr_name, secrets_masker.redact(attr, attr_name)) - for attr_name, attr in ( - (attr_name, getattr(task, attr_name)) for attr_name in filter(include_task_attrs, dir(task)) - ) - if not callable(attr) - ] - - # Color coding the special attributes that are code - special_attrs_rendered = { - attr_name: renderer(getattr(task, attr_name)) - for attr_name, renderer in attr_renderers.items() - if hasattr(task, attr_name) - } - - no_failed_deps_result = [ - ( - "Unknown", - "All dependencies are met but the task instance is not running. In most " - "cases this just means that the task will probably be scheduled soon " - "unless:
    \n- The scheduler is down or under heavy load
    \n{}\n" - "
    \nIf this task instance does not start soon please contact your " - "Airflow administrator for assistance.".format( - "- This task instance already ran and had it's state changed manually " - "(e.g. cleared in the UI)
    " - if ti and ti.state is None - else "" - ), - ) - ] - - # Use the scheduler's context to figure out which dependencies are not met - if ti is None: - failed_dep_reasons: list[tuple[str, str]] = [] - else: - dep_context = DepContext(SCHEDULER_QUEUED_DEPS) - failed_dep_reasons = [ - (dep.dep_name, dep.reason) for dep in ti.get_failed_dep_statuses(dep_context=dep_context) - ] - - title = "Task Instance Details" - return self.render_template( - "airflow/task.html", - show_trigger_form_if_no_params=conf.getboolean("webserver", "show_trigger_form_if_no_params"), - task_attrs=task_attrs, - ti_attrs=ti_attrs, - dag_run_id=ti.run_id if ti else "", - failed_dep_reasons=failed_dep_reasons or no_failed_deps_result, - task_id=task_id, - logical_date=logical_date, - map_index=map_index, - special_attrs_rendered=special_attrs_rendered, - form=form, - root=root, - dag=dag, - title=title, - task_display_name=task.task_display_name, - ) - - @expose("/xcom") - @auth.has_access_dag("GET", DagAccessEntity.XCOM) - @provide_session - def xcom(self, session: Session = NEW_SESSION): - """Retrieve XCOM.""" - dag_id = request.args["dag_id"] - task_id = request.args.get("task_id") - map_index = request.args.get("map_index", -1, type=int) - # Carrying logical_date through, even though it's irrelevant for - # this context - logical_date = request.args.get("logical_date") - dttm = _safe_parse_datetime(logical_date) - - form = DateTimeForm(data={"logical_date": dttm}) - root = request.args.get("root", "") - dag = DagModel.get_dagmodel(dag_id) - ti: TaskInstance = session.scalar( - select(TaskInstance).filter_by(dag_id=dag_id, task_id=task_id).limit(1) - ) - - if not ti: - flash(f"Task [{dag_id}.{task_id}] doesn't seem to exist at the moment", "error") - return redirect(url_for("Airflow.index")) - - xcom_query = session.scalars( - select(XCom).where( - XCom.dag_id == dag_id, - XCom.task_id == task_id, - XCom.logical_date == dttm, - XCom.map_index == map_index, - ) - ) - attributes = [(xcom.key, xcom.value) for xcom in xcom_query if not xcom.key.startswith("_")] - - title = "XCom" - return self.render_template( - "airflow/xcom.html", - show_trigger_form_if_no_params=conf.getboolean("webserver", "show_trigger_form_if_no_params"), - attributes=attributes, - task_id=task_id, - dag_run_id=ti.run_id if ti else "", - task_display_name=ti.task_display_name, - logical_date=logical_date, - map_index=map_index, - form=form, - root=root, - dag=dag, - title=title, - ) - - @expose("/delete", methods=["POST"]) - @auth.has_access_dag("DELETE") - @action_logging - def delete(self): - """Delete DAG.""" - from airflow.api.common import delete_dag - from airflow.exceptions import DagNotFound - - dag_id = request.values.get("dag_id") - origin = get_safe_url(request.values.get("origin")) - redirect_url = get_safe_url(request.values.get("redirect_url")) - - try: - delete_dag.delete_dag(dag_id) - except DagNotFound: - flash(f"DAG with id {dag_id} not found. Cannot delete", "error") - return redirect(redirect_url) - except AirflowException: - flash( - f"Cannot delete DAG with id {dag_id} because some task instances of the DAG " - "are still running. Please mark the task instances as " - "failed/succeeded before deleting the DAG", - "error", - ) - return redirect(redirect_url) - - flash(f"Deleting DAG with id {dag_id}. May take a couple minutes to fully disappear.") - - # Upon success return to origin. - return redirect(origin) - - @expose("/dags//trigger", methods=["POST", "GET"]) - @auth.has_access_dag("POST", DagAccessEntity.RUN) - @action_logging - @provide_session - def trigger(self, dag_id: str, session: Session = NEW_SESSION): - """Triggers DAG Run.""" - run_id = request.values.get("run_id", "") - origin = get_safe_url(request.values.get("origin")) - unpause = request.values.get("unpause") - request_conf = request.values.get("conf") - request_logical_date = request.values.get("logical_date") - request_run_after = request.values.get("run_after", default=timezone.utcnow().isoformat()) - is_dag_run_conf_overrides_params = conf.getboolean("core", "dag_run_conf_overrides_params") - dag = get_airflow_app().dag_bag.get_dag(dag_id) - dag_orm: DagModel = session.scalar(select(DagModel).where(DagModel.dag_id == dag_id).limit(1)) - - # Prepare form fields with param struct details to render a proper form with schema information - form_fields = {} - for k, v in dag.params.items(): - form_fields[k] = v.dump() - form_field: dict = form_fields[k] - # If no schema is provided, auto-detect on default values - if "schema" not in form_field: - form_field["schema"] = {} - form_field_schema: dict = form_field["schema"] - if "type" not in form_field_schema: - form_field_value = form_field["value"] - if isinstance(form_field_value, bool): - form_field_schema["type"] = "boolean" - elif isinstance(form_field_value, int): - form_field_schema["type"] = ["integer", "null"] - elif isinstance(form_field_value, list): - form_field_schema["type"] = ["array", "null"] - elif isinstance(form_field_value, dict): - form_field_schema["type"] = ["object", "null"] - if "description_md" in form_field_schema: - form_field["description"] = wwwutils.wrapped_markdown(form_field_schema["description_md"]) - # Check for default values and pre-populate - if k in request.values: - if form_field_schema.get("type", None) in [ - "boolean", - "array", - ["array", "null"], - "object", - ["object", "null"], - ]: - try: - form_field["value"] = json.loads(request.values.get(k, "")) - except JSONDecodeError: - flash( - f'Could not pre-populate field "{k}" due to parsing error of value "{request.values.get(k)}"' - ) - else: - form_field["value"] = request.values.get(k) - - ui_fields_defined = any("const" not in f["schema"] for f in form_fields.values()) - show_trigger_form_if_no_params = conf.getboolean("webserver", "show_trigger_form_if_no_params") - - if not dag_orm: - flash(f"Cannot find dag {dag_id}") - return redirect(origin) - - if dag_orm.has_import_errors: - flash(f"Cannot create dagruns because the dag {dag_id} has import errors", "error") - return redirect(origin) - - num_recent_confs = conf.getint("webserver", "num_recent_configurations_for_trigger") - recent_runs = session.execute( - select(DagRun.conf, func.max(DagRun.run_id).label("run_id"), func.max(DagRun.logical_date)) - .where( - DagRun.dag_id == dag_id, - DagRun.run_type == DagRunType.MANUAL, - DagRun.conf.isnot(None), - ) - .group_by(DagRun.conf) - .order_by(func.max(DagRun.logical_date).desc()) - .limit(num_recent_confs) - ) - recent_confs = { - run_id: json.dumps(run_conf, cls=utils_json.WebEncoder) - for run_id, run_conf in ((run.run_id, run.conf) for run in recent_runs) - if isinstance(run_conf, dict) and any(run_conf) - } - render_params = { - "dag": dag, - "dag_id": dag_id, - "run_id": run_id, - "origin": origin, - "doc_md": wwwutils.wrapped_markdown(getattr(dag, "doc_md", None)), - "recent_confs": recent_confs, - "is_dag_run_conf_overrides_params": is_dag_run_conf_overrides_params, - } - - if request.method == "GET" or ( - not request_conf and (ui_fields_defined or show_trigger_form_if_no_params) - ): - # Populate conf textarea with conf requests parameter, or dag.params - default_conf = "" - - form = DateTimeForm(data={"logical_date": request_logical_date}) - - if request_conf: - default_conf = request_conf - else: - try: - default_conf = json.dumps( - { - str(k): v.resolve( - value=request.values.get(k, default=NOTSET), suppress_exception=True - ) - for k, v in dag.params.items() - }, - indent=4, - ensure_ascii=False, - cls=utils_json.WebEncoder, - ) - except TypeError: - flash("Could not pre-populate conf field due to non-JSON-serializable data-types") - return self.render_template( - "airflow/trigger.html", - form_fields=form_fields, - **render_params, - conf=default_conf, - form=form, - ) - - try: - logical_date = timezone.parse(request_logical_date, strict=True) if request_logical_date else None - except ParserError: - flash("Invalid logical date", "error") - form = DateTimeForm(data={"logical_date": timezone.utcnow().isoformat()}) - return self.render_template( - "airflow/trigger.html", - form_fields=form_fields, - **render_params, - conf=request_conf or {}, - form=form, - ) - - try: - run_after = timezone.parse(request_run_after, strict=True) - except ParserError: - flash("Invalid run_after", "error") - form = DateTimeForm(data={"run_after": timezone.utcnow().isoformat()}) - return self.render_template( - "airflow/trigger.html", - form_fields=form_fields, - **render_params, - conf=request_conf or {}, - form=form, - ) - - dr = DagRun.find_duplicate(dag_id=dag_id, run_id=run_id, session=session) - if dr: - if dr.run_id == run_id: - message = f"The run ID {run_id} already exists" - else: - message = f"The logical date {logical_date} already exists" - flash(message, "error") - return redirect(origin) - - regex = conf.get("scheduler", "allowed_run_id_pattern") - if run_id and not re2.match(RUN_ID_REGEX, run_id): - if not regex.strip() or not re2.match(regex.strip(), run_id): - flash( - f"The provided run ID '{run_id}' is invalid. It does not match either " - f"the configured pattern: '{regex}' or the built-in pattern: '{RUN_ID_REGEX}'", - "error", - ) - - form = DateTimeForm(data={"logical_date": logical_date}) - return self.render_template( - "airflow/trigger.html", - form_fields=form_fields, - **render_params, - conf=request_conf, - form=form, - ) - - run_conf = {} - if request_conf: - try: - run_conf = json.loads(request_conf) - if not isinstance(run_conf, dict): - flash("Invalid JSON configuration, must be a dict", "error") - form = DateTimeForm(data={"logical_date": logical_date}) - return self.render_template( - "airflow/trigger.html", - form_fields=form_fields, - **render_params, - conf=request_conf, - form=form, - ) - except json.decoder.JSONDecodeError: - flash("Invalid JSON configuration, not parseable", "error") - form = DateTimeForm(data={"logical_date": logical_date}) - return self.render_template( - "airflow/trigger.html", - form_fields=form_fields, - **render_params, - conf=request_conf, - form=form, - ) - - if dag.get_is_paused(): - if unpause or not ui_fields_defined: - flash(f"Unpaused DAG {dag_id}.") - dag_model = models.DagModel.get_dagmodel(dag_id) - if dag_model is not None: - dag_model.set_is_paused(is_paused=False) - else: - flash( - f"DAG {dag_id} is paused, unpause if you want to have the triggered run being executed.", - "warning", - ) - - if logical_date: - data_interval = dag.timetable.infer_manual_data_interval(run_after=logical_date or run_after) - run_after = data_interval.end - else: - data_interval = None - - if not run_id: - run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, - logical_date=logical_date, - run_after=run_after, - ) - - try: - dag_run = dag.create_dagrun( - run_id=run_id, - logical_date=logical_date, - data_interval=data_interval, - run_after=run_after, - conf=run_conf, - run_type=DagRunType.MANUAL, - triggered_by=DagRunTriggeredByType.UI, - dag_version=DagVersion.get_latest_version(dag.dag_id), - state=DagRunState.QUEUED, - session=session, - ) - except (ValueError, ParamValidationError) as ve: - flash(f"{ve}", "error") - form = DateTimeForm(data={"logical_date": logical_date}) - # Take over "bad" submitted fields for new form display - for k in form_fields: - if k in run_conf: - form_fields[k]["value"] = run_conf[k] - return self.render_template( - "airflow/trigger.html", - form_fields=form_fields, - **render_params, - conf=request_conf, - form=form, - ) - - flash(f"Triggered {dag_id} with new Run ID {dag_run.run_id}, it should start any moment now.") - if "/grid?" in origin: - path, query = origin.split("?", 1) - params = {param.split("=")[0]: param.split("=")[1] for param in query.split("&")} - params["dag_run_id"] = dag_run.run_id - origin = f"{path}?{urlencode(params)}" - elif origin.endswith("/grid"): - origin += f"?{urlencode({'dag_run_id': dag_run.run_id})}" - return redirect(origin) - - def _clear_dag_tis( - self, - dag: DAG, - start_date: datetime.datetime | None, - end_date: datetime.datetime | None, - *, - origin: str | None, - task_ids: Collection[str | tuple[str, int]] | None = None, - recursive: bool = False, - confirmed: bool = False, - only_failed: bool = False, - session: Session, - ): - if confirmed: - count = dag.clear( - start_date=start_date, - end_date=end_date, - task_ids=task_ids, - only_failed=only_failed, - session=session, - ) - - msg = f"{count} task instances have been cleared" - return redirect_or_json(origin, msg) - - try: - tis = dag.clear( - start_date=start_date, - end_date=end_date, - task_ids=task_ids, - only_failed=only_failed, - dry_run=True, - session=session, - ) - except AirflowException as ex: - return redirect_or_json(origin, msg=str(ex), status="error", status_code=500) - - if not isinstance(tis, collections.abc.Iterable): - raise AssertionError( - f"Expected dag.clear() to return an iterable for dry runs, got {tis} instead." - ) - - details = [str(t) for t in tis] - - if not details: - return redirect_or_json(origin, "No task instances to clear", status="error", status_code=404) - elif request.headers.get("Accept") == "application/json": - if confirmed: - return htmlsafe_json_dumps(details, separators=(",", ":")) - return htmlsafe_json_dumps( - [{"task_id": ti.task_id, "map_index": ti.map_index, "run_id": ti.run_id} for ti in tis], - separators=(",", ":"), - ) - return self.render_template( - "airflow/confirm.html", - endpoint=None, - message="Task instances you are about to clear:", - details="\n".join(details), - ) - - @expose("/clear", methods=["POST"]) - @auth.has_access_dag("PUT", DagAccessEntity.TASK_INSTANCE) - @action_logging - @provide_session - def clear(self, *, session: Session = NEW_SESSION): - """Clear DAG tasks.""" - dag_id = request.form.get("dag_id") - task_id = request.form.get("task_id") - origin = get_safe_url(request.form.get("origin")) - dag = get_airflow_app().dag_bag.get_dag(dag_id) - group_id = request.form.get("group_id") - - if "map_index" not in request.form: - map_indexes: list[int] | None = None - else: - map_indexes = request.form.getlist("map_index", type=int) - - logical_date_str = request.form.get("logical_date") - logical_date = _safe_parse_datetime(logical_date_str) - confirmed = request.form.get("confirmed") == "true" - upstream = request.form.get("upstream") == "true" - downstream = request.form.get("downstream") == "true" - future = request.form.get("future") == "true" - past = request.form.get("past") == "true" - recursive = request.form.get("recursive") == "true" - only_failed = request.form.get("only_failed") == "true" - - task_ids: list[str | tuple[str, int]] = [] - - end_date = logical_date if not future else None - start_date = logical_date if not past else None - - locked_dag_run_ids: list[int] = [] - - if group_id is not None: - task_group_dict = dag.task_group.get_task_group_dict() - task_group = task_group_dict.get(group_id) - if task_group is None: - return redirect_or_json( - origin, msg=f"TaskGroup {group_id} could not be found", status="error", status_code=404 - ) - task_ids = task_ids_or_regex = [t.task_id for t in task_group.iter_tasks()] - - # Lock the related dag runs to prevent from possible dead lock. - # https://github.com/apache/airflow/pull/26658 - dag_runs_query = select(DagRun.id).where(DagRun.dag_id == dag_id).with_for_update() - - if start_date is None and end_date is None: - dag_runs_query = dag_runs_query.where(DagRun.logical_date == start_date) - else: - if start_date is not None: - dag_runs_query = dag_runs_query.where(DagRun.logical_date >= start_date) - - if end_date is not None: - dag_runs_query = dag_runs_query.where(DagRun.logical_date <= end_date) - - locked_dag_run_ids = session.scalars(dag_runs_query).all() - elif task_id: - if map_indexes is None: - task_ids = [task_id] - else: - task_ids = [(task_id, map_index) for map_index in map_indexes] - task_ids_or_regex = [task_id] - - dag = dag.partial_subset( - task_ids_or_regex=task_ids_or_regex, - include_downstream=downstream, - include_upstream=upstream, - ) - - if len(dag.task_dict) > 1: - # If we had upstream/downstream etc then also include those! - task_ids.extend(tid for tid in dag.task_dict if tid != task_id) - - response = self._clear_dag_tis( - dag, - start_date, - end_date, - origin=origin, - task_ids=task_ids, - recursive=recursive, - confirmed=confirmed, - only_failed=only_failed, - session=session, - ) - - del locked_dag_run_ids - - return response - - @expose("/dagrun_clear", methods=["POST"]) - @auth.has_access_dag("PUT", DagAccessEntity.TASK_INSTANCE) - @action_logging - @provide_session - def dagrun_clear(self, *, session: Session = NEW_SESSION): - """Clear the DagRun.""" - dag_id = request.form.get("dag_id") - dag_run_id = request.form.get("dag_run_id") - confirmed = request.form.get("confirmed") == "true" - only_failed = request.form.get("only_failed") == "true" - - dag = get_airflow_app().dag_bag.get_dag(dag_id) - dr = dag.get_dagrun(run_id=dag_run_id, session=session) - start_date = dr.logical_date - end_date = dr.logical_date - - return self._clear_dag_tis( - dag, - start_date, - end_date, - origin=None, - recursive=True, - confirmed=confirmed, - only_failed=only_failed, - session=session, - ) - - @expose("/blocked", methods=["POST"]) - @auth.has_access_dag("GET", DagAccessEntity.RUN) - @provide_session - def blocked(self, session: Session = NEW_SESSION): - """Retrieve active_dag_runs and max_active_runs information for running Dags.""" - allowed_dag_ids = get_auth_manager().get_permitted_dag_ids(user=g.user) - - # Filter by post parameters - selected_dag_ids = {unquote(dag_id) for dag_id in request.form.getlist("dag_ids") if dag_id} - - if selected_dag_ids: - filter_dag_ids = selected_dag_ids.intersection(allowed_dag_ids) - else: - filter_dag_ids = allowed_dag_ids - - if not filter_dag_ids: - return flask.json.jsonify([]) - - dags = session.execute( - select(DagRun.dag_id, sqla.func.count(DagRun.id)) - .where(DagRun.state == DagRunState.RUNNING) - .where(DagRun.dag_id.in_(filter_dag_ids)) - .group_by(DagRun.dag_id) - ) - - payload = [] - for dag_id, active_dag_runs in dags: - max_active_runs = 0 - dag = get_airflow_app().dag_bag.get_dag(dag_id) - if dag: - # TODO: Make max_active_runs a column so we can query for it directly - max_active_runs = dag.max_active_runs - payload.append( - { - "dag_id": dag_id, - "active_dag_run": active_dag_runs, - "max_active_runs": max_active_runs, - } - ) - return flask.json.jsonify(payload) - - def _mark_dagrun_state_as_failed(self, dag_id, dag_run_id, confirmed): - if not dag_run_id: - return {"status": "error", "message": "Invalid dag_run_id"} - - dag = get_airflow_app().dag_bag.get_dag(dag_id) - - if not dag: - return {"status": "error", "message": f"Cannot find DAG: {dag_id}"} - - new_dag_state = set_dag_run_state_to_failed(dag=dag, run_id=dag_run_id, commit=confirmed) - - if confirmed: - return {"status": "success", "message": f"Marked failed on {len(new_dag_state)} task instances"} - else: - details = [str(t) for t in new_dag_state] - - return htmlsafe_json_dumps(details, separators=(",", ":")) - - def _mark_dagrun_state_as_success(self, dag_id, dag_run_id, confirmed): - if not dag_run_id: - return {"status": "error", "message": "Invalid dag_run_id"} - - dag = get_airflow_app().dag_bag.get_dag(dag_id) - - if not dag: - return {"status": "error", "message": f"Cannot find DAG: {dag_id}"} - - new_dag_state = set_dag_run_state_to_success(dag=dag, run_id=dag_run_id, commit=confirmed) - - if confirmed: - return {"status": "success", "message": f"Marked success on {len(new_dag_state)} task instances"} - else: - details = [str(t) for t in new_dag_state] - - return htmlsafe_json_dumps(details, separators=(",", ":")) - - @provide_session - def _mark_dagrun_state_as_queued( - self, - dag_id: str, - dag_run_id: str, - confirmed: bool, - session: Session = NEW_SESSION, - ): - if not dag_run_id: - return {"status": "error", "message": "Invalid dag_run_id"} - - dag = get_airflow_app().dag_bag.get_dag(dag_id) - - if not dag: - return {"status": "error", "message": f"Cannot find DAG: {dag_id}"} - - set_dag_run_state_to_queued(dag=dag, run_id=dag_run_id, commit=confirmed) - - if confirmed: - return {"status": "success", "message": "Marked the DagRun as queued."} - - else: - # Identify tasks that will be queued up to run when confirmed - all_task_ids = [task.task_id for task in dag.tasks] - - existing_tis = session.execute( - select(TaskInstance.task_id).where( - TaskInstance.dag_id == dag.dag_id, - TaskInstance.run_id == dag_run_id, - ) - ) - - completed_tis_ids = [task_id for (task_id,) in existing_tis] - tasks_with_no_state = list(set(all_task_ids) - set(completed_tis_ids)) - details = [str(t) for t in tasks_with_no_state] - - return htmlsafe_json_dumps(details, separators=(",", ":")) - - @expose("/dagrun_failed", methods=["POST"]) - @auth.has_access_dag("PUT", DagAccessEntity.RUN) - @action_logging - def dagrun_failed(self): - """Mark DagRun failed.""" - dag_id = request.form.get("dag_id") - dag_run_id = request.form.get("dag_run_id") - confirmed = request.form.get("confirmed") == "true" - return self._mark_dagrun_state_as_failed(dag_id, dag_run_id, confirmed) - - @expose("/dagrun_success", methods=["POST"]) - @auth.has_access_dag("PUT", DagAccessEntity.RUN) - @action_logging - def dagrun_success(self): - """Mark DagRun success.""" - dag_id = request.form.get("dag_id") - dag_run_id = request.form.get("dag_run_id") - confirmed = request.form.get("confirmed") == "true" - return self._mark_dagrun_state_as_success(dag_id, dag_run_id, confirmed) - - @expose("/dagrun_queued", methods=["POST"]) - @auth.has_access_dag("PUT", DagAccessEntity.RUN) - @action_logging - def dagrun_queued(self): - """Queue DagRun so tasks that haven't run yet can be started.""" - dag_id = request.form.get("dag_id") - dag_run_id = request.form.get("dag_run_id") - confirmed = request.form.get("confirmed") == "true" - return self._mark_dagrun_state_as_queued(dag_id, dag_run_id, confirmed) - - @expose("/dagrun_details") - def dagrun_details(self): - """Redirect to the Grid DagRun page. This is avoids breaking links.""" - dag_id = request.args.get("dag_id") - run_id = request.args.get("run_id") - return redirect(url_for("Airflow.grid", dag_id=dag_id, dag_run_id=run_id)) - - def _mark_task_instance_state( - self, - *, - dag_id: str, - run_id: str, - task_id: str, - map_indexes: list[int] | None, - origin: str, - upstream: bool, - downstream: bool, - future: bool, - past: bool, - state: TaskInstanceState, - ): - dag: DAG = get_airflow_app().dag_bag.get_dag(dag_id) - - if not run_id: - flash(f"Cannot mark tasks as {state}, seem that DAG {dag_id} has never run", "error") - return redirect(origin) - - altered = dag.set_task_instance_state( - task_id=task_id, - map_indexes=map_indexes, - run_id=run_id, - state=state, - upstream=upstream, - downstream=downstream, - future=future, - past=past, - ) - - flash(f"Marked {state} on {len(altered)} task instances") - return redirect(origin) - - def _mark_task_group_state( - self, - *, - dag_id: str, - run_id: str, - group_id: str, - origin: str, - upstream: bool, - downstream: bool, - future: bool, - past: bool, - state: TaskInstanceState, - ): - dag: DAG = get_airflow_app().dag_bag.get_dag(dag_id) - - if not run_id: - flash(f"Cannot mark tasks as {state}, as DAG {dag_id} has never run", "error") - return redirect(origin) - - altered = dag.set_task_group_state( - group_id=group_id, - run_id=run_id, - state=state, - upstream=upstream, - downstream=downstream, - future=future, - past=past, - ) - - flash(f"Marked {state} on {len(altered)} task instances") - return redirect(origin) - - @expose("/confirm", methods=["GET"]) - @auth.has_access_dag("PUT", DagAccessEntity.TASK_INSTANCE) - @action_logging - def confirm(self): - """Show confirmation page for marking tasks as success or failed.""" - args = request.args - dag_id = args.get("dag_id") - task_id = args.get("task_id") - dag_run_id = args.get("dag_run_id") - state = args.get("state") - origin = get_safe_url(args.get("origin")) - group_id = args.get("group_id") - - if "map_index" not in args: - map_indexes: list[int] | None = None - else: - map_indexes = args.getlist("map_index", type=int) - - upstream = to_boolean(args.get("upstream")) - downstream = to_boolean(args.get("downstream")) - future = to_boolean(args.get("future")) - past = to_boolean(args.get("past")) - origin = origin or url_for("Airflow.index") - - if not exactly_one(task_id, group_id): - raise ValueError("Exactly one of task_id or group_id must be provided") - - dag = get_airflow_app().dag_bag.get_dag(dag_id) - if not dag: - msg = f"DAG {dag_id} not found" - return redirect_or_json(origin, msg, status="error", status_code=404) - - if state not in ( - "success", - "failed", - ): - msg = f"Invalid state {state}, must be either 'success' or 'failed'" - return redirect_or_json(origin, msg, status="error", status_code=400) - - latest_logical_date = dag.get_latest_logical_date() - if not latest_logical_date: - msg = f"Cannot mark tasks as {state}, seem that dag {dag_id} has never run" - return redirect_or_json(origin, msg, status="error", status_code=400) - - tasks: list[Operator | tuple[Operator, int]] = [] - - if group_id: - task_group_dict = dag.task_group.get_task_group_dict() - task_group = task_group_dict.get(group_id) - if task_group is None: - return redirect_or_json( - origin, msg=f"TaskGroup {group_id} could not be found", status="error", status_code=404 - ) - tasks = list(task_group.iter_tasks()) - elif task_id: - try: - task = dag.get_task(task_id) - except airflow.exceptions.TaskNotFound: - msg = f"Task {task_id} not found" - return redirect_or_json(origin, msg, status="error", status_code=404) - task.dag = dag - if map_indexes is None: - tasks = [task] - else: - tasks = [(task, map_index) for map_index in map_indexes] - - to_be_altered = set_state( - tasks=tasks, - run_id=dag_run_id, - upstream=upstream, - downstream=downstream, - future=future, - past=past, - state=state, - commit=False, - ) - - if request.headers.get("Accept") == "application/json": - return htmlsafe_json_dumps( - [ - {"task_id": ti.task_id, "map_index": ti.map_index, "run_id": ti.run_id} - for ti in to_be_altered - ], - separators=(",", ":"), - ) - - details = "\n".join(str(t) for t in to_be_altered) - - response = self.render_template( - "airflow/confirm.html", - endpoint=url_for(f"Airflow.{state}"), - message=f"Task instances you are about to mark as {state}:", - details=details, - ) - - return response - - @expose("/failed", methods=["POST"]) - @auth.has_access_dag("PUT", DagAccessEntity.TASK_INSTANCE) - @action_logging - def failed(self): - """Mark task or task_group as failed.""" - args = request.form - dag_id = args.get("dag_id") - task_id = args.get("task_id") - run_id = args.get("dag_run_id") - group_id = args.get("group_id") - - if not exactly_one(task_id, group_id): - raise ValueError("Exactly one of task_id or group_id must be provided") - - if "map_index" not in args: - map_indexes: list[int] | None = None - else: - map_indexes = args.getlist("map_index", type=int) - - origin = get_safe_url(args.get("origin")) - upstream = to_boolean(args.get("upstream")) - downstream = to_boolean(args.get("downstream")) - future = to_boolean(args.get("future")) - past = to_boolean(args.get("past")) - - if task_id: - return self._mark_task_instance_state( - dag_id=dag_id, - run_id=run_id, - task_id=task_id, - map_indexes=map_indexes, - origin=origin, - upstream=upstream, - downstream=downstream, - future=future, - past=past, - state=TaskInstanceState.FAILED, - ) - elif group_id: - return self._mark_task_group_state( - dag_id=dag_id, - run_id=run_id, - group_id=group_id, - origin=origin, - upstream=upstream, - downstream=downstream, - future=future, - past=past, - state=TaskInstanceState.FAILED, - ) - - @expose("/success", methods=["POST"]) - @auth.has_access_dag("PUT", DagAccessEntity.TASK_INSTANCE) - @action_logging - def success(self): - """Mark task or task_group as success.""" - args = request.form - dag_id = args.get("dag_id") - task_id = args.get("task_id") - run_id = args.get("dag_run_id") - group_id = args.get("group_id") - - if not exactly_one(task_id, group_id): - raise ValueError("Exactly one of task_id or group_id must be provided") - - if "map_index" not in args: - map_indexes: list[int] | None = None - else: - map_indexes = args.getlist("map_index", type=int) - - origin = get_safe_url(args.get("origin")) - upstream = to_boolean(args.get("upstream")) - downstream = to_boolean(args.get("downstream")) - future = to_boolean(args.get("future")) - past = to_boolean(args.get("past")) - - if task_id: - return self._mark_task_instance_state( - dag_id=dag_id, - run_id=run_id, - task_id=task_id, - map_indexes=map_indexes, - origin=origin, - upstream=upstream, - downstream=downstream, - future=future, - past=past, - state=TaskInstanceState.SUCCESS, - ) - elif group_id: - return self._mark_task_group_state( - dag_id=dag_id, - run_id=run_id, - group_id=group_id, - origin=origin, - upstream=upstream, - downstream=downstream, - future=future, - past=past, - state=TaskInstanceState.SUCCESS, - ) - - @expose("/dags/") - def dag(self, dag_id): - """Redirect to default DAG view.""" - kwargs = {**sanitize_args(request.args), "dag_id": dag_id} - return redirect(url_for("Airflow.grid", **kwargs)) - - @expose("/tree") - def legacy_tree(self): - """Redirect to the replacement - grid view. Kept for backwards compatibility.""" - return redirect(url_for("Airflow.grid", **sanitize_args(request.args))) - - @expose("/dags//grid") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - @gzipped - @provide_session - def grid(self, dag_id: str, session: Session = NEW_SESSION): - """Get Dag's grid view.""" - color_log_error_keywords = conf.get("logging", "color_log_error_keywords", fallback="") - color_log_warning_keywords = conf.get("logging", "color_log_warning_keywords", fallback="") - - dag = get_airflow_app().dag_bag.get_dag(dag_id, session=session) - url_serializer = URLSafeSerializer(current_app.config["SECRET_KEY"]) - dag_model = DagModel.get_dagmodel(dag_id, session=session) - if not dag or not dag_model: - flash(f'DAG "{dag_id}" seems to be missing from DagBag.', "error") - return redirect(url_for("Airflow.index")) - wwwutils.check_import_errors(dag.fileloc, dag_model.bundle_name, session) - wwwutils.check_dag_warnings(dag.dag_id, session) - - included_events_raw = conf.get("webserver", "audit_view_included_events", fallback="") - excluded_events_raw = conf.get("webserver", "audit_view_excluded_events", fallback="") - - root = request.args.get("root") - if root: - dag = dag.partial_subset(task_ids_or_regex=root, include_downstream=False, include_upstream=True) - - num_runs = request.args.get("num_runs", type=int) - if num_runs is None: - num_runs = conf.getint("webserver", "default_dag_run_display_number") - - doc_md = wwwutils.wrapped_markdown(getattr(dag, "doc_md", None)) - - task_log_reader = TaskLogReader() - if task_log_reader.supports_external_link: - external_log_name = task_log_reader.log_handler.log_name - else: - external_log_name = None - - default_dag_run_display_number = conf.getint("webserver", "default_dag_run_display_number") - - num_runs_options = [5, 25, 50, 100, 365] - - if default_dag_run_display_number not in num_runs_options: - insort_left(num_runs_options, default_dag_run_display_number) - - can_edit_taskinstance = get_auth_manager().is_authorized_dag( - method="PUT", - access_entity=DagAccessEntity.TASK_INSTANCE, - user=g.user, - ) - - return self.render_template( - "airflow/grid.html", - show_trigger_form_if_no_params=conf.getboolean("webserver", "show_trigger_form_if_no_params"), - root=root, - dag=dag, - doc_md=doc_md, - num_runs=num_runs, - can_edit_taskinstance=can_edit_taskinstance, - show_external_log_redirect=task_log_reader.supports_external_link, - external_log_name=external_log_name, - dag_model=dag_model, - auto_refresh_interval=conf.getint("webserver", "auto_refresh_interval"), - default_dag_run_display_number=default_dag_run_display_number, - default_wrap=conf.getboolean("webserver", "default_wrap"), - filters_drop_down_values=htmlsafe_json_dumps( - { - "taskStates": [state.value for state in TaskInstanceState], - "dagStates": [state.value for state in State.dag_states], - "runTypes": [run_type.value for run_type in DagRunType], - "numRuns": num_runs_options, - } - ), - included_events_raw=included_events_raw, - excluded_events_raw=excluded_events_raw, - color_log_error_keywords=color_log_error_keywords, - color_log_warning_keywords=color_log_warning_keywords, - dag_file_token=url_serializer.dumps(dag.fileloc), - ) - - @expose("/calendar") - def legacy_calendar(self): - """Redirect from url param.""" - return redirect(url_for("Airflow.calendar", **sanitize_args(request.args))) - - @expose("/dags//calendar") - def calendar(self, dag_id: str): - """Redirect to the replacement - grid + calendar. Kept for backwards compatibility.""" - kwargs = {**sanitize_args(request.args), "dag_id": dag_id, "tab": "calendar"} - - return redirect(url_for("Airflow.grid", **kwargs)) - - @expose("/object/calendar_data") - @auth.has_access_dag("GET", DagAccessEntity.RUN) - @gzipped - @provide_session - def calendar_data(self, session: Session = NEW_SESSION): - """Get DAG runs as calendar.""" - dag_id = request.args.get("dag_id") - dag = get_airflow_app().dag_bag.get_dag(dag_id, session=session) - if not dag: - return {"error": f"can't find dag {dag_id}"}, 404 - - dag_states = session.execute( - select( - func.date(DagRun.logical_date).label("date"), - DagRun.state, - func.max(DagRun.data_interval_start).label("data_interval_start"), - func.max(DagRun.data_interval_end).label("data_interval_end"), - func.count("*").label("count"), - ) - .where(DagRun.dag_id == dag.dag_id) - .group_by(func.date(DagRun.logical_date), DagRun.state) - .order_by(func.date(DagRun.logical_date).asc()) - ).all() - - data_dag_states = [ - { - # DATE() in SQLite and MySQL behave differently: - # SQLite returns a string, MySQL returns a date. - "date": dr.date if isinstance(dr.date, str) else dr.date.isoformat(), - "state": dr.state, - "count": dr.count, - } - for dr in dag_states - ] - - # Upper limit of how many planned runs we should iterate through - max_planned_runs = 2000 - total_planned = 0 - - # Interpret the schedule and show planned dag runs in calendar - if ( - dag_states - and dag_states[-1].data_interval_start - and dag_states[-1].data_interval_end - and not isinstance(dag.timetable, ContinuousTimetable) - ): - last_automated_data_interval = DataInterval( - timezone.coerce_datetime(dag_states[-1].data_interval_start), - timezone.coerce_datetime(dag_states[-1].data_interval_end), - ) - - year = last_automated_data_interval.end.year - restriction = TimeRestriction(dag.start_date, dag.end_date, False) - dates: dict[datetime.date, int] = collections.Counter() - - if isinstance(dag.timetable, CronMixin): - # Optimized calendar generation for timetables based on a cron expression. - dates_iter: Iterator[datetime.datetime | None] = croniter( - dag.timetable._expression, - start_time=last_automated_data_interval.end, - ret_type=datetime.datetime, - ) - for dt in dates_iter: - if dt is None: - break - if dt.year != year: - break - if dag.end_date and dt > dag.end_date: - break - dates[dt.date()] += 1 - else: - prev_logical_date = DateTime.min - while True: - curr_info = dag.timetable.next_dagrun_info( - last_automated_data_interval=last_automated_data_interval, - restriction=restriction, - ) - if curr_info is None: - break # Reached the end. - if curr_info.logical_date <= prev_logical_date: - break # We're not progressing. Maybe a malformed timetable? Give up. - if curr_info.logical_date.year != year: - break # Crossed the year boundary. - last_automated_data_interval = curr_info.data_interval - dates[curr_info.logical_date.date()] += 1 - prev_logical_date = curr_info.logical_date - total_planned += 1 - if total_planned > max_planned_runs: - break - - data_dag_states.extend( - {"date": date.isoformat(), "state": "planned", "count": count} - for (date, count) in dates.items() - ) - - data = { - "dag_states": data_dag_states, - } - - return ( - htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), - {"Content-Type": "application/json; charset=utf-8"}, - ) - - @expose("/graph") - def legacy_graph(self): - """Redirect from url param.""" - return redirect(url_for("Airflow.graph", **sanitize_args(request.args))) - - @expose("/dags//graph") - @gzipped - @provide_session - def graph(self, dag_id: str, session: Session = NEW_SESSION): - """Redirect to the replacement - grid + graph. Kept for backwards compatibility.""" - dag = get_airflow_app().dag_bag.get_dag(dag_id, session=session) - if not dag: - flash(f'DAG "{dag_id}" seems to be missing from DagBag.', "error") - return redirect(url_for("Airflow.index")) - dt_nr_dr_data = get_date_time_num_runs_dag_runs_form_data(request, session, dag) - run_id = session.scalar( - select(DagRun.run_id) - .where(DagRun.logical_date == dt_nr_dr_data["dttm"]) - .order_by(DagRun.id.desc()) - .limit(1) - ) - dag_run = dag.get_dagrun( - run_id=run_id, - session=session, - ) - dag_run_id = dag_run.run_id if dag_run else None - - kwargs = { - **sanitize_args(request.args), - "dag_id": dag_id, - "tab": "graph", - "dag_run_id": dag_run_id, - } - - return redirect(url_for("Airflow.grid", **kwargs)) - - @expose("/duration") - def legacy_duration(self): - """Redirect from url param.""" - return redirect(url_for("Airflow.duration", **sanitize_args(request.args))) - - @expose("/dags//duration") - def duration(self, dag_id: str): - """Redirect to Grid view.""" - return redirect(url_for("Airflow.grid", dag_id=dag_id)) - - @expose("/tries") - def legacy_tries(self): - """Redirect from url param.""" - return redirect(url_for("Airflow.tries", **sanitize_args(request.args))) - - @expose("/dags//tries") - def tries(self, dag_id: str): - """Redirect to grid view.""" - kwargs = { - **sanitize_args(request.args), - "dag_id": dag_id, - } - return redirect(url_for("Airflow.grid", **kwargs)) - - @expose("/landing_times") - def legacy_landing_times(self): - """Redirect from url param.""" - return redirect(url_for("Airflow.landing_times", **sanitize_args(request.args))) - - @expose("/dags//landing-times") - def landing_times(self, dag_id: str): - """Redirect to run duration page.""" - kwargs = { - **sanitize_args(request.args), - "dag_id": dag_id, - "tab": "run_duration", - } - - return redirect(url_for("Airflow.grid", **kwargs)) - - @expose("/paused", methods=["POST"]) - @auth.has_access_dag("PUT") - @action_logging - def paused(self): - """Toggle paused.""" - dag_id = request.args.get("dag_id") - is_paused = request.args.get("is_paused") == "false" - models.DagModel.get_dagmodel(dag_id).set_is_paused(is_paused=is_paused) - return "OK" - - @expose("/gantt") - def legacy_gantt(self): - """Redirect from url param.""" - return redirect(url_for("Airflow.gantt", **sanitize_args(request.args))) - - @expose("/dags//gantt") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - @provide_session - def gantt(self, dag_id: str, session: Session = NEW_SESSION): - """Redirect to the replacement - grid + gantt. Kept for backwards compatibility.""" - dag = get_airflow_app().dag_bag.get_dag(dag_id, session=session) - dt_nr_dr_data = get_date_time_num_runs_dag_runs_form_data(request, session, dag) - dttm = dt_nr_dr_data["dttm"] - run_id = session.scalar( - select(DagRun.run_id).where(DagRun.logical_date == dttm).order_by(DagRun.id.desc()).limit(1) - ) - dag_run = dag.get_dagrun( - run_id=run_id, - session=session, - ) - dag_run_id = dag_run.run_id if dag_run else None - - kwargs = {**sanitize_args(request.args), "dag_id": dag_id, "tab": "gantt", "dag_run_id": dag_run_id} - - return redirect(url_for("Airflow.grid", **kwargs)) - - @expose("/extra_links") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - @provide_session - def extra_links(self, *, session: Session = NEW_SESSION): - """ - Return external links for a given Operator. - - It queries the operator that sent the request for the links it wishes - to provide for a given external link name. - - API: GET - Args: dag_id: The id of the dag containing the task in question - task_id: The id of the task in question - logical_date: The date of execution of the task - link_name: The name of the link reference to find the actual URL for - - Returns: - 200: {url: , error: None} - returned when there was no problem - finding the URL - 404: {url: None, error: } - returned when the operator does - not return a URL - """ - dag_id = request.args.get("dag_id") - task_id = request.args.get("task_id") - map_index = request.args.get("map_index", -1, type=int) - logical_date = request.args.get("logical_date") - dttm = _safe_parse_datetime(logical_date) - dag = get_airflow_app().dag_bag.get_dag(dag_id) - - if not dag or task_id not in dag.task_ids: - return {"url": None, "error": f"can't find dag {dag} or task_id {task_id}"}, 404 - - task = dag.get_task(task_id) - link_name = request.args.get("link_name") - if link_name is None: - return {"url": None, "error": "Link name not passed"}, 400 - - ti = session.scalar( - select(TaskInstance) - .filter_by(dag_id=dag_id, task_id=task_id, logical_date=dttm, map_index=map_index) - .options(joinedload(TaskInstance.dag_run)) - .limit(1) - ) - - if not ti: - return {"url": None, "error": "Task Instances not found"}, 404 - try: - url = task.get_extra_links(ti, link_name) - except ValueError as err: - return {"url": None, "error": str(err)}, 404 - if url: - return {"error": None, "url": url} - else: - return {"url": None, "error": f"No URL found for {link_name}"}, 404 - - @mark_fastapi_migration_done - @expose("/object/graph_data") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - @gzipped - def graph_data(self): - """Get Graph Data.""" - dag_id = request.args.get("dag_id") - dag = get_airflow_app().dag_bag.get_dag(dag_id) - root = request.args.get("root") - if root: - filter_upstream = request.args.get("filter_upstream") == "true" - filter_downstream = request.args.get("filter_downstream") == "true" - dag = dag.partial_subset( - task_ids_or_regex=root, include_upstream=filter_upstream, include_downstream=filter_downstream - ) - - nodes = task_group_to_dict_legacy(dag.task_group) - edges = dag_edges(dag) - - data = { - "arrange": dag.orientation, - "nodes": nodes, - "edges": edges, - } - return ( - htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), - {"Content-Type": "application/json; charset=utf-8"}, - ) - - @expose("/object/task_instances") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - def task_instances(self): - """Show task instances.""" - dag_id = request.args.get("dag_id") - dag = get_airflow_app().dag_bag.get_dag(dag_id) - - dttm = request.args.get("logical_date") - if dttm: - dttm = _safe_parse_datetime(dttm) - else: - return {"error": f"Invalid logical_date {dttm}"}, 400 - - with create_session() as session: - task_instances = { - ti.task_id: wwwutils.get_instance_with_map(ti, session) - for ti in dag.get_task_instances(dttm, dttm) - } - - return flask.json.jsonify(task_instances) - - @mark_fastapi_migration_done - @expose("/object/grid_data") - @auth.has_access_dag("GET", DagAccessEntity.TASK_INSTANCE) - def grid_data(self): - """Return grid data.""" - dag_id = request.args.get("dag_id") - dag = get_airflow_app().dag_bag.get_dag(dag_id) - - if not dag: - return {"error": f"can't find dag {dag_id}"}, 404 - - root = request.args.get("root") - if root: - filter_upstream = request.args.get("filter_upstream") == "true" - filter_downstream = request.args.get("filter_downstream") == "true" - dag = dag.partial_subset( - task_ids_or_regex=root, include_upstream=filter_upstream, include_downstream=filter_downstream - ) - - num_runs = request.args.get("num_runs", type=int) - if num_runs is None: - num_runs = conf.getint("webserver", "default_dag_run_display_number") - - try: - base_date = timezone.parse(request.args["base_date"], strict=True) - except (KeyError, ValueError): - base_date = dag.get_latest_logical_date() or timezone.utcnow() - - with create_session() as session: - query = select(DagRun).where(DagRun.dag_id == dag.dag_id, DagRun.logical_date <= base_date) - - run_types = request.args.getlist("run_type") - if run_types: - query = query.where(DagRun.run_type.in_(run_types)) - - run_states = request.args.getlist("run_state") - if run_states: - query = query.where(DagRun.state.in_(run_states)) - - # Retrieve, sort and encode the previous DAG Runs - dag_runs = wwwutils.sorted_dag_runs( - query, ordering=dag.timetable.run_ordering, limit=num_runs, session=session - ) - encoded_runs = [] - encoding_errors = [] - for dr in dag_runs: - encoded_dr, error = wwwutils.encode_dag_run(dr, json_encoder=utils_json.WebEncoder) - if error: - encoding_errors.append(error) - else: - encoded_runs.append(encoded_dr) - - data = { - "groups": dag_to_grid(dag, dag_runs, session), - "dag_runs": encoded_runs, - "ordering": dag.timetable.run_ordering, - "errors": encoding_errors, - } - # avoid spaces to reduce payload size - return ( - htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), - {"Content-Type": "application/json; charset=utf-8"}, - ) - - @expose("/object/historical_metrics_data") - @auth.has_access_view(AccessView.CLUSTER_ACTIVITY) - @mark_fastapi_migration_done - def historical_metrics_data(self): - """Return cluster activity historical metrics.""" - start_date = _safe_parse_datetime(request.args.get("start_date")) - end_date = _safe_parse_datetime(request.args.get("end_date")) - - with create_session() as session: - # DagRuns - dag_run_types = session.execute( - select(DagRun.run_type, func.count(DagRun.run_id)) - .where( - DagRun.start_date >= start_date, - func.coalesce(DagRun.end_date, timezone.utcnow()) <= end_date, - ) - .group_by(DagRun.run_type) - ).all() - - dag_run_states = session.execute( - select(DagRun.state, func.count(DagRun.run_id)) - .where( - DagRun.start_date >= start_date, - func.coalesce(DagRun.end_date, timezone.utcnow()) <= end_date, - ) - .group_by(DagRun.state) - ).all() - - # TaskInstances - task_instance_states = session.execute( - select(TaskInstance.state, func.count(TaskInstance.run_id)) - .join(TaskInstance.dag_run) - .where( - DagRun.start_date >= start_date, - func.coalesce(DagRun.end_date, timezone.utcnow()) <= end_date, - ) - .group_by(TaskInstance.state) - ).all() - - data = { - "dag_run_types": { - **{dag_run_type.value: 0 for dag_run_type in DagRunType}, - **dict(dag_run_types), - }, - "dag_run_states": { - **{dag_run_state.value: 0 for dag_run_state in DagRunState}, - **dict(dag_run_states), - }, - "task_instance_states": { - "no_status": 0, - **{ti_state.value: 0 for ti_state in TaskInstanceState}, - **{ti_state or "no_status": sum_value for ti_state, sum_value in task_instance_states}, - }, - } - - return ( - htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), - {"Content-Type": "application/json; charset=utf-8"}, - ) - - @expose("/object/next_run_assets/") - @auth.has_access_dag("GET", DagAccessEntity.RUN) - @auth.has_access_asset("GET") - @mark_fastapi_migration_done - def next_run_assets(self, dag_id): - """Return assets necessary, and their status, for the next dag run.""" - dag = get_airflow_app().dag_bag.get_dag(dag_id) - if not dag: - return {"error": f"can't find dag {dag_id}"}, 404 - - with create_session() as session: - dag_model = DagModel.get_dagmodel(dag_id, session=session) - - latest_run = dag_model.get_last_dagrun(session=session) - - events = [ - dict(info._mapping) - for info in session.execute( - select( - AssetModel.id, - AssetModel.uri, - func.max(AssetEvent.timestamp).label("lastUpdate"), - ) - .join(DagScheduleAssetReference, DagScheduleAssetReference.asset_id == AssetModel.id) - .join( - AssetDagRunQueue, - and_( - AssetDagRunQueue.asset_id == AssetModel.id, - AssetDagRunQueue.target_dag_id == DagScheduleAssetReference.dag_id, - ), - isouter=True, - ) - .join( - AssetEvent, - and_( - AssetEvent.asset_id == AssetModel.id, - ( - AssetEvent.timestamp >= latest_run.logical_date - if latest_run and latest_run.logical_date - else True - ), - ), - isouter=True, - ) - .where(DagScheduleAssetReference.dag_id == dag_id, AssetModel.active.has()) - .group_by(AssetModel.id, AssetModel.uri) - .order_by(AssetModel.uri) - ) - ] - data = {"asset_expression": dag_model.asset_expression, "events": events} - return ( - htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), - {"Content-Type": "application/json; charset=utf-8"}, - ) - - @expose("/object/asset_dependencies") - @auth.has_access_dag("GET", DagAccessEntity.DEPENDENCIES) - def asset_dependencies(self): - """Return asset dependencies graph.""" - nodes_dict: dict[str, Any] = {} - edge_tuples: set[dict[str, str]] = set() - - for dag, dependencies in SerializedDagModel.get_dag_dependencies().items(): - dag_node_id = f"dag:{dag}" - if dag_node_id not in nodes_dict: - for dep in dependencies: - if dep.dependency_type in ("dag", "asset", "asset-alias"): - # add node - nodes_dict[dag_node_id] = node_dict(dag_node_id, dag, "dag") - if dep.node_id not in nodes_dict: - nodes_dict[dep.node_id] = node_dict( - dep.node_id, dep.dependency_id, dep.dependency_type - ) - - # add edge - - # not start dep - if dep.source != dep.dependency_type: - source = dep.source if ":" in dep.source else f"dag:{dep.source}" - target = dep.node_id - edge_tuples.add((source, target)) - - # not end dep - if dep.target != dep.dependency_type: - source = dep.node_id - target = dep.target if ":" in dep.target else f"dag:{dep.target}" - edge_tuples.add((source, target)) - - nodes = list(nodes_dict.values()) - edges = [{"source": source, "target": target} for source, target in edge_tuples] - - data = { - "nodes": nodes, - "edges": edges, - } - - return ( - htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), - {"Content-Type": "application/json; charset=utf-8"}, - ) - - @expose("/object/assets_summary") - @auth.has_access_asset("GET") - def assets_summary(self): - """ - Get a summary of assets. - - Includes the datetime they were last updated and how many updates they've ever had. - """ - allowed_attrs = ["uri", "last_asset_update"] - - # Grab query parameters - limit = int(request.args.get("limit", 25)) - offset = int(request.args.get("offset", 0)) - order_by = request.args.get("order_by", "uri") - uri_pattern = request.args.get("uri_pattern", "") - lstripped_orderby = order_by.lstrip("-") - updated_after = _safe_parse_datetime(request.args.get("updated_after"), allow_empty=True) - updated_before = _safe_parse_datetime(request.args.get("updated_before"), allow_empty=True) - - # Check and clean up query parameters - limit = min(50, limit) - - uri_pattern = uri_pattern[:4000] - - if lstripped_orderby not in allowed_attrs: - return { - "detail": ( - f"Ordering with '{lstripped_orderby}' is disallowed or the attribute does not " - "exist on the model" - ) - }, 400 - - with create_session() as session: - if lstripped_orderby == "uri": - if order_by.startswith("-"): - order_by = (AssetModel.uri.desc(),) - else: - order_by = (AssetModel.uri.asc(),) - elif lstripped_orderby == "last_asset_update": - if order_by.startswith("-"): - order_by = ( - func.max(AssetEvent.timestamp).desc(), - AssetModel.uri.asc(), - ) - if session.bind.dialect.name == "postgresql": - order_by = (order_by[0].nulls_last(), *order_by[1:]) - else: - order_by = ( - func.max(AssetEvent.timestamp).asc(), - AssetModel.uri.desc(), - ) - if session.bind.dialect.name == "postgresql": - order_by = (order_by[0].nulls_first(), *order_by[1:]) - - count_query = select(func.count(AssetModel.id)) - - has_event_filters = bool(updated_before or updated_after) - - query = ( - select( - AssetModel.id, - AssetModel.uri, - func.max(AssetEvent.timestamp).label("last_asset_update"), - func.sum(case((AssetEvent.id.is_not(None), 1), else_=0)).label("total_updates"), - ) - .join(AssetEvent, AssetEvent.asset_id == AssetModel.id, isouter=not has_event_filters) - .group_by( - AssetModel.id, - AssetModel.uri, - ) - .order_by(*order_by) - ) - - if has_event_filters: - count_query = count_query.join(AssetEvent, AssetEvent.asset_id == AssetModel.id) - - filters = [AssetModel.active.has()] - if uri_pattern: - filters.append(AssetModel.uri.ilike(f"%{uri_pattern}%")) - if updated_after: - filters.append(AssetEvent.timestamp >= updated_after) - if updated_before: - filters.append(AssetEvent.timestamp <= updated_before) - - query = query.where(*filters).offset(offset).limit(limit) - count_query = count_query.where(*filters) - - query = session.execute(query) - assets = [dict(asset._mapping) for asset in query] - data = {"assets": assets, "total_entries": session.scalar(count_query)} - - return ( - htmlsafe_json_dumps(data, separators=(",", ":"), cls=utils_json.WebEncoder), - {"Content-Type": "application/json; charset=utf-8"}, - ) - - @expose("/robots.txt") - @action_logging - def robots(self): - """ - Return a robots.txt file for blocking certain search engine crawlers. - - This mitigates some of the risk associated with exposing Airflow to the public - internet, however it does not address the real security risks associated with - such a deployment. - """ - return send_from_directory(get_airflow_app().static_folder, "robots.txt") - - @expose("/audit_log") - def legacy_audit_log(self): - """Redirect from url param.""" - return redirect(url_for("Airflow.audit_log", **sanitize_args(request.args))) - - @expose("/dags//audit_log") - def audit_log(self, dag_id: str): - current_page = request.args.get("page") - arg_sorting_key = request.args.get("sorting_key") - arg_sorting_direction = request.args.get("sorting_direction") - sort_args = { - "offset": current_page, - f"sort.{arg_sorting_key}": arg_sorting_direction, - "limit": PAGE_SIZE, - } - kwargs = { - **sanitize_args(sort_args), - "dag_id": dag_id, - "tab": "audit_log", - } - - return redirect(url_for("Airflow.grid", **kwargs)) - - @expose("/parseDagFile/") - def parse_dag(self, file_token: str): - from airflow.api_connexion.endpoints.dag_parsing import reparse_dag_file - - with create_session() as session: - response = reparse_dag_file(file_token=file_token, session=session) - response_messages = { - 201: ["Reparsing request submitted successfully", "info"], - 401: ["Unauthenticated request", "error"], - 403: ["Permission Denied", "error"], - 404: ["DAG not found", "error"], - } - flash(response_messages[response.status_code][0], response_messages[response.status_code][1]) - redirect_url = get_safe_url(request.values.get("redirect_url")) - return redirect(redirect_url) - - -class ConfigurationView(AirflowBaseView): - """View to show Airflow Configurations.""" - - default_view = "conf" - - class_permission_name = permissions.RESOURCE_CONFIG - base_permissions = [ - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - @expose("/configuration") - @auth.has_access_configuration("GET") - def conf(self): - """Show configuration.""" - raw = request.args.get("raw") == "true" - title = "Airflow Configuration" - expose_config = conf.get("webserver", "expose_config").lower() - - # TODO remove "if raw" usage in Airflow 3.0. Configuration can be fetched via the REST API. - if raw: - if expose_config == "non-sensitive-only": - updater = configupdater.ConfigUpdater() - updater.read(AIRFLOW_CONFIG) - for sect, key in conf.sensitive_config_values: - if updater.has_option(sect, key): - updater[sect][key].value = "< hidden >" - config = str(updater) - elif expose_config in {"true", "t", "1"}: - with open(AIRFLOW_CONFIG) as file: - config = file.read() - else: - config = ( - "# Your Airflow administrator chose not to expose the configuration, " - "most likely for security reasons." - ) - - return Response( - response=config, - status=200, - mimetype="application/text", - headers={"Deprecation": "Endpoint will be removed in Airflow 3.0, use the REST API instead."}, - ) - - if expose_config in {"non-sensitive-only", "true", "t", "1"}: - display_sensitive = expose_config != "non-sensitive-only" - - table = [ - (section, key, str(value), source) - for section, parameters in conf.as_dict(True, display_sensitive).items() - for key, (value, source) in parameters.items() - ] - - return self.render_template( - template="airflow/config.html", - title=title, - table=table, - ) - - else: - return self.render_template( - "airflow/config.html", - title=title, - hide_config_msg=( - "Your Airflow administrator chose not to expose the configuration, " - "most likely for security reasons." - ), - ) - - -class RedocView(AirflowBaseView): - """Redoc Open API documentation.""" - - default_view = "redoc" - - @expose("/redoc") - def redoc(self): - """Redoc API documentation.""" - openapi_spec_url = url_for("/api/v1./api/v1_openapi_yaml") - return self.render_template("airflow/redoc.html", openapi_spec_url=openapi_spec_url) - - -###################################################################################### -# ModelViews -###################################################################################### - - -class DagFilter(BaseFilter): - """Filter using DagIDs.""" - - def apply(self, query, func): - if get_auth_manager().is_authorized_dag(method="GET", user=g.user): - return query - if get_auth_manager().is_authorized_dag(method="PUT", user=g.user): - return query - filter_dag_ids = get_auth_manager().get_permitted_dag_ids(user=g.user) - return query.where(self.model.dag_id.in_(filter_dag_ids)) - - -class AirflowModelView(ModelView): - """ - Airflow Model View. - - Overridden `__getattribute__` to wraps REST methods with action_logger - """ - - list_widget = AirflowModelListWidget - page_size = PAGE_SIZE - - CustomSQLAInterface = wwwutils.CustomSQLAInterface - - def __getattribute__(self, attr): - """ - Wrap action REST methods with `action_logging` wrapper. - - Overriding enables differentiating resource and generation of event name at the decorator level. - - if attr in ["show", "list", "read", "get", "get_list"]: - return action_logging(event="RESOURCE_NAME"."action_name")(attr) - else: - return attr - """ - attribute = object.__getattribute__(self, attr) - if ( - callable(attribute) - and hasattr(attribute, "_permission_name") - and attribute._permission_name in self.method_permission_name - ): - permission_str = self.method_permission_name[attribute._permission_name] - if permission_str not in ["show", "list", "read", "get", "get_list"]: - return action_logging(event=f"{self.route_base.strip('/')}.{permission_str}")(attribute) - return attribute - - @expose("/show/", methods=["GET"]) - @auth.has_access_with_pk - def show(self, pk): - """ - Show view. - - Same implementation as - https://github.com/dpgaspar/Flask-AppBuilder/blob/1c3af9b665ed9a3daf36673fee3327d0abf43e5b/flask_appbuilder/views.py#L566 - - Override it to use a custom ``has_access_with_pk`` decorator to take into consideration resource for - fined-grained access. - """ - pk = self._deserialize_pk_if_composite(pk) - widgets = self._show(pk) - return self.render_template( - self.show_template, - pk=pk, - title=self.show_title, - widgets=widgets, - related_views=self._related_views, - ) - - @expose("/edit/", methods=["GET", "POST"]) - @auth.has_access_with_pk - def edit(self, pk): - """ - Edit view. - - Same implementation as - https://github.com/dpgaspar/Flask-AppBuilder/blob/1c3af9b665ed9a3daf36673fee3327d0abf43e5b/flask_appbuilder/views.py#L602 - - Override it to use a custom ``has_access_with_pk`` decorator to take into consideration resource for - fined-grained access. - """ - pk = self._deserialize_pk_if_composite(pk) - widgets = self._edit(pk) - if not widgets: - return self.post_edit_redirect() - else: - return self.render_template( - self.edit_template, - title=self.edit_title, - widgets=widgets, - related_views=self._related_views, - ) - - @expose("/delete/", methods=["GET", "POST"]) - @auth.has_access_with_pk - def delete(self, pk): - """ - Delete view. - - Same implementation as - https://github.com/dpgaspar/Flask-AppBuilder/blob/1c3af9b665ed9a3daf36673fee3327d0abf43e5b/flask_appbuilder/views.py#L623 - - Override it to use a custom ``has_access_with_pk`` decorator to take into consideration resource for - fined-grained access. - """ - # Maintains compatibility but refuses to delete on GET methods if CSRF is enabled - if not self.is_get_mutation_allowed(): - self.update_redirect() - logger.warning("CSRF is enabled and a delete using GET was invoked") - flash(as_unicode(FLAMSG_ERR_SEC_ACCESS_DENIED), "danger") - return self.post_delete_redirect() - pk = self._deserialize_pk_if_composite(pk) - self._delete(pk) - return self.post_delete_redirect() - - @expose("/action_post", methods=["POST"]) - def action_post(self): - """ - Handle multiple records selected from a list view. - - Same implementation as - https://github.com/dpgaspar/Flask-AppBuilder/blob/2c5763371b81cd679d88b9971ba5d1fc4d71d54b/flask_appbuilder/views.py#L677 - - The difference is, it no longer check permissions with ``self.appbuilder.sm.has_access``, - it executes the function without verifying permissions. - Thus, each action need to be annotated individually with ``@auth.has_access_*`` to check user - permissions. - """ - name = request.form["action"] - pks = request.form.getlist("rowid") - action = self.actions.get(name) - items = [self.datamodel.get(self._deserialize_pk_if_composite(pk)) for pk in pks] - return action.func(items) - - -class XComModelView(AirflowModelView): - """View to show records from XCom table.""" - - route_base = "/xcom" - - list_title = "List XComs" - - datamodel = AirflowModelView.CustomSQLAInterface(XCom) - - class_permission_name = permissions.RESOURCE_XCOM - method_permission_name = { - "list": "read", - "delete": "delete", - "action_muldelete": "delete", - } - base_permissions = [ - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_DELETE, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - add_exclude_columns = edit_exclude_columns = ["value"] - - search_columns = ["key", "timestamp", "dag_id", "task_id", "run_id", "logical_date"] - list_columns = ["key", "value", "timestamp", "dag_id", "task_id", "run_id", "map_index", "logical_date"] - base_order = ("dag_run_id", "desc") - - order_columns = [ - "key", - "value", - "timestamp", - "dag_id", - "task_id", - "run_id", - "map_index", - # "logical_date", # logical_date sorting is not working and crashing the UI, disabled for now. - ] - - base_filters = [["dag_id", DagFilter, list]] - - formatters_columns = { - "task_id": wwwutils.task_instance_link, - "timestamp": wwwutils.datetime_f("timestamp"), - "dag_id": wwwutils.dag_link, - "map_index": wwwutils.format_map_index, - "logical_date": wwwutils.datetime_f("logical_date"), - } - - @action("muldelete", "Delete", "Are you sure you want to delete selected records?", single=False) - @auth.has_access_dag_entities("DELETE", DagAccessEntity.XCOM) - def action_muldelete(self, items): - """Multiple delete action.""" - self.datamodel.delete_all(items) - self.update_redirect() - return redirect(self.get_redirect()) - - def pre_add(self, item): - """Pre add hook.""" - item.logical_date = timezone.make_aware(item.logical_date) - item.value = XCom.serialize_value( - value=item.value, - key=item.key, - task_id=item.task_id, - dag_id=item.dag_id, - run_id=item.run_id, - map_index=item.map_index, - ) - - def pre_update(self, item): - """Pre update hook.""" - item.logical_date = timezone.make_aware(item.logical_date) - item.value = XCom.serialize_value( - value=item.value, - key=item.key, - task_id=item.task_id, - dag_id=item.dag_id, - run_id=item.run_id, - map_index=item.map_index, - ) - - -# Used to store a dictionary of field behaviours used to dynamically change available -# fields in ConnectionForm based on type of connection chosen -# See airflow.hooks.base_hook.DiscoverableHook for details on how to customize your Hooks. -# -# Additionally, a list of connection types that support testing via Airflow REST API is stored to dynamically -# enable/disable the Test Connection button. -# -# These field behaviours and testable connection types are rendered as scripts in the conn_create.html and -# conn_edit.html templates. -class ConnectionFormWidget(FormWidget): - """Form widget used to display connection.""" - - @cached_property - def field_behaviours(self) -> str: - return json.dumps(ProvidersManager().field_behaviours) - - @cached_property - def testable_connection_types(self) -> list[str]: - return [ - connection_type - for connection_type, hook_info in ProvidersManager().hooks.items() - if hook_info and hook_info.connection_testable - ] - - -class ConnectionFormProxy: - """ - A stand-in for the connection form class. - - Flask-Appbuilder model views only ever call the ``refresh()`` function on - the form class, so this is the perfect place to make the form generation - dynamic. See docstring of ``create_connection_form_class`` for rationales. - """ - - @staticmethod - def refresh(obj=None): - return create_connection_form_class().refresh(obj) - - -class ConnectionModelView(AirflowModelView): - """View to show records from Connections table.""" - - route_base = "/connection" - - datamodel = AirflowModelView.CustomSQLAInterface(Connection) # type: ignore - - class_permission_name = permissions.RESOURCE_CONNECTION - method_permission_name = { - "add": "create", - "list": "read", - "edit": "edit", - "delete": "delete", - "action_muldelete": "delete", - "action_mulduplicate": "create", - } - - base_permissions = [ - permissions.ACTION_CAN_CREATE, - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_EDIT, - permissions.ACTION_CAN_DELETE, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - list_columns = [ - "conn_id", - "conn_type", - "description", - "host", - "port", - "is_encrypted", - "is_extra_encrypted", - ] - - # The real add_columns and edit_columns are dynamically generated at runtime - # so we can delay calculating entries relying on providers to make webserver - # start up faster. - _add_columns = _edit_columns = [ - "conn_id", - "conn_type", - "description", - "host", - "schema", - "login", - "password", - "port", - "extra", - ] - - # We will generate the actual ConnectionForm when it is actually needed, - # i.e. when the web form views are displayed and submitted. - add_form = edit_form = ConnectionFormProxy - - add_template = "airflow/conn_create.html" - edit_template = "airflow/conn_edit.html" - - add_widget = ConnectionFormWidget - edit_widget = ConnectionFormWidget - - base_order = ("conn_id", "asc") - - def _iter_extra_field_names_and_sensitivity(self) -> Iterator[tuple[str, str, bool]]: - """ - Iterate through provider-backed connection fields. - - Note that this cannot be a property (including a cached property) - because Flask-Appbuilder attempts to access all members on startup, and - using a property would initialize the providers manager too eagerly. - - Returns tuple of: - - * key - * field_name - * whether the field is sensitive - """ - return ( - (k, v.field_name, v.is_sensitive) for k, v in ProvidersManager().connection_form_widgets.items() - ) - - @property - def add_columns(self) -> list[str]: - """ - A list of columns to show in the Add form. - - This dynamically calculates additional fields from providers and add - them to the backing list. This calculation is done exactly once (by - checking we're referencing the class-level variable instead of the - instance-level), and only after we enter the request context (to skip - superfuluous checks done by Flask-Appbuilder on startup). - """ - if self._add_columns is type(self)._add_columns and has_request_context(): - self._add_columns = [ - *self._add_columns, - *(k for k, _, _ in self._iter_extra_field_names_and_sensitivity()), - ] - return self._add_columns - - @property - def edit_columns(self) -> list[str]: - """ - A list of columns to show in the Edit form. - - This dynamically calculates additional fields from providers and add - them to the backing list. This calculation is done exactly once (by - checking we're referencing the class-level variable instead of the - instance-level), and only after we enter the request context (to skip - superfuluous checks done by Flask-Appbuilder on startup). - """ - if self._edit_columns is type(self)._edit_columns and has_request_context(): - self._edit_columns = [ - *self._edit_columns, - *(k for k, _, _ in self._iter_extra_field_names_and_sensitivity()), - ] - return self._edit_columns - - @action("muldelete", "Delete", "Are you sure you want to delete selected records?", single=False) - @auth.has_access_connection("DELETE") - def action_muldelete(self, connections): - """Multiple delete.""" - self.datamodel.delete_all(connections) - self.update_redirect() - return redirect(self.get_redirect()) - - @action( - "mulduplicate", - "Duplicate", - "Are you sure you want to duplicate the selected connections?", - single=False, - ) - @provide_session - @auth.has_access_connection("POST") - @auth.has_access_connection("GET") - def action_mulduplicate(self, connections, session: Session = NEW_SESSION): - """Duplicate Multiple connections.""" - for selected_conn in connections: - new_conn_id = selected_conn.conn_id - match = re2.search(r"_copy(\d+)$", selected_conn.conn_id) - - base_conn_id = selected_conn.conn_id - if match: - base_conn_id = base_conn_id.split("_copy")[0] - - potential_connection_ids = [f"{base_conn_id}_copy{i}" for i in range(1, 11)] - - query = session.scalars( - select(Connection.conn_id).where(Connection.conn_id.in_(potential_connection_ids)) - ) - - found_conn_id_set = set(query) - - possible_conn_id_iter = ( - connection_id - for connection_id in potential_connection_ids - if connection_id not in found_conn_id_set - ) - try: - new_conn_id = next(possible_conn_id_iter) - except StopIteration: - flash( - f"Connection {new_conn_id} can't be added because it already exists, " - f"Please rename the existing connections", - "warning", - ) - else: - dup_conn = Connection( - new_conn_id, - selected_conn.conn_type, - selected_conn.description, - selected_conn.host, - selected_conn.login, - selected_conn.password, - selected_conn.schema, - selected_conn.port, - selected_conn.extra, - ) - - try: - session.add(dup_conn) - session.commit() - flash(f"Connection {new_conn_id} added successfully.", "success") - except IntegrityError: - flash( - f"Connection {new_conn_id} can't be added. Integrity error, " - f"probably unique constraint.", - "warning", - ) - session.rollback() - - self.update_redirect() - return redirect(self.get_redirect()) - - def process_form(self, form, is_created): - """Process form data.""" - conn_id = form.data["conn_id"] - conn_type = form.data["conn_type"] - # The extra value is the combination of custom fields for this conn_type and the Extra field. - # The extra form field with all extra values (including custom fields) is in the form being processed - # so we start with those values, and override them with anything in the custom fields. - extra = {} - - extra_json = form.data.get("extra") - - if extra_json: - try: - extra.update(json.loads(extra_json)) - except (JSONDecodeError, TypeError): - flash( - Markup( - "

    The Extra connection field contained an invalid value for Conn ID: " - "{conn_id}.

    " - "

    If connection parameters need to be added to Extra, " - "please make sure they are in the form of a single, valid JSON object.


    " - "The following Extra parameters were not added to the connection:
    " - "{extra_json}" - ).format(conn_id=conn_id, extra_json=extra_json), - category="error", - ) - del form.extra - del extra_json - for key, field_name, _ in self._iter_extra_field_names_and_sensitivity(): - if key in form.data and key.startswith("extra__"): - conn_type_from_extra_field = key.split("__")[1] - if conn_type_from_extra_field == conn_type: - value = form.data[key] - # Some extra fields have a default value of False so we need to explicitly check the - # value isn't an empty string. - if value != "": - extra[field_name] = value - elif field_name in extra: - del extra[field_name] - if extra.keys(): - sensitive_unchanged_keys = set() - for key, value in extra.items(): - if value == SENSITIVE_FIELD_PLACEHOLDER: - sensitive_unchanged_keys.add(key) - if sensitive_unchanged_keys: - try: - conn = BaseHook.get_connection(conn_id) - except AirflowNotFoundException: - conn = None - for key in sensitive_unchanged_keys: - if conn and conn.extra_dejson.get(key): - extra[key] = conn.extra_dejson.get(key) - else: - del extra[key] - form.extra.data = json.dumps(extra) - - def prefill_form(self, form, pk): - """Prefill the form.""" - try: - extra = form.data.get("extra") - if extra is None: - extra_dictionary = {} - else: - extra_dictionary = json.loads(extra) - except JSONDecodeError: - extra_dictionary = {} - - if not isinstance(extra_dictionary, dict): - logger.warning("extra field for %s is not a dictionary", form.data.get("conn_id", "")) - return - - for field_key, field_name, is_sensitive in self._iter_extra_field_names_and_sensitivity(): - value = extra_dictionary.get(field_name, "") - - if not value: - # check if connection `extra` json is using old prefixed field name style - value = extra_dictionary.get(field_key, "") - - if value: - field = getattr(form, field_key) - field.data = value - if is_sensitive and field_name in extra_dictionary: - extra_dictionary[field_name] = SENSITIVE_FIELD_PLACEHOLDER - # form.data is a property that builds the dictionary from fields so we have to modify the fields - if extra_dictionary: - form.extra.data = json.dumps(extra_dictionary) - else: - form.extra.data = None - - -class PluginView(AirflowBaseView): - """View to show Airflow Plugins.""" - - default_view = "list" - - class_permission_name = permissions.RESOURCE_PLUGIN - - method_permission_name = { - "list": "read", - } - - base_permissions = [ - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - plugins_attributes_to_dump = PLUGINS_ATTRIBUTES_TO_DUMP - - @expose("/plugin") - @auth.has_access_view(AccessView.PLUGINS) - def list(self): - """List loaded plugins.""" - plugins_manager.ensure_plugins_loaded() - plugins_manager.initialize_extra_operators_links_plugins() - plugins_manager.initialize_flask_plugins() - plugins_manager.initialize_fastapi_plugins() - - plugins = [] - for plugin_no, plugin in enumerate(plugins_manager.plugins, 1): - plugin_data = { - "plugin_no": plugin_no, - "plugin_name": plugin.name, - "attrs": {}, - } - for attr_name in self.plugins_attributes_to_dump: - attr_value = getattr(plugin, attr_name) - plugin_data["attrs"][attr_name] = attr_value - - plugins.append(plugin_data) - - title = "Airflow Plugins" - doc_url = get_docs_url("plugins.html") - return self.render_template( - "airflow/plugin.html", - plugins=plugins, - title=title, - doc_url=doc_url, - ) - - -class ProviderView(AirflowBaseView): - """View to show Airflow Providers.""" - - default_view = "list" - - class_permission_name = permissions.RESOURCE_PROVIDER - - method_permission_name = { - "list": "read", - } - - base_permissions = [ - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - @expose("/provider") - @auth.has_access_view(AccessView.PROVIDERS) - def list(self): - """List providers.""" - providers_manager = ProvidersManager() - - providers = [] - for pi in providers_manager.providers.values(): - provider_info = pi.data - provider_data = { - "package_name": provider_info["package-name"], - "description": self._clean_description(provider_info["description"]), - "version": pi.version, - "documentation_url": get_doc_url_for_provider(provider_info["package-name"], pi.version), - } - providers.append(provider_data) - - title = "Providers" - doc_url = get_docs_url("apache-airflow-providers/index.html") - return self.render_template( - "airflow/providers.html", - providers=providers, - title=title, - doc_url=doc_url, - ) - - def _clean_description(self, description): - def _build_link(match_obj): - text = match_obj.group(1) - url = match_obj.group(2) - - # parsing the url to check if it's a valid url - parsed_url = urlparse(url) - if not (parsed_url.scheme == "http" or parsed_url.scheme == "https"): - # returning the original raw text - return escape(match_obj.group(0)) - - return Markup(f'{text}') - - cd = escape(description) - cd = re2.sub(r"`(.*)[\s+]+<(.*)>`__", _build_link, cd) - return Markup(cd) - - -class PoolModelView(AirflowModelView): - """View to show records from Pool table.""" - - route_base = "/pool" - - list_template = "airflow/pool_list.html" - - datamodel = AirflowModelView.CustomSQLAInterface(models.Pool) # type: ignore - - class_permission_name = permissions.RESOURCE_POOL - method_permission_name = { - "add": "create", - "list": "read", - "edit": "edit", - "delete": "delete", - "action_muldelete": "delete", - } - - base_permissions = [ - permissions.ACTION_CAN_CREATE, - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_EDIT, - permissions.ACTION_CAN_DELETE, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - list_columns = [ - "pool", - "description", - "slots", - "running_slots", - "queued_slots", - "scheduled_slots", - "deferred_slots", - ] - add_columns = ["pool", "slots", "description", "include_deferred"] - edit_columns = ["pool", "slots", "description", "include_deferred"] - - # include_deferred is non-nullable, but as a checkbox in the resulting form we want to allow it unchecked - include_deferred_field = BooleanField( - validators=[validators.Optional()], - description="Check to include deferred tasks when calculating open pool slots.", - ) - edit_form_extra_fields = {"include_deferred": include_deferred_field} - add_form_extra_fields = {"include_deferred": include_deferred_field} - - base_order = ("pool", "asc") - - @action("muldelete", "Delete", "Are you sure you want to delete selected records?", single=False) - @auth.has_access_pool("DELETE") - def action_muldelete(self, items): - """Multiple delete.""" - if any(item.pool == models.Pool.DEFAULT_POOL_NAME for item in items): - flash(f"{models.Pool.DEFAULT_POOL_NAME} cannot be deleted", "error") - self.update_redirect() - return redirect(self.get_redirect()) - self.datamodel.delete_all(items) - self.update_redirect() - return redirect(self.get_redirect()) - - @expose("/delete/", methods=["GET", "POST"]) - @auth.has_access_with_pk - def delete(self, pk): - """Single delete.""" - if models.Pool.is_default_pool(pk): - flash(f"{models.Pool.DEFAULT_POOL_NAME} cannot be deleted", "error") - self.update_redirect() - return redirect(self.get_redirect()) - - return super().delete(pk) - - def pool_link(self): - """Pool link rendering.""" - pool_id = self.get("pool") - if pool_id is not None: - url = url_for("TaskInstanceModelView.list", _flt_3_pool=pool_id) - return Markup("{pool_id}").format(url=url, pool_id=pool_id) - else: - return Markup('Invalid') - - def frunning_slots(self): - """Format running slots rendering.""" - pool_id = self.get("pool") - running_slots = self.get("running_slots") - if pool_id is not None and running_slots is not None: - url = url_for("TaskInstanceModelView.list", _flt_3_pool=pool_id, _flt_3_state="running") - return Markup("{running_slots}").format(url=url, running_slots=running_slots) - else: - return Markup('Invalid') - - def fqueued_slots(self): - """Queued slots rendering.""" - pool_id = self.get("pool") - queued_slots = self.get("queued_slots") - if pool_id is not None and queued_slots is not None: - url = url_for("TaskInstanceModelView.list", _flt_3_pool=pool_id, _flt_3_state="queued") - return Markup("{queued_slots}").format(url=url, queued_slots=queued_slots) - else: - return Markup('Invalid') - - def fscheduled_slots(self): - """Scheduled slots rendering.""" - pool_id = self.get("pool") - scheduled_slots = self.get("scheduled_slots") - if pool_id is not None and scheduled_slots is not None: - url = url_for("TaskInstanceModelView.list", _flt_3_pool=pool_id, _flt_3_state="scheduled") - return Markup("{scheduled_slots}").format( - url=url, scheduled_slots=scheduled_slots - ) - else: - return Markup('Invalid') - - def fdeferred_slots(self): - """Deferred slots rendering.""" - pool_id = self.get("pool") - deferred_slots = self.get("deferred_slots") - if pool_id is not None and deferred_slots is not None: - url = url_for("TaskInstanceModelView.list", _flt_3_pool=pool_id, _flt_3_state="deferred") - return Markup("{deferred_slots}").format( - url=url, deferred_slots=deferred_slots - ) - else: - return Markup('Invalid') - - formatters_columns = { - "pool": pool_link, - "running_slots": frunning_slots, - "queued_slots": fqueued_slots, - "scheduled_slots": fscheduled_slots, - "deferred_slots": fdeferred_slots, - } - - validators_columns = {"pool": [validators.DataRequired()], "slots": [validators.NumberRange(min=-1)]} - - -class VariableModelView(AirflowModelView): - """View to show records from Variable table.""" - - route_base = "/variable" - - list_template = "airflow/variable_list.html" - edit_template = "airflow/variable_edit.html" - show_template = "airflow/variable_show.html" - - show_widget = AirflowVariableShowWidget - - datamodel = AirflowModelView.CustomSQLAInterface(models.Variable) # type: ignore - - class_permission_name = permissions.RESOURCE_VARIABLE - method_permission_name = { - "add": "create", - "list": "read", - "edit": "edit", - "show": "read", - "delete": "delete", - "action_muldelete": "delete", - "action_varexport": "read", - } - base_permissions = [ - permissions.ACTION_CAN_CREATE, - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_EDIT, - permissions.ACTION_CAN_DELETE, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - list_columns = ["key", "val", "description", "is_encrypted"] - add_columns = ["key", "val", "description"] - edit_columns = ["key", "val", "description"] - show_columns = ["key", "val", "description"] - search_columns = ["key", "val"] - - base_order = ("key", "asc") - - def hidden_field_formatter(self): - """Format hidden fields.""" - key = self.get("key") - val = self.get("val") - if secrets_masker.should_hide_value_for_key(key): - return Markup("*" * 8) - if val: - return val - else: - return Markup('Invalid') - - formatters_columns = { - "val": hidden_field_formatter, - } - - validators_columns = {"key": [validators.DataRequired()]} - - def prefill_form(self, form, request_id): - if secrets_masker.should_hide_value_for_key(form.key.data): - form.val.data = "*" * 8 - - def prefill_show(self, item): - if secrets_masker.should_hide_value_for_key(item.key): - item.val = "*" * 8 - - def _show(self, pk): - pages = get_page_args() - page_sizes = get_page_size_args() - orders = get_order_args() - - item = self.datamodel.get(pk, self._base_filters) - if not item: - abort(404) - self.prefill_show(item) - widgets = self._get_show_widget(pk, item) - self.update_redirect() - - return self._get_related_views_widgets( - item, orders=orders, pages=pages, page_sizes=page_sizes, widgets=widgets - ) - - extra_args = { - "can_create_variable": lambda: get_auth_manager().is_authorized_variable(method="POST", user=g.user) - } - - @action("muldelete", "Delete", "Are you sure you want to delete selected records?", single=False) - @auth.has_access_variable("DELETE") - def action_muldelete(self, items): - """Multiple delete.""" - self.datamodel.delete_all(items) - self.update_redirect() - return redirect(self.get_redirect()) - - @action("varexport", "Export", "", single=False) - @auth.has_access_variable("GET") - def action_varexport(self, items): - """Export variables.""" - var_dict = {} - decoder = json.JSONDecoder() - for var in items: - try: - val = decoder.decode(var.val) - except Exception: - val = var.val - var_dict[var.key] = val - - response = make_response(json.dumps(var_dict, sort_keys=True, indent=4)) - response.headers["Content-Disposition"] = "attachment; filename=variables.json" - response.headers["Content-Type"] = "application/json; charset=utf-8" - return response - - @expose("/varimport", methods=["POST"]) - @auth.has_access_variable("POST") - @action_logging(event=f"{permissions.RESOURCE_VARIABLE.lower()}.varimport") - @provide_session - def varimport(self, session): - """Import variables.""" - try: - variable_dict = json.loads(request.files["file"].read()) - action_on_existing = request.form.get("action_if_exists", "overwrite").lower() - except Exception: - self.update_redirect() - flash("Missing file or syntax error.", "error") - return redirect(self.get_redirect()) - else: - existing_keys = set() - if action_on_existing != "overwrite": - existing_keys = set( - session.scalars(select(models.Variable.key).where(models.Variable.key.in_(variable_dict))) - ) - if action_on_existing == "fail" and existing_keys: - failed_repr = ", ".join(repr(k) for k in sorted(existing_keys)) - flash(f"Failed. The variables with these keys: {failed_repr} already exists.") - logger.error("Failed. The variables with these keys: %s already exists.", failed_repr) - self.update_redirect() - return redirect(self.get_redirect()) - skipped = set() - suc_count = fail_count = 0 - for k, v in variable_dict.items(): - if action_on_existing == "skip" and k in existing_keys: - logger.warning("Variable: %s already exists, skipping.", k) - skipped.add(k) - continue - try: - models.Variable.set(k, v, serialize_json=not isinstance(v, str)) - except Exception as exc: - logger.info("Variable import failed: %r", exc) - fail_count += 1 - else: - suc_count += 1 - flash(f"{suc_count} variable(s) successfully updated.") - if fail_count: - flash(f"{fail_count} variable(s) failed to be updated.", "error") - if skipped: - skipped_repr = ", ".join(repr(k) for k in sorted(skipped)) - flash( - f"The variables with these keys: {skipped_repr} were skipped " - "because they already exists", - "warning", - ) - self.update_redirect() - return redirect(self.get_redirect()) - - -class JobModelView(AirflowModelView): - """View to show records from Job table.""" - - route_base = "/job" - - datamodel = AirflowModelView.CustomSQLAInterface(Job) # type: ignore - - class_permission_name = permissions.RESOURCE_JOB - method_permission_name = { - "list": "read", - } - base_permissions = [ - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - list_columns = [ - "id", - "dag_id", - "state", - "job_type", - "start_date", - "end_date", - "latest_heartbeat", - "executor_class", - "hostname", - "unixname", - ] - search_columns = [ - "id", - "dag_id", - "state", - "job_type", - "start_date", - "end_date", - "latest_heartbeat", - "executor_class", - "hostname", - "unixname", - ] - - base_order = ("start_date", "desc") - - base_filters = [["dag_id", DagFilter, list]] - - formatters_columns = { - "start_date": wwwutils.datetime_f("start_date"), - "end_date": wwwutils.datetime_f("end_date"), - "hostname": wwwutils.nobr_f("hostname"), - "state": wwwutils.state_f, - "latest_heartbeat": wwwutils.datetime_f("latest_heartbeat"), - } - - -class DagRunModelView(AirflowModelView): - """View to show records from DagRun table.""" - - route_base = "/dagrun" - - datamodel = wwwutils.DagRunCustomSQLAInterface(models.DagRun) # type: ignore - - class_permission_name = permissions.RESOURCE_DAG_RUN - method_permission_name = { - "delete": "delete", - "edit": "edit", - "list": "read", - "action_clear": "edit", - "action_muldelete": "delete", - "action_set_queued": "edit", - "action_set_running": "edit", - "action_set_failed": "edit", - "action_set_success": "edit", - } - base_permissions = [ - permissions.ACTION_CAN_CREATE, - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_EDIT, - permissions.ACTION_CAN_DELETE, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - add_exclude_columns = ["conf"] - - list_columns = [ - "state", - "dag_id", - "logical_date", - "run_id", - "run_type", - "queued_at", - "start_date", - "end_date", - "note", - "conf", - "duration", - ] - search_columns = [ - "state", - "dag_id", - "logical_date", - "run_id", - "run_type", - "start_date", - "end_date", - "note", - ] - label_columns = { - "logical_date": "Logical Date", - } - edit_columns = [ - "state", - "dag_id", - "logical_date", - "start_date", - "end_date", - "run_id", - "note", - ] - - # duration is not a DB column, its derived - order_columns = [ - "state", - "dag_id", - "logical_date", - "run_id", - "run_type", - "queued_at", - "start_date", - "end_date", - # "note", # todo: maybe figure out how to re-enable this - "conf", - ] - - base_order = ("logical_date", "desc") - - base_filters = [["dag_id", DagFilter, list]] - - edit_form = DagRunEditForm - - def duration_f(self): - """Duration calculation.""" - end_date = self.get("end_date") - start_date = self.get("start_date") - - difference = "0s" - if start_date and end_date: - difference = td_format(end_date - start_date) - - return difference - - formatters_columns = { - "logical_date": wwwutils.datetime_f("logical_date"), - "state": wwwutils.state_f, - "start_date": wwwutils.datetime_f("start_date"), - "end_date": wwwutils.datetime_f("end_date"), - "queued_at": wwwutils.datetime_f("queued_at"), - "dag_id": wwwutils.dag_link, - "run_id": wwwutils.dag_run_link, - "conf": wwwutils.json_f("conf"), - "duration": duration_f, - } - - @action("muldelete", "Delete", "Are you sure you want to delete selected records?", single=False) - @auth.has_access_dag_entities("DELETE", DagAccessEntity.RUN) - @action_logging - def action_muldelete(self, items: list[DagRun]): - """Multiple delete.""" - self.datamodel.delete_all(items) - self.update_redirect() - return redirect(self.get_redirect()) - - @action("set_queued", "Set state to 'queued'", "", single=False) - @auth.has_access_dag_entities("PUT", DagAccessEntity.RUN) - @action_logging - def action_set_queued(self, drs: list[DagRun]): - """Set state to queued.""" - return self._set_dag_runs_to_active_state(drs, DagRunState.QUEUED) - - @action("set_running", "Set state to 'running'", "", single=False) - @auth.has_access_dag_entities("PUT", DagAccessEntity.RUN) - @action_logging - def action_set_running(self, drs: list[DagRun]): - """Set state to running.""" - return self._set_dag_runs_to_active_state(drs, DagRunState.RUNNING) - - @provide_session - def _set_dag_runs_to_active_state( - self, - drs: list[DagRun], - state: DagRunState, - session: Session = NEW_SESSION, - ): - """Set dag run to active state; this routine only supports Running and Queued state.""" - try: - count = 0 - for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): - count += 1 - if state == DagRunState.RUNNING: - dr.start_date = timezone.utcnow() - dr.state = state - session.commit() - flash(f"{count} dag runs were set to {state}.") - except Exception as ex: - flash(str(ex), "error") - flash("Failed to set state", "error") - return redirect(self.get_default_url()) - - @action( - "set_failed", - "Set state to 'failed'", - "All running task instances would also be marked as failed, are you sure?", - single=False, - ) - @auth.has_access_dag_entities("PUT", DagAccessEntity.RUN) - @provide_session - @action_logging - def action_set_failed(self, drs: list[DagRun], session: Session = NEW_SESSION): - """Set state to failed.""" - try: - count = 0 - altered_tis = [] - for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): - count += 1 - altered_tis += set_dag_run_state_to_failed( - dag=get_airflow_app().dag_bag.get_dag(dr.dag_id), - run_id=dr.run_id, - commit=True, - session=session, - ) - altered_ti_count = len(altered_tis) - flash(f"{count} dag runs and {altered_ti_count} task instances were set to failed") - except Exception: - flash("Failed to set state", "error") - return redirect(self.get_default_url()) - - @action( - "set_success", - "Set state to 'success'", - "All task instances would also be marked as success, are you sure?", - single=False, - ) - @auth.has_access_dag_entities("PUT", DagAccessEntity.RUN) - @provide_session - @action_logging - def action_set_success(self, drs: list[DagRun], session: Session = NEW_SESSION): - """Set state to success.""" - try: - count = 0 - altered_tis = [] - for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): - count += 1 - altered_tis += set_dag_run_state_to_success( - dag=get_airflow_app().dag_bag.get_dag(dr.dag_id), - run_id=dr.run_id, - commit=True, - session=session, - ) - altered_ti_count = len(altered_tis) - flash(f"{count} dag runs and {altered_ti_count} task instances were set to success") - except Exception: - flash("Failed to set state", "error") - return redirect(self.get_default_url()) - - @action("clear", "Clear the state", "All task instances would be cleared, are you sure?", single=False) - @auth.has_access_dag_entities("PUT", DagAccessEntity.RUN) - @provide_session - @action_logging - def action_clear(self, drs: list[DagRun], session: Session = NEW_SESSION): - """Clear the state.""" - try: - count = 0 - cleared_ti_count = 0 - dag_to_tis: dict[DAG, list[TaskInstance]] = {} - for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): - count += 1 - dag = get_airflow_app().dag_bag.get_dag(dr.dag_id) - tis_to_clear = dag_to_tis.setdefault(dag, []) - tis_to_clear += dr.get_task_instances() - - for dag, tis in dag_to_tis.items(): - cleared_ti_count += len(tis) - models.clear_task_instances(tis, session, dag=dag) - - flash(f"{count} dag runs and {cleared_ti_count} task instances were cleared") - except Exception: - flash("Failed to clear state", "error") - return redirect(self.get_default_url()) - - -class LogModelView(AirflowModelView): - """View to show records from Log table.""" - - route_base = "/log" - - datamodel = AirflowModelView.CustomSQLAInterface(Log) # type:ignore - - class_permission_name = permissions.RESOURCE_AUDIT_LOG - method_permission_name = { - "list": "read", - } - base_permissions = [ - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - list_columns = [ - "id", - "dttm", - "dag_id", - "task_id", - "run_id", - "event", - "logical_date", - "owner", - "owner_display_name", - "extra", - ] - search_columns = [ - "dttm", - "dag_id", - "task_id", - "run_id", - "event", - "logical_date", - "owner", - "owner_display_name", - "extra", - ] - - label_columns = { - "logical_date": "Logical Date", - "owner": "Owner ID", - "owner_display_name": "Owner Name", - } - - base_order = ("dttm", "desc") - - base_filters = [["dag_id", DagFilter, list]] - - formatters_columns = { - "dttm": wwwutils.datetime_f("dttm"), - "logical_date": wwwutils.datetime_f("logical_date"), - "dag_id": wwwutils.dag_link, - } - - -class TaskRescheduleModelView(AirflowModelView): - """View to show records from Task Reschedule table.""" - - route_base = "/taskreschedule" - - datamodel = AirflowModelView.CustomSQLAInterface(models.TaskReschedule) # type: ignore - related_views = [DagRunModelView] - - class_permission_name = permissions.RESOURCE_TASK_RESCHEDULE - method_permission_name = { - "list": "read", - } - - base_permissions = [ - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - list_columns = [ - "id", - "dag_id", - "run_id", - "dag_run.logical_date", - "task_id", - "map_index", - "try_number", - "start_date", - "end_date", - "duration", - "reschedule_date", - ] - - label_columns = { - "dag_run.logical_date": "Logical Date", - } - - search_columns = [ - "dag_id", - "task_id", - "run_id", - "logical_date", - "start_date", - "end_date", - "reschedule_date", - ] - - base_order = ("id", "desc") - - base_filters = [["dag_id", DagFilter, list]] - - def duration_f(self): - """Duration calculation.""" - end_date = self.get("end_date") - duration = self.get("duration") - if end_date and duration: - return td_format(datetime.timedelta(seconds=duration)) - return None - - formatters_columns = { - "dag_id": wwwutils.dag_link, - "task_id": wwwutils.task_instance_link, - "start_date": wwwutils.datetime_f("start_date"), - "end_date": wwwutils.datetime_f("end_date"), - "dag_run.logical_date": wwwutils.datetime_f("dag_run.logical_date"), - "reschedule_date": wwwutils.datetime_f("reschedule_date"), - "duration": duration_f, - "map_index": wwwutils.format_map_index, - } - - -class TriggerModelView(AirflowModelView): - """View to show records from Task Reschedule table.""" - - route_base = "/triggerview" - - datamodel = AirflowModelView.CustomSQLAInterface(models.Trigger) # type: ignore - - class_permission_name = permissions.RESOURCE_TRIGGER - method_permission_name = { - "list": "read", - } - - base_permissions = [ - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - list_columns = [ - "id", - "classpath", - "created_date", - "triggerer_id", - ] - - search_columns = [ - "id", - "classpath", - "created_date", - "triggerer_id", - ] - - base_order = ("id", "created_date") - - formatters_columns = { - "created_date": wwwutils.datetime_f("created_date"), - } - - -class TaskInstanceModelView(AirflowModelView): - """View to show records from TaskInstance table.""" - - route_base = "/taskinstance" - - datamodel = AirflowModelView.CustomSQLAInterface(models.TaskInstance) # type: ignore - - class_permission_name = permissions.RESOURCE_TASK_INSTANCE - method_permission_name = { - "list": "read", - "delete": "delete", - "action_clear": "edit", - "action_clear_downstream": "edit", - "action_muldelete": "delete", - "action_set_failed": "edit", - "action_set_success": "edit", - "action_set_retry": "edit", - "action_set_skipped": "edit", - } - base_permissions = [ - permissions.ACTION_CAN_CREATE, - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_EDIT, - permissions.ACTION_CAN_DELETE, - permissions.ACTION_CAN_ACCESS_MENU, - ] - - page_size = PAGE_SIZE - - list_columns = [ - "state", - "dag_id", - "task_id", - "run_id", - "map_index", - "dag_run.logical_date", - "operator", - "start_date", - "end_date", - "duration", - "note", - "hostname", - "unixname", - "priority_weight", - "queue", - "queued_dttm", - "prev_attempted_tries", - "pool", - "queued_by_job_id", - "external_executor_id", - "log_url", - ] - - order_columns = [ - "state", - "dag_id", - "task_id", - "run_id", - "map_index", - "dag_run.logical_date", - "operator", - "start_date", - "end_date", - "duration", - # "note", # TODO: Maybe figure out how to re-enable this. - "hostname", - "unixname", - "priority_weight", - "queue", - "queued_dttm", - "pool", - "queued_by_job_id", - ] - # todo: don't use prev_attempted_tries; just use try_number - label_columns = {"dag_run.logical_date": "Logical Date", "prev_attempted_tries": "Try Number"} - - search_columns = [ - "state", - "dag_id", - "task_id", - "run_id", - "map_index", - "rendered_map_index", - "logical_date", - "operator", - "start_date", - "end_date", - "note", - "hostname", - "priority_weight", - "queue", - "queued_dttm", - "try_number", - "pool", - "queued_by_job_id", - ] - - edit_columns = [ - "dag_id", - "task_id", - "logical_date", - "start_date", - "end_date", - "state", - "note", - ] - - add_exclude_columns = ["next_method", "next_kwargs", "trigger_id"] - - edit_form = TaskInstanceEditForm - - base_order = ("queued_dttm", "asc") - - base_filters = [["dag_id", DagFilter, list]] - - def log_url_formatter(self): - """Format log URL.""" - dag_id = self.get("dag_id") - task_id = self.get("task_id") - run_id = self.get("run_id") - map_index = self.get("map_index", None) - if map_index == -1: - map_index = None - - url = url_for( - "Airflow.grid", - dag_id=dag_id, - task_id=task_id, - dag_run_id=run_id, - map_index=map_index, - tab="logs", - ) - return Markup( - '' - ).format(log_url=url) - - def duration_f(self): - """Format duration.""" - end_date = self.get("end_date") - duration = self.get("duration") - if end_date and duration: - return td_format(datetime.timedelta(seconds=duration)) - return None - - formatters_columns = { - "log_url": log_url_formatter, - "task_id": wwwutils.task_instance_link, - "run_id": wwwutils.dag_run_link, - "map_index": wwwutils.format_map_index, - "hostname": wwwutils.nobr_f("hostname"), - "state": wwwutils.state_f, - "dag_run.logical_date": wwwutils.datetime_f("dag_run.logical_date"), - "start_date": wwwutils.datetime_f("start_date"), - "end_date": wwwutils.datetime_f("end_date"), - "queued_dttm": wwwutils.datetime_f("queued_dttm"), - "dag_id": wwwutils.dag_link, - "duration": duration_f, - } - - def _clear_task_instances( - self, task_instances: list[TaskInstance], session: Session, clear_downstream: bool = False - ) -> tuple[int, int]: - """ - Clear task instances, optionally including their downstream dependencies. - - :param task_instances: list of TIs to clear - :param clear_downstream: should downstream task instances be cleared as well? - - :return: a tuple with: - - count of cleared task instances actually selected by the user - - count of downstream task instances that were additionally cleared - """ - cleared_tis_count = 0 - cleared_downstream_tis_count = 0 - - # Group TIs by dag id in order to call `get_dag` only once per dag - tis_grouped_by_dag_id = itertools.groupby(task_instances, lambda ti: ti.dag_id) - - for dag_id, dag_tis in tis_grouped_by_dag_id: - dag = get_airflow_app().dag_bag.get_dag(dag_id) - - tis_to_clear = list(dag_tis) - downstream_tis_to_clear = [] - - if clear_downstream: - tis_to_clear_grouped_by_dag_run = itertools.groupby(tis_to_clear, lambda ti: ti.dag_run) - - for dag_run, dag_run_tis in tis_to_clear_grouped_by_dag_run: - # Determine tasks that are downstream of the cleared TIs and fetch associated TIs - # This has to be run for each dag run because the user may clear different TIs across runs - task_ids_to_clear = [ti.task_id for ti in dag_run_tis] - - partial_dag = dag.partial_subset( - task_ids_or_regex=task_ids_to_clear, include_downstream=True, include_upstream=False - ) - - downstream_task_ids_to_clear = [ - task_id for task_id in partial_dag.task_dict if task_id not in task_ids_to_clear - ] - - # dag.clear returns TIs when in dry run mode - downstream_tis_to_clear.extend( - dag.clear( - start_date=dag_run.logical_date, - end_date=dag_run.logical_date, - task_ids=downstream_task_ids_to_clear, - session=session, - dry_run=True, - ) - ) - - # Once all TIs are fetched, perform the actual clearing - models.clear_task_instances(tis=tis_to_clear + downstream_tis_to_clear, session=session, dag=dag) - - cleared_tis_count += len(tis_to_clear) - cleared_downstream_tis_count += len(downstream_tis_to_clear) - - return cleared_tis_count, cleared_downstream_tis_count - - @action( - "clear", - lazy_gettext("Clear"), - lazy_gettext( - "Are you sure you want to clear the state of the selected task" - " instance(s) and set their dagruns to the QUEUED state?" - ), - single=False, - ) - @auth.has_access_dag_entities("PUT", DagAccessEntity.TASK_INSTANCE) - @provide_session - @action_logging - def action_clear(self, task_instances, session: Session = NEW_SESSION): - """Clear an arbitrary number of task instances.""" - try: - count, _ = self._clear_task_instances( - task_instances=task_instances, session=session, clear_downstream=False - ) - session.commit() - flash(f"{count} task instance{'s have' if count > 1 else ' has'} been cleared") - except Exception as e: - flash(f'Failed to clear task instances: "{e}"', "error") - - self.update_redirect() - return redirect(self.get_redirect()) - - @action( - "clear_downstream", - lazy_gettext("Clear (including downstream tasks)"), - lazy_gettext( - "Are you sure you want to clear the state of the selected task" - " instance(s) and all their downstream dependencies, and set their dagruns to the QUEUED state?" - ), - single=False, - ) - @auth.has_access_dag_entities("PUT", DagAccessEntity.TASK_INSTANCE) - @provide_session - @action_logging - def action_clear_downstream(self, task_instances, session: Session = NEW_SESSION): - """Clear an arbitrary number of task instances, including downstream dependencies.""" - try: - selected_ti_count, downstream_ti_count = self._clear_task_instances( - task_instances=task_instances, session=session, clear_downstream=True - ) - session.commit() - flash( - f"Cleared {selected_ti_count} selected task instance{'s' if selected_ti_count > 1 else ''} " - f"and {downstream_ti_count} downstream dependencies" - ) - except Exception as e: - flash(f'Failed to clear task instances: "{e}"', "error") - - self.update_redirect() - return redirect(self.get_redirect()) - - @action("muldelete", "Delete", "Are you sure you want to delete selected records?", single=False) - @auth.has_access_dag_entities("DELETE", DagAccessEntity.TASK_INSTANCE) - @action_logging - def action_muldelete(self, items): - self.datamodel.delete_all(items) - self.update_redirect() - return redirect(self.get_redirect()) - - @provide_session - def set_task_instance_state( - self, - tis: Collection[TaskInstance], - target_state: TaskInstanceState, - session: Session = NEW_SESSION, - ) -> None: - """Set task instance state.""" - try: - count = len(tis) - for ti in tis: - ti.set_state(target_state, session) - session.commit() - flash(f"{count} task instances were set to '{target_state}'") - except Exception: - flash("Failed to set state", "error") - - @action("set_failed", "Set state to 'failed'", "", single=False) - @auth.has_access_dag_entities("PUT", DagAccessEntity.TASK_INSTANCE) - @action_logging - def action_set_failed(self, tis): - """Set state to 'failed'.""" - self.set_task_instance_state(tis, TaskInstanceState.FAILED) - self.update_redirect() - return redirect(self.get_redirect()) - - @action("set_success", "Set state to 'success'", "", single=False) - @auth.has_access_dag_entities("PUT", DagAccessEntity.TASK_INSTANCE) - @action_logging - def action_set_success(self, tis): - """Set state to 'success'.""" - self.set_task_instance_state(tis, TaskInstanceState.SUCCESS) - self.update_redirect() - return redirect(self.get_redirect()) - - @action("set_retry", "Set state to 'up_for_retry'", "", single=False) - @auth.has_access_dag_entities("PUT", DagAccessEntity.TASK_INSTANCE) - @action_logging - def action_set_retry(self, tis): - """Set state to 'up_for_retry'.""" - self.set_task_instance_state(tis, TaskInstanceState.UP_FOR_RETRY) - self.update_redirect() - return redirect(self.get_redirect()) - - @action("set_skipped", "Set state to 'skipped'", "", single=False) - @auth.has_access_dag_entities("PUT", DagAccessEntity.TASK_INSTANCE) - @action_logging - def action_set_skipped(self, tis): - """Set state to skipped.""" - self.set_task_instance_state(tis, TaskInstanceState.SKIPPED) - self.update_redirect() - return redirect(self.get_redirect()) - - -class AutocompleteView(AirflowBaseView): - """View to provide autocomplete results.""" - - @provide_session - @expose("/dagmodel/autocomplete") - def autocomplete(self, session: Session = NEW_SESSION): - """Autocomplete.""" - query = unquote(request.args.get("query", "")) - - if not query: - return flask.json.jsonify([]) - - # Provide suggestions of dag_ids and owners - dag_ids_query = select( - sqla.literal("dag").label("type"), - DagModel.dag_id.label("name"), - DagModel._dag_display_property_value.label("dag_display_name"), - ).where( - DagModel.is_active, - or_( - DagModel.dag_id.ilike(f"%{query}%"), - DagModel._dag_display_property_value.ilike(f"%{query}%"), - ), - ) - - owners_query = ( - select( - sqla.literal("owner").label("type"), - DagModel.owners.label("name"), - sqla.literal(None).label("dag_display_name"), - ) - .distinct() - .where(DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) - ) - - # Hide DAGs if not showing status: "all" - status = flask_session.get(FILTER_STATUS_COOKIE) - if status == "active": - dag_ids_query = dag_ids_query.where(~DagModel.is_paused) - owners_query = owners_query.where(~DagModel.is_paused) - elif status == "paused": - dag_ids_query = dag_ids_query.where(DagModel.is_paused) - owners_query = owners_query.where(DagModel.is_paused) - - filter_dag_ids = get_auth_manager().get_permitted_dag_ids(user=g.user) - - dag_ids_query = dag_ids_query.where(DagModel.dag_id.in_(filter_dag_ids)) - owners_query = owners_query.where(DagModel.dag_id.in_(filter_dag_ids)) - payload = [ - row._asdict() - for row in session.execute(dag_ids_query.union(owners_query).order_by("name").limit(10)) - ] - return flask.json.jsonify(payload) - - -class DagDependenciesView(AirflowBaseView): - """View to show dependencies between DAGs.""" - - refresh_interval = datetime.timedelta( - seconds=conf.getint( - "webserver", - "dag_dependencies_refresh_interval", - fallback=300, - ) - ) - last_refresh = timezone.utcnow() - refresh_interval - nodes: list[dict[str, Any]] = [] - edges: list[dict[str, str]] = [] - - @expose("/dag-dependencies") - @auth.has_access_dag("GET", DagAccessEntity.DEPENDENCIES) - @gzipped - def list(self): - """Display DAG dependencies.""" - title = "DAG Dependencies" - - if not self.nodes or not self.edges: - self._calculate_graph() - self.last_refresh = timezone.utcnow() - elif timezone.utcnow() > self.last_refresh + self.refresh_interval: - max_last_updated = SerializedDagModel.get_max_last_updated_datetime() - if max_last_updated is None or max_last_updated > self.last_refresh: - self._calculate_graph() - self.last_refresh = timezone.utcnow() - - return self.render_template( - "airflow/dag_dependencies.html", - title=title, - nodes=self.nodes, - edges=self.edges, - last_refresh=self.last_refresh, - arrange=conf.get("webserver", "dag_orientation"), - width=request.args.get("width", "100%"), - height=request.args.get("height", "800"), - ) - - def _calculate_graph(self): - nodes_dict: dict[str, Any] = {} - edge_tuples: set[dict[str, str]] = set() - - for dag, dependencies in SerializedDagModel.get_dag_dependencies().items(): - dag_node_id = f"dag:{dag}" - if dag_node_id not in nodes_dict: - nodes_dict[dag_node_id] = node_dict(dag_node_id, dag, "dag") - - for dep in dependencies: - if dep.node_id not in nodes_dict: - nodes_dict[dep.node_id] = node_dict(dep.node_id, dep.dependency_id, dep.dependency_type) - edge_tuples.add((f"dag:{dep.source}", dep.node_id)) - edge_tuples.add((dep.node_id, f"dag:{dep.target}")) - - self.nodes = list(nodes_dict.values()) - self.edges = [{"u": u, "v": v} for u, v in edge_tuples] - - -def add_user_permissions_to_dag(sender, template, context, **extra): - """ - Add `.can_edit`, `.can_trigger`, and `.can_delete` properties to DAG based on current user's permissions. - - Located in `views.py` rather than the DAG model to keep permissions logic out of the Airflow core. - """ - if "dag" not in context: - return - dag = context["dag"] - can_create_dag_run = get_auth_manager().is_authorized_dag( - method="POST", access_entity=DagAccessEntity.RUN, details=DagDetails(id=dag.dag_id), user=g.user - ) - - dag.can_edit = get_auth_manager().is_authorized_dag( - method="PUT", details=DagDetails(id=dag.dag_id), user=g.user - ) - dag.can_trigger = dag.can_edit and can_create_dag_run - dag.can_delete = get_auth_manager().is_authorized_dag( - method="DELETE", details=DagDetails(id=dag.dag_id), user=g.user - ) - context["dag"] = dag - - -############################################################################## -# # -# Development Views # -# # -############################################################################## - - -def restrict_to_dev(f): - def wrapper(*args, **kwargs): - if not os.environ.get("AIRFLOW_ENV", None) == "development": - logger.error( - "You can only access this view in development mode. Set AIRFLOW_ENV=development to view it." - ) - return abort(404) - return f(*args, **kwargs) - - return wrapper - - -class DevView(BaseView): - """ - View to show Airflow Dev Endpoints. - - This view should only be accessible in development mode. You can enable development mode by setting - `AIRFLOW_ENV=development` in your environment. - - :meta private: - """ - - route_base = "/dev" - - @expose("/coverage/") - @restrict_to_dev - def coverage(self, path): - return send_from_directory(Path("htmlcov").resolve(), path) - - -class DocsView(BaseView): - """ - View to show airflow dev docs endpoints. - - This view should only be accessible in development mode. You can enable development mode by setting - `AIRFLOW_ENV=development` in your environment. - """ - - route_base = "/docs" - - @expose("/") - @expose("/") - @restrict_to_dev - def home(self, filename="index.html"): - """Serve documentation from the build directory.""" - if filename != "index.html": - return send_from_directory(Path("docs/_build/docs/").resolve(), filename) - return send_from_directory(Path("docs/_build/").resolve(), filename) - - -# NOTE: Put this at the end of the file. Pylance is too clever and detects that -# before_render_template.connect() is declared as NoReturn, and marks everything -# after this line as unreachable code. It's technically correct based on the -# lint-time information, but that's not what actually happens at runtime. -before_render_template.connect(add_user_permissions_to_dag) diff --git a/airflow/www/webpack.config.js b/airflow/www/webpack.config.js deleted file mode 100644 index 9d5800f783f50..0000000000000 --- a/airflow/www/webpack.config.js +++ /dev/null @@ -1,287 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -const webpack = require("webpack"); -const path = require("path"); -const { WebpackManifestPlugin } = require("webpack-manifest-plugin"); -const cwplg = require("clean-webpack-plugin"); -const CopyWebpackPlugin = require("copy-webpack-plugin"); -const MiniCssExtractPlugin = require("mini-css-extract-plugin"); -const MomentLocalesPlugin = require("moment-locales-webpack-plugin"); -const CssMinimizerPlugin = require("css-minimizer-webpack-plugin"); -const LicensePlugin = require("webpack-license-plugin"); -const TerserPlugin = require("terser-webpack-plugin"); - -// Input Directory (airflow/www) -// noinspection JSUnresolvedVariable -const CSS_DIR = path.resolve(__dirname, "./static/css"); -const JS_DIR = path.resolve(__dirname, "./static/js"); - -// Output Directory (airflow/www/static/dist) -// noinspection JSUnresolvedVariable -const BUILD_DIR = path.resolve(__dirname, "./static/dist"); - -// Convert licenses json into a standard format for LICENSES.txt -const formatLicenses = (packages) => { - let text = `Apache Airflow -Copyright 2016-2023 The Apache Software Foundation - -This product includes software developed at The Apache Software -Foundation (http://www.apache.org/). - -======================================================================= -`; - packages.forEach((p) => { - text += `${p.name}|${p.version}:\n-----\n${p.license}\n${ - p.licenseText || p.author - }\n${p.repository || ""}\n\n\n`; - }); - return text; -}; - -const config = { - entry: { - airflowDefaultTheme: `${CSS_DIR}/bootstrap-theme.css`, - connectionForm: `${JS_DIR}/connection_form.js`, - chart: [`${CSS_DIR}/chart.css`], - dag: `${JS_DIR}/dag.js`, - dagDependencies: `${JS_DIR}/dag_dependencies.js`, - dags: [`${CSS_DIR}/dags.css`, `${JS_DIR}/dags.js`], - flash: `${CSS_DIR}/flash.css`, - graph: `${CSS_DIR}/graph.css`, - loadingDots: `${CSS_DIR}/loading-dots.css`, - login: `${JS_DIR}/login/index.tsx`, - main: [`${CSS_DIR}/main.css`, `${JS_DIR}/main.js`], - materialIcons: `${CSS_DIR}/material-icons.css`, - moment: "moment-timezone", - switch: `${CSS_DIR}/switch.css`, - task: `${JS_DIR}/task.js`, - taskInstances: `${JS_DIR}/task_instances.js`, - tiLog: `${JS_DIR}/ti_log.js`, - toggleTheme: `${JS_DIR}/toggle_theme.js`, - grid: `${JS_DIR}/dag/index.tsx`, - clusterActivity: `${JS_DIR}/cluster-activity/index.tsx`, - assets: `${JS_DIR}/assets/index.tsx`, - trigger: `${JS_DIR}/trigger.js`, - variableEdit: `${JS_DIR}/variable_edit.js`, - }, - output: { - path: BUILD_DIR, - filename: "[name].[chunkhash].js", - chunkFilename: "[name].[chunkhash].js", - library: ["Airflow", "[name]"], - libraryTarget: "umd", - publicPath: "", - }, - resolve: { - alias: { - // Be sure to update aliases in jest.config.js and tsconfig.json - src: path.resolve(__dirname, "static/js"), - }, - extensions: [".js", ".jsx", ".ts", ".tsx", ".css"], - }, - module: { - rules: [ - { - test: /\.(js|jsx|tsx|ts)$/, - exclude: /node_modules/, - use: [ - { - loader: "babel-loader", - options: { - presets: ["@babel/preset-react", "@babel/preset-typescript"], - }, - }, - ], - }, - // Extract css files - { - test: /\.css$/, - include: CSS_DIR, - exclude: /node_modules/, - use: [ - { - loader: MiniCssExtractPlugin.loader, - options: { - esModule: true, - }, - }, - "css-loader", - ], - }, - // Extract css files - { - test: /\.css$/, - exclude: CSS_DIR, - include: /node_modules/, - use: ["css-loader"], - }, - /* for css linking images */ - { - test: /\.(png|jpg|gif)$/i, - use: [ - { - loader: "url-loader", - options: { - limit: 100000, - }, - }, - ], - }, - /* for fonts */ - { - test: /\.woff(2)?(\?v=[0-9]\.[0-9]\.[0-9])?$/, - use: [ - { - loader: "url-loader", - options: { - limit: 100000, - mimetype: "application/font-woff", - }, - }, - ], - }, - { - test: /\.(ttf|eot|svg)(\?v=[0-9]\.[0-9]\.[0-9])?$/, - use: [ - { - loader: "file-loader", - }, - ], - }, - ], - }, - plugins: [ - new WebpackManifestPlugin({ - // d3-tip is named index.js in its dist folder which was confusing the manifest - map: (file) => - file.path === "d3-tip.js" ? { ...file, name: "d3-tip.js" } : file, - }), - new cwplg.CleanWebpackPlugin({ - verbose: true, - }), - new MiniCssExtractPlugin({ - filename: "[name].[chunkhash].css", - }), - - // MomentJS loads all the locale, making it a huge JS file. - // This will ignore the locales from momentJS - new MomentLocalesPlugin(), - - new webpack.DefinePlugin({ - "process.env": { - NODE_ENV: JSON.stringify(process.env.NODE_ENV), - }, - }), - // Since we have all the dependencies separated from hard-coded JS within HTML, - // this seems like an efficient solution for now. Will update that once - // we'll have the dependencies imported within the custom JS - new CopyWebpackPlugin({ - patterns: [ - { - from: "node_modules/d3/d3.min.*", - flatten: true, - }, - { - from: "node_modules/dagre-d3/dist/*.min.*", - flatten: true, - }, - { - from: "node_modules/d3-shape/dist/*.min.*", - flatten: true, - }, - { - from: "node_modules/d3-tip/dist/index.js", - to: "d3-tip.js", - flatten: true, - }, - - { - from: "node_modules/eonasdan-bootstrap-datetimepicker/build/css/bootstrap-datetimepicker.min.css", - flatten: true, - }, - { - from: "node_modules/jquery-ui/dist/jquery-ui.min.js", - flatten: true, - }, - { - from: "node_modules/jquery-ui/dist/themes/base/jquery-ui.min.css", - flatten: true, - }, - { - from: "node_modules/eonasdan-bootstrap-datetimepicker/build/js/bootstrap-datetimepicker.min.js", - flatten: true, - }, - { - from: "node_modules/redoc/bundles/redoc.standalone.*", - flatten: true, - }, - { - from: "node_modules/codemirror/lib/codemirror.*", - flatten: true, - }, - { - from: "node_modules/codemirror/addon/lint/**.*", - flatten: true, - }, - { - from: "node_modules/codemirror/mode/javascript/javascript.js", - flatten: true, - }, - { - from: "node_modules/jshint/dist/jshint.js", - flatten: true, - }, - { - from: "templates/swagger-ui", - to: `${BUILD_DIR}/swagger-ui`, - }, - { - from: "node_modules/swagger-ui-dist", - to: `${BUILD_DIR}/swagger-ui`, - }, - ], - }), - new LicensePlugin({ - additionalFiles: { - "../../../../3rd-party-licenses/LICENSES-ui.txt": formatLicenses, - }, - unacceptableLicenseTest: (licenseIdentifier) => - [ - "BCL", - "JSR", - "ASL", - "RSAL", - "SSPL", - "CPOL", - "NPL", - "BSD-4", - "QPL", - "GPL", - "LGPL", - ].includes(licenseIdentifier), - }), - ], - optimization: { - minimize: process.env.NODE_ENV === "production", - minimizer: [new CssMinimizerPlugin({}), new TerserPlugin()], - }, -}; - -module.exports = config; diff --git a/airflow/www/widgets.py b/airflow/www/widgets.py deleted file mode 100644 index d1a8539ec906a..0000000000000 --- a/airflow/www/widgets.py +++ /dev/null @@ -1,82 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from flask_appbuilder.fieldwidgets import BS3TextAreaFieldWidget, BS3TextFieldWidget -from flask_appbuilder.widgets import RenderTemplateWidget -from markupsafe import Markup -from wtforms.widgets import html_params - - -class AirflowModelListWidget(RenderTemplateWidget): - """Airflow model list.""" - - template = "airflow/model_list.html" - - -class AirflowDateTimePickerWidget: - """Airflow date time picker widget.""" - - data_template = ( - '
    ' - 'calendar_today' - "" - '' - "
    " - ) - - def __call__(self, field, **kwargs): - kwargs.setdefault("id", field.id) - kwargs.setdefault("name", field.name) - if not field.data: - field.data = "" - template = self.data_template - - return Markup( - template % {"text": html_params(type="text", value=field.data, required=True, **kwargs)} - ) - - -class AirflowDateTimePickerROWidget(AirflowDateTimePickerWidget): - """Airflow Read-only date time picker widget.""" - - def __call__(self, field, **kwargs): - kwargs["readonly"] = "true" - return super().__call__(field, **kwargs) - - -class BS3TextFieldROWidget(BS3TextFieldWidget): - """Read-only single-line text input Widget (BS3TextFieldWidget).""" - - def __call__(self, field, **kwargs): - kwargs["readonly"] = "true" - return super().__call__(field, **kwargs) - - -class BS3TextAreaROWidget(BS3TextAreaFieldWidget): - """Read-only multi-line text area Widget (BS3TextAreaROWidget).""" - - def __call__(self, field, **kwargs): - kwargs["readonly"] = "true" - return super().__call__(field, **kwargs) - - -class AirflowVariableShowWidget(RenderTemplateWidget): - """Airflow variable show widget.""" - - template = "airflow/variable_show_widget.html" diff --git a/airflow/www/yarn.lock b/airflow/www/yarn.lock deleted file mode 100644 index c474e4c764dca..0000000000000 --- a/airflow/www/yarn.lock +++ /dev/null @@ -1,12073 +0,0 @@ -# THIS IS AN AUTOGENERATED FILE. DO NOT EDIT THIS FILE DIRECTLY. -# yarn lockfile v1 - - -"@ampproject/remapping@^2.0.0": - version "2.1.1" - resolved "https://registry.yarnpkg.com/@ampproject/remapping/-/remapping-2.1.1.tgz#7922fb0817bf3166d8d9e258c57477e3fd1c3610" - integrity sha512-Aolwjd7HSC2PyY0fDj/wA/EimQT4HfEnFYNp5s9CQlrdhyvWTtvZ5YzrUPu6R6/1jKiUlxu8bUhkdSnKHNAHMA== - dependencies: - "@jridgewell/trace-mapping" "^0.3.0" - -"@ampproject/remapping@^2.2.0": - version "2.3.0" - resolved "https://registry.yarnpkg.com/@ampproject/remapping/-/remapping-2.3.0.tgz#ed441b6fa600072520ce18b43d2c8cc8caecc7f4" - integrity sha512-30iZtAPgz+LTIYoeivqYo853f02jBYSd5uGnGpkFV0M3xOt9aN73erkgYAmZU43x4VfqcnLxW9Kpg3R5LC4YYw== - dependencies: - "@jridgewell/gen-mapping" "^0.3.5" - "@jridgewell/trace-mapping" "^0.3.24" - -"@babel/code-frame@^7.0.0": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.14.5.tgz#23b08d740e83f49c5e59945fbf1b43e80bbf4edb" - integrity sha512-9pzDqyc6OLDaqe+zbACgFkb6fKMNG6CObKpnYXChRsvYGyEdc7CA2BaqeOM+vOtCS5ndmJicPJhKAwYRI6UfFw== - dependencies: - "@babel/highlight" "^7.14.5" - -"@babel/code-frame@^7.10.4", "@babel/code-frame@^7.12.13", "@babel/code-frame@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.16.0.tgz#0dfc80309beec8411e65e706461c408b0bb9b431" - integrity sha512-IF4EOMEV+bfYwOmNxGzSnjR2EmQod7f1UXOpZM3l4i4o4QNwzjtJAu/HxdjHq0aYBvdqMuQEY1eg0nqW9ZPORA== - dependencies: - "@babel/highlight" "^7.16.0" - -"@babel/code-frame@^7.16.7": - version "7.16.7" - resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.16.7.tgz#44416b6bd7624b998f5b1af5d470856c40138789" - integrity sha512-iAXqUn8IIeBTNd72xsFlgaXHkMBMt6y4HJp1tIaK465CWLT/fG1aqB7ykr95gHHmlBdGbFeWWfyB4NJJ0nmeIg== - dependencies: - "@babel/highlight" "^7.16.7" - -"@babel/code-frame@^7.22.13": - version "7.22.13" - resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.22.13.tgz#e3c1c099402598483b7a8c46a721d1038803755e" - integrity sha512-XktuhWlJ5g+3TJXc5upd9Ks1HutSArik6jf2eAjYFyIOf4ej3RN+184cZbzDvbPnuTJIUhPKKJE3cIsYTiAT3w== - dependencies: - "@babel/highlight" "^7.22.13" - chalk "^2.4.2" - -"@babel/code-frame@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.24.7.tgz#882fd9e09e8ee324e496bd040401c6f046ef4465" - integrity sha512-BcYH1CVJBO9tvyIZ2jVeXgSIMvGZ2FDRvDdOIVQyuklNKSsx+eppDEBq/g47Ayw+RqNFE+URvOShmf+f/qwAlA== - dependencies: - "@babel/highlight" "^7.24.7" - picocolors "^1.0.0" - -"@babel/compat-data@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/compat-data/-/compat-data-7.16.0.tgz#ea269d7f78deb3a7826c39a4048eecda541ebdaa" - integrity sha512-DGjt2QZse5SGd9nfOSqO4WLJ8NN/oHkijbXbPrxuoJO3oIPJL3TciZs9FX+cOHNiY9E9l0opL8g7BmLe3T+9ew== - -"@babel/compat-data@^7.16.4": - version "7.17.0" - resolved "https://registry.yarnpkg.com/@babel/compat-data/-/compat-data-7.17.0.tgz#86850b8597ea6962089770952075dcaabb8dba34" - integrity sha512-392byTlpGWXMv4FbyWw3sAZ/FrW/DrwqLGXpy0mbyNe9Taqv1mg9yON5/o0cnr8XYCkFTZbC1eV+c+LAROgrng== - -"@babel/compat-data@^7.22.6", "@babel/compat-data@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/compat-data/-/compat-data-7.24.7.tgz#d23bbea508c3883ba8251fb4164982c36ea577ed" - integrity sha512-qJzAIcv03PyaWqxRgO4mSU3lihncDT296vnyuE2O8uA4w3UHWI4S3hgeZd1L8W1Bft40w9JxJ2b412iDUFFRhw== - -"@babel/core@^7.1.0", "@babel/core@^7.12.3", "@babel/core@^7.7.2": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/core/-/core-7.16.0.tgz#c4ff44046f5fe310525cc9eb4ef5147f0c5374d4" - integrity sha512-mYZEvshBRHGsIAiyH5PzCFTCfbWfoYbO/jcSdXQSUQu1/pW0xDZAUP7KEc32heqWTAfAHhV9j1vH8Sav7l+JNQ== - dependencies: - "@babel/code-frame" "^7.16.0" - "@babel/generator" "^7.16.0" - "@babel/helper-compilation-targets" "^7.16.0" - "@babel/helper-module-transforms" "^7.16.0" - "@babel/helpers" "^7.16.0" - "@babel/parser" "^7.16.0" - "@babel/template" "^7.16.0" - "@babel/traverse" "^7.16.0" - "@babel/types" "^7.16.0" - convert-source-map "^1.7.0" - debug "^4.1.0" - gensync "^1.0.0-beta.2" - json5 "^2.1.2" - semver "^6.3.0" - source-map "^0.5.0" - -"@babel/core@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/core/-/core-7.24.7.tgz#b676450141e0b52a3d43bc91da86aa608f950ac4" - integrity sha512-nykK+LEK86ahTkX/3TgauT0ikKoNCfKHEaZYTUVupJdTLzGNvrblu4u6fa7DhZONAltdf8e662t/abY8idrd/g== - dependencies: - "@ampproject/remapping" "^2.2.0" - "@babel/code-frame" "^7.24.7" - "@babel/generator" "^7.24.7" - "@babel/helper-compilation-targets" "^7.24.7" - "@babel/helper-module-transforms" "^7.24.7" - "@babel/helpers" "^7.24.7" - "@babel/parser" "^7.24.7" - "@babel/template" "^7.24.7" - "@babel/traverse" "^7.24.7" - "@babel/types" "^7.24.7" - convert-source-map "^2.0.0" - debug "^4.1.0" - gensync "^1.0.0-beta.2" - json5 "^2.2.3" - semver "^6.3.1" - -"@babel/core@^7.8.0": - version "7.17.2" - resolved "https://registry.yarnpkg.com/@babel/core/-/core-7.17.2.tgz#2c77fc430e95139d816d39b113b31bf40fb22337" - integrity sha512-R3VH5G42VSDolRHyUO4V2cfag8WHcZyxdq5Z/m8Xyb92lW/Erm/6kM+XtRFGf3Mulre3mveni2NHfEUws8wSvw== - dependencies: - "@ampproject/remapping" "^2.0.0" - "@babel/code-frame" "^7.16.7" - "@babel/generator" "^7.17.0" - "@babel/helper-compilation-targets" "^7.16.7" - "@babel/helper-module-transforms" "^7.16.7" - "@babel/helpers" "^7.17.2" - "@babel/parser" "^7.17.0" - "@babel/template" "^7.16.7" - "@babel/traverse" "^7.17.0" - "@babel/types" "^7.17.0" - convert-source-map "^1.7.0" - debug "^4.1.0" - gensync "^1.0.0-beta.2" - json5 "^2.1.2" - semver "^6.3.0" - -"@babel/eslint-parser@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/eslint-parser/-/eslint-parser-7.24.7.tgz#27ebab1a1ec21f48ae191a8aaac5b82baf80d9c7" - integrity sha512-SO5E3bVxDuxyNxM5agFv480YA2HO6ohZbGxbazZdIk3KQOPOGVNw6q78I9/lbviIf95eq6tPozeYnJLbjnC8IA== - dependencies: - "@nicolo-ribaudo/eslint-scope-5-internals" "5.1.1-v1" - eslint-visitor-keys "^2.1.0" - semver "^6.3.1" - -"@babel/generator@^7.16.0", "@babel/generator@^7.7.2": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/generator/-/generator-7.16.0.tgz#d40f3d1d5075e62d3500bccb67f3daa8a95265b2" - integrity sha512-RR8hUCfRQn9j9RPKEVXo9LiwoxLPYn6hNZlvUOR8tSnaxlD0p0+la00ZP9/SnRt6HchKr+X0fO2r8vrETiJGew== - dependencies: - "@babel/types" "^7.16.0" - jsesc "^2.5.1" - source-map "^0.5.0" - -"@babel/generator@^7.17.0": - version "7.17.0" - resolved "https://registry.yarnpkg.com/@babel/generator/-/generator-7.17.0.tgz#7bd890ba706cd86d3e2f727322346ffdbf98f65e" - integrity sha512-I3Omiv6FGOC29dtlZhkfXO6pgkmukJSlT26QjVvS1DGZe/NzSVCPG41X0tS21oZkJYlovfj9qDWgKP+Cn4bXxw== - dependencies: - "@babel/types" "^7.17.0" - jsesc "^2.5.1" - source-map "^0.5.0" - -"@babel/generator@^7.23.0": - version "7.23.0" - resolved "https://registry.yarnpkg.com/@babel/generator/-/generator-7.23.0.tgz#df5c386e2218be505b34837acbcb874d7a983420" - integrity sha512-lN85QRR+5IbYrMWM6Y4pE/noaQtg4pNiqeNGX60eqOfo6gtEj6uw/JagelB8vVztSd7R6M5n1+PQkDbHbBRU4g== - dependencies: - "@babel/types" "^7.23.0" - "@jridgewell/gen-mapping" "^0.3.2" - "@jridgewell/trace-mapping" "^0.3.17" - jsesc "^2.5.1" - -"@babel/generator@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/generator/-/generator-7.24.7.tgz#1654d01de20ad66b4b4d99c135471bc654c55e6d" - integrity sha512-oipXieGC3i45Y1A41t4tAqpnEZWgB/lC6Ehh6+rOviR5XWpTtMmLN+fGjz9vOiNRt0p6RtO6DtD0pdU3vpqdSA== - dependencies: - "@babel/types" "^7.24.7" - "@jridgewell/gen-mapping" "^0.3.5" - "@jridgewell/trace-mapping" "^0.3.25" - jsesc "^2.5.1" - -"@babel/helper-annotate-as-pure@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-annotate-as-pure/-/helper-annotate-as-pure-7.24.7.tgz#5373c7bc8366b12a033b4be1ac13a206c6656aab" - integrity sha512-BaDeOonYvhdKw+JoMVkAixAAJzG2jVPIwWoKBPdYuY9b452e2rPuI9QPYh3KpofZ3pW2akOmwZLOiOsHMiqRAg== - dependencies: - "@babel/types" "^7.24.7" - -"@babel/helper-builder-binary-assignment-operator-visitor@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-builder-binary-assignment-operator-visitor/-/helper-builder-binary-assignment-operator-visitor-7.24.7.tgz#37d66feb012024f2422b762b9b2a7cfe27c7fba3" - integrity sha512-xZeCVVdwb4MsDBkkyZ64tReWYrLRHlMN72vP7Bdm3OUOuyFZExhsHUUnuWnm2/XOlAJzR0LfPpB56WXZn0X/lA== - dependencies: - "@babel/traverse" "^7.24.7" - "@babel/types" "^7.24.7" - -"@babel/helper-compilation-targets@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/helper-compilation-targets/-/helper-compilation-targets-7.16.0.tgz#01d615762e796c17952c29e3ede9d6de07d235a8" - integrity sha512-S7iaOT1SYlqK0sQaCi21RX4+13hmdmnxIEAnQUB/eh7GeAnRjOUgTYpLkUOiRXzD+yog1JxP0qyAQZ7ZxVxLVg== - dependencies: - "@babel/compat-data" "^7.16.0" - "@babel/helper-validator-option" "^7.14.5" - browserslist "^4.16.6" - semver "^6.3.0" - -"@babel/helper-compilation-targets@^7.16.7": - version "7.16.7" - resolved "https://registry.yarnpkg.com/@babel/helper-compilation-targets/-/helper-compilation-targets-7.16.7.tgz#06e66c5f299601e6c7da350049315e83209d551b" - integrity sha512-mGojBwIWcwGD6rfqgRXVlVYmPAv7eOpIemUG3dGnDdCY4Pae70ROij3XmfrH6Fa1h1aiDylpglbZyktfzyo/hA== - dependencies: - "@babel/compat-data" "^7.16.4" - "@babel/helper-validator-option" "^7.16.7" - browserslist "^4.17.5" - semver "^6.3.0" - -"@babel/helper-compilation-targets@^7.22.6", "@babel/helper-compilation-targets@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-compilation-targets/-/helper-compilation-targets-7.24.7.tgz#4eb6c4a80d6ffeac25ab8cd9a21b5dfa48d503a9" - integrity sha512-ctSdRHBi20qWOfy27RUb4Fhp07KSJ3sXcuSvTrXrc4aG8NSYDo1ici3Vhg9bg69y5bj0Mr1lh0aeEgTvc12rMg== - dependencies: - "@babel/compat-data" "^7.24.7" - "@babel/helper-validator-option" "^7.24.7" - browserslist "^4.22.2" - lru-cache "^5.1.1" - semver "^6.3.1" - -"@babel/helper-create-class-features-plugin@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-create-class-features-plugin/-/helper-create-class-features-plugin-7.24.7.tgz#2eaed36b3a1c11c53bdf80d53838b293c52f5b3b" - integrity sha512-kTkaDl7c9vO80zeX1rJxnuRpEsD5tA81yh11X1gQo+PhSti3JS+7qeZo9U4RHobKRiFPKaGK3svUAeb8D0Q7eg== - dependencies: - "@babel/helper-annotate-as-pure" "^7.24.7" - "@babel/helper-environment-visitor" "^7.24.7" - "@babel/helper-function-name" "^7.24.7" - "@babel/helper-member-expression-to-functions" "^7.24.7" - "@babel/helper-optimise-call-expression" "^7.24.7" - "@babel/helper-replace-supers" "^7.24.7" - "@babel/helper-skip-transparent-expression-wrappers" "^7.24.7" - "@babel/helper-split-export-declaration" "^7.24.7" - semver "^6.3.1" - -"@babel/helper-create-regexp-features-plugin@^7.18.6", "@babel/helper-create-regexp-features-plugin@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-create-regexp-features-plugin/-/helper-create-regexp-features-plugin-7.24.7.tgz#be4f435a80dc2b053c76eeb4b7d16dd22cfc89da" - integrity sha512-03TCmXy2FtXJEZfbXDTSqq1fRJArk7lX9DOFC/47VthYcxyIOx+eXQmdo6DOQvrbpIix+KfXwvuXdFDZHxt+rA== - dependencies: - "@babel/helper-annotate-as-pure" "^7.24.7" - regexpu-core "^5.3.1" - semver "^6.3.1" - -"@babel/helper-define-polyfill-provider@^0.6.1", "@babel/helper-define-polyfill-provider@^0.6.2": - version "0.6.2" - resolved "https://registry.yarnpkg.com/@babel/helper-define-polyfill-provider/-/helper-define-polyfill-provider-0.6.2.tgz#18594f789c3594acb24cfdb4a7f7b7d2e8bd912d" - integrity sha512-LV76g+C502biUK6AyZ3LK10vDpDyCzZnhZFXkH1L75zHPj68+qc8Zfpx2th+gzwA2MzyK+1g/3EPl62yFnVttQ== - dependencies: - "@babel/helper-compilation-targets" "^7.22.6" - "@babel/helper-plugin-utils" "^7.22.5" - debug "^4.1.1" - lodash.debounce "^4.0.8" - resolve "^1.14.2" - -"@babel/helper-environment-visitor@^7.16.7": - version "7.16.7" - resolved "https://registry.yarnpkg.com/@babel/helper-environment-visitor/-/helper-environment-visitor-7.16.7.tgz#ff484094a839bde9d89cd63cba017d7aae80ecd7" - integrity sha512-SLLb0AAn6PkUeAfKJCCOl9e1R53pQlGAfc4y4XuMRZfqeMYLE0dM1LMhqbGAlGQY0lfw5/ohoYWAe9V1yibRag== - dependencies: - "@babel/types" "^7.16.7" - -"@babel/helper-environment-visitor@^7.22.20": - version "7.22.20" - resolved "https://registry.yarnpkg.com/@babel/helper-environment-visitor/-/helper-environment-visitor-7.22.20.tgz#96159db61d34a29dba454c959f5ae4a649ba9167" - integrity sha512-zfedSIzFhat/gFhWfHtgWvlec0nqB9YEIVrpuwjruLlXfUSnA8cJB0miHKwqDnQ7d32aKo2xt88/xZptwxbfhA== - -"@babel/helper-environment-visitor@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-environment-visitor/-/helper-environment-visitor-7.24.7.tgz#4b31ba9551d1f90781ba83491dd59cf9b269f7d9" - integrity sha512-DoiN84+4Gnd0ncbBOM9AZENV4a5ZiL39HYMyZJGZ/AZEykHYdJw0wW3kdcsh9/Kn+BRXHLkkklZ51ecPKmI1CQ== - dependencies: - "@babel/types" "^7.24.7" - -"@babel/helper-function-name@^7.23.0": - version "7.23.0" - resolved "https://registry.yarnpkg.com/@babel/helper-function-name/-/helper-function-name-7.23.0.tgz#1f9a3cdbd5b2698a670c30d2735f9af95ed52759" - integrity sha512-OErEqsrxjZTJciZ4Oo+eoZqeW9UIiOcuYKRJA4ZAgV9myA+pOXhhmpfNCKjEH/auVfEYVFJ6y1Tc4r0eIApqiw== - dependencies: - "@babel/template" "^7.22.15" - "@babel/types" "^7.23.0" - -"@babel/helper-function-name@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-function-name/-/helper-function-name-7.24.7.tgz#75f1e1725742f39ac6584ee0b16d94513da38dd2" - integrity sha512-FyoJTsj/PEUWu1/TYRiXTIHc8lbw+TDYkZuoE43opPS5TrI7MyONBE1oNvfguEXAD9yhQRrVBnXdXzSLQl9XnA== - dependencies: - "@babel/template" "^7.24.7" - "@babel/types" "^7.24.7" - -"@babel/helper-hoist-variables@^7.22.5": - version "7.22.5" - resolved "https://registry.yarnpkg.com/@babel/helper-hoist-variables/-/helper-hoist-variables-7.22.5.tgz#c01a007dac05c085914e8fb652b339db50d823bb" - integrity sha512-wGjk9QZVzvknA6yKIUURb8zY3grXCcOZt+/7Wcy8O2uctxhplmUPkOdlgoNhmdVee2c92JXbf1xpMtVNbfoxRw== - dependencies: - "@babel/types" "^7.22.5" - -"@babel/helper-hoist-variables@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-hoist-variables/-/helper-hoist-variables-7.24.7.tgz#b4ede1cde2fd89436397f30dc9376ee06b0f25ee" - integrity sha512-MJJwhkoGy5c4ehfoRyrJ/owKeMl19U54h27YYftT0o2teQ3FJ3nQUf/I3LlJsX4l3qlw7WRXUmiyajvHXoTubQ== - dependencies: - "@babel/types" "^7.24.7" - -"@babel/helper-member-expression-to-functions@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/helper-member-expression-to-functions/-/helper-member-expression-to-functions-7.16.0.tgz#29287040efd197c77636ef75188e81da8bccd5a4" - integrity sha512-bsjlBFPuWT6IWhl28EdrQ+gTvSvj5tqVP5Xeftp07SEuz5pLnsXZuDkDD3Rfcxy0IsHmbZ+7B2/9SHzxO0T+sQ== - dependencies: - "@babel/types" "^7.16.0" - -"@babel/helper-member-expression-to-functions@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-member-expression-to-functions/-/helper-member-expression-to-functions-7.24.7.tgz#67613d068615a70e4ed5101099affc7a41c5225f" - integrity sha512-LGeMaf5JN4hAT471eJdBs/GK1DoYIJ5GCtZN/EsL6KUiiDZOvO/eKE11AMZJa2zP4zk4qe9V2O/hxAmkRc8p6w== - dependencies: - "@babel/traverse" "^7.24.7" - "@babel/types" "^7.24.7" - -"@babel/helper-module-imports@^7.12.13": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/helper-module-imports/-/helper-module-imports-7.14.5.tgz#6d1a44df6a38c957aa7c312da076429f11b422f3" - integrity sha512-SwrNHu5QWS84XlHwGYPDtCxcA0hrSlL2yhWYLgeOc0w7ccOl2qv4s/nARI0aYZW+bSwAL5CukeXA47B/1NKcnQ== - dependencies: - "@babel/types" "^7.14.5" - -"@babel/helper-module-imports@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/helper-module-imports/-/helper-module-imports-7.16.0.tgz#90538e60b672ecf1b448f5f4f5433d37e79a3ec3" - integrity sha512-kkH7sWzKPq0xt3H1n+ghb4xEMP8k0U7XV3kkB+ZGy69kDk2ySFW1qPi06sjKzFY3t1j6XbJSqr4mF9L7CYVyhg== - dependencies: - "@babel/types" "^7.16.0" - -"@babel/helper-module-imports@^7.16.7": - version "7.16.7" - resolved "https://registry.yarnpkg.com/@babel/helper-module-imports/-/helper-module-imports-7.16.7.tgz#25612a8091a999704461c8a222d0efec5d091437" - integrity sha512-LVtS6TqjJHFc+nYeITRo6VLXve70xmq7wPhWTqDJusJEgGmkAACWwMiTNrvfoQo6hEhFwAIixNkvB0jPXDL8Wg== - dependencies: - "@babel/types" "^7.16.7" - -"@babel/helper-module-imports@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-module-imports/-/helper-module-imports-7.24.7.tgz#f2f980392de5b84c3328fc71d38bd81bbb83042b" - integrity sha512-8AyH3C+74cgCVVXow/myrynrAGv+nTVg5vKu2nZph9x7RcRwzmh0VFallJuFTZ9mx6u4eSdXZfcOzSqTUm0HCA== - dependencies: - "@babel/traverse" "^7.24.7" - "@babel/types" "^7.24.7" - -"@babel/helper-module-transforms@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/helper-module-transforms/-/helper-module-transforms-7.16.0.tgz#1c82a8dd4cb34577502ebd2909699b194c3e9bb5" - integrity sha512-My4cr9ATcaBbmaEa8M0dZNA74cfI6gitvUAskgDtAFmAqyFKDSHQo5YstxPbN+lzHl2D9l/YOEFqb2mtUh4gfA== - dependencies: - "@babel/helper-module-imports" "^7.16.0" - "@babel/helper-replace-supers" "^7.16.0" - "@babel/helper-simple-access" "^7.16.0" - "@babel/helper-split-export-declaration" "^7.16.0" - "@babel/helper-validator-identifier" "^7.15.7" - "@babel/template" "^7.16.0" - "@babel/traverse" "^7.16.0" - "@babel/types" "^7.16.0" - -"@babel/helper-module-transforms@^7.16.7": - version "7.16.7" - resolved "https://registry.yarnpkg.com/@babel/helper-module-transforms/-/helper-module-transforms-7.16.7.tgz#7665faeb721a01ca5327ddc6bba15a5cb34b6a41" - integrity sha512-gaqtLDxJEFCeQbYp9aLAefjhkKdjKcdh6DB7jniIGU3Pz52WAmP268zK0VgPz9hUNkMSYeH976K2/Y6yPadpng== - dependencies: - "@babel/helper-environment-visitor" "^7.16.7" - "@babel/helper-module-imports" "^7.16.7" - "@babel/helper-simple-access" "^7.16.7" - "@babel/helper-split-export-declaration" "^7.16.7" - "@babel/helper-validator-identifier" "^7.16.7" - "@babel/template" "^7.16.7" - "@babel/traverse" "^7.16.7" - "@babel/types" "^7.16.7" - -"@babel/helper-module-transforms@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-module-transforms/-/helper-module-transforms-7.24.7.tgz#31b6c9a2930679498db65b685b1698bfd6c7daf8" - integrity sha512-1fuJEwIrp+97rM4RWdO+qrRsZlAeL1lQJoPqtCYWv0NL115XM93hIH4CSRln2w52SqvmY5hqdtauB6QFCDiZNQ== - dependencies: - "@babel/helper-environment-visitor" "^7.24.7" - "@babel/helper-module-imports" "^7.24.7" - "@babel/helper-simple-access" "^7.24.7" - "@babel/helper-split-export-declaration" "^7.24.7" - "@babel/helper-validator-identifier" "^7.24.7" - -"@babel/helper-optimise-call-expression@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/helper-optimise-call-expression/-/helper-optimise-call-expression-7.16.0.tgz#cecdb145d70c54096b1564f8e9f10cd7d193b338" - integrity sha512-SuI467Gi2V8fkofm2JPnZzB/SUuXoJA5zXe/xzyPP2M04686RzFKFHPK6HDVN6JvWBIEW8tt9hPR7fXdn2Lgpw== - dependencies: - "@babel/types" "^7.16.0" - -"@babel/helper-optimise-call-expression@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-optimise-call-expression/-/helper-optimise-call-expression-7.24.7.tgz#8b0a0456c92f6b323d27cfd00d1d664e76692a0f" - integrity sha512-jKiTsW2xmWwxT1ixIdfXUZp+P5yURx2suzLZr5Hi64rURpDYdMW0pv+Uf17EYk2Rd428Lx4tLsnjGJzYKDM/6A== - dependencies: - "@babel/types" "^7.24.7" - -"@babel/helper-plugin-utils@^7.0.0", "@babel/helper-plugin-utils@^7.10.4", "@babel/helper-plugin-utils@^7.12.13", "@babel/helper-plugin-utils@^7.14.5", "@babel/helper-plugin-utils@^7.8.0", "@babel/helper-plugin-utils@^7.8.3": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/helper-plugin-utils/-/helper-plugin-utils-7.14.5.tgz#5ac822ce97eec46741ab70a517971e443a70c5a9" - integrity sha512-/37qQCE3K0vvZKwoK4XU/irIJQdIfCJuhU5eKnNxpFDsOkgFaUAwbv+RYw6eYgsC0E4hS7r5KqGULUogqui0fQ== - -"@babel/helper-plugin-utils@^7.18.6", "@babel/helper-plugin-utils@^7.22.5", "@babel/helper-plugin-utils@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-plugin-utils/-/helper-plugin-utils-7.24.7.tgz#98c84fe6fe3d0d3ae7bfc3a5e166a46844feb2a0" - integrity sha512-Rq76wjt7yz9AAc1KnlRKNAi/dMSVWgDRx43FHoJEbcYU6xOWaE2dVPwcdTukJrjxS65GITyfbvEYHvkirZ6uEg== - -"@babel/helper-remap-async-to-generator@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-remap-async-to-generator/-/helper-remap-async-to-generator-7.24.7.tgz#b3f0f203628522713849d49403f1a414468be4c7" - integrity sha512-9pKLcTlZ92hNZMQfGCHImUpDOlAgkkpqalWEeftW5FBya75k8Li2ilerxkM/uBEj01iBZXcCIB/bwvDYgWyibA== - dependencies: - "@babel/helper-annotate-as-pure" "^7.24.7" - "@babel/helper-environment-visitor" "^7.24.7" - "@babel/helper-wrap-function" "^7.24.7" - -"@babel/helper-replace-supers@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/helper-replace-supers/-/helper-replace-supers-7.16.0.tgz#73055e8d3cf9bcba8ddb55cad93fedc860f68f17" - integrity sha512-TQxuQfSCdoha7cpRNJvfaYxxxzmbxXw/+6cS7V02eeDYyhxderSoMVALvwupA54/pZcOTtVeJ0xccp1nGWladA== - dependencies: - "@babel/helper-member-expression-to-functions" "^7.16.0" - "@babel/helper-optimise-call-expression" "^7.16.0" - "@babel/traverse" "^7.16.0" - "@babel/types" "^7.16.0" - -"@babel/helper-replace-supers@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-replace-supers/-/helper-replace-supers-7.24.7.tgz#f933b7eed81a1c0265740edc91491ce51250f765" - integrity sha512-qTAxxBM81VEyoAY0TtLrx1oAEJc09ZK67Q9ljQToqCnA+55eNwCORaxlKyu+rNfX86o8OXRUSNUnrtsAZXM9sg== - dependencies: - "@babel/helper-environment-visitor" "^7.24.7" - "@babel/helper-member-expression-to-functions" "^7.24.7" - "@babel/helper-optimise-call-expression" "^7.24.7" - -"@babel/helper-simple-access@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/helper-simple-access/-/helper-simple-access-7.16.0.tgz#21d6a27620e383e37534cf6c10bba019a6f90517" - integrity sha512-o1rjBT/gppAqKsYfUdfHq5Rk03lMQrkPHG1OWzHWpLgVXRH4HnMM9Et9CVdIqwkCQlobnGHEJMsgWP/jE1zUiw== - dependencies: - "@babel/types" "^7.16.0" - -"@babel/helper-simple-access@^7.16.7": - version "7.16.7" - resolved "https://registry.yarnpkg.com/@babel/helper-simple-access/-/helper-simple-access-7.16.7.tgz#d656654b9ea08dbb9659b69d61063ccd343ff0f7" - integrity sha512-ZIzHVyoeLMvXMN/vok/a4LWRy8G2v205mNP0XOuf9XRLyX5/u9CnVulUtDgUTama3lT+bf/UqucuZjqiGuTS1g== - dependencies: - "@babel/types" "^7.16.7" - -"@babel/helper-simple-access@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-simple-access/-/helper-simple-access-7.24.7.tgz#bcade8da3aec8ed16b9c4953b74e506b51b5edb3" - integrity sha512-zBAIvbCMh5Ts+b86r/CjU+4XGYIs+R1j951gxI3KmmxBMhCg4oQMsv6ZXQ64XOm/cvzfU1FmoCyt6+owc5QMYg== - dependencies: - "@babel/traverse" "^7.24.7" - "@babel/types" "^7.24.7" - -"@babel/helper-skip-transparent-expression-wrappers@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-skip-transparent-expression-wrappers/-/helper-skip-transparent-expression-wrappers-7.24.7.tgz#5f8fa83b69ed5c27adc56044f8be2b3ea96669d9" - integrity sha512-IO+DLT3LQUElMbpzlatRASEyQtfhSE0+m465v++3jyyXeBTBUjtVZg28/gHeV5mrTJqvEKhKroBGAvhW+qPHiQ== - dependencies: - "@babel/traverse" "^7.24.7" - "@babel/types" "^7.24.7" - -"@babel/helper-split-export-declaration@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.16.0.tgz#29672f43663e936df370aaeb22beddb3baec7438" - integrity sha512-0YMMRpuDFNGTHNRiiqJX19GjNXA4H0E8jZ2ibccfSxaCogbm3am5WN/2nQNj0YnQwGWM1J06GOcQ2qnh3+0paw== - dependencies: - "@babel/types" "^7.16.0" - -"@babel/helper-split-export-declaration@^7.16.7": - version "7.16.7" - resolved "https://registry.yarnpkg.com/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.16.7.tgz#0b648c0c42da9d3920d85ad585f2778620b8726b" - integrity sha512-xbWoy/PFoxSWazIToT9Sif+jJTlrMcndIsaOKvTA6u7QEo7ilkRZpjew18/W3c7nm8fXdUDXh02VXTbZ0pGDNw== - dependencies: - "@babel/types" "^7.16.7" - -"@babel/helper-split-export-declaration@^7.22.6": - version "7.22.6" - resolved "https://registry.yarnpkg.com/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.22.6.tgz#322c61b7310c0997fe4c323955667f18fcefb91c" - integrity sha512-AsUnxuLhRYsisFiaJwvp1QF+I3KjD5FOxut14q/GzovUe6orHLesW2C7d754kRm53h5gqrz6sFl6sxc4BVtE/g== - dependencies: - "@babel/types" "^7.22.5" - -"@babel/helper-split-export-declaration@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.24.7.tgz#83949436890e07fa3d6873c61a96e3bbf692d856" - integrity sha512-oy5V7pD+UvfkEATUKvIjvIAH/xCzfsFVw7ygW2SI6NClZzquT+mwdTfgfdbUiceh6iQO0CHtCPsyze/MZ2YbAA== - dependencies: - "@babel/types" "^7.24.7" - -"@babel/helper-string-parser@^7.22.5": - version "7.22.5" - resolved "https://registry.yarnpkg.com/@babel/helper-string-parser/-/helper-string-parser-7.22.5.tgz#533f36457a25814cf1df6488523ad547d784a99f" - integrity sha512-mM4COjgZox8U+JcXQwPijIZLElkgEpO5rsERVDJTc2qfCDfERyob6k5WegS14SX18IIjv+XD+GrqNumY5JRCDw== - -"@babel/helper-string-parser@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-string-parser/-/helper-string-parser-7.24.7.tgz#4d2d0f14820ede3b9807ea5fc36dfc8cd7da07f2" - integrity sha512-7MbVt6xrwFQbunH2DNQsAP5sTGxfqQtErvBIvIMi6EQnbgUOuVYanvREcmFrOPhoXBrTtjhhP+lW+o5UfK+tDg== - -"@babel/helper-validator-identifier@^7.14.5": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.14.5.tgz#d0f0e277c512e0c938277faa85a3968c9a44c0e8" - integrity sha512-5lsetuxCLilmVGyiLEfoHBRX8UCFD+1m2x3Rj97WrW3V7H3u4RWRXA4evMjImCsin2J2YT0QaVDGf+z8ondbAg== - -"@babel/helper-validator-identifier@^7.15.7": - version "7.15.7" - resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.15.7.tgz#220df993bfe904a4a6b02ab4f3385a5ebf6e2389" - integrity sha512-K4JvCtQqad9OY2+yTU8w+E82ywk/fe+ELNlt1G8z3bVGlZfn/hOcQQsUhGhW/N+tb3fxK800wLtKOE/aM0m72w== - -"@babel/helper-validator-identifier@^7.16.7": - version "7.16.7" - resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.16.7.tgz#e8c602438c4a8195751243da9031d1607d247cad" - integrity sha512-hsEnFemeiW4D08A5gUAZxLBTXpZ39P+a+DGDsHw1yxqyQ/jzFEnxf5uTEGp+3bzAbNOxU1paTgYS4ECU/IgfDw== - -"@babel/helper-validator-identifier@^7.22.20": - version "7.22.20" - resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.22.20.tgz#c4ae002c61d2879e724581d96665583dbc1dc0e0" - integrity sha512-Y4OZ+ytlatR8AI+8KZfKuL5urKp7qey08ha31L8b3BwewJAoJamTzyvxPR/5D+KkdJCGPq/+8TukHBlY10FX9A== - -"@babel/helper-validator-identifier@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.24.7.tgz#75b889cfaf9e35c2aaf42cf0d72c8e91719251db" - integrity sha512-rR+PBcQ1SMQDDyF6X0wxtG8QyLCgUB0eRAGguqRLfkCA87l7yAP7ehq8SNj96OOGTO8OBV70KhuFYcIkHXOg0w== - -"@babel/helper-validator-option@^7.14.5": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/helper-validator-option/-/helper-validator-option-7.14.5.tgz#6e72a1fff18d5dfcb878e1e62f1a021c4b72d5a3" - integrity sha512-OX8D5eeX4XwcroVW45NMvoYaIuFI+GQpA2a8Gi+X/U/cDUIRsV37qQfF905F0htTRCREQIB4KqPeaveRJUl3Ow== - -"@babel/helper-validator-option@^7.16.7": - version "7.16.7" - resolved "https://registry.yarnpkg.com/@babel/helper-validator-option/-/helper-validator-option-7.16.7.tgz#b203ce62ce5fe153899b617c08957de860de4d23" - integrity sha512-TRtenOuRUVo9oIQGPC5G9DgK4743cdxvtOw0weQNpZXaS16SCBi5MNjZF8vba3ETURjZpTbVn7Vvcf2eAwFozQ== - -"@babel/helper-validator-option@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-validator-option/-/helper-validator-option-7.24.7.tgz#24c3bb77c7a425d1742eec8fb433b5a1b38e62f6" - integrity sha512-yy1/KvjhV/ZCL+SM7hBrvnZJ3ZuT9OuZgIJAGpPEToANvc3iM6iDvBnRjtElWibHU6n8/LPR/EjX9EtIEYO3pw== - -"@babel/helper-wrap-function@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helper-wrap-function/-/helper-wrap-function-7.24.7.tgz#52d893af7e42edca7c6d2c6764549826336aae1f" - integrity sha512-N9JIYk3TD+1vq/wn77YnJOqMtfWhNewNE+DJV4puD2X7Ew9J4JvrzrFDfTfyv5EgEXVy9/Wt8QiOErzEmv5Ifw== - dependencies: - "@babel/helper-function-name" "^7.24.7" - "@babel/template" "^7.24.7" - "@babel/traverse" "^7.24.7" - "@babel/types" "^7.24.7" - -"@babel/helpers@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/helpers/-/helpers-7.16.0.tgz#875519c979c232f41adfbd43a3b0398c2e388183" - integrity sha512-dVRM0StFMdKlkt7cVcGgwD8UMaBfWJHl3A83Yfs8GQ3MO0LHIIIMvK7Fa0RGOGUQ10qikLaX6D7o5htcQWgTMQ== - dependencies: - "@babel/template" "^7.16.0" - "@babel/traverse" "^7.16.0" - "@babel/types" "^7.16.0" - -"@babel/helpers@^7.17.2": - version "7.17.2" - resolved "https://registry.yarnpkg.com/@babel/helpers/-/helpers-7.17.2.tgz#23f0a0746c8e287773ccd27c14be428891f63417" - integrity sha512-0Qu7RLR1dILozr/6M0xgj+DFPmi6Bnulgm9M8BVa9ZCWxDqlSnqt3cf8IDPB5m45sVXUZ0kuQAgUrdSFFH79fQ== - dependencies: - "@babel/template" "^7.16.7" - "@babel/traverse" "^7.17.0" - "@babel/types" "^7.17.0" - -"@babel/helpers@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/helpers/-/helpers-7.24.7.tgz#aa2ccda29f62185acb5d42fb4a3a1b1082107416" - integrity sha512-NlmJJtvcw72yRJRcnCmGvSi+3jDEg8qFu3z0AFoymmzLx5ERVWyzd9kVXr7Th9/8yIJi2Zc6av4Tqz3wFs8QWg== - dependencies: - "@babel/template" "^7.24.7" - "@babel/types" "^7.24.7" - -"@babel/highlight@^7.14.5": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.14.5.tgz#6861a52f03966405001f6aa534a01a24d99e8cd9" - integrity sha512-qf9u2WFWVV0MppaL877j2dBtQIDgmidgjGk5VIMw3OadXvYaXn66U1BFlH2t4+t3i+8PhedppRv+i40ABzd+gg== - dependencies: - "@babel/helper-validator-identifier" "^7.14.5" - chalk "^2.0.0" - js-tokens "^4.0.0" - -"@babel/highlight@^7.16.0": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.16.0.tgz#6ceb32b2ca4b8f5f361fb7fd821e3fddf4a1725a" - integrity sha512-t8MH41kUQylBtu2+4IQA3atqevA2lRgqA2wyVB/YiWmsDSuylZZuXOUy9ric30hfzauEFfdsuk/eXTRrGrfd0g== - dependencies: - "@babel/helper-validator-identifier" "^7.15.7" - chalk "^2.0.0" - js-tokens "^4.0.0" - -"@babel/highlight@^7.16.7": - version "7.16.10" - resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.16.10.tgz#744f2eb81579d6eea753c227b0f570ad785aba88" - integrity sha512-5FnTQLSLswEj6IkgVw5KusNUUFY9ZGqe/TRFnP/BKYHYgfh7tc+C7mwiy95/yNP7Dh9x580Vv8r7u7ZfTBFxdw== - dependencies: - "@babel/helper-validator-identifier" "^7.16.7" - chalk "^2.0.0" - js-tokens "^4.0.0" - -"@babel/highlight@^7.22.13": - version "7.22.20" - resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.22.20.tgz#4ca92b71d80554b01427815e06f2df965b9c1f54" - integrity sha512-dkdMCN3py0+ksCgYmGG8jKeGA/8Tk+gJwSYYlFGxG5lmhfKNoAy004YpLxpS1W2J8m/EK2Ew+yOs9pVRwO89mg== - dependencies: - "@babel/helper-validator-identifier" "^7.22.20" - chalk "^2.4.2" - js-tokens "^4.0.0" - -"@babel/highlight@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.24.7.tgz#a05ab1df134b286558aae0ed41e6c5f731bf409d" - integrity sha512-EStJpq4OuY8xYfhGVXngigBJRWxftKX9ksiGDnmlY3o7B/V7KIAc9X4oiK87uPJSc/vs5L869bem5fhZa8caZw== - dependencies: - "@babel/helper-validator-identifier" "^7.24.7" - chalk "^2.4.2" - js-tokens "^4.0.0" - picocolors "^1.0.0" - -"@babel/parser@^7.1.0", "@babel/parser@^7.14.7", "@babel/parser@^7.16.0": - version "7.16.2" - resolved "https://registry.yarnpkg.com/@babel/parser/-/parser-7.16.2.tgz#3723cd5c8d8773eef96ce57ea1d9b7faaccd12ac" - integrity sha512-RUVpT0G2h6rOZwqLDTrKk7ksNv7YpAilTnYe1/Q+eDjxEceRMKVWbCsX7t8h6C1qCFi/1Y8WZjcEPBAFG27GPw== - -"@babel/parser@^7.16.7", "@babel/parser@^7.17.0": - version "7.17.0" - resolved "https://registry.yarnpkg.com/@babel/parser/-/parser-7.17.0.tgz#f0ac33eddbe214e4105363bb17c3341c5ffcc43c" - integrity sha512-VKXSCQx5D8S04ej+Dqsr1CzYvvWgf20jIw2D+YhQCrIlr2UZGaDds23Y0xg75/skOxpLCRpUZvk/1EAVkGoDOw== - -"@babel/parser@^7.22.15", "@babel/parser@^7.23.0": - version "7.23.0" - resolved "https://registry.yarnpkg.com/@babel/parser/-/parser-7.23.0.tgz#da950e622420bf96ca0d0f2909cdddac3acd8719" - integrity sha512-vvPKKdMemU85V9WE/l5wZEmImpCtLqbnTvqDS2U1fJ96KrxoW7KrXhNsNCblQlg8Ck4b85yxdTyelsMUgFUXiw== - -"@babel/parser@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/parser/-/parser-7.24.7.tgz#9a5226f92f0c5c8ead550b750f5608e766c8ce85" - integrity sha512-9uUYRm6OqQrCqQdG1iCBwBPZgN8ciDBro2nIOFaiRz1/BCxaI7CNvQbDHvsArAC7Tw9Hda/B3U+6ui9u4HWXPw== - -"@babel/plugin-bugfix-firefox-class-in-computed-class-key@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-bugfix-firefox-class-in-computed-class-key/-/plugin-bugfix-firefox-class-in-computed-class-key-7.24.7.tgz#fd059fd27b184ea2b4c7e646868a9a381bbc3055" - integrity sha512-TiT1ss81W80eQsN+722OaeQMY/G4yTb4G9JrqeiDADs3N8lbPMGldWi9x8tyqCW5NLx1Jh2AvkE6r6QvEltMMQ== - dependencies: - "@babel/helper-environment-visitor" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression/-/plugin-bugfix-safari-id-destructuring-collision-in-function-expression-7.24.7.tgz#468096ca44bbcbe8fcc570574e12eb1950e18107" - integrity sha512-unaQgZ/iRu/By6tsjMZzpeBZjChYfLYry6HrEXPoz3KmfF0sVBQ1l8zKMQ4xRGLWVsjuvB8nQfjNP/DcfEOCsg== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining/-/plugin-bugfix-v8-spread-parameters-in-optional-chaining-7.24.7.tgz#e4eabdd5109acc399b38d7999b2ef66fc2022f89" - integrity sha512-+izXIbke1T33mY4MSNnrqhPXDz01WYhEf3yF5NbnUtkiNnm+XBZJl3kNfoK6NKmYlz/D07+l2GWVK/QfDkNCuQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-skip-transparent-expression-wrappers" "^7.24.7" - "@babel/plugin-transform-optional-chaining" "^7.24.7" - -"@babel/plugin-bugfix-v8-static-class-fields-redefine-readonly@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-bugfix-v8-static-class-fields-redefine-readonly/-/plugin-bugfix-v8-static-class-fields-redefine-readonly-7.24.7.tgz#71b21bb0286d5810e63a1538aa901c58e87375ec" - integrity sha512-utA4HuR6F4Vvcr+o4DnjL8fCOlgRFGbeeBEGNg3ZTrLFw6VWG5XmUrvcQ0FjIYMU2ST4XcR2Wsp7t9qOAPnxMg== - dependencies: - "@babel/helper-environment-visitor" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-proposal-private-property-in-object@7.21.0-placeholder-for-preset-env.2": - version "7.21.0-placeholder-for-preset-env.2" - resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-private-property-in-object/-/plugin-proposal-private-property-in-object-7.21.0-placeholder-for-preset-env.2.tgz#7844f9289546efa9febac2de4cfe358a050bd703" - integrity sha512-SOSkfJDddaM7mak6cPEpswyTRnuRltl429hMraQEglW+OkovnCzsiszTmsrlY//qLFjCpQDFRvjdm2wA5pPm9w== - -"@babel/plugin-syntax-async-generators@^7.8.4": - version "7.8.4" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-async-generators/-/plugin-syntax-async-generators-7.8.4.tgz#a983fb1aeb2ec3f6ed042a210f640e90e786fe0d" - integrity sha512-tycmZxkGfZaxhMRbXlPXuVFpdWlXpir2W4AMhSJgRKzk/eDlIXOhb2LHWoLpDF7TEHylV5zNhykX6KAgHJmTNw== - dependencies: - "@babel/helper-plugin-utils" "^7.8.0" - -"@babel/plugin-syntax-bigint@^7.8.3": - version "7.8.3" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-bigint/-/plugin-syntax-bigint-7.8.3.tgz#4c9a6f669f5d0cdf1b90a1671e9a146be5300cea" - integrity sha512-wnTnFlG+YxQm3vDxpGE57Pj0srRU4sHE/mDkt1qv2YJJSeUAec2ma4WLUnUPeKjyrfntVwe/N6dCXpU+zL3Npg== - dependencies: - "@babel/helper-plugin-utils" "^7.8.0" - -"@babel/plugin-syntax-class-properties@^7.12.13", "@babel/plugin-syntax-class-properties@^7.8.3": - version "7.12.13" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-class-properties/-/plugin-syntax-class-properties-7.12.13.tgz#b5c987274c4a3a82b89714796931a6b53544ae10" - integrity sha512-fm4idjKla0YahUNgFNLCB0qySdsoPiZP3iQE3rky0mBUtMZ23yDJ9SJdg6dXTSDnulOVqiF3Hgr9nbXvXTQZYA== - dependencies: - "@babel/helper-plugin-utils" "^7.12.13" - -"@babel/plugin-syntax-class-static-block@^7.14.5": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-class-static-block/-/plugin-syntax-class-static-block-7.14.5.tgz#195df89b146b4b78b3bf897fd7a257c84659d406" - integrity sha512-b+YyPmr6ldyNnM6sqYeMWE+bgJcJpO6yS4QD7ymxgH34GBPNDM/THBh8iunyvKIZztiwLH4CJZ0RxTk9emgpjw== - dependencies: - "@babel/helper-plugin-utils" "^7.14.5" - -"@babel/plugin-syntax-dynamic-import@^7.8.3": - version "7.8.3" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-dynamic-import/-/plugin-syntax-dynamic-import-7.8.3.tgz#62bf98b2da3cd21d626154fc96ee5b3cb68eacb3" - integrity sha512-5gdGbFon+PszYzqs83S3E5mpi7/y/8M9eC90MRTZfduQOYW76ig6SOSPNe41IG5LoP3FGBn2N0RjVDSQiS94kQ== - dependencies: - "@babel/helper-plugin-utils" "^7.8.0" - -"@babel/plugin-syntax-export-namespace-from@^7.8.3": - version "7.8.3" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-export-namespace-from/-/plugin-syntax-export-namespace-from-7.8.3.tgz#028964a9ba80dbc094c915c487ad7c4e7a66465a" - integrity sha512-MXf5laXo6c1IbEbegDmzGPwGNTsHZmEy6QGznu5Sh2UCWvueywb2ee+CCE4zQiZstxU9BMoQO9i6zUFSY0Kj0Q== - dependencies: - "@babel/helper-plugin-utils" "^7.8.3" - -"@babel/plugin-syntax-import-assertions@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-import-assertions/-/plugin-syntax-import-assertions-7.24.7.tgz#2a0b406b5871a20a841240586b1300ce2088a778" - integrity sha512-Ec3NRUMoi8gskrkBe3fNmEQfxDvY8bgfQpz6jlk/41kX9eUjvpyqWU7PBP/pLAvMaSQjbMNKJmvX57jP+M6bPg== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-syntax-import-attributes@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-import-attributes/-/plugin-syntax-import-attributes-7.24.7.tgz#b4f9ea95a79e6912480c4b626739f86a076624ca" - integrity sha512-hbX+lKKeUMGihnK8nvKqmXBInriT3GVjzXKFriV3YC6APGxMbP8RZNFwy91+hocLXq90Mta+HshoB31802bb8A== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-syntax-import-meta@^7.10.4", "@babel/plugin-syntax-import-meta@^7.8.3": - version "7.10.4" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-import-meta/-/plugin-syntax-import-meta-7.10.4.tgz#ee601348c370fa334d2207be158777496521fd51" - integrity sha512-Yqfm+XDx0+Prh3VSeEQCPU81yC+JWZ2pDPFSS4ZdpfZhp4MkFMaDC1UqseovEKwSUpnIL7+vK+Clp7bfh0iD7g== - dependencies: - "@babel/helper-plugin-utils" "^7.10.4" - -"@babel/plugin-syntax-json-strings@^7.8.3": - version "7.8.3" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-json-strings/-/plugin-syntax-json-strings-7.8.3.tgz#01ca21b668cd8218c9e640cb6dd88c5412b2c96a" - integrity sha512-lY6kdGpWHvjoe2vk4WrAapEuBR69EMxZl+RoGRhrFGNYVK8mOPAW8VfbT/ZgrFbXlDNiiaxQnAtgVCZ6jv30EA== - dependencies: - "@babel/helper-plugin-utils" "^7.8.0" - -"@babel/plugin-syntax-jsx@^7.12.13": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.14.5.tgz#000e2e25d8673cce49300517a3eda44c263e4201" - integrity sha512-ohuFIsOMXJnbOMRfX7/w7LocdR6R7whhuRD4ax8IipLcLPlZGJKkBxgHp++U4N/vKyU16/YDQr2f5seajD3jIw== - dependencies: - "@babel/helper-plugin-utils" "^7.14.5" - -"@babel/plugin-syntax-jsx@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.24.7.tgz#39a1fa4a7e3d3d7f34e2acc6be585b718d30e02d" - integrity sha512-6ddciUPe/mpMnOKv/U+RSd2vvVy+Yw/JfBB0ZHYjEZt9NLHmCUylNYlsbqCCS1Bffjlb0fCwC9Vqz+sBz6PsiQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-syntax-logical-assignment-operators@^7.10.4", "@babel/plugin-syntax-logical-assignment-operators@^7.8.3": - version "7.10.4" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-logical-assignment-operators/-/plugin-syntax-logical-assignment-operators-7.10.4.tgz#ca91ef46303530448b906652bac2e9fe9941f699" - integrity sha512-d8waShlpFDinQ5MtvGU9xDAOzKH47+FFoney2baFIoMr952hKOLp1HR7VszoZvOsV/4+RRszNY7D17ba0te0ig== - dependencies: - "@babel/helper-plugin-utils" "^7.10.4" - -"@babel/plugin-syntax-nullish-coalescing-operator@^7.8.3": - version "7.8.3" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-nullish-coalescing-operator/-/plugin-syntax-nullish-coalescing-operator-7.8.3.tgz#167ed70368886081f74b5c36c65a88c03b66d1a9" - integrity sha512-aSff4zPII1u2QD7y+F8oDsz19ew4IGEJg9SVW+bqwpwtfFleiQDMdzA/R+UlWDzfnHFCxxleFT0PMIrR36XLNQ== - dependencies: - "@babel/helper-plugin-utils" "^7.8.0" - -"@babel/plugin-syntax-numeric-separator@^7.10.4", "@babel/plugin-syntax-numeric-separator@^7.8.3": - version "7.10.4" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-numeric-separator/-/plugin-syntax-numeric-separator-7.10.4.tgz#b9b070b3e33570cd9fd07ba7fa91c0dd37b9af97" - integrity sha512-9H6YdfkcK/uOnY/K7/aA2xpzaAgkQn37yzWUMRK7OaPOqOpGS1+n0H5hxT9AUw9EsSjPW8SVyMJwYRtWs3X3ug== - dependencies: - "@babel/helper-plugin-utils" "^7.10.4" - -"@babel/plugin-syntax-object-rest-spread@^7.8.3": - version "7.8.3" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-object-rest-spread/-/plugin-syntax-object-rest-spread-7.8.3.tgz#60e225edcbd98a640332a2e72dd3e66f1af55871" - integrity sha512-XoqMijGZb9y3y2XskN+P1wUGiVwWZ5JmoDRwx5+3GmEplNyVM2s2Dg8ILFQm8rWM48orGy5YpI5Bl8U1y7ydlA== - dependencies: - "@babel/helper-plugin-utils" "^7.8.0" - -"@babel/plugin-syntax-optional-catch-binding@^7.8.3": - version "7.8.3" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-optional-catch-binding/-/plugin-syntax-optional-catch-binding-7.8.3.tgz#6111a265bcfb020eb9efd0fdfd7d26402b9ed6c1" - integrity sha512-6VPD0Pc1lpTqw0aKoeRTMiB+kWhAoT24PA+ksWSBrFtl5SIRVpZlwN3NNPQjehA2E/91FV3RjLWoVTglWcSV3Q== - dependencies: - "@babel/helper-plugin-utils" "^7.8.0" - -"@babel/plugin-syntax-optional-chaining@^7.8.3": - version "7.8.3" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-optional-chaining/-/plugin-syntax-optional-chaining-7.8.3.tgz#4f69c2ab95167e0180cd5336613f8c5788f7d48a" - integrity sha512-KoK9ErH1MBlCPxV0VANkXW2/dw4vlbGDrFgz8bmUsBGYkFRcbRwMh6cIJubdPrkxRwuGdtCk0v/wPTKbQgBjkg== - dependencies: - "@babel/helper-plugin-utils" "^7.8.0" - -"@babel/plugin-syntax-private-property-in-object@^7.14.5": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-private-property-in-object/-/plugin-syntax-private-property-in-object-7.14.5.tgz#0dc6671ec0ea22b6e94a1114f857970cd39de1ad" - integrity sha512-0wVnp9dxJ72ZUJDV27ZfbSj6iHLoytYZmh3rFcxNnvsJF3ktkzLDZPy/mA17HGsaQT3/DQsWYX1f1QGWkCoVUg== - dependencies: - "@babel/helper-plugin-utils" "^7.14.5" - -"@babel/plugin-syntax-top-level-await@^7.14.5", "@babel/plugin-syntax-top-level-await@^7.8.3": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-top-level-await/-/plugin-syntax-top-level-await-7.14.5.tgz#c1cfdadc35a646240001f06138247b741c34d94c" - integrity sha512-hx++upLv5U1rgYfwe1xBQUhRmU41NEvpUvrp8jkrSCdvGSnM5/qdRMtylJ6PG5OFkBaHkbTAKTnd3/YyESRHFw== - dependencies: - "@babel/helper-plugin-utils" "^7.14.5" - -"@babel/plugin-syntax-typescript@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.24.7.tgz#58d458271b4d3b6bb27ee6ac9525acbb259bad1c" - integrity sha512-c/+fVeJBB0FeKsFvwytYiUD+LBvhHjGSI0g446PRGdSVGZLRNArBUno2PETbAly3tpiNAQR5XaZ+JslxkotsbA== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-syntax-typescript@^7.7.2": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.16.0.tgz#2feeb13d9334cc582ea9111d3506f773174179bb" - integrity sha512-Xv6mEXqVdaqCBfJFyeab0fH2DnUoMsDmhamxsSi4j8nLd4Vtw213WMJr55xxqipC/YVWyPY3K0blJncPYji+dQ== - dependencies: - "@babel/helper-plugin-utils" "^7.14.5" - -"@babel/plugin-syntax-unicode-sets-regex@^7.18.6": - version "7.18.6" - resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-unicode-sets-regex/-/plugin-syntax-unicode-sets-regex-7.18.6.tgz#d49a3b3e6b52e5be6740022317580234a6a47357" - integrity sha512-727YkEAPwSIQTv5im8QHz3upqp92JTWhidIC81Tdx4VJYIte/VndKf1qKrfnnhPLiPghStWfvC/iFaMCQu7Nqg== - dependencies: - "@babel/helper-create-regexp-features-plugin" "^7.18.6" - "@babel/helper-plugin-utils" "^7.18.6" - -"@babel/plugin-transform-arrow-functions@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-arrow-functions/-/plugin-transform-arrow-functions-7.24.7.tgz#4f6886c11e423bd69f3ce51dbf42424a5f275514" - integrity sha512-Dt9LQs6iEY++gXUwY03DNFat5C2NbO48jj+j/bSAz6b3HgPs39qcPiYt77fDObIcFwj3/C2ICX9YMwGflUoSHQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-async-generator-functions@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-async-generator-functions/-/plugin-transform-async-generator-functions-7.24.7.tgz#7330a5c50e05181ca52351b8fd01642000c96cfd" - integrity sha512-o+iF77e3u7ZS4AoAuJvapz9Fm001PuD2V3Lp6OSE4FYQke+cSewYtnek+THqGRWyQloRCyvWL1OkyfNEl9vr/g== - dependencies: - "@babel/helper-environment-visitor" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-remap-async-to-generator" "^7.24.7" - "@babel/plugin-syntax-async-generators" "^7.8.4" - -"@babel/plugin-transform-async-to-generator@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-async-to-generator/-/plugin-transform-async-to-generator-7.24.7.tgz#72a3af6c451d575842a7e9b5a02863414355bdcc" - integrity sha512-SQY01PcJfmQ+4Ash7NE+rpbLFbmqA2GPIgqzxfFTL4t1FKRq4zTms/7htKpoCUI9OcFYgzqfmCdH53s6/jn5fA== - dependencies: - "@babel/helper-module-imports" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-remap-async-to-generator" "^7.24.7" - -"@babel/plugin-transform-block-scoped-functions@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-block-scoped-functions/-/plugin-transform-block-scoped-functions-7.24.7.tgz#a4251d98ea0c0f399dafe1a35801eaba455bbf1f" - integrity sha512-yO7RAz6EsVQDaBH18IDJcMB1HnrUn2FJ/Jslc/WtPPWcjhpUJXU/rjbwmluzp7v/ZzWcEhTMXELnnsz8djWDwQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-block-scoping@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-block-scoping/-/plugin-transform-block-scoping-7.24.7.tgz#42063e4deb850c7bd7c55e626bf4e7ab48e6ce02" - integrity sha512-Nd5CvgMbWc+oWzBsuaMcbwjJWAcp5qzrbg69SZdHSP7AMY0AbWFqFO0WTFCA1jxhMCwodRwvRec8k0QUbZk7RQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-class-properties@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-class-properties/-/plugin-transform-class-properties-7.24.7.tgz#256879467b57b0b68c7ddfc5b76584f398cd6834" - integrity sha512-vKbfawVYayKcSeSR5YYzzyXvsDFWU2mD8U5TFeXtbCPLFUqe7GyCgvO6XDHzje862ODrOwy6WCPmKeWHbCFJ4w== - dependencies: - "@babel/helper-create-class-features-plugin" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-class-static-block@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-class-static-block/-/plugin-transform-class-static-block-7.24.7.tgz#c82027ebb7010bc33c116d4b5044fbbf8c05484d" - integrity sha512-HMXK3WbBPpZQufbMG4B46A90PkuuhN9vBCb5T8+VAHqvAqvcLi+2cKoukcpmUYkszLhScU3l1iudhrks3DggRQ== - dependencies: - "@babel/helper-create-class-features-plugin" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-class-static-block" "^7.14.5" - -"@babel/plugin-transform-classes@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-classes/-/plugin-transform-classes-7.24.7.tgz#4ae6ef43a12492134138c1e45913f7c46c41b4bf" - integrity sha512-CFbbBigp8ln4FU6Bpy6g7sE8B/WmCmzvivzUC6xDAdWVsjYTXijpuuGJmYkAaoWAzcItGKT3IOAbxRItZ5HTjw== - dependencies: - "@babel/helper-annotate-as-pure" "^7.24.7" - "@babel/helper-compilation-targets" "^7.24.7" - "@babel/helper-environment-visitor" "^7.24.7" - "@babel/helper-function-name" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-replace-supers" "^7.24.7" - "@babel/helper-split-export-declaration" "^7.24.7" - globals "^11.1.0" - -"@babel/plugin-transform-computed-properties@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-computed-properties/-/plugin-transform-computed-properties-7.24.7.tgz#4cab3214e80bc71fae3853238d13d097b004c707" - integrity sha512-25cS7v+707Gu6Ds2oY6tCkUwsJ9YIDbggd9+cu9jzzDgiNq7hR/8dkzxWfKWnTic26vsI3EsCXNd4iEB6e8esQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/template" "^7.24.7" - -"@babel/plugin-transform-destructuring@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-destructuring/-/plugin-transform-destructuring-7.24.7.tgz#a097f25292defb6e6cc16d6333a4cfc1e3c72d9e" - integrity sha512-19eJO/8kdCQ9zISOf+SEUJM/bAUIsvY3YDnXZTupUCQ8LgrWnsG/gFB9dvXqdXnRXMAM8fvt7b0CBKQHNGy1mw== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-dotall-regex@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-dotall-regex/-/plugin-transform-dotall-regex-7.24.7.tgz#5f8bf8a680f2116a7207e16288a5f974ad47a7a0" - integrity sha512-ZOA3W+1RRTSWvyqcMJDLqbchh7U4NRGqwRfFSVbOLS/ePIP4vHB5e8T8eXcuqyN1QkgKyj5wuW0lcS85v4CrSw== - dependencies: - "@babel/helper-create-regexp-features-plugin" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-duplicate-keys@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-duplicate-keys/-/plugin-transform-duplicate-keys-7.24.7.tgz#dd20102897c9a2324e5adfffb67ff3610359a8ee" - integrity sha512-JdYfXyCRihAe46jUIliuL2/s0x0wObgwwiGxw/UbgJBr20gQBThrokO4nYKgWkD7uBaqM7+9x5TU7NkExZJyzw== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-dynamic-import@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-dynamic-import/-/plugin-transform-dynamic-import-7.24.7.tgz#4d8b95e3bae2b037673091aa09cd33fecd6419f4" - integrity sha512-sc3X26PhZQDb3JhORmakcbvkeInvxz+A8oda99lj7J60QRuPZvNAk9wQlTBS1ZynelDrDmTU4pw1tyc5d5ZMUg== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-dynamic-import" "^7.8.3" - -"@babel/plugin-transform-exponentiation-operator@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-exponentiation-operator/-/plugin-transform-exponentiation-operator-7.24.7.tgz#b629ee22645f412024297d5245bce425c31f9b0d" - integrity sha512-Rqe/vSc9OYgDajNIK35u7ot+KeCoetqQYFXM4Epf7M7ez3lWlOjrDjrwMei6caCVhfdw+mIKD4cgdGNy5JQotQ== - dependencies: - "@babel/helper-builder-binary-assignment-operator-visitor" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-export-namespace-from@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-export-namespace-from/-/plugin-transform-export-namespace-from-7.24.7.tgz#176d52d8d8ed516aeae7013ee9556d540c53f197" - integrity sha512-v0K9uNYsPL3oXZ/7F9NNIbAj2jv1whUEtyA6aujhekLs56R++JDQuzRcP2/z4WX5Vg/c5lE9uWZA0/iUoFhLTA== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-export-namespace-from" "^7.8.3" - -"@babel/plugin-transform-for-of@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-for-of/-/plugin-transform-for-of-7.24.7.tgz#f25b33f72df1d8be76399e1b8f3f9d366eb5bc70" - integrity sha512-wo9ogrDG1ITTTBsy46oGiN1dS9A7MROBTcYsfS8DtsImMkHk9JXJ3EWQM6X2SUw4x80uGPlwj0o00Uoc6nEE3g== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-skip-transparent-expression-wrappers" "^7.24.7" - -"@babel/plugin-transform-function-name@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-function-name/-/plugin-transform-function-name-7.24.7.tgz#6d8601fbffe665c894440ab4470bc721dd9131d6" - integrity sha512-U9FcnA821YoILngSmYkW6FjyQe2TyZD5pHt4EVIhmcTkrJw/3KqcrRSxuOo5tFZJi7TE19iDyI1u+weTI7bn2w== - dependencies: - "@babel/helper-compilation-targets" "^7.24.7" - "@babel/helper-function-name" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-json-strings@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-json-strings/-/plugin-transform-json-strings-7.24.7.tgz#f3e9c37c0a373fee86e36880d45b3664cedaf73a" - integrity sha512-2yFnBGDvRuxAaE/f0vfBKvtnvvqU8tGpMHqMNpTN2oWMKIR3NqFkjaAgGwawhqK/pIN2T3XdjGPdaG0vDhOBGw== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-json-strings" "^7.8.3" - -"@babel/plugin-transform-literals@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-literals/-/plugin-transform-literals-7.24.7.tgz#36b505c1e655151a9d7607799a9988fc5467d06c" - integrity sha512-vcwCbb4HDH+hWi8Pqenwnjy+UiklO4Kt1vfspcQYFhJdpthSnW8XvWGyDZWKNVrVbVViI/S7K9PDJZiUmP2fYQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-logical-assignment-operators@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-logical-assignment-operators/-/plugin-transform-logical-assignment-operators-7.24.7.tgz#a58fb6eda16c9dc8f9ff1c7b1ba6deb7f4694cb0" - integrity sha512-4D2tpwlQ1odXmTEIFWy9ELJcZHqrStlzK/dAOWYyxX3zT0iXQB6banjgeOJQXzEc4S0E0a5A+hahxPaEFYftsw== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-logical-assignment-operators" "^7.10.4" - -"@babel/plugin-transform-member-expression-literals@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-member-expression-literals/-/plugin-transform-member-expression-literals-7.24.7.tgz#3b4454fb0e302e18ba4945ba3246acb1248315df" - integrity sha512-T/hRC1uqrzXMKLQ6UCwMT85S3EvqaBXDGf0FaMf4446Qx9vKwlghvee0+uuZcDUCZU5RuNi4781UQ7R308zzBw== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-modules-amd@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-amd/-/plugin-transform-modules-amd-7.24.7.tgz#65090ed493c4a834976a3ca1cde776e6ccff32d7" - integrity sha512-9+pB1qxV3vs/8Hdmz/CulFB8w2tuu6EB94JZFsjdqxQokwGa9Unap7Bo2gGBGIvPmDIVvQrom7r5m/TCDMURhg== - dependencies: - "@babel/helper-module-transforms" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-modules-commonjs@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-commonjs/-/plugin-transform-modules-commonjs-7.24.7.tgz#9fd5f7fdadee9085886b183f1ad13d1ab260f4ab" - integrity sha512-iFI8GDxtevHJ/Z22J5xQpVqFLlMNstcLXh994xifFwxxGslr2ZXXLWgtBeLctOD63UFDArdvN6Tg8RFw+aEmjQ== - dependencies: - "@babel/helper-module-transforms" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-simple-access" "^7.24.7" - -"@babel/plugin-transform-modules-systemjs@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-systemjs/-/plugin-transform-modules-systemjs-7.24.7.tgz#f8012316c5098f6e8dee6ecd58e2bc6f003d0ce7" - integrity sha512-GYQE0tW7YoaN13qFh3O1NCY4MPkUiAH3fiF7UcV/I3ajmDKEdG3l+UOcbAm4zUE3gnvUU+Eni7XrVKo9eO9auw== - dependencies: - "@babel/helper-hoist-variables" "^7.24.7" - "@babel/helper-module-transforms" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-validator-identifier" "^7.24.7" - -"@babel/plugin-transform-modules-umd@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-umd/-/plugin-transform-modules-umd-7.24.7.tgz#edd9f43ec549099620df7df24e7ba13b5c76efc8" - integrity sha512-3aytQvqJ/h9z4g8AsKPLvD4Zqi2qT+L3j7XoFFu1XBlZWEl2/1kWnhmAbxpLgPrHSY0M6UA02jyTiwUVtiKR6A== - dependencies: - "@babel/helper-module-transforms" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-named-capturing-groups-regex@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-named-capturing-groups-regex/-/plugin-transform-named-capturing-groups-regex-7.24.7.tgz#9042e9b856bc6b3688c0c2e4060e9e10b1460923" - integrity sha512-/jr7h/EWeJtk1U/uz2jlsCioHkZk1JJZVcc8oQsJ1dUlaJD83f4/6Zeh2aHt9BIFokHIsSeDfhUmju0+1GPd6g== - dependencies: - "@babel/helper-create-regexp-features-plugin" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-new-target@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-new-target/-/plugin-transform-new-target-7.24.7.tgz#31ff54c4e0555cc549d5816e4ab39241dfb6ab00" - integrity sha512-RNKwfRIXg4Ls/8mMTza5oPF5RkOW8Wy/WgMAp1/F1yZ8mMbtwXW+HDoJiOsagWrAhI5f57Vncrmr9XeT4CVapA== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-nullish-coalescing-operator@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-nullish-coalescing-operator/-/plugin-transform-nullish-coalescing-operator-7.24.7.tgz#1de4534c590af9596f53d67f52a92f12db984120" - integrity sha512-Ts7xQVk1OEocqzm8rHMXHlxvsfZ0cEF2yomUqpKENHWMF4zKk175Y4q8H5knJes6PgYad50uuRmt3UJuhBw8pQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" - -"@babel/plugin-transform-numeric-separator@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-numeric-separator/-/plugin-transform-numeric-separator-7.24.7.tgz#bea62b538c80605d8a0fac9b40f48e97efa7de63" - integrity sha512-e6q1TiVUzvH9KRvicuxdBTUj4AdKSRwzIyFFnfnezpCfP2/7Qmbb8qbU2j7GODbl4JMkblitCQjKYUaX/qkkwA== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-numeric-separator" "^7.10.4" - -"@babel/plugin-transform-object-rest-spread@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-object-rest-spread/-/plugin-transform-object-rest-spread-7.24.7.tgz#d13a2b93435aeb8a197e115221cab266ba6e55d6" - integrity sha512-4QrHAr0aXQCEFni2q4DqKLD31n2DL+RxcwnNjDFkSG0eNQ/xCavnRkfCUjsyqGC2OviNJvZOF/mQqZBw7i2C5Q== - dependencies: - "@babel/helper-compilation-targets" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-object-rest-spread" "^7.8.3" - "@babel/plugin-transform-parameters" "^7.24.7" - -"@babel/plugin-transform-object-super@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-object-super/-/plugin-transform-object-super-7.24.7.tgz#66eeaff7830bba945dd8989b632a40c04ed625be" - integrity sha512-A/vVLwN6lBrMFmMDmPPz0jnE6ZGx7Jq7d6sT/Ev4H65RER6pZ+kczlf1DthF5N0qaPHBsI7UXiE8Zy66nmAovg== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-replace-supers" "^7.24.7" - -"@babel/plugin-transform-optional-catch-binding@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-optional-catch-binding/-/plugin-transform-optional-catch-binding-7.24.7.tgz#00eabd883d0dd6a60c1c557548785919b6e717b4" - integrity sha512-uLEndKqP5BfBbC/5jTwPxLh9kqPWWgzN/f8w6UwAIirAEqiIVJWWY312X72Eub09g5KF9+Zn7+hT7sDxmhRuKA== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" - -"@babel/plugin-transform-optional-chaining@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-optional-chaining/-/plugin-transform-optional-chaining-7.24.7.tgz#b8f6848a80cf2da98a8a204429bec04756c6d454" - integrity sha512-tK+0N9yd4j+x/4hxF3F0e0fu/VdcxU18y5SevtyM/PCFlQvXbR0Zmlo2eBrKtVipGNFzpq56o8WsIIKcJFUCRQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-skip-transparent-expression-wrappers" "^7.24.7" - "@babel/plugin-syntax-optional-chaining" "^7.8.3" - -"@babel/plugin-transform-parameters@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-parameters/-/plugin-transform-parameters-7.24.7.tgz#5881f0ae21018400e320fc7eb817e529d1254b68" - integrity sha512-yGWW5Rr+sQOhK0Ot8hjDJuxU3XLRQGflvT4lhlSY0DFvdb3TwKaY26CJzHtYllU0vT9j58hc37ndFPsqT1SrzA== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-private-methods@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-private-methods/-/plugin-transform-private-methods-7.24.7.tgz#e6318746b2ae70a59d023d5cc1344a2ba7a75f5e" - integrity sha512-COTCOkG2hn4JKGEKBADkA8WNb35TGkkRbI5iT845dB+NyqgO8Hn+ajPbSnIQznneJTa3d30scb6iz/DhH8GsJQ== - dependencies: - "@babel/helper-create-class-features-plugin" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-private-property-in-object@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-private-property-in-object/-/plugin-transform-private-property-in-object-7.24.7.tgz#4eec6bc701288c1fab5f72e6a4bbc9d67faca061" - integrity sha512-9z76mxwnwFxMyxZWEgdgECQglF2Q7cFLm0kMf8pGwt+GSJsY0cONKj/UuO4bOH0w/uAel3ekS4ra5CEAyJRmDA== - dependencies: - "@babel/helper-annotate-as-pure" "^7.24.7" - "@babel/helper-create-class-features-plugin" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-private-property-in-object" "^7.14.5" - -"@babel/plugin-transform-property-literals@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-property-literals/-/plugin-transform-property-literals-7.24.7.tgz#f0d2ed8380dfbed949c42d4d790266525d63bbdc" - integrity sha512-EMi4MLQSHfd2nrCqQEWxFdha2gBCqU4ZcCng4WBGZ5CJL4bBRW0ptdqqDdeirGZcpALazVVNJqRmsO8/+oNCBA== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-react-display-name@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-display-name/-/plugin-transform-react-display-name-7.24.7.tgz#9caff79836803bc666bcfe210aeb6626230c293b" - integrity sha512-H/Snz9PFxKsS1JLI4dJLtnJgCJRoo0AUm3chP6NYr+9En1JMKloheEiLIhlp5MDVznWo+H3AAC1Mc8lmUEpsgg== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-react-jsx-development@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-jsx-development/-/plugin-transform-react-jsx-development-7.24.7.tgz#eaee12f15a93f6496d852509a850085e6361470b" - integrity sha512-QG9EnzoGn+Qar7rxuW+ZOsbWOt56FvvI93xInqsZDC5fsekx1AlIO4KIJ5M+D0p0SqSH156EpmZyXq630B8OlQ== - dependencies: - "@babel/plugin-transform-react-jsx" "^7.24.7" - -"@babel/plugin-transform-react-jsx@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-jsx/-/plugin-transform-react-jsx-7.24.7.tgz#17cd06b75a9f0e2bd076503400e7c4b99beedac4" - integrity sha512-+Dj06GDZEFRYvclU6k4bme55GKBEWUmByM/eoKuqg4zTNQHiApWRhQph5fxQB2wAEFvRzL1tOEj1RJ19wJrhoA== - dependencies: - "@babel/helper-annotate-as-pure" "^7.24.7" - "@babel/helper-module-imports" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-jsx" "^7.24.7" - "@babel/types" "^7.24.7" - -"@babel/plugin-transform-react-pure-annotations@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-pure-annotations/-/plugin-transform-react-pure-annotations-7.24.7.tgz#bdd9d140d1c318b4f28b29a00fb94f97ecab1595" - integrity sha512-PLgBVk3fzbmEjBJ/u8kFzOqS9tUeDjiaWud/rRym/yjCo/M9cASPlnrd2ZmmZpQT40fOOrvR8jh+n8jikrOhNA== - dependencies: - "@babel/helper-annotate-as-pure" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-regenerator@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-regenerator/-/plugin-transform-regenerator-7.24.7.tgz#021562de4534d8b4b1851759fd7af4e05d2c47f8" - integrity sha512-lq3fvXPdimDrlg6LWBoqj+r/DEWgONuwjuOuQCSYgRroXDH/IdM1C0IZf59fL5cHLpjEH/O6opIRBbqv7ELnuA== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - regenerator-transform "^0.15.2" - -"@babel/plugin-transform-reserved-words@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-reserved-words/-/plugin-transform-reserved-words-7.24.7.tgz#80037fe4fbf031fc1125022178ff3938bb3743a4" - integrity sha512-0DUq0pHcPKbjFZCfTss/pGkYMfy3vFWydkUBd9r0GHpIyfs2eCDENvqadMycRS9wZCXR41wucAfJHJmwA0UmoQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-runtime@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-runtime/-/plugin-transform-runtime-7.24.7.tgz#00a5bfaf8c43cf5c8703a8a6e82b59d9c58f38ca" - integrity sha512-YqXjrk4C+a1kZjewqt+Mmu2UuV1s07y8kqcUf4qYLnoqemhR4gRQikhdAhSVJioMjVTu6Mo6pAbaypEA3jY6fw== - dependencies: - "@babel/helper-module-imports" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - babel-plugin-polyfill-corejs2 "^0.4.10" - babel-plugin-polyfill-corejs3 "^0.10.1" - babel-plugin-polyfill-regenerator "^0.6.1" - semver "^6.3.1" - -"@babel/plugin-transform-shorthand-properties@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-shorthand-properties/-/plugin-transform-shorthand-properties-7.24.7.tgz#85448c6b996e122fa9e289746140aaa99da64e73" - integrity sha512-KsDsevZMDsigzbA09+vacnLpmPH4aWjcZjXdyFKGzpplxhbeB4wYtury3vglQkg6KM/xEPKt73eCjPPf1PgXBA== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-spread@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-spread/-/plugin-transform-spread-7.24.7.tgz#e8a38c0fde7882e0fb8f160378f74bd885cc7bb3" - integrity sha512-x96oO0I09dgMDxJaANcRyD4ellXFLLiWhuwDxKZX5g2rWP1bTPkBSwCYv96VDXVT1bD9aPj8tppr5ITIh8hBng== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-skip-transparent-expression-wrappers" "^7.24.7" - -"@babel/plugin-transform-sticky-regex@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-sticky-regex/-/plugin-transform-sticky-regex-7.24.7.tgz#96ae80d7a7e5251f657b5cf18f1ea6bf926f5feb" - integrity sha512-kHPSIJc9v24zEml5geKg9Mjx5ULpfncj0wRpYtxbvKyTtHCYDkVE3aHQ03FrpEo4gEe2vrJJS1Y9CJTaThA52g== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-template-literals@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-template-literals/-/plugin-transform-template-literals-7.24.7.tgz#a05debb4a9072ae8f985bcf77f3f215434c8f8c8" - integrity sha512-AfDTQmClklHCOLxtGoP7HkeMw56k1/bTQjwsfhL6pppo/M4TOBSq+jjBUBLmV/4oeFg4GWMavIl44ZeCtmmZTw== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-typeof-symbol@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-typeof-symbol/-/plugin-transform-typeof-symbol-7.24.7.tgz#f074be466580d47d6e6b27473a840c9f9ca08fb0" - integrity sha512-VtR8hDy7YLB7+Pet9IarXjg/zgCMSF+1mNS/EQEiEaUPoFXCVsHG64SIxcaaI2zJgRiv+YmgaQESUfWAdbjzgg== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-typescript@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-typescript/-/plugin-transform-typescript-7.24.7.tgz#b006b3e0094bf0813d505e0c5485679eeaf4a881" - integrity sha512-iLD3UNkgx2n/HrjBesVbYX6j0yqn/sJktvbtKKgcaLIQ4bTTQ8obAypc1VpyHPD2y4Phh9zHOaAt8e/L14wCpw== - dependencies: - "@babel/helper-annotate-as-pure" "^7.24.7" - "@babel/helper-create-class-features-plugin" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/plugin-syntax-typescript" "^7.24.7" - -"@babel/plugin-transform-unicode-escapes@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-unicode-escapes/-/plugin-transform-unicode-escapes-7.24.7.tgz#2023a82ced1fb4971630a2e079764502c4148e0e" - integrity sha512-U3ap1gm5+4edc2Q/P+9VrBNhGkfnf+8ZqppY71Bo/pzZmXhhLdqgaUl6cuB07O1+AQJtCLfaOmswiNbSQ9ivhw== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-unicode-property-regex@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-unicode-property-regex/-/plugin-transform-unicode-property-regex-7.24.7.tgz#9073a4cd13b86ea71c3264659590ac086605bbcd" - integrity sha512-uH2O4OV5M9FZYQrwc7NdVmMxQJOCCzFeYudlZSzUAHRFeOujQefa92E74TQDVskNHCzOXoigEuoyzHDhaEaK5w== - dependencies: - "@babel/helper-create-regexp-features-plugin" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-unicode-regex@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-unicode-regex/-/plugin-transform-unicode-regex-7.24.7.tgz#dfc3d4a51127108099b19817c0963be6a2adf19f" - integrity sha512-hlQ96MBZSAXUq7ltkjtu3FJCCSMx/j629ns3hA3pXnBXjanNP0LHi+JpPeA81zaWgVK1VGH95Xuy7u0RyQ8kMg== - dependencies: - "@babel/helper-create-regexp-features-plugin" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/plugin-transform-unicode-sets-regex@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/plugin-transform-unicode-sets-regex/-/plugin-transform-unicode-sets-regex-7.24.7.tgz#d40705d67523803a576e29c63cef6e516b858ed9" - integrity sha512-2G8aAvF4wy1w/AGZkemprdGMRg5o6zPNhbHVImRz3lss55TYCBd6xStN19rt8XJHq20sqV0JbyWjOWwQRwV/wg== - dependencies: - "@babel/helper-create-regexp-features-plugin" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - -"@babel/preset-env@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/preset-env/-/preset-env-7.24.7.tgz#ff067b4e30ba4a72f225f12f123173e77b987f37" - integrity sha512-1YZNsc+y6cTvWlDHidMBsQZrZfEFjRIo/BZCT906PMdzOyXtSLTgqGdrpcuTDCXyd11Am5uQULtDIcCfnTc8fQ== - dependencies: - "@babel/compat-data" "^7.24.7" - "@babel/helper-compilation-targets" "^7.24.7" - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-validator-option" "^7.24.7" - "@babel/plugin-bugfix-firefox-class-in-computed-class-key" "^7.24.7" - "@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression" "^7.24.7" - "@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining" "^7.24.7" - "@babel/plugin-bugfix-v8-static-class-fields-redefine-readonly" "^7.24.7" - "@babel/plugin-proposal-private-property-in-object" "7.21.0-placeholder-for-preset-env.2" - "@babel/plugin-syntax-async-generators" "^7.8.4" - "@babel/plugin-syntax-class-properties" "^7.12.13" - "@babel/plugin-syntax-class-static-block" "^7.14.5" - "@babel/plugin-syntax-dynamic-import" "^7.8.3" - "@babel/plugin-syntax-export-namespace-from" "^7.8.3" - "@babel/plugin-syntax-import-assertions" "^7.24.7" - "@babel/plugin-syntax-import-attributes" "^7.24.7" - "@babel/plugin-syntax-import-meta" "^7.10.4" - "@babel/plugin-syntax-json-strings" "^7.8.3" - "@babel/plugin-syntax-logical-assignment-operators" "^7.10.4" - "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" - "@babel/plugin-syntax-numeric-separator" "^7.10.4" - "@babel/plugin-syntax-object-rest-spread" "^7.8.3" - "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" - "@babel/plugin-syntax-optional-chaining" "^7.8.3" - "@babel/plugin-syntax-private-property-in-object" "^7.14.5" - "@babel/plugin-syntax-top-level-await" "^7.14.5" - "@babel/plugin-syntax-unicode-sets-regex" "^7.18.6" - "@babel/plugin-transform-arrow-functions" "^7.24.7" - "@babel/plugin-transform-async-generator-functions" "^7.24.7" - "@babel/plugin-transform-async-to-generator" "^7.24.7" - "@babel/plugin-transform-block-scoped-functions" "^7.24.7" - "@babel/plugin-transform-block-scoping" "^7.24.7" - "@babel/plugin-transform-class-properties" "^7.24.7" - "@babel/plugin-transform-class-static-block" "^7.24.7" - "@babel/plugin-transform-classes" "^7.24.7" - "@babel/plugin-transform-computed-properties" "^7.24.7" - "@babel/plugin-transform-destructuring" "^7.24.7" - "@babel/plugin-transform-dotall-regex" "^7.24.7" - "@babel/plugin-transform-duplicate-keys" "^7.24.7" - "@babel/plugin-transform-dynamic-import" "^7.24.7" - "@babel/plugin-transform-exponentiation-operator" "^7.24.7" - "@babel/plugin-transform-export-namespace-from" "^7.24.7" - "@babel/plugin-transform-for-of" "^7.24.7" - "@babel/plugin-transform-function-name" "^7.24.7" - "@babel/plugin-transform-json-strings" "^7.24.7" - "@babel/plugin-transform-literals" "^7.24.7" - "@babel/plugin-transform-logical-assignment-operators" "^7.24.7" - "@babel/plugin-transform-member-expression-literals" "^7.24.7" - "@babel/plugin-transform-modules-amd" "^7.24.7" - "@babel/plugin-transform-modules-commonjs" "^7.24.7" - "@babel/plugin-transform-modules-systemjs" "^7.24.7" - "@babel/plugin-transform-modules-umd" "^7.24.7" - "@babel/plugin-transform-named-capturing-groups-regex" "^7.24.7" - "@babel/plugin-transform-new-target" "^7.24.7" - "@babel/plugin-transform-nullish-coalescing-operator" "^7.24.7" - "@babel/plugin-transform-numeric-separator" "^7.24.7" - "@babel/plugin-transform-object-rest-spread" "^7.24.7" - "@babel/plugin-transform-object-super" "^7.24.7" - "@babel/plugin-transform-optional-catch-binding" "^7.24.7" - "@babel/plugin-transform-optional-chaining" "^7.24.7" - "@babel/plugin-transform-parameters" "^7.24.7" - "@babel/plugin-transform-private-methods" "^7.24.7" - "@babel/plugin-transform-private-property-in-object" "^7.24.7" - "@babel/plugin-transform-property-literals" "^7.24.7" - "@babel/plugin-transform-regenerator" "^7.24.7" - "@babel/plugin-transform-reserved-words" "^7.24.7" - "@babel/plugin-transform-shorthand-properties" "^7.24.7" - "@babel/plugin-transform-spread" "^7.24.7" - "@babel/plugin-transform-sticky-regex" "^7.24.7" - "@babel/plugin-transform-template-literals" "^7.24.7" - "@babel/plugin-transform-typeof-symbol" "^7.24.7" - "@babel/plugin-transform-unicode-escapes" "^7.24.7" - "@babel/plugin-transform-unicode-property-regex" "^7.24.7" - "@babel/plugin-transform-unicode-regex" "^7.24.7" - "@babel/plugin-transform-unicode-sets-regex" "^7.24.7" - "@babel/preset-modules" "0.1.6-no-external-plugins" - babel-plugin-polyfill-corejs2 "^0.4.10" - babel-plugin-polyfill-corejs3 "^0.10.4" - babel-plugin-polyfill-regenerator "^0.6.1" - core-js-compat "^3.31.0" - semver "^6.3.1" - -"@babel/preset-modules@0.1.6-no-external-plugins": - version "0.1.6-no-external-plugins" - resolved "https://registry.yarnpkg.com/@babel/preset-modules/-/preset-modules-0.1.6-no-external-plugins.tgz#ccb88a2c49c817236861fee7826080573b8a923a" - integrity sha512-HrcgcIESLm9aIR842yhJ5RWan/gebQUJ6E/E5+rf0y9o6oj7w0Br+sWuL6kEQ/o/AdfvR1Je9jG18/gnpwjEyA== - dependencies: - "@babel/helper-plugin-utils" "^7.0.0" - "@babel/types" "^7.4.4" - esutils "^2.0.2" - -"@babel/preset-react@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/preset-react/-/preset-react-7.24.7.tgz#480aeb389b2a798880bf1f889199e3641cbb22dc" - integrity sha512-AAH4lEkpmzFWrGVlHaxJB7RLH21uPQ9+He+eFLWHmF9IuFQVugz8eAsamaW0DXRrTfco5zj1wWtpdcXJUOfsag== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-validator-option" "^7.24.7" - "@babel/plugin-transform-react-display-name" "^7.24.7" - "@babel/plugin-transform-react-jsx" "^7.24.7" - "@babel/plugin-transform-react-jsx-development" "^7.24.7" - "@babel/plugin-transform-react-pure-annotations" "^7.24.7" - -"@babel/preset-typescript@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/preset-typescript/-/preset-typescript-7.24.7.tgz#66cd86ea8f8c014855671d5ea9a737139cbbfef1" - integrity sha512-SyXRe3OdWwIwalxDg5UtJnJQO+YPcTfwiIY2B0Xlddh9o7jpWLvv8X1RthIeDOxQ+O1ML5BLPCONToObyVQVuQ== - dependencies: - "@babel/helper-plugin-utils" "^7.24.7" - "@babel/helper-validator-option" "^7.24.7" - "@babel/plugin-syntax-jsx" "^7.24.7" - "@babel/plugin-transform-modules-commonjs" "^7.24.7" - "@babel/plugin-transform-typescript" "^7.24.7" - -"@babel/regjsgen@^0.8.0": - version "0.8.0" - resolved "https://registry.yarnpkg.com/@babel/regjsgen/-/regjsgen-0.8.0.tgz#f0ba69b075e1f05fb2825b7fad991e7adbb18310" - integrity sha512-x/rqGMdzj+fWZvCOYForTghzbtqPDZ5gPwaoNGHdgDfF2QA/XZbCBp4Moo5scrkAMPhB7z26XM/AaHuIJdgauA== - -"@babel/runtime-corejs3@^7.10.2": - version "7.15.4" - resolved "https://registry.yarnpkg.com/@babel/runtime-corejs3/-/runtime-corejs3-7.15.4.tgz#403139af262b9a6e8f9ba04a6fdcebf8de692bf1" - integrity sha512-lWcAqKeB624/twtTc3w6w/2o9RqJPaNBhPGK6DKLSiwuVWC7WFkypWyNg+CpZoyJH0jVzv1uMtXZ/5/lQOLtCg== - dependencies: - core-js-pure "^3.16.0" - regenerator-runtime "^0.13.4" - -"@babel/runtime@^7.0.0", "@babel/runtime@^7.12.13", "@babel/runtime@^7.5.5", "@babel/runtime@^7.6.2": - version "7.17.2" - resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.17.2.tgz#66f68591605e59da47523c631416b18508779941" - integrity sha512-hzeyJyMA1YGdJTuWU0e/j4wKXrU4OMFvY2MSlaI9B7VQb0r5cxTE3EAIS2Q7Tn2RIcDkRvTA/v2JsAEhxe99uw== - dependencies: - regenerator-runtime "^0.13.4" - -"@babel/runtime@^7.10.2": - version "7.15.4" - resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.15.4.tgz#fd17d16bfdf878e6dd02d19753a39fa8a8d9c84a" - integrity sha512-99catp6bHCaxr4sJ/DbTGgHS4+Rs2RVd2g7iOap6SLGPDknRK9ztKNsE/Fg6QhSeh1FGE5f6gHGQmvvn3I3xhw== - dependencies: - regenerator-runtime "^0.13.4" - -"@babel/runtime@^7.12.0", "@babel/runtime@^7.16.3", "@babel/runtime@^7.8.7": - version "7.18.3" - resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.18.3.tgz#c7b654b57f6f63cf7f8b418ac9ca04408c4579f4" - integrity sha512-38Y8f7YUhce/K7RMwTp7m0uCumpv9hZkitCbBClqQIow1qSbCvGkcegKOXpEWCQLfWmevgRiWokZ1GkpfhbZug== - dependencies: - regenerator-runtime "^0.13.4" - -"@babel/runtime@^7.12.5", "@babel/runtime@^7.8.4", "@babel/runtime@^7.9.2": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.16.0.tgz#e27b977f2e2088ba24748bf99b5e1dece64e4f0b" - integrity sha512-Nht8L0O8YCktmsDV6FqFue7vQLRx3Hb0B37lS5y0jDRqRxlBG4wIJHnf9/bgSE2UyipKFA01YtS+npRdTWBUyw== - dependencies: - regenerator-runtime "^0.13.4" - -"@babel/runtime@^7.13.10", "@babel/runtime@^7.7.2": - version "7.15.3" - resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.15.3.tgz#2e1c2880ca118e5b2f9988322bd8a7656a32502b" - integrity sha512-OvwMLqNXkCXSz1kSm58sEsNuhqOx/fKpnUnKnFB5v8uDda5bLNEHNgKPvhDN6IU0LDcnHQ90LlJ0Q6jnyBSIBA== - dependencies: - regenerator-runtime "^0.13.4" - -"@babel/runtime@^7.14.0": - version "7.14.6" - resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.14.6.tgz#535203bc0892efc7dec60bdc27b2ecf6e409062d" - integrity sha512-/PCB2uJ7oM44tz8YhC4Z/6PeOKXp4K588f+5M3clr1M4zbqztlo0XEfJ2LEzj/FgwfgGcIdl8n7YYjTCI0BYwg== - dependencies: - regenerator-runtime "^0.13.4" - -"@babel/runtime@^7.3.1": - version "7.21.5" - resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.21.5.tgz#8492dddda9644ae3bda3b45eabe87382caee7200" - integrity sha512-8jI69toZqqcsnqGGqwGS4Qb1VwLOEp4hz+CXPywcvjs60u3B4Pom/U/7rm4W8tMOYEB+E9wgD0mW1l3r8qlI9Q== - dependencies: - regenerator-runtime "^0.13.11" - -"@babel/runtime@^7.7.6": - version "7.17.9" - resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.17.9.tgz#d19fbf802d01a8cb6cf053a64e472d42c434ba72" - integrity sha512-lSiBBvodq29uShpWGNbgFdKYNiFDo5/HIYsaCEY9ff4sb10x9jizo2+pRrSyF4jKZCXqgzuqBOQKbUm90gQwJg== - dependencies: - regenerator-runtime "^0.13.4" - -"@babel/template@^7.16.0", "@babel/template@^7.3.3": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/template/-/template-7.16.0.tgz#d16a35ebf4cd74e202083356fab21dd89363ddd6" - integrity sha512-MnZdpFD/ZdYhXwiunMqqgyZyucaYsbL0IrjoGjaVhGilz+x8YB++kRfygSOIj1yOtWKPlx7NBp+9I1RQSgsd5A== - dependencies: - "@babel/code-frame" "^7.16.0" - "@babel/parser" "^7.16.0" - "@babel/types" "^7.16.0" - -"@babel/template@^7.16.7": - version "7.16.7" - resolved "https://registry.yarnpkg.com/@babel/template/-/template-7.16.7.tgz#8d126c8701fde4d66b264b3eba3d96f07666d155" - integrity sha512-I8j/x8kHUrbYRTUxXrrMbfCa7jxkE7tZre39x3kjr9hvI82cK1FfqLygotcWN5kdPGWcLdWMHpSBavse5tWw3w== - dependencies: - "@babel/code-frame" "^7.16.7" - "@babel/parser" "^7.16.7" - "@babel/types" "^7.16.7" - -"@babel/template@^7.22.15": - version "7.22.15" - resolved "https://registry.yarnpkg.com/@babel/template/-/template-7.22.15.tgz#09576efc3830f0430f4548ef971dde1350ef2f38" - integrity sha512-QPErUVm4uyJa60rkI73qneDacvdvzxshT3kksGqlGWYdOTIUOwJ7RDUL8sGqslY1uXWSL6xMFKEXDS3ox2uF0w== - dependencies: - "@babel/code-frame" "^7.22.13" - "@babel/parser" "^7.22.15" - "@babel/types" "^7.22.15" - -"@babel/template@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/template/-/template-7.24.7.tgz#02efcee317d0609d2c07117cb70ef8fb17ab7315" - integrity sha512-jYqfPrU9JTF0PmPy1tLYHW4Mp4KlgxJD9l2nP9fD6yT/ICi554DmrWBAEYpIelzjHf1msDP3PxJIRt/nFNfBig== - dependencies: - "@babel/code-frame" "^7.24.7" - "@babel/parser" "^7.24.7" - "@babel/types" "^7.24.7" - -"@babel/traverse@^7.16.0", "@babel/traverse@^7.16.7", "@babel/traverse@^7.17.0", "@babel/traverse@^7.7.2": - version "7.23.2" - resolved "https://registry.yarnpkg.com/@babel/traverse/-/traverse-7.23.2.tgz#329c7a06735e144a506bdb2cad0268b7f46f4ad8" - integrity sha512-azpe59SQ48qG6nu2CzcMLbxUudtN+dOM9kDbUqGq3HXUJRlo7i8fvPoxQUzYgLZ4cMVmuZgm8vvBpNeRhd6XSw== - dependencies: - "@babel/code-frame" "^7.22.13" - "@babel/generator" "^7.23.0" - "@babel/helper-environment-visitor" "^7.22.20" - "@babel/helper-function-name" "^7.23.0" - "@babel/helper-hoist-variables" "^7.22.5" - "@babel/helper-split-export-declaration" "^7.22.6" - "@babel/parser" "^7.23.0" - "@babel/types" "^7.23.0" - debug "^4.1.0" - globals "^11.1.0" - -"@babel/traverse@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/traverse/-/traverse-7.24.7.tgz#de2b900163fa741721ba382163fe46a936c40cf5" - integrity sha512-yb65Ed5S/QAcewNPh0nZczy9JdYXkkAbIsEo+P7BE7yO3txAY30Y/oPa3QkQ5It3xVG2kpKMg9MsdxZaO31uKA== - dependencies: - "@babel/code-frame" "^7.24.7" - "@babel/generator" "^7.24.7" - "@babel/helper-environment-visitor" "^7.24.7" - "@babel/helper-function-name" "^7.24.7" - "@babel/helper-hoist-variables" "^7.24.7" - "@babel/helper-split-export-declaration" "^7.24.7" - "@babel/parser" "^7.24.7" - "@babel/types" "^7.24.7" - debug "^4.3.1" - globals "^11.1.0" - -"@babel/types@^7.0.0", "@babel/types@^7.16.0", "@babel/types@^7.3.0", "@babel/types@^7.3.3", "@babel/types@^7.4.4": - version "7.16.0" - resolved "https://registry.yarnpkg.com/@babel/types/-/types-7.16.0.tgz#db3b313804f96aadd0b776c4823e127ad67289ba" - integrity sha512-PJgg/k3SdLsGb3hhisFvtLOw5ts113klrpLuIPtCJIU+BB24fqq6lf8RWqKJEjzqXR9AEH1rIb5XTqwBHB+kQg== - dependencies: - "@babel/helper-validator-identifier" "^7.15.7" - to-fast-properties "^2.0.0" - -"@babel/types@^7.14.5": - version "7.14.5" - resolved "https://registry.yarnpkg.com/@babel/types/-/types-7.14.5.tgz#3bb997ba829a2104cedb20689c4a5b8121d383ff" - integrity sha512-M/NzBpEL95I5Hh4dwhin5JlE7EzO5PHMAuzjxss3tiOBD46KfQvVedN/3jEPZvdRvtsK2222XfdHogNIttFgcg== - dependencies: - "@babel/helper-validator-identifier" "^7.14.5" - to-fast-properties "^2.0.0" - -"@babel/types@^7.16.7", "@babel/types@^7.17.0": - version "7.17.0" - resolved "https://registry.yarnpkg.com/@babel/types/-/types-7.17.0.tgz#a826e368bccb6b3d84acd76acad5c0d87342390b" - integrity sha512-TmKSNO4D5rzhL5bjWFcVHHLETzfQ/AmbKpKPOSjlP0WoHZ6L911fgoOKY4Alp/emzG4cHJdyN49zpgkbXFEHHw== - dependencies: - "@babel/helper-validator-identifier" "^7.16.7" - to-fast-properties "^2.0.0" - -"@babel/types@^7.22.15", "@babel/types@^7.22.5", "@babel/types@^7.23.0": - version "7.23.0" - resolved "https://registry.yarnpkg.com/@babel/types/-/types-7.23.0.tgz#8c1f020c9df0e737e4e247c0619f58c68458aaeb" - integrity sha512-0oIyUfKoI3mSqMvsxBdclDwxXKXAUA8v/apZbc+iSyARYou1o8ZGDxbUYyLFoW2arqS2jDGqJuZvv1d/io1axg== - dependencies: - "@babel/helper-string-parser" "^7.22.5" - "@babel/helper-validator-identifier" "^7.22.20" - to-fast-properties "^2.0.0" - -"@babel/types@^7.24.7": - version "7.24.7" - resolved "https://registry.yarnpkg.com/@babel/types/-/types-7.24.7.tgz#6027fe12bc1aa724cd32ab113fb7f1988f1f66f2" - integrity sha512-XEFXSlxiG5td2EJRe8vOmRbaXVgfcBlszKujvVmWIK/UpywWljQCfzAv3RQCGujWQ1RD4YYWEAqDXfuJiy8f5Q== - dependencies: - "@babel/helper-string-parser" "^7.24.7" - "@babel/helper-validator-identifier" "^7.24.7" - to-fast-properties "^2.0.0" - -"@bcoe/v8-coverage@^0.2.3": - version "0.2.3" - resolved "https://registry.yarnpkg.com/@bcoe/v8-coverage/-/v8-coverage-0.2.3.tgz#75a2e8b51cb758a7553d6804a5932d7aace75c39" - integrity sha512-0hYQ8SB4Db5zvZB4axdMHGwEaQjkZzFjQiN9LVYvIFB2nSUHW9tYpxWriPrWDASIxiaXax83REcLxuSdnGPZtw== - -"@chakra-ui/accordion@2.1.4": - version "2.1.4" - resolved "https://registry.yarnpkg.com/@chakra-ui/accordion/-/accordion-2.1.4.tgz#a3eca38f8e52d5a5f4b9528fb9d269dcdcb035ac" - integrity sha512-PQFW6kr+Bdru0DjKA8akC4BAz1VAJisLgo4TsJwjPO2gTS0zr99C+3bBs9uoDnjSJAf18/Q5zdXv11adA8n2XA== - dependencies: - "@chakra-ui/descendant" "3.0.11" - "@chakra-ui/icon" "3.0.13" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-use-controllable-state" "2.0.6" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@chakra-ui/transition" "2.0.12" - -"@chakra-ui/alert@2.0.13": - version "2.0.13" - resolved "https://registry.yarnpkg.com/@chakra-ui/alert/-/alert-2.0.13.tgz#11d48346e501988074affe12a448add1a6060296" - integrity sha512-7LqPv6EUBte4XM/Q2qBFIT5o4BC0dSlni9BHOH2BgAc5B1NF+pBAMDTUH7JNBiN7RHTV7EHAIWDziiX/NK28+Q== - dependencies: - "@chakra-ui/icon" "3.0.13" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/spinner" "2.0.11" - -"@chakra-ui/anatomy@2.1.0": - version "2.1.0" - resolved "https://registry.yarnpkg.com/@chakra-ui/anatomy/-/anatomy-2.1.0.tgz#8aeb9b753f0412f262743adf68519dfa85120b3e" - integrity sha512-E3jMPGqKuGTbt7mKtc8g/MOOenw2c4wqRC1vOypyFgmC8wsewdY+DJJNENF3atXAK7p5VMBKQfZ7ipNlHnDAwA== - -"@chakra-ui/anatomy@^2.2.2": - version "2.2.2" - resolved "https://registry.yarnpkg.com/@chakra-ui/anatomy/-/anatomy-2.2.2.tgz#2d0e14cba2534d92077ca28abf8c183b6e27897b" - integrity sha512-MV6D4VLRIHr4PkW4zMyqfrNS1mPlCTiCXwvYGtDFQYr+xHFfonhAuf9WjsSc0nyp2m0OdkSLnzmVKkZFLo25Tg== - -"@chakra-ui/avatar@2.2.1": - version "2.2.1" - resolved "https://registry.yarnpkg.com/@chakra-ui/avatar/-/avatar-2.2.1.tgz#3946d8c3b1d49dc425aa80f22d2f53661395e394" - integrity sha512-sgiogfLM8vas8QJTt7AJI4XxNXYdViCWj+xYJwyOwUN93dWKImqqx3O2ihCXoXTIqQWg1rcEgoJ5CxCg6rQaQQ== - dependencies: - "@chakra-ui/image" "2.0.12" - "@chakra-ui/react-children-utils" "2.0.4" - "@chakra-ui/react-context" "2.0.5" - -"@chakra-ui/breadcrumb@2.1.1": - version "2.1.1" - resolved "https://registry.yarnpkg.com/@chakra-ui/breadcrumb/-/breadcrumb-2.1.1.tgz#e8a682a4909cf8ee5771f7b287524df2be383b8a" - integrity sha512-OSa+F9qJ1xmF0zVxC1GU46OWbbhGf0kurHioSB729d+tRw/OMzmqrrfCJ7KVUUN8NEnTZXT5FIgokMvHGEt+Hg== - dependencies: - "@chakra-ui/react-children-utils" "2.0.4" - "@chakra-ui/react-context" "2.0.5" - -"@chakra-ui/breakpoint-utils@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/breakpoint-utils/-/breakpoint-utils-2.0.5.tgz#55b571038b66e9f6d41633c102ea904c679dac5c" - integrity sha512-8uhrckMwoR/powlAhxiFZPM0s8vn0B2yEyEaRcwpy5NmRAJSTEotC2WkSyQl/Cjysx9scredumB5g+fBX7IqGQ== - -"@chakra-ui/button@2.0.13": - version "2.0.13" - resolved "https://registry.yarnpkg.com/@chakra-ui/button/-/button-2.0.13.tgz#5db6aa3425a6bebc2102cd9f58e434d5508dd999" - integrity sha512-T9W/zHpHZVcbx/BMg0JIXCgRycut/eYoTYee/E+eBxyPCH45n308AsYU2bZ8TgZxUwbYNRgMp4qRL/KHUQDv5g== - dependencies: - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@chakra-ui/spinner" "2.0.11" - -"@chakra-ui/card@2.1.1": - version "2.1.1" - resolved "https://registry.yarnpkg.com/@chakra-ui/card/-/card-2.1.1.tgz#b981a68d81d0f6447eb0d4d3fdcd7846bab2111f" - integrity sha512-vvmfuNn6gkfv6bGcXQe6kvWHspziPZgYnnffiEjPaZYtaf98WRszpjyPbFv0oQR/2H1RSE1oaTqa/J1rHrzw3A== - dependencies: - "@chakra-ui/react-context" "2.0.5" - -"@chakra-ui/checkbox@2.2.5": - version "2.2.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/checkbox/-/checkbox-2.2.5.tgz#ce1c409647d11bf947ff0316bc397bc7cf25316c" - integrity sha512-7fNH+Q2nB2uMSnYAPtYxnuwZ1MOJqblZHa/ScfZ/fjiPDyEae1m068ZP/l9yJ5zlawYMTkp83m/JVcu5QFYurA== - dependencies: - "@chakra-ui/form-control" "2.0.13" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-types" "2.0.5" - "@chakra-ui/react-use-callback-ref" "2.0.5" - "@chakra-ui/react-use-controllable-state" "2.0.6" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@chakra-ui/react-use-safe-layout-effect" "2.0.3" - "@chakra-ui/react-use-update-effect" "2.0.5" - "@chakra-ui/visually-hidden" "2.0.13" - "@zag-js/focus-visible" "0.1.0" - -"@chakra-ui/clickable@2.0.11": - version "2.0.11" - resolved "https://registry.yarnpkg.com/@chakra-ui/clickable/-/clickable-2.0.11.tgz#d0afcdb40ed1b1ceeabb4ac3e9f2f51fd3cbdac7" - integrity sha512-5Y2dl5cxNgOxHbjxyxsL6Vdze4wUUvwsMCCW3kXwgz2OUI2y5UsBZNcvhNJx3RchJEd0fylMKiKoKmnZMHN2aw== - dependencies: - "@chakra-ui/react-use-merge-refs" "2.0.5" - -"@chakra-ui/close-button@2.0.13": - version "2.0.13" - resolved "https://registry.yarnpkg.com/@chakra-ui/close-button/-/close-button-2.0.13.tgz#c549d682c66f3e08b1f37e98a83ebe0421846496" - integrity sha512-ZI/3p84FPlW0xoDCZYqsnIvR6bTc2d/TlhwyTHsDDxq9ZOWp9c2JicVn6WTdWGdshk8itnZZdG50IcnizGnimA== - dependencies: - "@chakra-ui/icon" "3.0.13" - -"@chakra-ui/color-mode@2.1.10": - version "2.1.10" - resolved "https://registry.yarnpkg.com/@chakra-ui/color-mode/-/color-mode-2.1.10.tgz#8d446550af80cf01a2ccd7470861cb0180112049" - integrity sha512-aUPouOUPn7IPm1v00/9AIkRuNrkCwJlbjVL1kJzLzxijYjbHvEHPxntITt+JWjtXPT8xdOq6mexLYCOGA67JwQ== - dependencies: - "@chakra-ui/react-use-safe-layout-effect" "2.0.3" - -"@chakra-ui/control-box@2.0.11": - version "2.0.11" - resolved "https://registry.yarnpkg.com/@chakra-ui/control-box/-/control-box-2.0.11.tgz#b2deec368fc83f6675964785f823e4c0c1f5d4ac" - integrity sha512-UJb4vqq+/FPuwTCuaPeHa2lwtk6u7eFvLuwDCST2e/sBWGJC1R+1/Il5pHccnWs09FWxyZ9v/Oxkg/CG3jZR4Q== - -"@chakra-ui/counter@2.0.11": - version "2.0.11" - resolved "https://registry.yarnpkg.com/@chakra-ui/counter/-/counter-2.0.11.tgz#b49aa76423e5f4a4a8e717750c190fa5050a3dca" - integrity sha512-1YRt/jom+m3iWw9J9trcM6rAHDvD4lwThiO9raxUK7BRsYUhnPZvsMpcXU1Moax218C4rRpbI9KfPLaig0m1xQ== - dependencies: - "@chakra-ui/number-utils" "2.0.5" - "@chakra-ui/react-use-callback-ref" "2.0.5" - -"@chakra-ui/css-reset@2.0.10": - version "2.0.10" - resolved "https://registry.yarnpkg.com/@chakra-ui/css-reset/-/css-reset-2.0.10.tgz#cb6cd97ee38f8069789f08c31a828bf3a7e339ea" - integrity sha512-FwHOfw2P4ckbpSahDZef2KoxcvHPUg09jlicWdp24/MjdsOO5PAB/apm2UBvQflY4WAJyOqYaOdnXFlR6nF4cQ== - -"@chakra-ui/descendant@3.0.11": - version "3.0.11" - resolved "https://registry.yarnpkg.com/@chakra-ui/descendant/-/descendant-3.0.11.tgz#cb8bca7b6e8915afc58cdb1444530a2d1b03efd3" - integrity sha512-sNLI6NS6uUgrvYS6Imhoc1YlI6bck6pfxMBJcnXVSfdIjD6XjCmeY2YgzrtDS+o+J8bB3YJeIAG/vsVy5USE5Q== - dependencies: - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-use-merge-refs" "2.0.5" - -"@chakra-ui/dom-utils@2.0.4": - version "2.0.4" - resolved "https://registry.yarnpkg.com/@chakra-ui/dom-utils/-/dom-utils-2.0.4.tgz#367fffecbd287e16836e093d4030dc6e3785d402" - integrity sha512-P936+WKinz5fgHzfwiUQjE/t7NC8bU89Tceim4tbn8CIm/9b+CsHX64eNw4vyJqRwt78TXQK7aGBIbS18R0q5Q== - -"@chakra-ui/editable@2.0.16": - version "2.0.16" - resolved "https://registry.yarnpkg.com/@chakra-ui/editable/-/editable-2.0.16.tgz#8a45ff26d77f06841ea986484d71ede312b4c22e" - integrity sha512-kIFPufzIlViNv7qi2PxxWWBvjLb+3IP5hUGmqOA9qcYz5TAdqblQqDClm0iajlIDNUFWnS4h056o8jKsQ42a5A== - dependencies: - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-types" "2.0.5" - "@chakra-ui/react-use-callback-ref" "2.0.5" - "@chakra-ui/react-use-controllable-state" "2.0.6" - "@chakra-ui/react-use-focus-on-pointer-down" "2.0.4" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@chakra-ui/react-use-safe-layout-effect" "2.0.3" - "@chakra-ui/react-use-update-effect" "2.0.5" - "@chakra-ui/shared-utils" "2.0.3" - -"@chakra-ui/event-utils@2.0.6": - version "2.0.6" - resolved "https://registry.yarnpkg.com/@chakra-ui/event-utils/-/event-utils-2.0.6.tgz#5e04d68ea070ef52ce212c2a99be9afcc015cfaf" - integrity sha512-ZIoqUbgJ5TcCbZRchMv4n7rOl1JL04doMebED88LO5mux36iVP9er/nnOY4Oke1bANKKURMrQf5VTT9hoYeA7A== - -"@chakra-ui/focus-lock@2.0.13": - version "2.0.13" - resolved "https://registry.yarnpkg.com/@chakra-ui/focus-lock/-/focus-lock-2.0.13.tgz#19d6ca35555965a9aaa241b991a67bbc875ee53d" - integrity sha512-AVSJt+3Ukia/m9TCZZgyWvTY7pw88jArivWVJ2gySGYYIs6z/FJMnlwbCVldV2afS0g3cYaii7aARb/WrlG34Q== - dependencies: - "@chakra-ui/dom-utils" "2.0.4" - react-focus-lock "^2.9.1" - -"@chakra-ui/form-control@2.0.13": - version "2.0.13" - resolved "https://registry.yarnpkg.com/@chakra-ui/form-control/-/form-control-2.0.13.tgz#51831f981a2e937b0258b4fd2dd4ceacda03c01a" - integrity sha512-J964JlgrxP+LP3kYmLk1ttbl73u6ghT+JQDjEjkEUc8lSS9Iv4u9XkRDQHuz2t2y0KHjQdH12PUfUfBqcITbYw== - dependencies: - "@chakra-ui/icon" "3.0.13" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-types" "2.0.5" - "@chakra-ui/react-use-merge-refs" "2.0.5" - -"@chakra-ui/hooks@2.1.2": - version "2.1.2" - resolved "https://registry.yarnpkg.com/@chakra-ui/hooks/-/hooks-2.1.2.tgz#1e413f6624e97b854569e8a19846c9162a4ec153" - integrity sha512-/vDBOqqnho9q++lay0ZcvnH8VuE0wT2OkZj+qDwFwjiHAtGPVxHCSpu9KC8BIHME5TlWjyO6riVyUCb2e2ip6w== - dependencies: - "@chakra-ui/react-utils" "2.0.9" - "@chakra-ui/utils" "2.0.12" - compute-scroll-into-view "1.0.14" - copy-to-clipboard "3.3.1" - -"@chakra-ui/icon@3.0.13": - version "3.0.13" - resolved "https://registry.yarnpkg.com/@chakra-ui/icon/-/icon-3.0.13.tgz#1782f8bd81946eabb39d4dde9eccebba1e5571ba" - integrity sha512-RaDLC4psd8qyInY2RX4AlYRfpLBNw3VsMih17BFf8EESVhBXNJcYy7Q9eMV/K4NvZfZT42vuVqGVNFmkG89lBQ== - dependencies: - "@chakra-ui/shared-utils" "2.0.3" - -"@chakra-ui/image@2.0.12": - version "2.0.12" - resolved "https://registry.yarnpkg.com/@chakra-ui/image/-/image-2.0.12.tgz#e90b1d2a5f87fff90b1ef86ca75bfe6b44ac545d" - integrity sha512-uclFhs0+wq2qujGu8Wk4eEWITA3iZZQTitGiFSEkO9Ws5VUH+Gqtn3mUilH0orubrI5srJsXAmjVTuVwge1KJQ== - dependencies: - "@chakra-ui/react-use-safe-layout-effect" "2.0.3" - -"@chakra-ui/input@2.0.14": - version "2.0.14" - resolved "https://registry.yarnpkg.com/@chakra-ui/input/-/input-2.0.14.tgz#eec3d04834ab1ac7dab344e9bf14d779c4f4da31" - integrity sha512-CkSrUJeKWogOSt2pUf2vVv5s0bUVcAi4/XGj1JVCCfyIX6a6h1m8R69MShTPxPiQ0Mdebq5ATrW/aZQQXZzRGQ== - dependencies: - "@chakra-ui/form-control" "2.0.13" - "@chakra-ui/object-utils" "2.0.5" - "@chakra-ui/react-children-utils" "2.0.4" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/shared-utils" "2.0.3" - -"@chakra-ui/layout@2.1.11": - version "2.1.11" - resolved "https://registry.yarnpkg.com/@chakra-ui/layout/-/layout-2.1.11.tgz#6b0005dd897a901f2fded99c19fe47f60db80cb3" - integrity sha512-UP19V8EeI/DEODbWrZlqC0sg248bpFaWpMiM/+g9Bsxs9aof3yexpMD/7gb0yrfbIrkdvSBrcQeqxXGzbfoopw== - dependencies: - "@chakra-ui/breakpoint-utils" "2.0.5" - "@chakra-ui/icon" "3.0.13" - "@chakra-ui/object-utils" "2.0.5" - "@chakra-ui/react-children-utils" "2.0.4" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/shared-utils" "2.0.3" - -"@chakra-ui/lazy-utils@2.0.3": - version "2.0.3" - resolved "https://registry.yarnpkg.com/@chakra-ui/lazy-utils/-/lazy-utils-2.0.3.tgz#5ba459a2541ad0c98cd98b20a8054664c129e9b4" - integrity sha512-SQ5I5rJrcHpVUcEftHLOh8UyeY+06R8Gv3k2RjcpvM6mb2Gktlz/4xl2GcUh3LWydgGQDW/7Rse5rQhKWgzmcg== - -"@chakra-ui/live-region@2.0.11": - version "2.0.11" - resolved "https://registry.yarnpkg.com/@chakra-ui/live-region/-/live-region-2.0.11.tgz#1008c5b629aa4120e5158be53f13d8d34bc2d71a" - integrity sha512-ltObaKQekP75GCCbN+vt1/mGABSCaRdQELmotHTBc5AioA3iyCDHH69ev+frzEwLvKFqo+RomAdAAgqBIMJ02Q== - -"@chakra-ui/media-query@3.2.8": - version "3.2.8" - resolved "https://registry.yarnpkg.com/@chakra-ui/media-query/-/media-query-3.2.8.tgz#7d5feccb7ac52891426c060dd2ed1df37420956d" - integrity sha512-djmEg/eJ5Qrjn7SArTqjsvlwF6mNeMuiawrTwnU+0EKq9Pq/wVSb7VaIhxdQYJLA/DbRhE/KPMogw1LNVKa4Rw== - dependencies: - "@chakra-ui/breakpoint-utils" "2.0.5" - "@chakra-ui/react-env" "2.0.11" - -"@chakra-ui/menu@2.1.5": - version "2.1.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/menu/-/menu-2.1.5.tgz#73b6db93d6dec9d04ab7c2630a7984e3180fd769" - integrity sha512-2UusrQtxHcqcO9n/0YobNN3RJC8yAZU6oJbRPuvsQ9IL89scEWCTIxXEYrnIjeh/5zikcSEDGo9zM9Udg/XcsA== - dependencies: - "@chakra-ui/clickable" "2.0.11" - "@chakra-ui/descendant" "3.0.11" - "@chakra-ui/lazy-utils" "2.0.3" - "@chakra-ui/popper" "3.0.10" - "@chakra-ui/react-children-utils" "2.0.4" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-use-animation-state" "2.0.6" - "@chakra-ui/react-use-controllable-state" "2.0.6" - "@chakra-ui/react-use-disclosure" "2.0.6" - "@chakra-ui/react-use-focus-effect" "2.0.7" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@chakra-ui/react-use-outside-click" "2.0.5" - "@chakra-ui/react-use-update-effect" "2.0.5" - "@chakra-ui/transition" "2.0.12" - -"@chakra-ui/modal@2.2.4": - version "2.2.4" - resolved "https://registry.yarnpkg.com/@chakra-ui/modal/-/modal-2.2.4.tgz#dbe884a9245ed840b6511a4f06b4a622fa86de4c" - integrity sha512-K2cafyNI0b4OSAB55qIXt5DLZqj7E1G0+Fza02ZOBZpgTCNQyDtc0KzdVMJZ9ryxKd16LUk5UmKHugY/VpHEWQ== - dependencies: - "@chakra-ui/close-button" "2.0.13" - "@chakra-ui/focus-lock" "2.0.13" - "@chakra-ui/portal" "2.0.11" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-types" "2.0.5" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@chakra-ui/transition" "2.0.12" - aria-hidden "^1.1.1" - react-remove-scroll "^2.5.4" - -"@chakra-ui/number-input@2.0.14": - version "2.0.14" - resolved "https://registry.yarnpkg.com/@chakra-ui/number-input/-/number-input-2.0.14.tgz#e6336228b9210f9543fe440bfc6478537810d59c" - integrity sha512-IARUAbP4pn1gP5fY2dK4wtbR3ONjzHgTjH4Zj3ErZvdu/yTURLaZmlb6UGHwgqjWLyioactZ/+n4Njj5CRjs8w== - dependencies: - "@chakra-ui/counter" "2.0.11" - "@chakra-ui/form-control" "2.0.13" - "@chakra-ui/icon" "3.0.13" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-types" "2.0.5" - "@chakra-ui/react-use-callback-ref" "2.0.5" - "@chakra-ui/react-use-event-listener" "2.0.5" - "@chakra-ui/react-use-interval" "2.0.3" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@chakra-ui/react-use-safe-layout-effect" "2.0.3" - "@chakra-ui/react-use-update-effect" "2.0.5" - -"@chakra-ui/number-utils@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/number-utils/-/number-utils-2.0.5.tgz#c7595fc919fca7c43fe172bfd6c5197c653ee572" - integrity sha512-Thhohnlqze0i5HBJO9xkfOPq1rv3ji/hNPf2xh1fh4hxrNzdm3HCkz0c6lyRQwGuVoeltEHysYZLH/uWLFTCSQ== - -"@chakra-ui/object-utils@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/object-utils/-/object-utils-2.0.5.tgz#231602066ddb96ae91dcc7da243b97ad46972398" - integrity sha512-/rIMoYI3c2uLtFIrnTFOPRAI8StUuu335WszqKM0KAW1lwG9H6uSbxqlpZT1Pxi/VQqZKfheGiMQOx5lfTmM/A== - -"@chakra-ui/pin-input@2.0.16": - version "2.0.16" - resolved "https://registry.yarnpkg.com/@chakra-ui/pin-input/-/pin-input-2.0.16.tgz#d31a6e2bce85aa2d1351ccb4cd9bf7a5134d3fb9" - integrity sha512-51cioNYpBSgi9/jq6CrzoDvo8fpMwFXu3SaFRbKO47s9Dz/OAW0MpjyabTfSpwOv0xKZE+ayrYGJopCzZSWXPg== - dependencies: - "@chakra-ui/descendant" "3.0.11" - "@chakra-ui/react-children-utils" "2.0.4" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-use-controllable-state" "2.0.6" - "@chakra-ui/react-use-merge-refs" "2.0.5" - -"@chakra-ui/popover@2.1.4": - version "2.1.4" - resolved "https://registry.yarnpkg.com/@chakra-ui/popover/-/popover-2.1.4.tgz#c44df775875faabe09ef13ce32150a2631c768dd" - integrity sha512-NXVtyMxYzDKzzQph/+GFRSM3tEj3gNvlCX/xGRsCOt9I446zJ1InCd/boXQKAc813coEN9McSOjNWgo+NCBD+Q== - dependencies: - "@chakra-ui/close-button" "2.0.13" - "@chakra-ui/lazy-utils" "2.0.3" - "@chakra-ui/popper" "3.0.10" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-types" "2.0.5" - "@chakra-ui/react-use-animation-state" "2.0.6" - "@chakra-ui/react-use-disclosure" "2.0.6" - "@chakra-ui/react-use-focus-effect" "2.0.7" - "@chakra-ui/react-use-focus-on-pointer-down" "2.0.4" - "@chakra-ui/react-use-merge-refs" "2.0.5" - -"@chakra-ui/popper@3.0.10": - version "3.0.10" - resolved "https://registry.yarnpkg.com/@chakra-ui/popper/-/popper-3.0.10.tgz#5d382c36359615e349e679445eb58c139dbb4d4f" - integrity sha512-6LacbBGX0piHWY/DYxOGCTTFAoRGRHpGIRzTgfNy8jxw4f+rukaVudd4Pc2fwjCTdobJKM8nGNYIYNv9/Dmq9Q== - dependencies: - "@chakra-ui/react-types" "2.0.5" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@popperjs/core" "^2.9.3" - -"@chakra-ui/portal@2.0.11": - version "2.0.11" - resolved "https://registry.yarnpkg.com/@chakra-ui/portal/-/portal-2.0.11.tgz#7a6b3ebc621bb28b46550fcfb36b94926d0111a5" - integrity sha512-Css61i4WKzKO8ou1aGjBzcsXMy9LnfnpkOFfvaNCpUUNEd6c47z6+FhZNq7Gc38PGNjSfMLAd4LmH+H0ZanYIA== - dependencies: - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-use-safe-layout-effect" "2.0.3" - -"@chakra-ui/progress@2.1.1": - version "2.1.1" - resolved "https://registry.yarnpkg.com/@chakra-ui/progress/-/progress-2.1.1.tgz#b94399af12e9324737f9e690201f78546572ac59" - integrity sha512-ddAXaYGNObGqH1stRAYxkdospf6J4CDOhB0uyw9BeHRSsYkCUQWkUBd/melJuZeGHEH2ItF9T7FZ4JhcepP3GA== - dependencies: - "@chakra-ui/react-context" "2.0.5" - -"@chakra-ui/provider@2.0.24": - version "2.0.24" - resolved "https://registry.yarnpkg.com/@chakra-ui/provider/-/provider-2.0.24.tgz#733f0eacf779d39029cee164027af7bf5c6c66c3" - integrity sha512-32+DGfoXAOUOXwjLstdGQ+k/YoCwdFxWbwnEAp7WleislYsMcl0JeINDAbvksQH0piBty77swTuWfUU5cIox7g== - dependencies: - "@chakra-ui/css-reset" "2.0.10" - "@chakra-ui/portal" "2.0.11" - "@chakra-ui/react-env" "2.0.11" - "@chakra-ui/system" "2.3.4" - "@chakra-ui/utils" "2.0.12" - -"@chakra-ui/radio@2.0.14": - version "2.0.14" - resolved "https://registry.yarnpkg.com/@chakra-ui/radio/-/radio-2.0.14.tgz#f214f728235782a2ac49c0eb507f151612e31b2e" - integrity sha512-e/hY1g92Xdu5d5A27NFfa1+ccE2q/A5H7sc/M7p0fId6KO33Dst25Hy+HThtqnYN0Y3Om58fiXEKo5SsdtvSfA== - dependencies: - "@chakra-ui/form-control" "2.0.13" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-types" "2.0.5" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@zag-js/focus-visible" "0.1.0" - -"@chakra-ui/react-children-utils@2.0.4": - version "2.0.4" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-children-utils/-/react-children-utils-2.0.4.tgz#6e4284297a8a9b4e6f5f955b099bb6c2c6bbf8b9" - integrity sha512-qsKUEfK/AhDbMexWo5JhmdlkxLg5WEw2dFh4XorvU1/dTYsRfP6cjFfO8zE+X3F0ZFNsgKz6rbN5oU349GLEFw== - -"@chakra-ui/react-context@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-context/-/react-context-2.0.5.tgz#c434013ecc46c780539791d756dafdfc7c64320e" - integrity sha512-WYS0VBl5Q3/kNShQ26BP+Q0OGMeTQWco3hSiJWvO2wYLY7N1BLq6dKs8vyKHZfpwKh2YL2bQeAObi+vSkXp6tQ== - -"@chakra-ui/react-env@2.0.11": - version "2.0.11" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-env/-/react-env-2.0.11.tgz#d9d65fb695de7aff15e1d0e97d57bb7bedce5fa2" - integrity sha512-rPwUHReSWh7rbCw0HePa8Pvc+Q82fUFvVjHTIbXKnE6d+01cCE7j4f1NLeRD9pStKPI6sIZm9xTGvOCzl8F8iw== - -"@chakra-ui/react-types@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-types/-/react-types-2.0.5.tgz#1e4c99ef0e59b5fe9263d0e186cd66afdfb6c87b" - integrity sha512-GApp+R/VjS1UV5ms5irrij5LOIgUM0dqSVHagyEFEz88LRKkqMD9RuO577ZsVd4Gn0ULsacVJCUA0HtNUBJNzA== - -"@chakra-ui/react-use-animation-state@2.0.6": - version "2.0.6" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-animation-state/-/react-use-animation-state-2.0.6.tgz#2a324d3c67015a542ed589f899672d681889e1e7" - integrity sha512-M2kUzZkSBgDpfvnffh3kTsMIM3Dvn+CTMqy9zfY97NL4P3LAWL1MuFtKdlKfQ8hs/QpwS/ew8CTmCtaywn4sKg== - dependencies: - "@chakra-ui/dom-utils" "2.0.4" - "@chakra-ui/react-use-event-listener" "2.0.5" - -"@chakra-ui/react-use-callback-ref@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-callback-ref/-/react-use-callback-ref-2.0.5.tgz#862430dfbab8e1f0b8e04476e5e25469bd044ec9" - integrity sha512-vKnXleD2PzB0nGabY35fRtklMid4z7cecbMG0fkasNNsgWmrQcXJOuEKUUVCynL6FBU6gBnpKFi5Aqj6x+K4tw== - -"@chakra-ui/react-use-controllable-state@2.0.6": - version "2.0.6" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-controllable-state/-/react-use-controllable-state-2.0.6.tgz#ec62aff9b9c00324a0a4c9a4523824a9ad5ef9aa" - integrity sha512-7WuKrhQkpSRoiI5PKBvuIsO46IIP0wsRQgXtStSaIXv+FIvIJl9cxQXTbmZ5q1Ds641QdAUKx4+6v0K/zoZEHg== - dependencies: - "@chakra-ui/react-use-callback-ref" "2.0.5" - -"@chakra-ui/react-use-disclosure@2.0.6": - version "2.0.6" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-disclosure/-/react-use-disclosure-2.0.6.tgz#db707ee119db829e9b21ff1c05e867938f1e27ba" - integrity sha512-4UPePL+OcCY37KZ585iLjg8i6J0sjpLm7iZG3PUwmb97oKHVHq6DpmWIM0VfSjcT6AbSqyGcd5BXZQBgwt8HWQ== - dependencies: - "@chakra-ui/react-use-callback-ref" "2.0.5" - -"@chakra-ui/react-use-event-listener@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-event-listener/-/react-use-event-listener-2.0.5.tgz#949aa99878b25b23709452d3c80a1570c99747cc" - integrity sha512-etLBphMigxy/cm7Yg22y29gQ8u/K3PniR5ADZX7WVX61Cgsa8ciCqjTE9sTtlJQWAQySbWxt9+mjlT5zaf+6Zw== - dependencies: - "@chakra-ui/react-use-callback-ref" "2.0.5" - -"@chakra-ui/react-use-focus-effect@2.0.7": - version "2.0.7" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-focus-effect/-/react-use-focus-effect-2.0.7.tgz#bd03290cac32e0de6a71ce987f939a5e697bca04" - integrity sha512-wI8OUNwfbkusajLac8QtjfSyNmsNu1D5pANmnSHIntHhui6Jwv75Pxx7RgmBEnfBEpleBndhR9E75iCjPLhZ/A== - dependencies: - "@chakra-ui/dom-utils" "2.0.4" - "@chakra-ui/react-use-event-listener" "2.0.5" - "@chakra-ui/react-use-safe-layout-effect" "2.0.3" - "@chakra-ui/react-use-update-effect" "2.0.5" - -"@chakra-ui/react-use-focus-on-pointer-down@2.0.4": - version "2.0.4" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-focus-on-pointer-down/-/react-use-focus-on-pointer-down-2.0.4.tgz#aeba543c451ac1b0138093234e71d334044daf84" - integrity sha512-L3YKouIi77QbXH9mSLGEFzJbJDhyrPlcRcuu+TSC7mYaK9E+3Ap+RVSAVxj+CfQz7hCWpikPecKDuspIPWlyuA== - dependencies: - "@chakra-ui/react-use-event-listener" "2.0.5" - -"@chakra-ui/react-use-interval@2.0.3": - version "2.0.3" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-interval/-/react-use-interval-2.0.3.tgz#d5c7bce117fb25edb54e3e2c666e900618bb5bb2" - integrity sha512-Orbij5c5QkL4NuFyU4mfY/nyRckNBgoGe9ic8574VVNJIXfassevZk0WB+lvqBn5XZeLf2Tj+OGJrg4j4H9wzw== - dependencies: - "@chakra-ui/react-use-callback-ref" "2.0.5" - -"@chakra-ui/react-use-latest-ref@2.0.3": - version "2.0.3" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-latest-ref/-/react-use-latest-ref-2.0.3.tgz#27cf703858e65ecb5a0eef26215c794ad2a5353d" - integrity sha512-exNSQD4rPclDSmNwtcChUCJ4NuC2UJ4amyNGBqwSjyaK5jNHk2kkM7rZ6I0I8ul+26lvrXlSuhyv6c2PFwbFQQ== - -"@chakra-ui/react-use-merge-refs@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-merge-refs/-/react-use-merge-refs-2.0.5.tgz#13181e1a43219c6a04a01f84de0188df042ee92e" - integrity sha512-uc+MozBZ8asaUpO8SWcK6D4svRPACN63jv5uosUkXJR+05jQJkUofkfQbf2HeGVbrWCr0XZsftLIm4Mt/QMoVw== - -"@chakra-ui/react-use-outside-click@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-outside-click/-/react-use-outside-click-2.0.5.tgz#6a9896d2c2d35f3c301c3bb62bed1bf5290d1e60" - integrity sha512-WmtXUeVaMtxP9aUGGG+GQaDeUn/Bvf8TI3EU5mE1+TtqLHxyA9wtvQurynrogvpilLaBADwn/JeBeqs2wHpvqA== - dependencies: - "@chakra-ui/react-use-callback-ref" "2.0.5" - -"@chakra-ui/react-use-pan-event@2.0.6": - version "2.0.6" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-pan-event/-/react-use-pan-event-2.0.6.tgz#e489d61672e6f473b7fd362d816e2e27ed3b2af6" - integrity sha512-Vtgl3c+Mj4hdehFRFIgruQVXctwnG1590Ein1FiU8sVnlqO6bpug6Z+B14xBa+F+X0aK+DxnhkJFyWI93Pks2g== - dependencies: - "@chakra-ui/event-utils" "2.0.6" - "@chakra-ui/react-use-latest-ref" "2.0.3" - framesync "5.3.0" - -"@chakra-ui/react-use-previous@2.0.3": - version "2.0.3" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-previous/-/react-use-previous-2.0.3.tgz#9da3d53fd75f1c3da902bd6af71dcb1a7be37f31" - integrity sha512-A2ODOa0rm2HM4aqXfxxI0zPLcn5Q7iBEjRyfIQhb+EH+d2OFuj3L2slVoIpp6e/km3Xzv2d+u/WbjgTzdQ3d0w== - -"@chakra-ui/react-use-safe-layout-effect@2.0.3": - version "2.0.3" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-safe-layout-effect/-/react-use-safe-layout-effect-2.0.3.tgz#bf63ac8c94460aa1b20b6b601a0ea873556ffb1b" - integrity sha512-dlTvQURzmdfyBbNdydgO4Wy2/HV8aJN8LszTtyb5vRZsyaslDM/ftcxo8E8QjHwRLD/V1Epb/A8731QfimfVaQ== - -"@chakra-ui/react-use-size@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-size/-/react-use-size-2.0.5.tgz#4bbffb64f97dcfe1d7edeb0f03bb1d5fbc48df64" - integrity sha512-4arAApdiXk5uv5ZeFKltEUCs5h3yD9dp6gTIaXbAdq+/ENK3jMWTwlqzNbJtCyhwoOFrblLSdBrssBMIsNQfZQ== - dependencies: - "@zag-js/element-size" "0.1.0" - -"@chakra-ui/react-use-timeout@2.0.3": - version "2.0.3" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-timeout/-/react-use-timeout-2.0.3.tgz#16ca397dbca55a64811575884cb81a348d86d4e2" - integrity sha512-rBBUkZSQq3nJQ8fuMkgZNY2Sgg4vKiKNp05GxAwlT7TitOfVZyoTriqQpqz296bWlmkICTZxlqCWfE5fWpsTsg== - dependencies: - "@chakra-ui/react-use-callback-ref" "2.0.5" - -"@chakra-ui/react-use-update-effect@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-use-update-effect/-/react-use-update-effect-2.0.5.tgz#aede8f13f2b3de254b4ffa3b8cec1b70bd2876c5" - integrity sha512-y9tCMr1yuDl8ATYdh64Gv8kge5xE1DMykqPDZw++OoBsTaWr3rx40wblA8NIWuSyJe5ErtKP2OeglvJkYhryJQ== - -"@chakra-ui/react-utils@2.0.9": - version "2.0.9" - resolved "https://registry.yarnpkg.com/@chakra-ui/react-utils/-/react-utils-2.0.9.tgz#5cdf0bc8dee57c15f15ace04fbba574ec8aa6ecc" - integrity sha512-nlwPBVlQmcl1PiLzZWyrT3FSnt3vKSkBMzQ0EF4SJWA/nOIqTvmffb5DCzCqPzgQaE/Da1Xgus+JufFGM8GLCQ== - dependencies: - "@chakra-ui/utils" "2.0.12" - -"@chakra-ui/react@2.4.2": - version "2.4.2" - resolved "https://registry.yarnpkg.com/@chakra-ui/react/-/react-2.4.2.tgz#60d0cf80965d77ab6e280e28124b800a0d7a5f8c" - integrity sha512-lPDCCuY3S7XSeIK+P+ypGIL+lFqEZQt8H3Iyq4coblULMsj8skdSUqaoQ4I9fGgOi1koTPe4OlXb+rmqwQQ9MQ== - dependencies: - "@chakra-ui/accordion" "2.1.4" - "@chakra-ui/alert" "2.0.13" - "@chakra-ui/avatar" "2.2.1" - "@chakra-ui/breadcrumb" "2.1.1" - "@chakra-ui/button" "2.0.13" - "@chakra-ui/card" "2.1.1" - "@chakra-ui/checkbox" "2.2.5" - "@chakra-ui/close-button" "2.0.13" - "@chakra-ui/control-box" "2.0.11" - "@chakra-ui/counter" "2.0.11" - "@chakra-ui/css-reset" "2.0.10" - "@chakra-ui/editable" "2.0.16" - "@chakra-ui/form-control" "2.0.13" - "@chakra-ui/hooks" "2.1.2" - "@chakra-ui/icon" "3.0.13" - "@chakra-ui/image" "2.0.12" - "@chakra-ui/input" "2.0.14" - "@chakra-ui/layout" "2.1.11" - "@chakra-ui/live-region" "2.0.11" - "@chakra-ui/media-query" "3.2.8" - "@chakra-ui/menu" "2.1.5" - "@chakra-ui/modal" "2.2.4" - "@chakra-ui/number-input" "2.0.14" - "@chakra-ui/pin-input" "2.0.16" - "@chakra-ui/popover" "2.1.4" - "@chakra-ui/popper" "3.0.10" - "@chakra-ui/portal" "2.0.11" - "@chakra-ui/progress" "2.1.1" - "@chakra-ui/provider" "2.0.24" - "@chakra-ui/radio" "2.0.14" - "@chakra-ui/react-env" "2.0.11" - "@chakra-ui/select" "2.0.14" - "@chakra-ui/skeleton" "2.0.18" - "@chakra-ui/slider" "2.0.14" - "@chakra-ui/spinner" "2.0.11" - "@chakra-ui/stat" "2.0.13" - "@chakra-ui/styled-system" "2.4.0" - "@chakra-ui/switch" "2.0.17" - "@chakra-ui/system" "2.3.4" - "@chakra-ui/table" "2.0.12" - "@chakra-ui/tabs" "2.1.5" - "@chakra-ui/tag" "2.0.13" - "@chakra-ui/textarea" "2.0.14" - "@chakra-ui/theme" "2.2.2" - "@chakra-ui/theme-utils" "2.0.5" - "@chakra-ui/toast" "4.0.4" - "@chakra-ui/tooltip" "2.2.2" - "@chakra-ui/transition" "2.0.12" - "@chakra-ui/utils" "2.0.12" - "@chakra-ui/visually-hidden" "2.0.13" - -"@chakra-ui/select@2.0.14": - version "2.0.14" - resolved "https://registry.yarnpkg.com/@chakra-ui/select/-/select-2.0.14.tgz#b2230702e31d2b9b4cc7848b18ba7ae8e4c89bdb" - integrity sha512-fvVGxAtLaIXGOMicrzSa6imMw5h26S1ar3xyNmXgR40dbpTPHmtQJkbHBf9FwwQXgSgKWgBzsztw5iDHCpPVzA== - dependencies: - "@chakra-ui/form-control" "2.0.13" - -"@chakra-ui/shared-utils@2.0.3": - version "2.0.3" - resolved "https://registry.yarnpkg.com/@chakra-ui/shared-utils/-/shared-utils-2.0.3.tgz#97cbc11282e381ebd9f581c603088f9d60ead451" - integrity sha512-pCU+SUGdXzjAuUiUT8mriekL3tJVfNdwSTIaNeip7k/SWDzivrKGMwAFBxd3XVTDevtVusndkO4GJuQ3yILzDg== - -"@chakra-ui/skeleton@2.0.18": - version "2.0.18" - resolved "https://registry.yarnpkg.com/@chakra-ui/skeleton/-/skeleton-2.0.18.tgz#a2af241f0b1b692db4d10b90a887107a5e401c7d" - integrity sha512-qjcD8BgVx4kL8Lmb8EvmmDGM2ICl6CqhVE2LShJrgG7PDM6Rt6rYM617kqLurLYZjbJUiwgf9VXWifS0IpT31Q== - dependencies: - "@chakra-ui/media-query" "3.2.8" - "@chakra-ui/react-use-previous" "2.0.3" - -"@chakra-ui/slider@2.0.14": - version "2.0.14" - resolved "https://registry.yarnpkg.com/@chakra-ui/slider/-/slider-2.0.14.tgz#8fa8fb5df292525d8b97ea3c3c666e400fb365f2" - integrity sha512-z4Q5rWtYVTdFgBVvR6aUhSMg3CQuAgjJGHvLHEGDCUjYCuBXrb3SmWyvv03uKyjSbwRyKqSsvAnSCxtmHODt/w== - dependencies: - "@chakra-ui/number-utils" "2.0.5" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-types" "2.0.5" - "@chakra-ui/react-use-callback-ref" "2.0.5" - "@chakra-ui/react-use-controllable-state" "2.0.6" - "@chakra-ui/react-use-latest-ref" "2.0.3" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@chakra-ui/react-use-pan-event" "2.0.6" - "@chakra-ui/react-use-size" "2.0.5" - "@chakra-ui/react-use-update-effect" "2.0.5" - -"@chakra-ui/spinner@2.0.11": - version "2.0.11" - resolved "https://registry.yarnpkg.com/@chakra-ui/spinner/-/spinner-2.0.11.tgz#a5dd76b6cb0f3524d9b90b73fa4acfb6adc69f33" - integrity sha512-piO2ghWdJzQy/+89mDza7xLhPnW7pA+ADNbgCb1vmriInWedS41IBKe+pSPz4IidjCbFu7xwKE0AerFIbrocCA== - -"@chakra-ui/stat@2.0.13": - version "2.0.13" - resolved "https://registry.yarnpkg.com/@chakra-ui/stat/-/stat-2.0.13.tgz#1805817ab54f9d9b663b465fcb255285d22d0152" - integrity sha512-6XeuE/7w0BjyCHSxMbsf6/rNOOs8BSit1NS7g7+Jd/40Pc/SKlNWLd3kxXPid4eT3RwyNIdMPtm30OActr9nqQ== - dependencies: - "@chakra-ui/icon" "3.0.13" - "@chakra-ui/react-context" "2.0.5" - -"@chakra-ui/styled-system@2.4.0": - version "2.4.0" - resolved "https://registry.yarnpkg.com/@chakra-ui/styled-system/-/styled-system-2.4.0.tgz#4b50079606331e4e8fda7ea59da9db51b446d40c" - integrity sha512-G4HpbFERq4C1cBwKNDNkpCiliOICLXjYwKI/e/6hxNY+GlPxt8BCzz3uhd3vmEoG2vRM4qjidlVjphhWsf6vRQ== - dependencies: - csstype "^3.0.11" - lodash.mergewith "4.6.2" - -"@chakra-ui/switch@2.0.17": - version "2.0.17" - resolved "https://registry.yarnpkg.com/@chakra-ui/switch/-/switch-2.0.17.tgz#1d6904b6cde2469212bbd8311b749b96c653a9a3" - integrity sha512-BQabfC6qYi5xBJvEFPzKq0yl6fTtTNNEHTid5r7h0PWcCnAiHwQJTpQRpxp+AjK569LMLtTXReTZvNBrzEwOrA== - dependencies: - "@chakra-ui/checkbox" "2.2.5" - -"@chakra-ui/system@2.3.4": - version "2.3.4" - resolved "https://registry.yarnpkg.com/@chakra-ui/system/-/system-2.3.4.tgz#425bf7eebf61bd92aa68f60a6b62c380274fbe4e" - integrity sha512-/2m8hFfFzOMO2OlwHxTWqINOBJMjxWwU5V/AcB7C0qS51Dcj9c7kupilM6QdqiOLLdMS7mIVRSYr8jn8gMw9fA== - dependencies: - "@chakra-ui/color-mode" "2.1.10" - "@chakra-ui/react-utils" "2.0.9" - "@chakra-ui/styled-system" "2.4.0" - "@chakra-ui/theme-utils" "2.0.5" - "@chakra-ui/utils" "2.0.12" - react-fast-compare "3.2.0" - -"@chakra-ui/table@2.0.12": - version "2.0.12" - resolved "https://registry.yarnpkg.com/@chakra-ui/table/-/table-2.0.12.tgz#387653cf660318b13086b6497aca2b671deb055a" - integrity sha512-TSxzpfrOoB+9LTdNTMnaQC6OTsp36TlCRxJ1+1nAiCmlk+m+FiNzTQsmBalDDhc29rm+6AdRsxSPsjGWB8YVwg== - dependencies: - "@chakra-ui/react-context" "2.0.5" - -"@chakra-ui/tabs@2.1.5": - version "2.1.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/tabs/-/tabs-2.1.5.tgz#827b0e71eb173c09c31dcbbe05fc1146f4267229" - integrity sha512-XmnKDclAJe0FoW4tdC8AlnZpPN5fcj92l4r2sqiL9WyYVEM71hDxZueETIph/GTtfMelG7Z8e5vBHP4rh1RT5g== - dependencies: - "@chakra-ui/clickable" "2.0.11" - "@chakra-ui/descendant" "3.0.11" - "@chakra-ui/lazy-utils" "2.0.3" - "@chakra-ui/react-children-utils" "2.0.4" - "@chakra-ui/react-context" "2.0.5" - "@chakra-ui/react-use-controllable-state" "2.0.6" - "@chakra-ui/react-use-merge-refs" "2.0.5" - "@chakra-ui/react-use-safe-layout-effect" "2.0.3" - -"@chakra-ui/tag@2.0.13": - version "2.0.13" - resolved "https://registry.yarnpkg.com/@chakra-ui/tag/-/tag-2.0.13.tgz#ad7349bfcdd5642d3894fadb43728acc0f061101" - integrity sha512-W1urf+tvGMt6J3cc31HudybYSl+B5jYUP5DJxzXM9p+n3JrvXWAo4D6LmpLBHY5zT2mNne14JF1rVeRcG4Rtdg== - dependencies: - "@chakra-ui/icon" "3.0.13" - "@chakra-ui/react-context" "2.0.5" - -"@chakra-ui/textarea@2.0.14": - version "2.0.14" - resolved "https://registry.yarnpkg.com/@chakra-ui/textarea/-/textarea-2.0.14.tgz#a79a3fdd850a3303e6ebb68d64b7c334de03da4d" - integrity sha512-r8hF1rCi+GseLtY/IGeVWXFN0Uve2b820UQumRj4qxj7PsPqw1hFg7Cecbbb9zwF38K/m+D3IdwFeJzI1MtgRA== - dependencies: - "@chakra-ui/form-control" "2.0.13" - -"@chakra-ui/theme-tools@2.0.14": - version "2.0.14" - resolved "https://registry.yarnpkg.com/@chakra-ui/theme-tools/-/theme-tools-2.0.14.tgz#6c523284ab384ca57a3aef1fcfa7c32ed357fbde" - integrity sha512-lVcDmq5pyU0QbsIFKjt/iVUFDap7di2QHvPvGChA1YSjtg1PtuUi+BxEXWzp3Nfgw/N4rMvlBs+S0ynJypdwbg== - dependencies: - "@chakra-ui/anatomy" "2.1.0" - color2k "^2.0.0" - -"@chakra-ui/theme-utils@2.0.5": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@chakra-ui/theme-utils/-/theme-utils-2.0.5.tgz#ad1e53fc7f71326d15b9b01a157c7e2a029f3dda" - integrity sha512-QQowSM8fvQlTmT0w9wtqUlWOB4i+9eA7P4XRm4bfhBMZ7XpK4ctV95sPeGqaXVccsz5m0q1AuGWa+j6eMCbrrg== - dependencies: - "@chakra-ui/styled-system" "2.4.0" - "@chakra-ui/theme" "2.2.2" - lodash.mergewith "4.6.2" - -"@chakra-ui/theme@2.2.2": - version "2.2.2" - resolved "https://registry.yarnpkg.com/@chakra-ui/theme/-/theme-2.2.2.tgz#5ea69adde78ee6ea59f9dce674947ed8be2ebc62" - integrity sha512-7DlOQiXmnaqYyqXwqmfFSCWGkUonuqmNC5mmUCwxI435KgHNCaE2bIm6DI7N2NcIcuVcfc8Vn0UqrDoGU3zJBg== - dependencies: - "@chakra-ui/anatomy" "2.1.0" - "@chakra-ui/theme-tools" "2.0.14" - -"@chakra-ui/toast@4.0.4": - version "4.0.4" - resolved "https://registry.yarnpkg.com/@chakra-ui/toast/-/toast-4.0.4.tgz#254fb5c4c5bde0a373aab574927c654442fb0411" - integrity sha512-Gv52UQ4fJtziL9Qg0Yterb76C1GgzViryPDf2dxSzTlnCcKIbY4ktEhehyFBjDXYoGkFb47NZUEyhy+u8p3GUA== - dependencies: - "@chakra-ui/alert" "2.0.13" - "@chakra-ui/close-button" "2.0.13" - "@chakra-ui/portal" "2.0.11" - "@chakra-ui/react-use-timeout" "2.0.3" - "@chakra-ui/react-use-update-effect" "2.0.5" - "@chakra-ui/styled-system" "2.4.0" - "@chakra-ui/theme" "2.2.2" - -"@chakra-ui/tooltip@2.2.2": - version "2.2.2" - resolved "https://registry.yarnpkg.com/@chakra-ui/tooltip/-/tooltip-2.2.2.tgz#8ac0759fbc5adacec6e0ac7419c8055a67a95b5c" - integrity sha512-WDgQVEMHdsyUpKG9Nogy2FKLBgfdJG7hTSrSbH1WLvHsPkpPLknL4i5Z/pCvpa4A7SzTa6ps350mxtJ054MeMg== - dependencies: - "@chakra-ui/popper" "3.0.10" - "@chakra-ui/portal" "2.0.11" - "@chakra-ui/react-types" "2.0.5" - "@chakra-ui/react-use-disclosure" "2.0.6" - "@chakra-ui/react-use-event-listener" "2.0.5" - "@chakra-ui/react-use-merge-refs" "2.0.5" - -"@chakra-ui/transition@2.0.12": - version "2.0.12" - resolved "https://registry.yarnpkg.com/@chakra-ui/transition/-/transition-2.0.12.tgz#876c6ed24e442a720a8570490a93cb1f87008700" - integrity sha512-ff6eU+m08ccYfCkk0hKfY/XlmGxCrfbBgsKgV4mirZ4SKUL1GVye8CYuHwWQlBJo+8s0yIpsTNxAuX4n/cW9/w== - -"@chakra-ui/utils@2.0.12": - version "2.0.12" - resolved "https://registry.yarnpkg.com/@chakra-ui/utils/-/utils-2.0.12.tgz#5ab8a4529fca68d9f8c6722004f6a5129b0b75e9" - integrity sha512-1Z1MgsrfMQhNejSdrPJk8v5J4gCefHo+1wBmPPHTz5bGEbAAbZ13aXAfXy8w0eFy0Nvnawn0EHW7Oynp/MdH+Q== - dependencies: - "@types/lodash.mergewith" "4.6.6" - css-box-model "1.2.1" - framesync "5.3.0" - lodash.mergewith "4.6.2" - -"@chakra-ui/visually-hidden@2.0.13": - version "2.0.13" - resolved "https://registry.yarnpkg.com/@chakra-ui/visually-hidden/-/visually-hidden-2.0.13.tgz#6553467d93f206d17716bcbe6e895a84eef87472" - integrity sha512-sDEeeEjLfID333EC46NdCbhK2HyMXlpl5HzcJjuwWIpyVz4E1gKQ9hlwpq6grijvmzeSywQ5D3tTwUrvZck4KQ== - -"@csstools/css-parser-algorithms@^2.3.0": - version "2.3.0" - resolved "https://registry.yarnpkg.com/@csstools/css-parser-algorithms/-/css-parser-algorithms-2.3.0.tgz#0cc3a656dc2d638370ecf6f98358973bfbd00141" - integrity sha512-dTKSIHHWc0zPvcS5cqGP+/TPFUJB0ekJ9dGKvMAFoNuBFhDPBt9OMGNZiIA5vTiNdGHHBeScYPXIGBMnVOahsA== - -"@csstools/css-tokenizer@^2.1.1": - version "2.1.1" - resolved "https://registry.yarnpkg.com/@csstools/css-tokenizer/-/css-tokenizer-2.1.1.tgz#07ae11a0a06365d7ec686549db7b729bc036528e" - integrity sha512-GbrTj2Z8MCTUv+52GE0RbFGM527xuXZ0Xa5g0Z+YN573uveS4G0qi6WNOMyz3yrFM/jaILTTwJ0+umx81EzqfA== - -"@csstools/media-query-list-parser@^2.1.2": - version "2.1.2" - resolved "https://registry.yarnpkg.com/@csstools/media-query-list-parser/-/media-query-list-parser-2.1.2.tgz#6ef642b728d30c1009bfbba3211c7e4c11302728" - integrity sha512-M8cFGGwl866o6++vIY7j1AKuq9v57cf+dGepScwCcbut9ypJNr4Cj+LLTWligYUZ0uyhEoJDKt5lvyBfh2L3ZQ== - -"@csstools/selector-specificity@^3.0.0": - version "3.0.0" - resolved "https://registry.yarnpkg.com/@csstools/selector-specificity/-/selector-specificity-3.0.0.tgz#798622546b63847e82389e473fd67f2707d82247" - integrity sha512-hBI9tfBtuPIi885ZsZ32IMEU/5nlZH/KOVYJCOh7gyMxaVLGmLedYqFN6Ui1LXkI8JlC8IsuC0rF0btcRZKd5g== - -"@discoveryjs/json-ext@^0.5.0": - version "0.5.7" - resolved "https://registry.yarnpkg.com/@discoveryjs/json-ext/-/json-ext-0.5.7.tgz#1d572bfbbe14b7704e0ba0f39b74815b84870d70" - integrity sha512-dBVuXR082gk3jsFp7Rd/JI4kytwGHecnCoTtXFb7DB6CNHp4rg5k1bhg0nWdLGLnOV71lmDzGQaLMy8iPLY0pw== - -"@emotion/babel-plugin@^11.3.0": - version "11.3.0" - resolved "https://registry.yarnpkg.com/@emotion/babel-plugin/-/babel-plugin-11.3.0.tgz#3a16850ba04d8d9651f07f3fb674b3436a4fb9d7" - integrity sha512-UZKwBV2rADuhRp+ZOGgNWg2eYgbzKzQXfQPtJbu/PLy8onurxlNCLvxMQEvlr1/GudguPI5IU9qIY1+2z1M5bA== - dependencies: - "@babel/helper-module-imports" "^7.12.13" - "@babel/plugin-syntax-jsx" "^7.12.13" - "@babel/runtime" "^7.13.10" - "@emotion/hash" "^0.8.0" - "@emotion/memoize" "^0.7.5" - "@emotion/serialize" "^1.0.2" - babel-plugin-macros "^2.6.1" - convert-source-map "^1.5.0" - escape-string-regexp "^4.0.0" - find-root "^1.1.0" - source-map "^0.5.7" - stylis "^4.0.3" - -"@emotion/babel-plugin@^11.7.1": - version "11.9.2" - resolved "https://registry.yarnpkg.com/@emotion/babel-plugin/-/babel-plugin-11.9.2.tgz#723b6d394c89fb2ef782229d92ba95a740576e95" - integrity sha512-Pr/7HGH6H6yKgnVFNEj2MVlreu3ADqftqjqwUvDy/OJzKFgxKeTQ+eeUf20FOTuHVkDON2iNa25rAXVYtWJCjw== - dependencies: - "@babel/helper-module-imports" "^7.12.13" - "@babel/plugin-syntax-jsx" "^7.12.13" - "@babel/runtime" "^7.13.10" - "@emotion/hash" "^0.8.0" - "@emotion/memoize" "^0.7.5" - "@emotion/serialize" "^1.0.2" - babel-plugin-macros "^2.6.1" - convert-source-map "^1.5.0" - escape-string-regexp "^4.0.0" - find-root "^1.1.0" - source-map "^0.5.7" - stylis "4.0.13" - -"@emotion/cache@^11.4.0", "@emotion/cache@^11.9.3": - version "11.9.3" - resolved "https://registry.yarnpkg.com/@emotion/cache/-/cache-11.9.3.tgz#96638449f6929fd18062cfe04d79b29b44c0d6cb" - integrity sha512-0dgkI/JKlCXa+lEXviaMtGBL0ynpx4osh7rjOXE71q9bIF8G+XhJgvi+wDu0B0IdCVx37BffiwXlN9I3UuzFvg== - dependencies: - "@emotion/memoize" "^0.7.4" - "@emotion/sheet" "^1.1.1" - "@emotion/utils" "^1.0.0" - "@emotion/weak-memoize" "^0.2.5" - stylis "4.0.13" - -"@emotion/hash@^0.8.0": - version "0.8.0" - resolved "https://registry.yarnpkg.com/@emotion/hash/-/hash-0.8.0.tgz#bbbff68978fefdbe68ccb533bc8cbe1d1afb5413" - integrity sha512-kBJtf7PH6aWwZ6fka3zQ0p6SBYzx4fl1LoZXE2RrnYST9Xljm7WfKJrU4g/Xr3Beg72MLrp1AWNUmuYJTL7Cow== - -"@emotion/is-prop-valid@^0.8.2": - version "0.8.8" - resolved "https://registry.yarnpkg.com/@emotion/is-prop-valid/-/is-prop-valid-0.8.8.tgz#db28b1c4368a259b60a97311d6a952d4fd01ac1a" - integrity sha512-u5WtneEAr5IDG2Wv65yhunPSMLIpuKsbuOktRojfrEiEvRyC85LgPMZI63cr7NUqT8ZIGdSVg8ZKGxIug4lXcA== - dependencies: - "@emotion/memoize" "0.7.4" - -"@emotion/is-prop-valid@^1.1.0": - version "1.1.0" - resolved "https://registry.yarnpkg.com/@emotion/is-prop-valid/-/is-prop-valid-1.1.0.tgz#29ef6be1e946fb4739f9707def860f316f668cde" - integrity sha512-9RkilvXAufQHsSsjQ3PIzSns+pxuX4EW8EbGeSPjZMHuMx6z/MOzb9LpqNieQX4F3mre3NWS2+X3JNRHTQztUQ== - dependencies: - "@emotion/memoize" "^0.7.4" - -"@emotion/memoize@0.7.4": - version "0.7.4" - resolved "https://registry.yarnpkg.com/@emotion/memoize/-/memoize-0.7.4.tgz#19bf0f5af19149111c40d98bb0cf82119f5d9eeb" - integrity sha512-Ja/Vfqe3HpuzRsG1oBtWTHk2PGZ7GR+2Vz5iYGelAw8dx32K0y7PjVuxK6z1nMpZOqAFsRUPCkK1YjJ56qJlgw== - -"@emotion/memoize@^0.7.4", "@emotion/memoize@^0.7.5": - version "0.7.5" - resolved "https://registry.yarnpkg.com/@emotion/memoize/-/memoize-0.7.5.tgz#2c40f81449a4e554e9fc6396910ed4843ec2be50" - integrity sha512-igX9a37DR2ZPGYtV6suZ6whr8pTFtyHL3K/oLUotxpSVO2ASaprmAe2Dkq7tBo7CRY7MMDrAa9nuQP9/YG8FxQ== - -"@emotion/react@^11.8.1", "@emotion/react@^11.9.3": - version "11.9.3" - resolved "https://registry.yarnpkg.com/@emotion/react/-/react-11.9.3.tgz#f4f4f34444f6654a2e550f5dab4f2d360c101df9" - integrity sha512-g9Q1GcTOlzOEjqwuLF/Zd9LC+4FljjPjDfxSM7KmEakm+hsHXk+bYZ2q+/hTJzr0OUNkujo72pXLQvXj6H+GJQ== - dependencies: - "@babel/runtime" "^7.13.10" - "@emotion/babel-plugin" "^11.7.1" - "@emotion/cache" "^11.9.3" - "@emotion/serialize" "^1.0.4" - "@emotion/utils" "^1.1.0" - "@emotion/weak-memoize" "^0.2.5" - hoist-non-react-statics "^3.3.1" - -"@emotion/serialize@^1.0.2": - version "1.0.2" - resolved "https://registry.yarnpkg.com/@emotion/serialize/-/serialize-1.0.2.tgz#77cb21a0571c9f68eb66087754a65fa97bfcd965" - integrity sha512-95MgNJ9+/ajxU7QIAruiOAdYNjxZX7G2mhgrtDWswA21VviYIRP1R5QilZ/bDY42xiKsaktP4egJb3QdYQZi1A== - dependencies: - "@emotion/hash" "^0.8.0" - "@emotion/memoize" "^0.7.4" - "@emotion/unitless" "^0.7.5" - "@emotion/utils" "^1.0.0" - csstype "^3.0.2" - -"@emotion/serialize@^1.0.4": - version "1.0.4" - resolved "https://registry.yarnpkg.com/@emotion/serialize/-/serialize-1.0.4.tgz#ff31fd11bb07999611199c2229e152faadc21a3c" - integrity sha512-1JHamSpH8PIfFwAMryO2bNka+y8+KA5yga5Ocf2d7ZEiJjb7xlLW7aknBGZqJLajuLOvJ+72vN+IBSwPlXD1Pg== - dependencies: - "@emotion/hash" "^0.8.0" - "@emotion/memoize" "^0.7.4" - "@emotion/unitless" "^0.7.5" - "@emotion/utils" "^1.0.0" - csstype "^3.0.2" - -"@emotion/sheet@^1.1.1": - version "1.1.1" - resolved "https://registry.yarnpkg.com/@emotion/sheet/-/sheet-1.1.1.tgz#015756e2a9a3c7c5f11d8ec22966a8dbfbfac787" - integrity sha512-J3YPccVRMiTZxYAY0IOq3kd+hUP8idY8Kz6B/Cyo+JuXq52Ek+zbPbSQUrVQp95aJ+lsAW7DPL1P2Z+U1jGkKA== - -"@emotion/styled@^11": - version "11.3.0" - resolved "https://registry.yarnpkg.com/@emotion/styled/-/styled-11.3.0.tgz#d63ee00537dfb6ff612e31b0e915c5cf9925a207" - integrity sha512-fUoLcN3BfMiLlRhJ8CuPUMEyKkLEoM+n+UyAbnqGEsCd5IzKQ7VQFLtzpJOaCD2/VR2+1hXQTnSZXVJeiTNltA== - dependencies: - "@babel/runtime" "^7.13.10" - "@emotion/babel-plugin" "^11.3.0" - "@emotion/is-prop-valid" "^1.1.0" - "@emotion/serialize" "^1.0.2" - "@emotion/utils" "^1.0.0" - -"@emotion/unitless@^0.7.5": - version "0.7.5" - resolved "https://registry.yarnpkg.com/@emotion/unitless/-/unitless-0.7.5.tgz#77211291c1900a700b8a78cfafda3160d76949ed" - integrity sha512-OWORNpfjMsSSUBVrRBVGECkhWcULOAJz9ZW8uK9qgxD+87M7jHRcvh/A96XXNhXTLmKcoYSQtBEX7lHMO7YRwg== - -"@emotion/utils@^1.0.0", "@emotion/utils@^1.1.0": - version "1.1.0" - resolved "https://registry.yarnpkg.com/@emotion/utils/-/utils-1.1.0.tgz#86b0b297f3f1a0f2bdb08eeac9a2f49afd40d0cf" - integrity sha512-iRLa/Y4Rs5H/f2nimczYmS5kFJEbpiVvgN3XVfZ022IYhuNA1IRSHEizcof88LtCTXtl9S2Cxt32KgaXEu72JQ== - -"@emotion/weak-memoize@^0.2.5": - version "0.2.5" - resolved "https://registry.yarnpkg.com/@emotion/weak-memoize/-/weak-memoize-0.2.5.tgz#8eed982e2ee6f7f4e44c253e12962980791efd46" - integrity sha512-6U71C2Wp7r5XtFtQzYrW5iKFT67OixrSxjI4MptCHzdSVlgabczzqLe0ZSgnub/5Kp4hSbpDB1tMytZY9pwxxA== - -"@eslint/eslintrc@^1.3.0": - version "1.3.0" - resolved "https://registry.yarnpkg.com/@eslint/eslintrc/-/eslintrc-1.3.0.tgz#29f92c30bb3e771e4a2048c95fa6855392dfac4f" - integrity sha512-UWW0TMTmk2d7hLcWD1/e2g5HDM/HQ3csaLSqXCfqwh4uNDuNqlaKWXmEsL4Cs41Z0KnILNvwbHAah3C2yt06kw== - dependencies: - ajv "^6.12.4" - debug "^4.3.2" - espree "^9.3.2" - globals "^13.15.0" - ignore "^5.2.0" - import-fresh "^3.2.1" - js-yaml "^4.1.0" - minimatch "^3.1.2" - strip-json-comments "^3.1.1" - -"@exodus/schemasafe@^1.0.0-rc.2": - version "1.0.0-rc.3" - resolved "https://registry.yarnpkg.com/@exodus/schemasafe/-/schemasafe-1.0.0-rc.3.tgz#dda2fbf3dafa5ad8c63dadff7e01d3fdf4736025" - integrity sha512-GoXw0U2Qaa33m3eUcxuHnHpNvHjNlLo0gtV091XBpaRINaB4X6FGCG5XKxSFNFiPpugUDqNruHzaqpTdDm4AOg== - -"@fastify/busboy@^2.0.0": - version "2.0.0" - resolved "https://registry.yarnpkg.com/@fastify/busboy/-/busboy-2.0.0.tgz#f22824caff3ae506b18207bad4126dbc6ccdb6b8" - integrity sha512-JUFJad5lv7jxj926GPgymrWQxxjPYuJNiNjNMzqT+HiuP6Vl3dk5xzG+8sTX96np0ZAluvaMzPsjhHZ5rNuNQQ== - -"@humanwhocodes/config-array@^0.9.2": - version "0.9.5" - resolved "https://registry.yarnpkg.com/@humanwhocodes/config-array/-/config-array-0.9.5.tgz#2cbaf9a89460da24b5ca6531b8bbfc23e1df50c7" - integrity sha512-ObyMyWxZiCu/yTisA7uzx81s40xR2fD5Cg/2Kq7G02ajkNubJf6BopgDTmDyc3U7sXpNKM8cYOw7s7Tyr+DnCw== - dependencies: - "@humanwhocodes/object-schema" "^1.2.1" - debug "^4.1.1" - minimatch "^3.0.4" - -"@humanwhocodes/object-schema@^1.2.1": - version "1.2.1" - resolved "https://registry.yarnpkg.com/@humanwhocodes/object-schema/-/object-schema-1.2.1.tgz#b520529ec21d8e5945a1851dfd1c32e94e39ff45" - integrity sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA== - -"@istanbuljs/load-nyc-config@^1.0.0": - version "1.1.0" - resolved "https://registry.yarnpkg.com/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz#fd3db1d59ecf7cf121e80650bb86712f9b55eced" - integrity sha512-VjeHSlIzpv/NyD3N0YuHfXOPDIixcA1q2ZV98wsMqcYlPmv2n3Yb2lYP9XMElnaFVXg5A7YLTeLu6V84uQDjmQ== - dependencies: - camelcase "^5.3.1" - find-up "^4.1.0" - get-package-type "^0.1.0" - js-yaml "^3.13.1" - resolve-from "^5.0.0" - -"@istanbuljs/schema@^0.1.2": - version "0.1.3" - resolved "https://registry.yarnpkg.com/@istanbuljs/schema/-/schema-0.1.3.tgz#e45e384e4b8ec16bce2fd903af78450f6bf7ec98" - integrity sha512-ZXRY4jNvVgSVQ8DL3LTcakaAtXwTVUxE81hslsyD2AtoXW/wVob10HkOJ1X/pAlcI7D+2YoZKg5do8G/w6RYgA== - -"@jest/console@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/console/-/console-27.5.1.tgz#260fe7239602fe5130a94f1aa386eff54b014bba" - integrity sha512-kZ/tNpS3NXn0mlXXXPNuDZnb4c0oZ20r4K5eemM2k30ZC3G0T02nXUvyhf5YdbXWHPEJLc9qGLxEZ216MdL+Zg== - dependencies: - "@jest/types" "^27.5.1" - "@types/node" "*" - chalk "^4.0.0" - jest-message-util "^27.5.1" - jest-util "^27.5.1" - slash "^3.0.0" - -"@jest/core@^27.3.1", "@jest/core@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/core/-/core-27.5.1.tgz#267ac5f704e09dc52de2922cbf3af9edcd64b626" - integrity sha512-AK6/UTrvQD0Cd24NSqmIA6rKsu0tKIxfiCducZvqxYdmMisOYAsdItspT+fQDQYARPf8XgjAFZi0ogW2agH5nQ== - dependencies: - "@jest/console" "^27.5.1" - "@jest/reporters" "^27.5.1" - "@jest/test-result" "^27.5.1" - "@jest/transform" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/node" "*" - ansi-escapes "^4.2.1" - chalk "^4.0.0" - emittery "^0.8.1" - exit "^0.1.2" - graceful-fs "^4.2.9" - jest-changed-files "^27.5.1" - jest-config "^27.5.1" - jest-haste-map "^27.5.1" - jest-message-util "^27.5.1" - jest-regex-util "^27.5.1" - jest-resolve "^27.5.1" - jest-resolve-dependencies "^27.5.1" - jest-runner "^27.5.1" - jest-runtime "^27.5.1" - jest-snapshot "^27.5.1" - jest-util "^27.5.1" - jest-validate "^27.5.1" - jest-watcher "^27.5.1" - micromatch "^4.0.4" - rimraf "^3.0.0" - slash "^3.0.0" - strip-ansi "^6.0.0" - -"@jest/environment@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/environment/-/environment-27.5.1.tgz#d7425820511fe7158abbecc010140c3fd3be9c74" - integrity sha512-/WQjhPJe3/ghaol/4Bq480JKXV/Rfw8nQdN7f41fM8VDHLcxKXou6QyXAh3EFr9/bVG3x74z1NWDkP87EiY8gA== - dependencies: - "@jest/fake-timers" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/node" "*" - jest-mock "^27.5.1" - -"@jest/fake-timers@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/fake-timers/-/fake-timers-27.5.1.tgz#76979745ce0579c8a94a4678af7a748eda8ada74" - integrity sha512-/aPowoolwa07k7/oM3aASneNeBGCmGQsc3ugN4u6s4C/+s5M64MFo/+djTdiwcbQlRfFElGuDXWzaWj6QgKObQ== - dependencies: - "@jest/types" "^27.5.1" - "@sinonjs/fake-timers" "^8.0.1" - "@types/node" "*" - jest-message-util "^27.5.1" - jest-mock "^27.5.1" - jest-util "^27.5.1" - -"@jest/globals@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/globals/-/globals-27.5.1.tgz#7ac06ce57ab966566c7963431cef458434601b2b" - integrity sha512-ZEJNB41OBQQgGzgyInAv0UUfDDj3upmHydjieSxFvTRuZElrx7tXg/uVQ5hYVEwiXs3+aMsAeEc9X7xiSKCm4Q== - dependencies: - "@jest/environment" "^27.5.1" - "@jest/types" "^27.5.1" - expect "^27.5.1" - -"@jest/reporters@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/reporters/-/reporters-27.5.1.tgz#ceda7be96170b03c923c37987b64015812ffec04" - integrity sha512-cPXh9hWIlVJMQkVk84aIvXuBB4uQQmFqZiacloFuGiP3ah1sbCxCosidXFDfqG8+6fO1oR2dTJTlsOy4VFmUfw== - dependencies: - "@bcoe/v8-coverage" "^0.2.3" - "@jest/console" "^27.5.1" - "@jest/test-result" "^27.5.1" - "@jest/transform" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/node" "*" - chalk "^4.0.0" - collect-v8-coverage "^1.0.0" - exit "^0.1.2" - glob "^7.1.2" - graceful-fs "^4.2.9" - istanbul-lib-coverage "^3.0.0" - istanbul-lib-instrument "^5.1.0" - istanbul-lib-report "^3.0.0" - istanbul-lib-source-maps "^4.0.0" - istanbul-reports "^3.1.3" - jest-haste-map "^27.5.1" - jest-resolve "^27.5.1" - jest-util "^27.5.1" - jest-worker "^27.5.1" - slash "^3.0.0" - source-map "^0.6.0" - string-length "^4.0.1" - terminal-link "^2.0.0" - v8-to-istanbul "^8.1.0" - -"@jest/source-map@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/source-map/-/source-map-27.5.1.tgz#6608391e465add4205eae073b55e7f279e04e8cf" - integrity sha512-y9NIHUYF3PJRlHk98NdC/N1gl88BL08aQQgu4k4ZopQkCw9t9cV8mtl3TV8b/YCB8XaVTFrmUTAJvjsntDireg== - dependencies: - callsites "^3.0.0" - graceful-fs "^4.2.9" - source-map "^0.6.0" - -"@jest/test-result@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/test-result/-/test-result-27.5.1.tgz#56a6585fa80f7cdab72b8c5fc2e871d03832f5bb" - integrity sha512-EW35l2RYFUcUQxFJz5Cv5MTOxlJIQs4I7gxzi2zVU7PJhOwfYq1MdC5nhSmYjX1gmMmLPvB3sIaC+BkcHRBfag== - dependencies: - "@jest/console" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/istanbul-lib-coverage" "^2.0.0" - collect-v8-coverage "^1.0.0" - -"@jest/test-sequencer@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/test-sequencer/-/test-sequencer-27.5.1.tgz#4057e0e9cea4439e544c6353c6affe58d095745b" - integrity sha512-LCheJF7WB2+9JuCS7VB/EmGIdQuhtqjRNI9A43idHv3E4KltCTsPsLxvdaubFHSYwY/fNjMWjl6vNRhDiN7vpQ== - dependencies: - "@jest/test-result" "^27.5.1" - graceful-fs "^4.2.9" - jest-haste-map "^27.5.1" - jest-runtime "^27.5.1" - -"@jest/transform@^27.3.1": - version "27.3.1" - resolved "https://registry.yarnpkg.com/@jest/transform/-/transform-27.3.1.tgz#ff80eafbeabe811e9025e4b6f452126718455220" - integrity sha512-3fSvQ02kuvjOI1C1ssqMVBKJpZf6nwoCiSu00zAKh5nrp3SptNtZy/8s5deayHnqxhjD9CWDJ+yqQwuQ0ZafXQ== - dependencies: - "@babel/core" "^7.1.0" - "@jest/types" "^27.2.5" - babel-plugin-istanbul "^6.0.0" - chalk "^4.0.0" - convert-source-map "^1.4.0" - fast-json-stable-stringify "^2.0.0" - graceful-fs "^4.2.4" - jest-haste-map "^27.3.1" - jest-regex-util "^27.0.6" - jest-util "^27.3.1" - micromatch "^4.0.4" - pirates "^4.0.1" - slash "^3.0.0" - source-map "^0.6.1" - write-file-atomic "^3.0.0" - -"@jest/transform@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/transform/-/transform-27.5.1.tgz#6c3501dcc00c4c08915f292a600ece5ecfe1f409" - integrity sha512-ipON6WtYgl/1329g5AIJVbUuEh0wZVbdpGwC99Jw4LwuoBNS95MVphU6zOeD9pDkon+LLbFL7lOQRapbB8SCHw== - dependencies: - "@babel/core" "^7.1.0" - "@jest/types" "^27.5.1" - babel-plugin-istanbul "^6.1.1" - chalk "^4.0.0" - convert-source-map "^1.4.0" - fast-json-stable-stringify "^2.0.0" - graceful-fs "^4.2.9" - jest-haste-map "^27.5.1" - jest-regex-util "^27.5.1" - jest-util "^27.5.1" - micromatch "^4.0.4" - pirates "^4.0.4" - slash "^3.0.0" - source-map "^0.6.1" - write-file-atomic "^3.0.0" - -"@jest/types@^27.2.5": - version "27.2.5" - resolved "https://registry.yarnpkg.com/@jest/types/-/types-27.2.5.tgz#420765c052605e75686982d24b061b4cbba22132" - integrity sha512-nmuM4VuDtCZcY+eTpw+0nvstwReMsjPoj7ZR80/BbixulhLaiX+fbv8oeLW8WZlJMcsGQsTmMKT/iTZu1Uy/lQ== - dependencies: - "@types/istanbul-lib-coverage" "^2.0.0" - "@types/istanbul-reports" "^3.0.0" - "@types/node" "*" - "@types/yargs" "^16.0.0" - chalk "^4.0.0" - -"@jest/types@^27.5.1": - version "27.5.1" - resolved "https://registry.yarnpkg.com/@jest/types/-/types-27.5.1.tgz#3c79ec4a8ba61c170bf937bcf9e98a9df175ec80" - integrity sha512-Cx46iJ9QpwQTjIdq5VJu2QTMMs3QlEjI0x1QbBP5W1+nMzyc2XmimiRR/CbX9TO0cPTeUlxWMOu8mslYsJ8DEw== - dependencies: - "@types/istanbul-lib-coverage" "^2.0.0" - "@types/istanbul-reports" "^3.0.0" - "@types/node" "*" - "@types/yargs" "^16.0.0" - chalk "^4.0.0" - -"@jridgewell/gen-mapping@^0.3.0": - version "0.3.2" - resolved "https://registry.yarnpkg.com/@jridgewell/gen-mapping/-/gen-mapping-0.3.2.tgz#c1aedc61e853f2bb9f5dfe6d4442d3b565b253b9" - integrity sha512-mh65xKQAzI6iBcFzwv28KVWSmCkdRBWoOh+bYQGW3+6OZvbbN3TqMGo5hqYxQniRcH9F2VZIoJCm4pa3BPDK/A== - dependencies: - "@jridgewell/set-array" "^1.0.1" - "@jridgewell/sourcemap-codec" "^1.4.10" - "@jridgewell/trace-mapping" "^0.3.9" - -"@jridgewell/gen-mapping@^0.3.2": - version "0.3.3" - resolved "https://registry.yarnpkg.com/@jridgewell/gen-mapping/-/gen-mapping-0.3.3.tgz#7e02e6eb5df901aaedb08514203b096614024098" - integrity sha512-HLhSWOLRi875zjjMG/r+Nv0oCW8umGb0BgEhyX3dDX3egwZtB8PqLnjz3yedt8R5StBrzcg4aBpnh8UA9D1BoQ== - dependencies: - "@jridgewell/set-array" "^1.0.1" - "@jridgewell/sourcemap-codec" "^1.4.10" - "@jridgewell/trace-mapping" "^0.3.9" - -"@jridgewell/gen-mapping@^0.3.5": - version "0.3.5" - resolved "https://registry.yarnpkg.com/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz#dcce6aff74bdf6dad1a95802b69b04a2fcb1fb36" - integrity sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg== - dependencies: - "@jridgewell/set-array" "^1.2.1" - "@jridgewell/sourcemap-codec" "^1.4.10" - "@jridgewell/trace-mapping" "^0.3.24" - -"@jridgewell/resolve-uri@^3.0.3": - version "3.1.0" - resolved "https://registry.yarnpkg.com/@jridgewell/resolve-uri/-/resolve-uri-3.1.0.tgz#2203b118c157721addfe69d47b70465463066d78" - integrity sha512-F2msla3tad+Mfht5cJq7LSXcdudKTWCVYUgw6pLFOOHSTtZlj6SWNYAp+AhuqLmWdBO2X5hPrLcu8cVP8fy28w== - -"@jridgewell/resolve-uri@^3.1.0": - version "3.1.1" - resolved "https://registry.yarnpkg.com/@jridgewell/resolve-uri/-/resolve-uri-3.1.1.tgz#c08679063f279615a3326583ba3a90d1d82cc721" - integrity sha512-dSYZh7HhCDtCKm4QakX0xFpsRDqjjtZf/kjI/v3T3Nwt5r8/qz/M19F9ySyOqU94SXBmeG9ttTul+YnR4LOxFA== - -"@jridgewell/set-array@^1.0.1": - version "1.1.2" - resolved "https://registry.yarnpkg.com/@jridgewell/set-array/-/set-array-1.1.2.tgz#7c6cf998d6d20b914c0a55a91ae928ff25965e72" - integrity sha512-xnkseuNADM0gt2bs+BvhO0p78Mk762YnZdsuzFV018NoG1Sj1SCQvpSqa7XUaTam5vAGasABV9qXASMKnFMwMw== - -"@jridgewell/set-array@^1.2.1": - version "1.2.1" - resolved "https://registry.yarnpkg.com/@jridgewell/set-array/-/set-array-1.2.1.tgz#558fb6472ed16a4c850b889530e6b36438c49280" - integrity sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A== - -"@jridgewell/source-map@^0.3.2": - version "0.3.2" - resolved "https://registry.yarnpkg.com/@jridgewell/source-map/-/source-map-0.3.2.tgz#f45351aaed4527a298512ec72f81040c998580fb" - integrity sha512-m7O9o2uR8k2ObDysZYzdfhb08VuEml5oWGiosa1VdaPZ/A6QyPkAJuwN0Q1lhULOf6B7MtQmHENS743hWtCrgw== - dependencies: - "@jridgewell/gen-mapping" "^0.3.0" - "@jridgewell/trace-mapping" "^0.3.9" - -"@jridgewell/source-map@^0.3.3": - version "0.3.6" - resolved "https://registry.yarnpkg.com/@jridgewell/source-map/-/source-map-0.3.6.tgz#9d71ca886e32502eb9362c9a74a46787c36df81a" - integrity sha512-1ZJTZebgqllO79ue2bm3rIGud/bOe0pP5BjSRCRxxYkEZS8STV7zN84UBbiYu7jy+eCKSnVIUgoWWE/tt+shMQ== - dependencies: - "@jridgewell/gen-mapping" "^0.3.5" - "@jridgewell/trace-mapping" "^0.3.25" - -"@jridgewell/sourcemap-codec@^1.4.10": - version "1.4.14" - resolved "https://registry.yarnpkg.com/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.14.tgz#add4c98d341472a289190b424efbdb096991bb24" - integrity sha512-XPSJHWmi394fuUuzDnGz1wiKqWfo1yXecHQMRf2l6hztTO+nPru658AyDngaBe7isIxEkRsPR3FZh+s7iVa4Uw== - -"@jridgewell/sourcemap-codec@^1.4.14": - version "1.4.15" - resolved "https://registry.yarnpkg.com/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.15.tgz#d7c6e6755c78567a951e04ab52ef0fd26de59f32" - integrity sha512-eF2rxCRulEKXHTRiDrDy6erMYWqNw4LPdQ8UQA4huuxaQsVeRPFl2oM8oDGxMFhJUWZf9McpLtJasDDZb/Bpeg== - -"@jridgewell/trace-mapping@^0.3.0": - version "0.3.4" - resolved "https://registry.yarnpkg.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.4.tgz#f6a0832dffd5b8a6aaa633b7d9f8e8e94c83a0c3" - integrity sha512-vFv9ttIedivx0ux3QSjhgtCVjPZd5l46ZOMDSCwnH1yUO2e964gO8LZGyv2QkqcgR6TnBU1v+1IFqmeoG+0UJQ== - dependencies: - "@jridgewell/resolve-uri" "^3.0.3" - "@jridgewell/sourcemap-codec" "^1.4.10" - -"@jridgewell/trace-mapping@^0.3.17": - version "0.3.19" - resolved "https://registry.yarnpkg.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.19.tgz#f8a3249862f91be48d3127c3cfe992f79b4b8811" - integrity sha512-kf37QtfW+Hwx/buWGMPcR60iF9ziHa6r/CZJIHbmcm4+0qrXiVdxegAH0F6yddEVQ7zdkjcGCgCzUu+BcbhQxw== - dependencies: - "@jridgewell/resolve-uri" "^3.1.0" - "@jridgewell/sourcemap-codec" "^1.4.14" - -"@jridgewell/trace-mapping@^0.3.20", "@jridgewell/trace-mapping@^0.3.24", "@jridgewell/trace-mapping@^0.3.25": - version "0.3.25" - resolved "https://registry.yarnpkg.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz#15f190e98895f3fc23276ee14bc76b675c2e50f0" - integrity sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ== - dependencies: - "@jridgewell/resolve-uri" "^3.1.0" - "@jridgewell/sourcemap-codec" "^1.4.14" - -"@jridgewell/trace-mapping@^0.3.9": - version "0.3.14" - resolved "https://registry.yarnpkg.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.14.tgz#b231a081d8f66796e475ad588a1ef473112701ed" - integrity sha512-bJWEfQ9lPTvm3SneWwRFVLzrh6nhjwqw7TUFFBEMzwvg7t7PCDenf2lDwqo4NQXzdpgBXyFgDWnQA+2vkruksQ== - dependencies: - "@jridgewell/resolve-uri" "^3.0.3" - "@jridgewell/sourcemap-codec" "^1.4.10" - -"@nicolo-ribaudo/eslint-scope-5-internals@5.1.1-v1": - version "5.1.1-v1" - resolved "https://registry.yarnpkg.com/@nicolo-ribaudo/eslint-scope-5-internals/-/eslint-scope-5-internals-5.1.1-v1.tgz#dbf733a965ca47b1973177dc0bb6c889edcfb129" - integrity sha512-54/JRvkLIzzDWshCWfuhadfrfZVPiElY8Fcgmg1HroEly/EDSszzhBAsarCux+D/kOslTRquNzuyGSmUSTTHGg== - dependencies: - eslint-scope "5.1.1" - -"@nodelib/fs.scandir@2.1.5": - version "2.1.5" - resolved "https://registry.yarnpkg.com/@nodelib/fs.scandir/-/fs.scandir-2.1.5.tgz#7619c2eb21b25483f6d167548b4cfd5a7488c3d5" - integrity sha512-vq24Bq3ym5HEQm2NKCr3yXDwjc7vTsEThRDnkp2DK9p1uqLR+DHurm/NOTo0KG7HYHU7eppKZj3MyqYuMBf62g== - dependencies: - "@nodelib/fs.stat" "2.0.5" - run-parallel "^1.1.9" - -"@nodelib/fs.stat@2.0.5", "@nodelib/fs.stat@^2.0.2": - version "2.0.5" - resolved "https://registry.yarnpkg.com/@nodelib/fs.stat/-/fs.stat-2.0.5.tgz#5bd262af94e9d25bd1e71b05deed44876a222e8b" - integrity sha512-RkhPPp2zrqDAQA/2jNhnztcPAlv64XdhIp7a7454A5ovI7Bukxgt7MX7udwAu3zg1DcpPU0rz3VV1SeaqvY4+A== - -"@nodelib/fs.walk@^1.2.3": - version "1.2.7" - resolved "https://registry.yarnpkg.com/@nodelib/fs.walk/-/fs.walk-1.2.7.tgz#94c23db18ee4653e129abd26fb06f870ac9e1ee2" - integrity sha512-BTIhocbPBSrRmHxOAJFtR18oLhxTtAFDAvL8hY1S3iU8k+E60W/YFs4jrixGzQjMpF4qPXxIQHcjVD9dz1C2QA== - dependencies: - "@nodelib/fs.scandir" "2.1.5" - fastq "^1.6.0" - -"@npmcli/move-file@^1.0.1": - version "1.1.2" - resolved "https://registry.yarnpkg.com/@npmcli/move-file/-/move-file-1.1.2.tgz#1a82c3e372f7cae9253eb66d72543d6b8685c674" - integrity sha512-1SUf/Cg2GzGDyaf15aR9St9TWlb+XvbZXWpDx8YKs7MLzMH/BCeopv+y9vzrzgkfykCGuWOlSu3mZhj2+FQcrg== - dependencies: - mkdirp "^1.0.4" - rimraf "^3.0.2" - -"@popperjs/core@^2.9.3": - version "2.11.2" - resolved "https://registry.yarnpkg.com/@popperjs/core/-/core-2.11.2.tgz#830beaec4b4091a9e9398ac50f865ddea52186b9" - integrity sha512-92FRmppjjqz29VMJ2dn+xdyXZBrMlE42AV6Kq6BwjWV7CNUW1hs2FtxSNLQE+gJhaZ6AAmYuO9y8dshhcBl7vA== - -"@reactflow/background@11.2.4": - version "11.2.4" - resolved "https://registry.yarnpkg.com/@reactflow/background/-/background-11.2.4.tgz#06cd4c9f222dbeb2d3ffb2a530b6d88bf130dd9c" - integrity sha512-SYQbCRCU0GuxT/40Tm7ZK+l5wByGnNJSLtZhbL9C/Hl7JhsJXV3UGXr0vrlhVZUBEtkWA7XhZM/5S9XEA5XSFA== - dependencies: - "@reactflow/core" "11.7.4" - classcat "^5.0.3" - zustand "^4.3.1" - -"@reactflow/controls@11.1.15": - version "11.1.15" - resolved "https://registry.yarnpkg.com/@reactflow/controls/-/controls-11.1.15.tgz#6dc823eb67f38a50907fffcc21b6a20e4fc00e7c" - integrity sha512-//33XfBYu8vQ6brfmlZwKrDoh+8hh93xO2d88XiqfIbrPEEb32SYjsb9mS9VuHKNlSIW+eB27fBA1Gt00mEj5w== - dependencies: - "@reactflow/core" "11.7.4" - classcat "^5.0.3" - zustand "^4.3.1" - -"@reactflow/core@11.7.4", "@reactflow/core@^11.6.0": - version "11.7.4" - resolved "https://registry.yarnpkg.com/@reactflow/core/-/core-11.7.4.tgz#1a7e4d6cabbd2ea888547133d507f1ab24896520" - integrity sha512-nt0T8ERp8TE7YCDQViaoEY9lb0StDPrWHVx3zBjhStFYET3wc88t8QRasZdf99xRTmyNtI3U3M40M5EBLNUpMw== - dependencies: - "@types/d3" "^7.4.0" - "@types/d3-drag" "^3.0.1" - "@types/d3-selection" "^3.0.3" - "@types/d3-zoom" "^3.0.1" - classcat "^5.0.3" - d3-drag "^3.0.0" - d3-selection "^3.0.0" - d3-zoom "^3.0.0" - zustand "^4.3.1" - -"@reactflow/minimap@11.5.4": - version "11.5.4" - resolved "https://registry.yarnpkg.com/@reactflow/minimap/-/minimap-11.5.4.tgz#b072094f7d827660f0205796d5f22fbbf6e31cc3" - integrity sha512-1tDBj2zX2gxu2oHU6qvH5RGNrOWRfRxu8369KhDotuuBN5yJrGXJzWIKikwhzjsNsQJYOB+B0cS44yWAfwSwzw== - dependencies: - "@reactflow/core" "11.7.4" - "@types/d3-selection" "^3.0.3" - "@types/d3-zoom" "^3.0.1" - classcat "^5.0.3" - d3-selection "^3.0.0" - d3-zoom "^3.0.0" - zustand "^4.3.1" - -"@reactflow/node-resizer@2.1.1": - version "2.1.1" - resolved "https://registry.yarnpkg.com/@reactflow/node-resizer/-/node-resizer-2.1.1.tgz#8f9b4e362e572dcddb54d43a67b5c5919b25937f" - integrity sha512-5Q+IBmZfpp/bYsw3+KRVJB1nUbj6W3XAp5ycx4uNWH+K98vbssymyQsW0vvKkIhxEPg6tkiMzO4UWRWvwBwt1g== - dependencies: - "@reactflow/core" "^11.6.0" - classcat "^5.0.4" - d3-drag "^3.0.0" - d3-selection "^3.0.0" - zustand "^4.3.1" - -"@reactflow/node-toolbar@1.2.3": - version "1.2.3" - resolved "https://registry.yarnpkg.com/@reactflow/node-toolbar/-/node-toolbar-1.2.3.tgz#8ff8408dffee7920752479cd19e7ab7c9c47b4d2" - integrity sha512-uFQy9xpog92s0G1wsPLniwV9nyH4i/MmL7QoMsWdnKaOi7XMhd8SJcCzUdHC3imR21HltsuQITff/XQ51ApMbg== - dependencies: - "@reactflow/core" "11.7.4" - classcat "^5.0.3" - zustand "^4.3.1" - -"@redocly/ajv@^8.6.4": - version "8.6.4" - resolved "https://registry.yarnpkg.com/@redocly/ajv/-/ajv-8.6.4.tgz#94053e7a9d4146d1a4feacd3813892873f229a85" - integrity sha512-y9qNj0//tZtWB2jfXNK3BX18BSBp9zNR7KE7lMysVHwbZtY392OJCjm6Rb/h4UHH2r1AqjNEHFD6bRn+DqU9Mw== - dependencies: - fast-deep-equal "^3.1.1" - json-schema-traverse "^1.0.0" - require-from-string "^2.0.2" - uri-js "^4.2.2" - -"@redocly/openapi-core@^1.0.0-beta.97": - version "1.0.0-beta.102" - resolved "https://registry.yarnpkg.com/@redocly/openapi-core/-/openapi-core-1.0.0-beta.102.tgz#e1cd049979f05812c594063fec71e618201319c4" - integrity sha512-3Fr3fg+9VEF4+4uoyvOOk+9ipmX2GYhlb18uZbpC4v3cUgGpkTRGZM2Qetfah7Tgx2LgqLuw8A1icDD6Zed2Gw== - dependencies: - "@redocly/ajv" "^8.6.4" - "@types/node" "^14.11.8" - colorette "^1.2.0" - js-levenshtein "^1.1.6" - js-yaml "^4.1.0" - lodash.isequal "^4.5.0" - minimatch "^5.0.1" - node-fetch "^2.6.1" - pluralize "^8.0.0" - yaml-ast-parser "0.0.43" - -"@sinonjs/commons@^1.7.0": - version "1.8.3" - resolved "https://registry.yarnpkg.com/@sinonjs/commons/-/commons-1.8.3.tgz#3802ddd21a50a949b6721ddd72da36e67e7f1b2d" - integrity sha512-xkNcLAn/wZaX14RPlwizcKicDk9G3F8m2nU3L7Ukm5zBgTwiT0wsoFAHx9Jq56fJA1z/7uKGtCRu16sOUCLIHQ== - dependencies: - type-detect "4.0.8" - -"@sinonjs/fake-timers@^8.0.1": - version "8.1.0" - resolved "https://registry.yarnpkg.com/@sinonjs/fake-timers/-/fake-timers-8.1.0.tgz#3fdc2b6cb58935b21bfb8d1625eb1300484316e7" - integrity sha512-OAPJUAtgeINhh/TAlUID4QTs53Njm7xzddaVlEs/SXwgtiD1tW22zAB/W1wdqfrpmikgaWQ9Fw6Ws+hsiRm5Vg== - dependencies: - "@sinonjs/commons" "^1.7.0" - -"@tanstack/react-table@^8.13.2": - version "8.13.2" - resolved "https://registry.yarnpkg.com/@tanstack/react-table/-/react-table-8.13.2.tgz#a3aa737ae464abc651f68daa7e82dca17813606c" - integrity sha512-b6mR3mYkjRtJ443QZh9sc7CvGTce81J35F/XMr0OoWbx0KIM7TTTdyNP2XKObvkLpYnLpCrYDwI3CZnLezWvpg== - dependencies: - "@tanstack/table-core" "8.13.2" - -"@tanstack/table-core@8.13.2": - version "8.13.2" - resolved "https://registry.yarnpkg.com/@tanstack/table-core/-/table-core-8.13.2.tgz#2512574dd3d20dc94b7db1f9f48090f0c18b5c85" - integrity sha512-/2saD1lWBUV6/uNAwrsg2tw58uvMJ07bO2F1IWMxjFRkJiXKQRuc3Oq2aufeobD3873+4oIM/DRySIw7+QsPPw== - -"@testing-library/dom@^8.5.0": - version "8.13.0" - resolved "https://registry.yarnpkg.com/@testing-library/dom/-/dom-8.13.0.tgz#bc00bdd64c7d8b40841e27a70211399ad3af46f5" - integrity sha512-9VHgfIatKNXQNaZTtLnalIy0jNZzY35a4S3oi08YAt9Hv1VsfZ/DfA45lM8D/UhtHBGJ4/lGwp0PZkVndRkoOQ== - dependencies: - "@babel/code-frame" "^7.10.4" - "@babel/runtime" "^7.12.5" - "@types/aria-query" "^4.2.0" - aria-query "^5.0.0" - chalk "^4.1.0" - dom-accessibility-api "^0.5.9" - lz-string "^1.4.4" - pretty-format "^27.0.2" - -"@testing-library/jest-dom@^5.16.0": - version "5.16.4" - resolved "https://registry.yarnpkg.com/@testing-library/jest-dom/-/jest-dom-5.16.4.tgz#938302d7b8b483963a3ae821f1c0808f872245cd" - integrity sha512-Gy+IoFutbMQcky0k+bqqumXZ1cTGswLsFqmNLzNdSKkU9KGV2u9oXhukCbbJ9/LRPKiqwxEE8VpV/+YZlfkPUA== - dependencies: - "@babel/runtime" "^7.9.2" - "@types/testing-library__jest-dom" "^5.9.1" - aria-query "^5.0.0" - chalk "^3.0.0" - css "^3.0.0" - css.escape "^1.5.1" - dom-accessibility-api "^0.5.6" - lodash "^4.17.15" - redent "^3.0.0" - -"@testing-library/react@^13.0.0": - version "13.3.0" - resolved "https://registry.yarnpkg.com/@testing-library/react/-/react-13.3.0.tgz#bf298bfbc5589326bbcc8052b211f3bb097a97c5" - integrity sha512-DB79aA426+deFgGSjnf5grczDPiL4taK3hFaa+M5q7q20Kcve9eQottOG5kZ74KEr55v0tU2CQormSSDK87zYQ== - dependencies: - "@babel/runtime" "^7.12.5" - "@testing-library/dom" "^8.5.0" - "@types/react-dom" "^18.0.0" - -"@tootallnate/once@1": - version "1.1.2" - resolved "https://registry.yarnpkg.com/@tootallnate/once/-/once-1.1.2.tgz#ccb91445360179a04e7fe6aff78c00ffc1eeaf82" - integrity sha512-RbzJvlNzmRq5c3O09UipeuXno4tA1FE6ikOjxZK0tuxVv3412l64l5t1W5pj4+rJq9vpkm/kwiR07aZXnsKPxw== - -"@trysound/sax@0.2.0": - version "0.2.0" - resolved "https://registry.yarnpkg.com/@trysound/sax/-/sax-0.2.0.tgz#cccaab758af56761eb7bf37af6f03f326dd798ad" - integrity sha512-L7z9BgrNEcYyUYtF+HaEfiS5ebkh9jXqbszz7pC0hRBPaatV0XjSD3+eHrpqFemQfgwiFF0QPIarnIihIDn7OA== - -"@types/aria-query@^4.2.0": - version "4.2.2" - resolved "https://registry.yarnpkg.com/@types/aria-query/-/aria-query-4.2.2.tgz#ed4e0ad92306a704f9fb132a0cfcf77486dbe2bc" - integrity sha512-HnYpAE1Y6kRyKM/XkEuiRQhTHvkzMBurTHnpFLYLBGPIylZNPs9jJcuOOYWxPLJCSEtmZT0Y8rHDokKN7rRTig== - -"@types/babel__core@^7.0.0", "@types/babel__core@^7.1.14": - version "7.1.16" - resolved "https://registry.yarnpkg.com/@types/babel__core/-/babel__core-7.1.16.tgz#bc12c74b7d65e82d29876b5d0baf5c625ac58702" - integrity sha512-EAEHtisTMM+KaKwfWdC3oyllIqswlznXCIVCt7/oRNrh+DhgT4UEBNC/jlADNjvw7UnfbcdkGQcPVZ1xYiLcrQ== - dependencies: - "@babel/parser" "^7.1.0" - "@babel/types" "^7.0.0" - "@types/babel__generator" "*" - "@types/babel__template" "*" - "@types/babel__traverse" "*" - -"@types/babel__generator@*": - version "7.6.3" - resolved "https://registry.yarnpkg.com/@types/babel__generator/-/babel__generator-7.6.3.tgz#f456b4b2ce79137f768aa130d2423d2f0ccfaba5" - integrity sha512-/GWCmzJWqV7diQW54smJZzWbSFf4QYtF71WCKhcx6Ru/tFyQIY2eiiITcCAeuPbNSvT9YCGkVMqqvSk2Z0mXiA== - dependencies: - "@babel/types" "^7.0.0" - -"@types/babel__template@*": - version "7.4.1" - resolved "https://registry.yarnpkg.com/@types/babel__template/-/babel__template-7.4.1.tgz#3d1a48fd9d6c0edfd56f2ff578daed48f36c8969" - integrity sha512-azBFKemX6kMg5Io+/rdGT0dkGreboUVR0Cdm3fz9QJWpaQGJRQXl7C+6hOTCZcMll7KFyEQpgbYI2lHdsS4U7g== - dependencies: - "@babel/parser" "^7.1.0" - "@babel/types" "^7.0.0" - -"@types/babel__traverse@*", "@types/babel__traverse@^7.0.4", "@types/babel__traverse@^7.0.6": - version "7.14.2" - resolved "https://registry.yarnpkg.com/@types/babel__traverse/-/babel__traverse-7.14.2.tgz#ffcd470bbb3f8bf30481678fb5502278ca833a43" - integrity sha512-K2waXdXBi2302XUdcHcR1jCeU0LL4TD9HRs/gk0N2Xvrht+G/BfJa4QObBQZfhMdxiCpV3COl5Nfq4uKTeTnJA== - dependencies: - "@babel/types" "^7.3.0" - -"@types/color-convert@*": - version "2.0.0" - resolved "https://registry.yarnpkg.com/@types/color-convert/-/color-convert-2.0.0.tgz#8f5ee6b9e863dcbee5703f5a517ffb13d3ea4e22" - integrity sha512-m7GG7IKKGuJUXvkZ1qqG3ChccdIM/qBBo913z+Xft0nKCX4hAU/IxKwZBU4cpRZ7GS5kV4vOblUkILtSShCPXQ== - dependencies: - "@types/color-name" "*" - -"@types/color-name@*": - version "1.1.1" - resolved "https://registry.yarnpkg.com/@types/color-name/-/color-name-1.1.1.tgz#1c1261bbeaa10a8055bbc5d8ab84b7b2afc846a0" - integrity sha512-rr+OQyAjxze7GgWrSaJwydHStIhHq2lvY3BOC2Mj7KnzI7XK0Uw1TOOdI9lDoajEbSWLiYgoo4f1R51erQfhPQ== - -"@types/color@^3.0.3": - version "3.0.3" - resolved "https://registry.yarnpkg.com/@types/color/-/color-3.0.3.tgz#e6d8d72b7aaef4bb9fe80847c26c7c786191016d" - integrity sha512-X//qzJ3d3Zj82J9sC/C18ZY5f43utPbAJ6PhYt/M7uG6etcF6MRpKdN880KBy43B0BMzSfeT96MzrsNjFI3GbA== - dependencies: - "@types/color-convert" "*" - -"@types/d3-array@*": - version "3.0.3" - resolved "https://registry.yarnpkg.com/@types/d3-array/-/d3-array-3.0.3.tgz#87d990bf504d14ad6b16766979d04e943c046dac" - integrity sha512-Reoy+pKnvsksN0lQUlcH6dOGjRZ/3WRwXR//m+/8lt1BXeI4xyaUZoqULNjyXXRuh0Mj4LNpkCvhUpQlY3X5xQ== - -"@types/d3-axis@*": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-axis/-/d3-axis-3.0.1.tgz#6afc20744fa5cc0cbc3e2bd367b140a79ed3e7a8" - integrity sha512-zji/iIbdd49g9WN0aIsGcwcTBUkgLsCSwB+uH+LPVDAiKWENMtI3cJEWt+7/YYwelMoZmbBfzA3qCdrZ2XFNnw== - dependencies: - "@types/d3-selection" "*" - -"@types/d3-brush@*": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-brush/-/d3-brush-3.0.1.tgz#ae5f17ce391935ca88b29000e60ee20452c6357c" - integrity sha512-B532DozsiTuQMHu2YChdZU0qsFJSio3Q6jmBYGYNp3gMDzBmuFFgPt9qKA4VYuLZMp4qc6eX7IUFUEsvHiXZAw== - dependencies: - "@types/d3-selection" "*" - -"@types/d3-chord@*": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-chord/-/d3-chord-3.0.1.tgz#54c8856c19c8e4ab36a53f73ba737de4768ad248" - integrity sha512-eQfcxIHrg7V++W8Qxn6QkqBNBokyhdWSAS73AbkbMzvLQmVVBviknoz2SRS/ZJdIOmhcmmdCRE/NFOm28Z1AMw== - -"@types/d3-color@*": - version "3.1.0" - resolved "https://registry.yarnpkg.com/@types/d3-color/-/d3-color-3.1.0.tgz#6594da178ded6c7c3842f3cc0ac84b156f12f2d4" - integrity sha512-HKuicPHJuvPgCD+np6Se9MQvS6OCbJmOjGvylzMJRlDwUXjKTTXs6Pwgk79O09Vj/ho3u1ofXnhFOaEWWPrlwA== - -"@types/d3-color@^1": - version "1.4.2" - resolved "https://registry.yarnpkg.com/@types/d3-color/-/d3-color-1.4.2.tgz#944f281d04a0f06e134ea96adbb68303515b2784" - integrity sha512-fYtiVLBYy7VQX+Kx7wU/uOIkGQn8aAEY8oWMoyja3N4dLd8Yf6XgSIR/4yWvMuveNOH5VShnqCgRqqh/UNanBA== - -"@types/d3-contour@*": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-contour/-/d3-contour-3.0.1.tgz#9ff4e2fd2a3910de9c5097270a7da8a6ef240017" - integrity sha512-C3zfBrhHZvrpAAK3YXqLWVAGo87A4SvJ83Q/zVJ8rFWJdKejUnDYaWZPkA8K84kb2vDA/g90LTQAz7etXcgoQQ== - dependencies: - "@types/d3-array" "*" - "@types/geojson" "*" - -"@types/d3-delaunay@*": - version "6.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-delaunay/-/d3-delaunay-6.0.1.tgz#006b7bd838baec1511270cb900bf4fc377bbbf41" - integrity sha512-tLxQ2sfT0p6sxdG75c6f/ekqxjyYR0+LwPrsO1mbC9YDBzPJhs2HbJJRrn8Ez1DBoHRo2yx7YEATI+8V1nGMnQ== - -"@types/d3-dispatch@*": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-dispatch/-/d3-dispatch-3.0.1.tgz#a1b18ae5fa055a6734cb3bd3cbc6260ef19676e3" - integrity sha512-NhxMn3bAkqhjoxabVJWKryhnZXXYYVQxaBnbANu0O94+O/nX9qSjrA1P1jbAQJxJf+VC72TxDX/YJcKue5bRqw== - -"@types/d3-drag@*", "@types/d3-drag@^3.0.1": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-drag/-/d3-drag-3.0.1.tgz#fb1e3d5cceeee4d913caa59dedf55c94cb66e80f" - integrity sha512-o1Va7bLwwk6h03+nSM8dpaGEYnoIG19P0lKqlic8Un36ymh9NSkNFX1yiXMKNMx8rJ0Kfnn2eovuFaL6Jvj0zA== - dependencies: - "@types/d3-selection" "*" - -"@types/d3-dsv@*": - version "3.0.0" - resolved "https://registry.yarnpkg.com/@types/d3-dsv/-/d3-dsv-3.0.0.tgz#f3c61fb117bd493ec0e814856feb804a14cfc311" - integrity sha512-o0/7RlMl9p5n6FQDptuJVMxDf/7EDEv2SYEO/CwdG2tr1hTfUVi0Iavkk2ax+VpaQ/1jVhpnj5rq1nj8vwhn2A== - -"@types/d3-ease@*": - version "3.0.0" - resolved "https://registry.yarnpkg.com/@types/d3-ease/-/d3-ease-3.0.0.tgz#c29926f8b596f9dadaeca062a32a45365681eae0" - integrity sha512-aMo4eaAOijJjA6uU+GIeW018dvy9+oH5Y2VPPzjjfxevvGQ/oRDs+tfYC9b50Q4BygRR8yE2QCLsrT0WtAVseA== - -"@types/d3-fetch@*": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-fetch/-/d3-fetch-3.0.1.tgz#f9fa88b81aa2eea5814f11aec82ecfddbd0b8fe0" - integrity sha512-toZJNOwrOIqz7Oh6Q7l2zkaNfXkfR7mFSJvGvlD/Ciq/+SQ39d5gynHJZ/0fjt83ec3WL7+u3ssqIijQtBISsw== - dependencies: - "@types/d3-dsv" "*" - -"@types/d3-force@*": - version "3.0.3" - resolved "https://registry.yarnpkg.com/@types/d3-force/-/d3-force-3.0.3.tgz#76cb20d04ae798afede1ea6e41750763ff5a9c82" - integrity sha512-z8GteGVfkWJMKsx6hwC3SiTSLspL98VNpmvLpEFJQpZPq6xpA1I8HNBDNSpukfK0Vb0l64zGFhzunLgEAcBWSA== - -"@types/d3-format@*": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-format/-/d3-format-3.0.1.tgz#194f1317a499edd7e58766f96735bdc0216bb89d" - integrity sha512-5KY70ifCCzorkLuIkDe0Z9YTf9RR2CjBX1iaJG+rgM/cPP+sO+q9YdQ9WdhQcgPj1EQiJ2/0+yUkkziTG6Lubg== - -"@types/d3-geo@*": - version "3.0.2" - resolved "https://registry.yarnpkg.com/@types/d3-geo/-/d3-geo-3.0.2.tgz#e7ec5f484c159b2c404c42d260e6d99d99f45d9a" - integrity sha512-DbqK7MLYA8LpyHQfv6Klz0426bQEf7bRTvhMy44sNGVyZoWn//B0c+Qbeg8Osi2Obdc9BLLXYAKpyWege2/7LQ== - dependencies: - "@types/geojson" "*" - -"@types/d3-hierarchy@*": - version "3.1.0" - resolved "https://registry.yarnpkg.com/@types/d3-hierarchy/-/d3-hierarchy-3.1.0.tgz#4561bb7ace038f247e108295ef77b6a82193ac25" - integrity sha512-g+sey7qrCa3UbsQlMZZBOHROkFqx7KZKvUpRzI/tAp/8erZWpYq7FgNKvYwebi2LaEiVs1klhUfd3WCThxmmWQ== - -"@types/d3-interpolate@*": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-interpolate/-/d3-interpolate-3.0.1.tgz#e7d17fa4a5830ad56fe22ce3b4fac8541a9572dc" - integrity sha512-jx5leotSeac3jr0RePOH1KdR9rISG91QIE4Q2PYTu4OymLTZfA3SrnURSLzKH48HmXVUru50b8nje4E79oQSQw== - dependencies: - "@types/d3-color" "*" - -"@types/d3-interpolate@^1.3.1": - version "1.4.2" - resolved "https://registry.yarnpkg.com/@types/d3-interpolate/-/d3-interpolate-1.4.2.tgz#88902a205f682773a517612299a44699285eed7b" - integrity sha512-ylycts6llFf8yAEs1tXzx2loxxzDZHseuhPokrqKprTQSTcD3JbJI1omZP1rphsELZO3Q+of3ff0ZS7+O6yVzg== - dependencies: - "@types/d3-color" "^1" - -"@types/d3-path@*": - version "3.0.0" - resolved "https://registry.yarnpkg.com/@types/d3-path/-/d3-path-3.0.0.tgz#939e3a784ae4f80b1fde8098b91af1776ff1312b" - integrity sha512-0g/A+mZXgFkQxN3HniRDbXMN79K3CdTpLsevj+PXiTcb2hVyvkZUBg37StmgCQkaD84cUJ4uaDAWq7UJOQy2Tg== - -"@types/d3-path@^1", "@types/d3-path@^1.0.8": - version "1.0.9" - resolved "https://registry.yarnpkg.com/@types/d3-path/-/d3-path-1.0.9.tgz#73526b150d14cd96e701597cbf346cfd1fd4a58c" - integrity sha512-NaIeSIBiFgSC6IGUBjZWcscUJEq7vpVu7KthHN8eieTV9d9MqkSOZLH4chq1PmcKy06PNe3axLeKmRIyxJ+PZQ== - -"@types/d3-polygon@*": - version "3.0.0" - resolved "https://registry.yarnpkg.com/@types/d3-polygon/-/d3-polygon-3.0.0.tgz#5200a3fa793d7736fa104285fa19b0dbc2424b93" - integrity sha512-D49z4DyzTKXM0sGKVqiTDTYr+DHg/uxsiWDAkNrwXYuiZVd9o9wXZIo+YsHkifOiyBkmSWlEngHCQme54/hnHw== - -"@types/d3-quadtree@*": - version "3.0.2" - resolved "https://registry.yarnpkg.com/@types/d3-quadtree/-/d3-quadtree-3.0.2.tgz#433112a178eb7df123aab2ce11c67f51cafe8ff5" - integrity sha512-QNcK8Jguvc8lU+4OfeNx+qnVy7c0VrDJ+CCVFS9srBo2GL9Y18CnIxBdTF3v38flrGy5s1YggcoAiu6s4fLQIw== - -"@types/d3-random@*": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-random/-/d3-random-3.0.1.tgz#5c8d42b36cd4c80b92e5626a252f994ca6bfc953" - integrity sha512-IIE6YTekGczpLYo/HehAy3JGF1ty7+usI97LqraNa8IiDur+L44d0VOjAvFQWJVdZOJHukUJw+ZdZBlgeUsHOQ== - -"@types/d3-scale-chromatic@*": - version "3.0.0" - resolved "https://registry.yarnpkg.com/@types/d3-scale-chromatic/-/d3-scale-chromatic-3.0.0.tgz#103124777e8cdec85b20b51fd3397c682ee1e954" - integrity sha512-dsoJGEIShosKVRBZB0Vo3C8nqSDqVGujJU6tPznsBJxNJNwMF8utmS83nvCBKQYPpjCzaaHcrf66iTRpZosLPw== - -"@types/d3-scale@*": - version "4.0.2" - resolved "https://registry.yarnpkg.com/@types/d3-scale/-/d3-scale-4.0.2.tgz#41be241126af4630524ead9cb1008ab2f0f26e69" - integrity sha512-Yk4htunhPAwN0XGlIwArRomOjdoBFXC3+kCxK2Ubg7I9shQlVSJy/pG/Ht5ASN+gdMIalpk8TJ5xV74jFsetLA== - dependencies: - "@types/d3-time" "*" - -"@types/d3-scale@^3.3.0": - version "3.3.2" - resolved "https://registry.yarnpkg.com/@types/d3-scale/-/d3-scale-3.3.2.tgz#18c94e90f4f1c6b1ee14a70f14bfca2bd1c61d06" - integrity sha512-gGqr7x1ost9px3FvIfUMi5XA/F/yAf4UkUDtdQhpH92XCT0Oa7zkkRzY61gPVJq+DxpHn/btouw5ohWkbBsCzQ== - dependencies: - "@types/d3-time" "^2" - -"@types/d3-selection@*", "@types/d3-selection@^3.0.3": - version "3.0.3" - resolved "https://registry.yarnpkg.com/@types/d3-selection/-/d3-selection-3.0.3.tgz#57be7da68e7d9c9b29efefd8ea5a9ef1171e42ba" - integrity sha512-Mw5cf6nlW1MlefpD9zrshZ+DAWL4IQ5LnWfRheW6xwsdaWOb6IRRu2H7XPAQcyXEx1D7XQWgdoKR83ui1/HlEA== - -"@types/d3-shape@*": - version "3.1.0" - resolved "https://registry.yarnpkg.com/@types/d3-shape/-/d3-shape-3.1.0.tgz#1d87a6ddcf28285ef1e5c278ca4bdbc0658f3505" - integrity sha512-jYIYxFFA9vrJ8Hd4Se83YI6XF+gzDL1aC5DCsldai4XYYiVNdhtpGbA/GM6iyQ8ayhSp3a148LY34hy7A4TxZA== - dependencies: - "@types/d3-path" "*" - -"@types/d3-shape@^1.3.1": - version "1.3.8" - resolved "https://registry.yarnpkg.com/@types/d3-shape/-/d3-shape-1.3.8.tgz#c3c15ec7436b4ce24e38de517586850f1fea8e89" - integrity sha512-gqfnMz6Fd5H6GOLYixOZP/xlrMtJms9BaS+6oWxTKHNqPGZ93BkWWupQSCYm6YHqx6h9wjRupuJb90bun6ZaYg== - dependencies: - "@types/d3-path" "^1" - -"@types/d3-time-format@*": - version "4.0.0" - resolved "https://registry.yarnpkg.com/@types/d3-time-format/-/d3-time-format-4.0.0.tgz#ee7b6e798f8deb2d9640675f8811d0253aaa1946" - integrity sha512-yjfBUe6DJBsDin2BMIulhSHmr5qNR5Pxs17+oW4DoVPyVIXZ+m6bs7j1UVKP08Emv6jRmYrYqxYzO63mQxy1rw== - -"@types/d3-time@*": - version "3.0.0" - resolved "https://registry.yarnpkg.com/@types/d3-time/-/d3-time-3.0.0.tgz#e1ac0f3e9e195135361fa1a1d62f795d87e6e819" - integrity sha512-sZLCdHvBUcNby1cB6Fd3ZBrABbjz3v1Vm90nysCQ6Vt7vd6e/h9Lt7SiJUoEX0l4Dzc7P5llKyhqSi1ycSf1Hg== - -"@types/d3-time@^2", "@types/d3-time@^2.0.0": - version "2.1.1" - resolved "https://registry.yarnpkg.com/@types/d3-time/-/d3-time-2.1.1.tgz#743fdc821c81f86537cbfece07093ac39b4bc342" - integrity sha512-9MVYlmIgmRR31C5b4FVSWtuMmBHh2mOWQYfl7XAYOa8dsnb7iEmUmRSWSFgXFtkjxO65d7hTUHQC+RhR/9IWFg== - -"@types/d3-timer@*": - version "3.0.0" - resolved "https://registry.yarnpkg.com/@types/d3-timer/-/d3-timer-3.0.0.tgz#e2505f1c21ec08bda8915238e397fb71d2fc54ce" - integrity sha512-HNB/9GHqu7Fo8AQiugyJbv6ZxYz58wef0esl4Mv828w1ZKpAshw/uFWVDUcIB9KKFeFKoxS3cHY07FFgtTRZ1g== - -"@types/d3-transition@*": - version "3.0.2" - resolved "https://registry.yarnpkg.com/@types/d3-transition/-/d3-transition-3.0.2.tgz#393dc3e3d55009a43cc6f252e73fccab6d78a8a4" - integrity sha512-jo5o/Rf+/u6uerJ/963Dc39NI16FQzqwOc54bwvksGAdVfvDrqDpVeq95bEvPtBwLCVZutAEyAtmSyEMxN7vxQ== - dependencies: - "@types/d3-selection" "*" - -"@types/d3-zoom@*", "@types/d3-zoom@^3.0.1": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/d3-zoom/-/d3-zoom-3.0.1.tgz#4bfc7e29625c4f79df38e2c36de52ec3e9faf826" - integrity sha512-7s5L9TjfqIYQmQQEUcpMAcBOahem7TRoSO/+Gkz02GbMVuULiZzjF2BOdw291dbO2aNon4m2OdFsRGaCq2caLQ== - dependencies: - "@types/d3-interpolate" "*" - "@types/d3-selection" "*" - -"@types/d3@^7.4.0": - version "7.4.0" - resolved "https://registry.yarnpkg.com/@types/d3/-/d3-7.4.0.tgz#fc5cac5b1756fc592a3cf1f3dc881bf08225f515" - integrity sha512-jIfNVK0ZlxcuRDKtRS/SypEyOQ6UHaFQBKv032X45VvxSJ6Yi5G9behy9h6tNTHTDGh5Vq+KbmBjUWLgY4meCA== - dependencies: - "@types/d3-array" "*" - "@types/d3-axis" "*" - "@types/d3-brush" "*" - "@types/d3-chord" "*" - "@types/d3-color" "*" - "@types/d3-contour" "*" - "@types/d3-delaunay" "*" - "@types/d3-dispatch" "*" - "@types/d3-drag" "*" - "@types/d3-dsv" "*" - "@types/d3-ease" "*" - "@types/d3-fetch" "*" - "@types/d3-force" "*" - "@types/d3-format" "*" - "@types/d3-geo" "*" - "@types/d3-hierarchy" "*" - "@types/d3-interpolate" "*" - "@types/d3-path" "*" - "@types/d3-polygon" "*" - "@types/d3-quadtree" "*" - "@types/d3-random" "*" - "@types/d3-scale" "*" - "@types/d3-scale-chromatic" "*" - "@types/d3-selection" "*" - "@types/d3-shape" "*" - "@types/d3-time" "*" - "@types/d3-time-format" "*" - "@types/d3-timer" "*" - "@types/d3-transition" "*" - "@types/d3-zoom" "*" - -"@types/debug@^4.0.0": - version "4.1.7" - resolved "https://registry.yarnpkg.com/@types/debug/-/debug-4.1.7.tgz#7cc0ea761509124709b8b2d1090d8f6c17aadb82" - integrity sha512-9AonUzyTjXXhEOa0DnqpzZi6VHlqKMswga9EXjpXnnqxwLtdvPPtlO8evrI5D9S6asFRCQ6v+wpiUKbw+vKqyg== - dependencies: - "@types/ms" "*" - -"@types/estree@^1.0.5": - version "1.0.5" - resolved "https://registry.yarnpkg.com/@types/estree/-/estree-1.0.5.tgz#a6ce3e556e00fd9895dd872dd172ad0d4bd687f4" - integrity sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw== - -"@types/geojson@*": - version "7946.0.10" - resolved "https://registry.yarnpkg.com/@types/geojson/-/geojson-7946.0.10.tgz#6dfbf5ea17142f7f9a043809f1cd4c448cb68249" - integrity sha512-Nmh0K3iWQJzniTuPRcJn5hxXkfB1T1pgB89SBig5PlJQU5yocazeu4jATJlaA0GYFKWMqDdvYemoSnF2pXgLVA== - -"@types/glob@^7.1.1": - version "7.1.3" - resolved "https://registry.yarnpkg.com/@types/glob/-/glob-7.1.3.tgz#e6ba80f36b7daad2c685acd9266382e68985c183" - integrity sha512-SEYeGAIQIQX8NN6LDKprLjbrd5dARM5EXsd8GI/A5l0apYI1fGMWgPHSe4ZKL4eozlAyI+doUE9XbYS4xCkQ1w== - dependencies: - "@types/minimatch" "*" - "@types/node" "*" - -"@types/graceful-fs@^4.1.2": - version "4.1.5" - resolved "https://registry.yarnpkg.com/@types/graceful-fs/-/graceful-fs-4.1.5.tgz#21ffba0d98da4350db64891f92a9e5db3cdb4e15" - integrity sha512-anKkLmZZ+xm4p8JWBf4hElkM4XR+EZeA2M9BAkkTldmcyDY4mbdIJnRghDJH3Ov5ooY7/UAoENtmdMSkaAd7Cw== - dependencies: - "@types/node" "*" - -"@types/hast@^2.0.0": - version "2.3.4" - resolved "https://registry.yarnpkg.com/@types/hast/-/hast-2.3.4.tgz#8aa5ef92c117d20d974a82bdfb6a648b08c0bafc" - integrity sha512-wLEm0QvaoawEDoTRwzTXp4b4jpwiJDvR5KMnFnVodm3scufTlBOWRD6N1OBf9TZMhjlNsSfcO5V+7AF4+Vy+9g== - dependencies: - "@types/unist" "*" - -"@types/istanbul-lib-coverage@*", "@types/istanbul-lib-coverage@^2.0.0", "@types/istanbul-lib-coverage@^2.0.1": - version "2.0.3" - resolved "https://registry.yarnpkg.com/@types/istanbul-lib-coverage/-/istanbul-lib-coverage-2.0.3.tgz#4ba8ddb720221f432e443bd5f9117fd22cfd4762" - integrity sha512-sz7iLqvVUg1gIedBOvlkxPlc8/uVzyS5OwGz1cKjXzkl3FpL3al0crU8YGU1WoHkxn0Wxbw5tyi6hvzJKNzFsw== - -"@types/istanbul-lib-report@*": - version "3.0.0" - resolved "https://registry.yarnpkg.com/@types/istanbul-lib-report/-/istanbul-lib-report-3.0.0.tgz#c14c24f18ea8190c118ee7562b7ff99a36552686" - integrity sha512-plGgXAPfVKFoYfa9NpYDAkseG+g6Jr294RqeqcqDixSbU34MZVJRi/P+7Y8GDpzkEwLaGZZOpKIEmeVZNtKsrg== - dependencies: - "@types/istanbul-lib-coverage" "*" - -"@types/istanbul-reports@^3.0.0": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/istanbul-reports/-/istanbul-reports-3.0.1.tgz#9153fe98bba2bd565a63add9436d6f0d7f8468ff" - integrity sha512-c3mAZEuK0lvBp8tmuL74XRKn1+y2dcwOUpH7x4WrF6gk1GIgiluDRgMYQtw2OFcBvAJWlt6ASU3tSqxp0Uu0Aw== - dependencies: - "@types/istanbul-lib-report" "*" - -"@types/jest@*": - version "27.0.2" - resolved "https://registry.yarnpkg.com/@types/jest/-/jest-27.0.2.tgz#ac383c4d4aaddd29bbf2b916d8d105c304a5fcd7" - integrity sha512-4dRxkS/AFX0c5XW6IPMNOydLn2tEhNhJV7DnYK+0bjoJZ+QTmfucBlihX7aoEsh/ocYtkLC73UbnBXBXIxsULA== - dependencies: - jest-diff "^27.0.0" - pretty-format "^27.0.0" - -"@types/json-schema@^7.0.5", "@types/json-schema@^7.0.7": - version "7.0.7" - resolved "https://registry.yarnpkg.com/@types/json-schema/-/json-schema-7.0.7.tgz#98a993516c859eb0d5c4c8f098317a9ea68db9ad" - integrity sha512-cxWFQVseBm6O9Gbw1IWb8r6OS4OhSt3hPZLkFApLjM8TEXROBuQGLAH2i2gZpcXdLBIrpXuTDhH7Vbm1iXmNGA== - -"@types/json-schema@^7.0.8", "@types/json-schema@^7.0.9": - version "7.0.11" - resolved "https://registry.yarnpkg.com/@types/json-schema/-/json-schema-7.0.11.tgz#d421b6c527a3037f7c84433fd2c4229e016863d3" - integrity sha512-wOuvG1SN4Us4rez+tylwwwCV1psiNVOkJeM3AUWUNWg/jDQY2+HE/444y5gc+jBmRqASOm2Oeh5c1axHobwRKQ== - -"@types/json-to-pretty-yaml@^1.2.1": - version "1.2.1" - resolved "https://registry.yarnpkg.com/@types/json-to-pretty-yaml/-/json-to-pretty-yaml-1.2.1.tgz#bf193455477295d83c78f73c08d956f74321193e" - integrity sha512-+uOlBkCPkny6CE2a5IAR0Q21/ZE+90MsK7EfDblDdutcey+rbMDrp3i93M6MTwbMHFB75aIFR5fVXVcnLCkAiw== - -"@types/json5@^0.0.29": - version "0.0.29" - resolved "https://registry.yarnpkg.com/@types/json5/-/json5-0.0.29.tgz#ee28707ae94e11d2b827bcbe5270bcea7f3e71ee" - integrity sha1-7ihweulOEdK4J7y+UnC86n8+ce4= - -"@types/lodash.mergewith@4.6.6": - version "4.6.6" - resolved "https://registry.yarnpkg.com/@types/lodash.mergewith/-/lodash.mergewith-4.6.6.tgz#c4698f5b214a433ff35cb2c75ee6ec7f99d79f10" - integrity sha512-RY/8IaVENjG19rxTZu9Nukqh0W2UrYgmBj5sdns4hWRZaV8PqR7wIKHFKzvOTjo4zVRV7sVI+yFhAJql12Kfqg== - dependencies: - "@types/lodash" "*" - -"@types/lodash@*": - version "4.14.178" - resolved "https://registry.yarnpkg.com/@types/lodash/-/lodash-4.14.178.tgz#341f6d2247db528d4a13ddbb374bcdc80406f4f8" - integrity sha512-0d5Wd09ItQWH1qFbEyQ7oTQ3GZrMfth5JkbN3EvTKLXcHLRDSXeLnlvlOn0wvxVIwK5o2M8JzP/OWz7T3NRsbw== - -"@types/lodash@^4.14.172": - version "4.14.182" - resolved "https://registry.yarnpkg.com/@types/lodash/-/lodash-4.14.182.tgz#05301a4d5e62963227eaafe0ce04dd77c54ea5c2" - integrity sha512-/THyiqyQAP9AfARo4pF+aCGcyiQ94tX/Is2I7HofNRqoYLgN1PBoOWu2/zTA5zMxzP5EFutMtWtGAFRKUe961Q== - -"@types/mdast@^3.0.0": - version "3.0.10" - resolved "https://registry.yarnpkg.com/@types/mdast/-/mdast-3.0.10.tgz#4724244a82a4598884cbbe9bcfd73dff927ee8af" - integrity sha512-W864tg/Osz1+9f4lrGTZpCSO5/z4608eUp19tbozkq2HJK6i3z1kT0H9tlADXuYIb1YYOBByU4Jsqkk75q48qA== - dependencies: - "@types/unist" "*" - -"@types/minimatch@*": - version "3.0.4" - resolved "https://registry.yarnpkg.com/@types/minimatch/-/minimatch-3.0.4.tgz#f0ec25dbf2f0e4b18647313ac031134ca5b24b21" - integrity sha512-1z8k4wzFnNjVK/tlxvrWuK5WMt6mydWWP7+zvH5eFep4oj+UkrfiJTRtjCeBXNpwaA/FYqqtb4/QS4ianFpIRA== - -"@types/minimist@^1.2.2": - version "1.2.2" - resolved "https://registry.yarnpkg.com/@types/minimist/-/minimist-1.2.2.tgz#ee771e2ba4b3dc5b372935d549fd9617bf345b8c" - integrity sha512-jhuKLIRrhvCPLqwPcx6INqmKeiA5EWrsCOPhrlFSrbrmU4ZMPjj5Ul/oLCMDO98XRUIwVm78xICz4EPCektzeQ== - -"@types/ms@*": - version "0.7.31" - resolved "https://registry.yarnpkg.com/@types/ms/-/ms-0.7.31.tgz#31b7ca6407128a3d2bbc27fe2d21b345397f6197" - integrity sha512-iiUgKzV9AuaEkZqkOLDIvlQiL6ltuZd9tGcW3gwpnX8JbuiuhFlEGmmFXEXkN50Cvq7Os88IY2v0dkDqXYWVgA== - -"@types/node@*": - version "15.12.2" - resolved "https://registry.yarnpkg.com/@types/node/-/node-15.12.2.tgz#1f2b42c4be7156ff4a6f914b2fb03d05fa84e38d" - integrity sha512-zjQ69G564OCIWIOHSXyQEEDpdpGl+G348RAKY0XXy9Z5kU9Vzv1GMNnkar/ZJ8dzXB3COzD9Mo9NtRZ4xfgUww== - -"@types/node@^14.11.8": - version "14.17.3" - resolved "https://registry.yarnpkg.com/@types/node/-/node-14.17.3.tgz#6d327abaa4be34a74e421ed6409a0ae2f47f4c3d" - integrity sha512-e6ZowgGJmTuXa3GyaPbTGxX17tnThl2aSSizrFthQ7m9uLGZBXiGhgE55cjRZTF5kjZvYn9EOPOMljdjwbflxw== - -"@types/normalize-package-data@^2.4.0": - version "2.4.0" - resolved "https://registry.yarnpkg.com/@types/normalize-package-data/-/normalize-package-data-2.4.0.tgz#e486d0d97396d79beedd0a6e33f4534ff6b4973e" - integrity sha512-f5j5b/Gf71L+dbqxIpQ4Z2WlmI/mPJ0fOkGGmFgtb6sAu97EPczzbS3/tJKxmcYDj55OX6ssqwDAWOHIYDRDGA== - -"@types/parse-json@^4.0.0": - version "4.0.0" - resolved "https://registry.yarnpkg.com/@types/parse-json/-/parse-json-4.0.0.tgz#2f8bb441434d163b35fb8ffdccd7138927ffb8c0" - integrity sha512-//oorEZjL6sbPcKUaCdIGlIUeH26mgzimjBB77G6XRgnDl/L5wOnpyBGRe/Mmf5CVW3PwEBE1NjiMZ/ssFh4wA== - -"@types/prettier@^2.1.5": - version "2.4.1" - resolved "https://registry.yarnpkg.com/@types/prettier/-/prettier-2.4.1.tgz#e1303048d5389563e130f5bdd89d37a99acb75eb" - integrity sha512-Fo79ojj3vdEZOHg3wR9ksAMRz4P3S5fDB5e/YWZiFnyFQI1WY2Vftu9XoXVVtJfxB7Bpce/QTqWSSntkz2Znrw== - -"@types/prop-types@*": - version "15.7.4" - resolved "https://registry.yarnpkg.com/@types/prop-types/-/prop-types-15.7.4.tgz#fcf7205c25dff795ee79af1e30da2c9790808f11" - integrity sha512-rZ5drC/jWjrArrS8BR6SIr4cWpW09RNTYt9AMZo3Jwwif+iacXAqgVjm0B0Bv/S1jhDXKHqRVNCbACkJ89RAnQ== - -"@types/prop-types@^15.0.0": - version "15.7.5" - resolved "https://registry.yarnpkg.com/@types/prop-types/-/prop-types-15.7.5.tgz#5f19d2b85a98e9558036f6a3cacc8819420f05cf" - integrity sha512-JCB8C6SnDoQf0cNycqd/35A7MjcnK+ZTqE7judS6o7utxUCg6imJg3QK2qzHKszlTjcj2cn+NwMB2i96ubpj7w== - -"@types/react-dom@^18.0.0", "@types/react-dom@^18.0.5": - version "18.0.5" - resolved "https://registry.yarnpkg.com/@types/react-dom/-/react-dom-18.0.5.tgz#330b2d472c22f796e5531446939eacef8378444a" - integrity sha512-OWPWTUrY/NIrjsAPkAk1wW9LZeIjSvkXRhclsFO8CZcZGCOg2G0YZy4ft+rOyYxy8B7ui5iZzi9OkDebZ7/QSA== - dependencies: - "@types/react" "*" - -"@types/react-syntax-highlighter@^15.5.6": - version "15.5.6" - resolved "https://registry.yarnpkg.com/@types/react-syntax-highlighter/-/react-syntax-highlighter-15.5.6.tgz#77c95e6b74d2be23208fcdcf187b93b47025f1b1" - integrity sha512-i7wFuLbIAFlabTeD2I1cLjEOrG/xdMa/rpx2zwzAoGHuXJDhSqp9BSfDlMHSh9JSuNfxHk9eEmMX6D55GiyjGg== - dependencies: - "@types/react" "*" - -"@types/react-table@^7.7.12": - version "7.7.12" - resolved "https://registry.yarnpkg.com/@types/react-table/-/react-table-7.7.12.tgz#628011d3cb695b07c678704a61f2f1d5b8e567fd" - integrity sha512-bRUent+NR/WwtDGwI/BqhZ8XnHghwHw0HUKeohzB5xN3K2qKWYE5w19e7GCuOkL1CXD9Gi1HFy7TIm2AvgWUHg== - dependencies: - "@types/react" "*" - -"@types/react-transition-group@^4.4.0": - version "4.4.5" - resolved "https://registry.yarnpkg.com/@types/react-transition-group/-/react-transition-group-4.4.5.tgz#aae20dcf773c5aa275d5b9f7cdbca638abc5e416" - integrity sha512-juKD/eiSM3/xZYzjuzH6ZwpP+/lejltmiS3QEzV/vmb/Q8+HfDmxu+Baga8UEMGBqV88Nbg4l2hY/K2DkyaLLA== - dependencies: - "@types/react" "*" - -"@types/react@*": - version "18.0.15" - resolved "https://registry.yarnpkg.com/@types/react/-/react-18.0.15.tgz#d355644c26832dc27f3e6cbf0c4f4603fc4ab7fe" - integrity sha512-iz3BtLuIYH1uWdsv6wXYdhozhqj20oD4/Hk2DNXIn1kFsmp9x8d9QB6FnPhfkbhd2PgEONt9Q1x/ebkwjfFLow== - dependencies: - "@types/prop-types" "*" - "@types/scheduler" "*" - csstype "^3.0.2" - -"@types/react@^18.0.12": - version "18.0.12" - resolved "https://registry.yarnpkg.com/@types/react/-/react-18.0.12.tgz#cdaa209d0a542b3fcf69cf31a03976ec4cdd8840" - integrity sha512-duF1OTASSBQtcigUvhuiTB1Ya3OvSy+xORCiEf20H0P0lzx+/KeVsA99U5UjLXSbyo1DRJDlLKqTeM1ngosqtg== - dependencies: - "@types/prop-types" "*" - "@types/scheduler" "*" - csstype "^3.0.2" - -"@types/scheduler@*": - version "0.16.2" - resolved "https://registry.yarnpkg.com/@types/scheduler/-/scheduler-0.16.2.tgz#1a62f89525723dde24ba1b01b092bf5df8ad4d39" - integrity sha512-hppQEBDmlwhFAXKJX2KnWLYu5yMfi91yazPb2l+lbJiwW+wdo1gNeRA+3RgNSO39WYX2euey41KEwnqesU2Jew== - -"@types/source-list-map@*": - version "0.1.2" - resolved "https://registry.yarnpkg.com/@types/source-list-map/-/source-list-map-0.1.2.tgz#0078836063ffaf17412349bba364087e0ac02ec9" - integrity sha512-K5K+yml8LTo9bWJI/rECfIPrGgxdpeNbj+d53lwN4QjW1MCwlkhUms+gtdzigTeUyBr09+u8BwOIY3MXvHdcsA== - -"@types/stack-utils@^2.0.0": - version "2.0.1" - resolved "https://registry.yarnpkg.com/@types/stack-utils/-/stack-utils-2.0.1.tgz#20f18294f797f2209b5f65c8e3b5c8e8261d127c" - integrity sha512-Hl219/BT5fLAaz6NDkSuhzasy49dwQS/DSdu4MdggFB8zcXv7vflBI3xp7FEmkmdDkBUI2bPUNeMttp2knYdxw== - -"@types/tapable@^1": - version "1.0.7" - resolved "https://registry.yarnpkg.com/@types/tapable/-/tapable-1.0.7.tgz#545158342f949e8fd3bfd813224971ecddc3fac4" - integrity sha512-0VBprVqfgFD7Ehb2vd8Lh9TG3jP98gvr8rgehQqzztZNI7o8zS8Ad4jyZneKELphpuE212D8J70LnSNQSyO6bQ== - -"@types/testing-library__jest-dom@^5.9.1": - version "5.14.1" - resolved "https://registry.yarnpkg.com/@types/testing-library__jest-dom/-/testing-library__jest-dom-5.14.1.tgz#014162a5cee6571819d48e999980694e2f657c3c" - integrity sha512-Gk9vaXfbzc5zCXI9eYE9BI5BNHEp4D3FWjgqBE/ePGYElLAP+KvxBcsdkwfIVvezs605oiyd/VrpiHe3Oeg+Aw== - dependencies: - "@types/jest" "*" - -"@types/uglify-js@*": - version "3.13.0" - resolved "https://registry.yarnpkg.com/@types/uglify-js/-/uglify-js-3.13.0.tgz#1cad8df1fb0b143c5aba08de5712ea9d1ff71124" - integrity sha512-EGkrJD5Uy+Pg0NUR8uA4bJ5WMfljyad0G+784vLCNUkD+QwOJXUbBYExXfVGf7YtyzdQp3L/XMYcliB987kL5Q== - dependencies: - source-map "^0.6.1" - -"@types/unist@*", "@types/unist@^2.0.0": - version "2.0.6" - resolved "https://registry.yarnpkg.com/@types/unist/-/unist-2.0.6.tgz#250a7b16c3b91f672a24552ec64678eeb1d3a08d" - integrity sha512-PBjIUxZHOuj0R15/xuwJYjFi+KZdNFrehocChv4g5hu6aFroHue8m0lBP0POdK2nKzbw0cgV1mws8+V/JAcEkQ== - -"@types/webpack-sources@*": - version "2.1.0" - resolved "https://registry.yarnpkg.com/@types/webpack-sources/-/webpack-sources-2.1.0.tgz#8882b0bd62d1e0ce62f183d0d01b72e6e82e8c10" - integrity sha512-LXn/oYIpBeucgP1EIJbKQ2/4ZmpvRl+dlrFdX7+94SKRUV3Evy3FsfMZY318vGhkWUS5MPhtOM3w1/hCOAOXcg== - dependencies: - "@types/node" "*" - "@types/source-list-map" "*" - source-map "^0.7.3" - -"@types/webpack@^4.4.31": - version "4.41.29" - resolved "https://registry.yarnpkg.com/@types/webpack/-/webpack-4.41.29.tgz#2e66c1de8223c440366469415c50a47d97625773" - integrity sha512-6pLaORaVNZxiB3FSHbyBiWM7QdazAWda1zvAq4SbZObZqHSDbWLi62iFdblVea6SK9eyBIVp5yHhKt/yNQdR7Q== - dependencies: - "@types/node" "*" - "@types/tapable" "^1" - "@types/uglify-js" "*" - "@types/webpack-sources" "*" - anymatch "^3.0.0" - source-map "^0.6.0" - -"@types/yargs-parser@*": - version "20.2.1" - resolved "https://registry.yarnpkg.com/@types/yargs-parser/-/yargs-parser-20.2.1.tgz#3b9ce2489919d9e4fea439b76916abc34b2df129" - integrity sha512-7tFImggNeNBVMsn0vLrpn1H1uPrUBdnARPTpZoitY37ZrdJREzf7I16tMrlK3hen349gr1NYh8CmZQa7CTG6Aw== - -"@types/yargs@^16.0.0": - version "16.0.4" - resolved "https://registry.yarnpkg.com/@types/yargs/-/yargs-16.0.4.tgz#26aad98dd2c2a38e421086ea9ad42b9e51642977" - integrity sha512-T8Yc9wt/5LbJyCaLiHPReJa0kApcIgJ7Bn735GjItUfh08Z1pJvu8QZqb9s+mMvKV6WUQRV7K2R46YbjMXTTJw== - dependencies: - "@types/yargs-parser" "*" - -"@typescript-eslint/eslint-plugin@^5.13.0": - version "5.27.1" - resolved "https://registry.yarnpkg.com/@typescript-eslint/eslint-plugin/-/eslint-plugin-5.27.1.tgz#fdf59c905354139046b41b3ed95d1609913d0758" - integrity sha512-6dM5NKT57ZduNnJfpY81Phe9nc9wolnMCnknb1im6brWi1RYv84nbMS3olJa27B6+irUVV1X/Wb+Am0FjJdGFw== - dependencies: - "@typescript-eslint/scope-manager" "5.27.1" - "@typescript-eslint/type-utils" "5.27.1" - "@typescript-eslint/utils" "5.27.1" - debug "^4.3.4" - functional-red-black-tree "^1.0.1" - ignore "^5.2.0" - regexpp "^3.2.0" - semver "^7.3.7" - tsutils "^3.21.0" - -"@typescript-eslint/parser@^5.0.0": - version "5.27.1" - resolved "https://registry.yarnpkg.com/@typescript-eslint/parser/-/parser-5.27.1.tgz#3a4dcaa67e45e0427b6ca7bb7165122c8b569639" - integrity sha512-7Va2ZOkHi5NP+AZwb5ReLgNF6nWLGTeUJfxdkVUAPPSaAdbWNnFZzLZ4EGGmmiCTg+AwlbE1KyUYTBglosSLHQ== - dependencies: - "@typescript-eslint/scope-manager" "5.27.1" - "@typescript-eslint/types" "5.27.1" - "@typescript-eslint/typescript-estree" "5.27.1" - debug "^4.3.4" - -"@typescript-eslint/scope-manager@5.27.1": - version "5.27.1" - resolved "https://registry.yarnpkg.com/@typescript-eslint/scope-manager/-/scope-manager-5.27.1.tgz#4d1504392d01fe5f76f4a5825991ec78b7b7894d" - integrity sha512-fQEOSa/QroWE6fAEg+bJxtRZJTH8NTskggybogHt4H9Da8zd4cJji76gA5SBlR0MgtwF7rebxTbDKB49YUCpAg== - dependencies: - "@typescript-eslint/types" "5.27.1" - "@typescript-eslint/visitor-keys" "5.27.1" - -"@typescript-eslint/type-utils@5.27.1": - version "5.27.1" - resolved "https://registry.yarnpkg.com/@typescript-eslint/type-utils/-/type-utils-5.27.1.tgz#369f695199f74c1876e395ebea202582eb1d4166" - integrity sha512-+UC1vVUWaDHRnC2cQrCJ4QtVjpjjCgjNFpg8b03nERmkHv9JV9X5M19D7UFMd+/G7T/sgFwX2pGmWK38rqyvXw== - dependencies: - "@typescript-eslint/utils" "5.27.1" - debug "^4.3.4" - tsutils "^3.21.0" - -"@typescript-eslint/types@5.27.1": - version "5.27.1" - resolved "https://registry.yarnpkg.com/@typescript-eslint/types/-/types-5.27.1.tgz#34e3e629501349d38be6ae97841298c03a6ffbf1" - integrity sha512-LgogNVkBhCTZU/m8XgEYIWICD6m4dmEDbKXESCbqOXfKZxRKeqpiJXQIErv66sdopRKZPo5l32ymNqibYEH/xg== - -"@typescript-eslint/typescript-estree@5.27.1": - version "5.27.1" - resolved "https://registry.yarnpkg.com/@typescript-eslint/typescript-estree/-/typescript-estree-5.27.1.tgz#7621ee78607331821c16fffc21fc7a452d7bc808" - integrity sha512-DnZvvq3TAJ5ke+hk0LklvxwYsnXpRdqUY5gaVS0D4raKtbznPz71UJGnPTHEFo0GDxqLOLdMkkmVZjSpET1hFw== - dependencies: - "@typescript-eslint/types" "5.27.1" - "@typescript-eslint/visitor-keys" "5.27.1" - debug "^4.3.4" - globby "^11.1.0" - is-glob "^4.0.3" - semver "^7.3.7" - tsutils "^3.21.0" - -"@typescript-eslint/utils@5.27.1": - version "5.27.1" - resolved "https://registry.yarnpkg.com/@typescript-eslint/utils/-/utils-5.27.1.tgz#b4678b68a94bc3b85bf08f243812a6868ac5128f" - integrity sha512-mZ9WEn1ZLDaVrhRaYgzbkXBkTPghPFsup8zDbbsYTxC5OmqrFE7skkKS/sraVsLP3TcT3Ki5CSyEFBRkLH/H/w== - dependencies: - "@types/json-schema" "^7.0.9" - "@typescript-eslint/scope-manager" "5.27.1" - "@typescript-eslint/types" "5.27.1" - "@typescript-eslint/typescript-estree" "5.27.1" - eslint-scope "^5.1.1" - eslint-utils "^3.0.0" - -"@typescript-eslint/visitor-keys@5.27.1": - version "5.27.1" - resolved "https://registry.yarnpkg.com/@typescript-eslint/visitor-keys/-/visitor-keys-5.27.1.tgz#05a62666f2a89769dac2e6baa48f74e8472983af" - integrity sha512-xYs6ffo01nhdJgPieyk7HAOpjhTsx7r/oB9LWEhwAXgwn33tkr+W8DI2ChboqhZlC4q3TC6geDYPoiX8ROqyOQ== - dependencies: - "@typescript-eslint/types" "5.27.1" - eslint-visitor-keys "^3.3.0" - -"@visx/curve@2.1.0": - version "2.1.0" - resolved "https://registry.yarnpkg.com/@visx/curve/-/curve-2.1.0.tgz#f614bfe3db66df7db7382db7a75ced1506b94602" - integrity sha512-9b6JOnx91gmOQiSPhUOxdsvcnW88fgqfTPKoVgQxidMsD/I3wksixtwo8TR/vtEz2aHzzsEEhlv1qK7Y3yaSDw== - dependencies: - "@types/d3-shape" "^1.3.1" - d3-shape "^1.0.6" - -"@visx/group@2.10.0", "@visx/group@^2.10.0": - version "2.10.0" - resolved "https://registry.yarnpkg.com/@visx/group/-/group-2.10.0.tgz#95839851832545621eb0d091866a61dafe552ae1" - integrity sha512-DNJDX71f65Et1+UgQvYlZbE66owYUAfcxTkC96Db6TnxV221VKI3T5l23UWbnMzwFBP9dR3PWUjjqhhF12N5pA== - dependencies: - "@types/react" "*" - classnames "^2.3.1" - prop-types "^15.6.2" - -"@visx/scale@2.2.2": - version "2.2.2" - resolved "https://registry.yarnpkg.com/@visx/scale/-/scale-2.2.2.tgz#b8eafabdcf92bb45ab196058fe184772ad80fd25" - integrity sha512-3aDySGUTpe6VykDQmF+g2nz5paFu9iSPTcCOEgkcru0/v5tmGzUdvivy8CkYbr87HN73V/Jc53lGm+kJUQcLBw== - dependencies: - "@types/d3-interpolate" "^1.3.1" - "@types/d3-scale" "^3.3.0" - "@types/d3-time" "^2.0.0" - d3-interpolate "^1.4.0" - d3-scale "^3.3.0" - d3-time "^2.1.1" - -"@visx/shape@^2.12.2": - version "2.12.2" - resolved "https://registry.yarnpkg.com/@visx/shape/-/shape-2.12.2.tgz#81ed88bf823aa84a4f5f32a9c9daf8371a606897" - integrity sha512-4gN0fyHWYXiJ+Ck8VAazXX0i8TOnLJvOc5jZBnaJDVxgnSIfCjJn0+Nsy96l9Dy/bCMTh4DBYUBv9k+YICBUOA== - dependencies: - "@types/d3-path" "^1.0.8" - "@types/d3-shape" "^1.3.1" - "@types/lodash" "^4.14.172" - "@types/react" "*" - "@visx/curve" "2.1.0" - "@visx/group" "2.10.0" - "@visx/scale" "2.2.2" - classnames "^2.3.1" - d3-path "^1.0.5" - d3-shape "^1.2.0" - lodash "^4.17.21" - prop-types "^15.5.10" - -"@webassemblyjs/ast@1.12.1", "@webassemblyjs/ast@^1.12.1": - version "1.12.1" - resolved "https://registry.yarnpkg.com/@webassemblyjs/ast/-/ast-1.12.1.tgz#bb16a0e8b1914f979f45864c23819cc3e3f0d4bb" - integrity sha512-EKfMUOPRRUTy5UII4qJDGPpqfwjOmZ5jeGFwid9mnoqIFK+e0vqoi1qH56JpmZSzEL53jKnNzScdmftJyG5xWg== - dependencies: - "@webassemblyjs/helper-numbers" "1.11.6" - "@webassemblyjs/helper-wasm-bytecode" "1.11.6" - -"@webassemblyjs/floating-point-hex-parser@1.11.6": - version "1.11.6" - resolved "https://registry.yarnpkg.com/@webassemblyjs/floating-point-hex-parser/-/floating-point-hex-parser-1.11.6.tgz#dacbcb95aff135c8260f77fa3b4c5fea600a6431" - integrity sha512-ejAj9hfRJ2XMsNHk/v6Fu2dGS+i4UaXBXGemOfQ/JfQ6mdQg/WXtwleQRLLS4OvfDhv8rYnVwH27YJLMyYsxhw== - -"@webassemblyjs/helper-api-error@1.11.6": - version "1.11.6" - resolved "https://registry.yarnpkg.com/@webassemblyjs/helper-api-error/-/helper-api-error-1.11.6.tgz#6132f68c4acd59dcd141c44b18cbebbd9f2fa768" - integrity sha512-o0YkoP4pVu4rN8aTJgAyj9hC2Sv5UlkzCHhxqWj8butaLvnpdc2jOwh4ewE6CX0txSfLn/UYaV/pheS2Txg//Q== - -"@webassemblyjs/helper-buffer@1.12.1": - version "1.12.1" - resolved "https://registry.yarnpkg.com/@webassemblyjs/helper-buffer/-/helper-buffer-1.12.1.tgz#6df20d272ea5439bf20ab3492b7fb70e9bfcb3f6" - integrity sha512-nzJwQw99DNDKr9BVCOZcLuJJUlqkJh+kVzVl6Fmq/tI5ZtEyWT1KZMyOXltXLZJmDtvLCDgwsyrkohEtopTXCw== - -"@webassemblyjs/helper-numbers@1.11.6": - version "1.11.6" - resolved "https://registry.yarnpkg.com/@webassemblyjs/helper-numbers/-/helper-numbers-1.11.6.tgz#cbce5e7e0c1bd32cf4905ae444ef64cea919f1b5" - integrity sha512-vUIhZ8LZoIWHBohiEObxVm6hwP034jwmc9kuq5GdHZH0wiLVLIPcMCdpJzG4C11cHoQ25TFIQj9kaVADVX7N3g== - dependencies: - "@webassemblyjs/floating-point-hex-parser" "1.11.6" - "@webassemblyjs/helper-api-error" "1.11.6" - "@xtuc/long" "4.2.2" - -"@webassemblyjs/helper-wasm-bytecode@1.11.6": - version "1.11.6" - resolved "https://registry.yarnpkg.com/@webassemblyjs/helper-wasm-bytecode/-/helper-wasm-bytecode-1.11.6.tgz#bb2ebdb3b83aa26d9baad4c46d4315283acd51e9" - integrity sha512-sFFHKwcmBprO9e7Icf0+gddyWYDViL8bpPjJJl0WHxCdETktXdmtWLGVzoHbqUcY4Be1LkNfwTmXOJUFZYSJdA== - -"@webassemblyjs/helper-wasm-section@1.12.1": - version "1.12.1" - resolved "https://registry.yarnpkg.com/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.12.1.tgz#3da623233ae1a60409b509a52ade9bc22a37f7bf" - integrity sha512-Jif4vfB6FJlUlSbgEMHUyk1j234GTNG9dBJ4XJdOySoj518Xj0oGsNi59cUQF4RRMS9ouBUxDDdyBVfPTypa5g== - dependencies: - "@webassemblyjs/ast" "1.12.1" - "@webassemblyjs/helper-buffer" "1.12.1" - "@webassemblyjs/helper-wasm-bytecode" "1.11.6" - "@webassemblyjs/wasm-gen" "1.12.1" - -"@webassemblyjs/ieee754@1.11.6": - version "1.11.6" - resolved "https://registry.yarnpkg.com/@webassemblyjs/ieee754/-/ieee754-1.11.6.tgz#bb665c91d0b14fffceb0e38298c329af043c6e3a" - integrity sha512-LM4p2csPNvbij6U1f19v6WR56QZ8JcHg3QIJTlSwzFcmx6WSORicYj6I63f9yU1kEUtrpG+kjkiIAkevHpDXrg== - dependencies: - "@xtuc/ieee754" "^1.2.0" - -"@webassemblyjs/leb128@1.11.6": - version "1.11.6" - resolved "https://registry.yarnpkg.com/@webassemblyjs/leb128/-/leb128-1.11.6.tgz#70e60e5e82f9ac81118bc25381a0b283893240d7" - integrity sha512-m7a0FhE67DQXgouf1tbN5XQcdWoNgaAuoULHIfGFIEVKA6tu/edls6XnIlkmS6FrXAquJRPni3ZZKjw6FSPjPQ== - dependencies: - "@xtuc/long" "4.2.2" - -"@webassemblyjs/utf8@1.11.6": - version "1.11.6" - resolved "https://registry.yarnpkg.com/@webassemblyjs/utf8/-/utf8-1.11.6.tgz#90f8bc34c561595fe156603be7253cdbcd0fab5a" - integrity sha512-vtXf2wTQ3+up9Zsg8sa2yWiQpzSsMyXj0qViVP6xKGCUT8p8YJ6HqI7l5eCnWx1T/FYdsv07HQs2wTFbbof/RA== - -"@webassemblyjs/wasm-edit@^1.12.1": - version "1.12.1" - resolved "https://registry.yarnpkg.com/@webassemblyjs/wasm-edit/-/wasm-edit-1.12.1.tgz#9f9f3ff52a14c980939be0ef9d5df9ebc678ae3b" - integrity sha512-1DuwbVvADvS5mGnXbE+c9NfA8QRcZ6iKquqjjmR10k6o+zzsRVesil54DKexiowcFCPdr/Q0qaMgB01+SQ1u6g== - dependencies: - "@webassemblyjs/ast" "1.12.1" - "@webassemblyjs/helper-buffer" "1.12.1" - "@webassemblyjs/helper-wasm-bytecode" "1.11.6" - "@webassemblyjs/helper-wasm-section" "1.12.1" - "@webassemblyjs/wasm-gen" "1.12.1" - "@webassemblyjs/wasm-opt" "1.12.1" - "@webassemblyjs/wasm-parser" "1.12.1" - "@webassemblyjs/wast-printer" "1.12.1" - -"@webassemblyjs/wasm-gen@1.12.1": - version "1.12.1" - resolved "https://registry.yarnpkg.com/@webassemblyjs/wasm-gen/-/wasm-gen-1.12.1.tgz#a6520601da1b5700448273666a71ad0a45d78547" - integrity sha512-TDq4Ojh9fcohAw6OIMXqiIcTq5KUXTGRkVxbSo1hQnSy6lAM5GSdfwWeSxpAo0YzgsgF182E/U0mDNhuA0tW7w== - dependencies: - "@webassemblyjs/ast" "1.12.1" - "@webassemblyjs/helper-wasm-bytecode" "1.11.6" - "@webassemblyjs/ieee754" "1.11.6" - "@webassemblyjs/leb128" "1.11.6" - "@webassemblyjs/utf8" "1.11.6" - -"@webassemblyjs/wasm-opt@1.12.1": - version "1.12.1" - resolved "https://registry.yarnpkg.com/@webassemblyjs/wasm-opt/-/wasm-opt-1.12.1.tgz#9e6e81475dfcfb62dab574ac2dda38226c232bc5" - integrity sha512-Jg99j/2gG2iaz3hijw857AVYekZe2SAskcqlWIZXjji5WStnOpVoat3gQfT/Q5tb2djnCjBtMocY/Su1GfxPBg== - dependencies: - "@webassemblyjs/ast" "1.12.1" - "@webassemblyjs/helper-buffer" "1.12.1" - "@webassemblyjs/wasm-gen" "1.12.1" - "@webassemblyjs/wasm-parser" "1.12.1" - -"@webassemblyjs/wasm-parser@1.12.1", "@webassemblyjs/wasm-parser@^1.12.1": - version "1.12.1" - resolved "https://registry.yarnpkg.com/@webassemblyjs/wasm-parser/-/wasm-parser-1.12.1.tgz#c47acb90e6f083391e3fa61d113650eea1e95937" - integrity sha512-xikIi7c2FHXysxXe3COrVUPSheuBtpcfhbpFj4gmu7KRLYOzANztwUU0IbsqvMqzuNK2+glRGWCEqZo1WCLyAQ== - dependencies: - "@webassemblyjs/ast" "1.12.1" - "@webassemblyjs/helper-api-error" "1.11.6" - "@webassemblyjs/helper-wasm-bytecode" "1.11.6" - "@webassemblyjs/ieee754" "1.11.6" - "@webassemblyjs/leb128" "1.11.6" - "@webassemblyjs/utf8" "1.11.6" - -"@webassemblyjs/wast-printer@1.12.1": - version "1.12.1" - resolved "https://registry.yarnpkg.com/@webassemblyjs/wast-printer/-/wast-printer-1.12.1.tgz#bcecf661d7d1abdaf989d8341a4833e33e2b31ac" - integrity sha512-+X4WAlOisVWQMikjbcvY2e0rwPsKQ9F688lksZhBcPycBBuii3O7m8FACbDMWDojpAqvjIncrG8J0XHKyQfVeA== - dependencies: - "@webassemblyjs/ast" "1.12.1" - "@xtuc/long" "4.2.2" - -"@webpack-cli/configtest@^1.2.0": - version "1.2.0" - resolved "https://registry.yarnpkg.com/@webpack-cli/configtest/-/configtest-1.2.0.tgz#7b20ce1c12533912c3b217ea68262365fa29a6f5" - integrity sha512-4FB8Tj6xyVkyqjj1OaTqCjXYULB9FMkqQ8yGrZjRDrYh0nOE+7Lhs45WioWQQMV+ceFlE368Ukhe6xdvJM9Egg== - -"@webpack-cli/info@^1.5.0": - version "1.5.0" - resolved "https://registry.yarnpkg.com/@webpack-cli/info/-/info-1.5.0.tgz#6c78c13c5874852d6e2dd17f08a41f3fe4c261b1" - integrity sha512-e8tSXZpw2hPl2uMJY6fsMswaok5FdlGNRTktvFk2sD8RjH0hE2+XistawJx1vmKteh4NmGmNUrp+Tb2w+udPcQ== - dependencies: - envinfo "^7.7.3" - -"@webpack-cli/serve@^1.7.0": - version "1.7.0" - resolved "https://registry.yarnpkg.com/@webpack-cli/serve/-/serve-1.7.0.tgz#e1993689ac42d2b16e9194376cfb6753f6254db1" - integrity sha512-oxnCNGj88fL+xzV+dacXs44HcDwf1ovs3AuEzvP7mqXw7fQntqIhQ1BRmynh4qEKQSSSRSWVyXRjmTbZIX9V2Q== - -"@xtuc/ieee754@^1.2.0": - version "1.2.0" - resolved "https://registry.yarnpkg.com/@xtuc/ieee754/-/ieee754-1.2.0.tgz#eef014a3145ae477a1cbc00cd1e552336dceb790" - integrity sha512-DX8nKgqcGwsc0eJSqYt5lwP4DH5FlHnmuWWBRy7X0NcaGR0ZtuyeESgMwTYVEtxmsNGY+qit4QYT/MIYTOTPeA== - -"@xtuc/long@4.2.2": - version "4.2.2" - resolved "https://registry.yarnpkg.com/@xtuc/long/-/long-4.2.2.tgz#d291c6a4e97989b5c61d9acf396ae4fe133a718d" - integrity sha512-NuHqBY1PB/D8xU6s/thBgOAiAP7HOYDQ32+BFZILJ8ivkUkAHQnWfn6WhL79Owj1qmUnoN/YPhktdIoucipkAQ== - -"@zag-js/element-size@0.1.0": - version "0.1.0" - resolved "https://registry.yarnpkg.com/@zag-js/element-size/-/element-size-0.1.0.tgz#dfdb3f66a70328d0c3149aae29b8f99c10590c22" - integrity sha512-QF8wp0+V8++z+FHXiIw93+zudtubYszOtYbNgK39fg3pi+nCZtuSm4L1jC5QZMatNZ83MfOzyNCfgUubapagJQ== - -"@zag-js/focus-visible@0.1.0": - version "0.1.0" - resolved "https://registry.yarnpkg.com/@zag-js/focus-visible/-/focus-visible-0.1.0.tgz#9777bbaff8316d0b3a14a9095631e1494f69dbc7" - integrity sha512-PeaBcTmdZWcFf7n1aM+oiOdZc+sy14qi0emPIeUuGMTjbP0xLGrZu43kdpHnWSXy7/r4Ubp/vlg50MCV8+9Isg== - -abab@^2.0.3, abab@^2.0.5: - version "2.0.5" - resolved "https://registry.yarnpkg.com/abab/-/abab-2.0.5.tgz#c0b678fb32d60fc1219c784d6a826fe385aeb79a" - integrity sha512-9IK9EadsbHo6jLWIpxpR6pL0sazTXV6+SQv25ZB+F7Bj9mJNaOc4nCRabwd5M/JwmUa8idz6Eci6eKfJryPs6Q== - -acorn-globals@^6.0.0: - version "6.0.0" - resolved "https://registry.yarnpkg.com/acorn-globals/-/acorn-globals-6.0.0.tgz#46cdd39f0f8ff08a876619b55f5ac8a6dc770b45" - integrity sha512-ZQl7LOWaF5ePqqcX4hLuv/bLXYQNfNWw2c0/yX/TsPRKamzHcTGQnlCjHT3TsmkOUVEPS3crCxiPfdzE/Trlhg== - dependencies: - acorn "^7.1.1" - acorn-walk "^7.1.1" - -acorn-import-attributes@^1.9.5: - version "1.9.5" - resolved "https://registry.yarnpkg.com/acorn-import-attributes/-/acorn-import-attributes-1.9.5.tgz#7eb1557b1ba05ef18b5ed0ec67591bfab04688ef" - integrity sha512-n02Vykv5uA3eHGM/Z2dQrcD56kL8TyDb2p1+0P83PClMnC/nc+anbQRhIOWnSq4Ke/KvDPrY3C9hDtC/A3eHnQ== - -acorn-jsx@^5.3.2: - version "5.3.2" - resolved "https://registry.yarnpkg.com/acorn-jsx/-/acorn-jsx-5.3.2.tgz#7ed5bb55908b3b2f1bc55c6af1653bada7f07937" - integrity sha512-rq9s+JNhf0IChjtDXxllJ7g41oZk5SlXtp0LHwyA5cejwn7vKmKp4pPri6YEePv2PU65sAsegbXtIinmDFDXgQ== - -acorn-walk@^7.1.1: - version "7.2.0" - resolved "https://registry.yarnpkg.com/acorn-walk/-/acorn-walk-7.2.0.tgz#0de889a601203909b0fbe07b8938dc21d2e967bc" - integrity sha512-OPdCF6GsMIP+Az+aWfAAOEt2/+iVDKE7oy6lJ098aoe59oAmK76qV6Gw60SbZ8jHuG2wH058GF4pLFbYamYrVA== - -acorn@^7.1.1: - version "7.4.1" - resolved "https://registry.yarnpkg.com/acorn/-/acorn-7.4.1.tgz#feaed255973d2e77555b83dbc08851a6c63520fa" - integrity sha512-nQyp0o1/mNdbTO1PO6kHkwSrmgZ0MT/jCCpNiwbUjGoRN4dlBhqJtoQuCnEOKzgTVwg0ZWiCoQy6SxMebQVh8A== - -acorn@^8.2.4, acorn@^8.5.0, acorn@^8.7.1, acorn@^8.8.2: - version "8.12.1" - resolved "https://registry.yarnpkg.com/acorn/-/acorn-8.12.1.tgz#71616bdccbe25e27a54439e0046e89ca76df2248" - integrity sha512-tcpGyI9zbizT9JbV6oYE477V6mTlXvvi0T0G3SNIYE2apm/G5huBa1+K89VGeovbg+jycCrfhl3ADxErOuO6Jg== - -agent-base@6: - version "6.0.2" - resolved "https://registry.yarnpkg.com/agent-base/-/agent-base-6.0.2.tgz#49fff58577cfee3f37176feab4c22e00f86d7f77" - integrity sha512-RZNwNclF7+MS/8bDg70amg32dyeZGZxiDuQmZxKLAlQjr3jGyLx+4Kkk58UO7D2QdgFIQCovuSuZESne6RG6XQ== - dependencies: - debug "4" - -aggregate-error@^3.0.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/aggregate-error/-/aggregate-error-3.1.0.tgz#92670ff50f5359bdb7a3e0d40d0ec30c5737687a" - integrity sha512-4I7Td01quW/RpocfNayFdFVk1qSuoh0E7JrbRJ16nH01HhKFQ88INq9Sd+nd72zqRySlr9BmDA8xlEJ6vJMrYA== - dependencies: - clean-stack "^2.0.0" - indent-string "^4.0.0" - -ajv-formats@^2.1.1: - version "2.1.1" - resolved "https://registry.yarnpkg.com/ajv-formats/-/ajv-formats-2.1.1.tgz#6e669400659eb74973bbf2e33327180a0996b520" - integrity sha512-Wx0Kx52hxE7C18hkMEggYlEifqWZtYaRgouJor+WMdPnQyEK13vgEWyVNup7SoeeoLMsr4kf5h6dOW11I15MUA== - dependencies: - ajv "^8.0.0" - -ajv-keywords@^3.5.2: - version "3.5.2" - resolved "https://registry.yarnpkg.com/ajv-keywords/-/ajv-keywords-3.5.2.tgz#31f29da5ab6e00d1c2d329acf7b5929614d5014d" - integrity sha512-5p6WTN0DdTGVQk6VjcEju19IgaHudalcfabD7yhDGeA6bcQnmL+CpveLJq/3hvfwd1aof6L386Ougkx6RfyMIQ== - -ajv-keywords@^5.0.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/ajv-keywords/-/ajv-keywords-5.1.0.tgz#69d4d385a4733cdbeab44964a1170a88f87f0e16" - integrity sha512-YCS/JNFAUyr5vAuhk1DWm1CBxRHW9LbJ2ozWeemrIqpbsqKjHVxYPyi5GC0rjZIT5JxJ3virVTS8wk4i/Z+krw== - dependencies: - fast-deep-equal "^3.1.3" - -ajv@^6.10.0, ajv@^6.12.4, ajv@^6.12.5: - version "6.12.6" - resolved "https://registry.yarnpkg.com/ajv/-/ajv-6.12.6.tgz#baf5a62e802b07d977034586f8c3baf5adf26df4" - integrity sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g== - dependencies: - fast-deep-equal "^3.1.1" - fast-json-stable-stringify "^2.0.0" - json-schema-traverse "^0.4.1" - uri-js "^4.2.2" - -ajv@^8.0.0, ajv@^8.8.0: - version "8.11.0" - resolved "https://registry.yarnpkg.com/ajv/-/ajv-8.11.0.tgz#977e91dd96ca669f54a11e23e378e33b884a565f" - integrity sha512-wGgprdCvMalC0BztXvitD2hC04YffAvtsUn93JbGXYLAtCUO4xd17mCCZQxUOItiBwZvJScWo8NIvQMQ71rdpg== - dependencies: - fast-deep-equal "^3.1.1" - json-schema-traverse "^1.0.0" - require-from-string "^2.0.2" - uri-js "^4.2.2" - -ajv@^8.0.1: - version "8.6.0" - resolved "https://registry.yarnpkg.com/ajv/-/ajv-8.6.0.tgz#60cc45d9c46a477d80d92c48076d972c342e5720" - integrity sha512-cnUG4NSBiM4YFBxgZIj/In3/6KX+rQ2l2YPRVcvAMQGWEPKuXoPIhxzwqh31jA3IPbI4qEOp/5ILI4ynioXsGQ== - dependencies: - fast-deep-equal "^3.1.1" - json-schema-traverse "^1.0.0" - require-from-string "^2.0.2" - uri-js "^4.2.2" - -ansi-escapes@^4.2.1: - version "4.3.2" - resolved "https://registry.yarnpkg.com/ansi-escapes/-/ansi-escapes-4.3.2.tgz#6b2291d1db7d98b6521d5f1efa42d0f3a9feb65e" - integrity sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ== - dependencies: - type-fest "^0.21.3" - -ansi-regex@^5.0.1: - version "5.0.1" - resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-5.0.1.tgz#082cb2c89c9fe8659a311a53bd6a4dc5301db304" - integrity sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ== - -ansi-styles@^3.2.1: - version "3.2.1" - resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-3.2.1.tgz#41fbb20243e50b12be0f04b8dedbf07520ce841d" - integrity sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA== - dependencies: - color-convert "^1.9.0" - -ansi-styles@^4.0.0, ansi-styles@^4.1.0: - version "4.3.0" - resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-4.3.0.tgz#edd803628ae71c04c85ae7a0906edad34b648937" - integrity sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg== - dependencies: - color-convert "^2.0.1" - -ansi-styles@^5.0.0: - version "5.2.0" - resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-5.2.0.tgz#07449690ad45777d1924ac2abb2fc8895dba836b" - integrity sha512-Cxwpt2SfTzTtXcfOlzGEee8O+c+MmUgGrNiBcXnuWxuFJHe6a5Hz7qwhwe5OgaSYI0IJvkLqWX1ASG+cJOkEiA== - -ansi_up@^6.0.2: - version "6.0.2" - resolved "https://registry.yarnpkg.com/ansi_up/-/ansi_up-6.0.2.tgz#083adb65be5b21ba283fd105d3102e64f3f0b092" - integrity sha512-3G3vKvl1ilEp7J1u6BmULpMA0xVoW/f4Ekqhl8RTrJrhEBkonKn5k3bUc5Xt+qDayA6iDX0jyUh3AbZjB/l0tw== - -anymatch@^3.0.0, anymatch@^3.0.3: - version "3.1.2" - resolved "https://registry.yarnpkg.com/anymatch/-/anymatch-3.1.2.tgz#c0557c096af32f106198f4f4e2a383537e378716" - integrity sha512-P43ePfOAIupkguHUycrc4qJ9kz8ZiuOUijaETwX7THt0Y/GNK7v0aa8rY816xWjZ7rJdA5XdMcpVFTKMq+RvWg== - dependencies: - normalize-path "^3.0.0" - picomatch "^2.0.4" - -argparse@^1.0.7: - version "1.0.10" - resolved "https://registry.yarnpkg.com/argparse/-/argparse-1.0.10.tgz#bcd6791ea5ae09725e17e5ad988134cd40b3d911" - integrity sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg== - dependencies: - sprintf-js "~1.0.2" - -argparse@^2.0.1: - version "2.0.1" - resolved "https://registry.yarnpkg.com/argparse/-/argparse-2.0.1.tgz#246f50f3ca78a3240f6c997e8a9bd1eac49e4b38" - integrity sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q== - -aria-hidden@^1.1.1: - version "1.2.3" - resolved "https://registry.yarnpkg.com/aria-hidden/-/aria-hidden-1.2.3.tgz#14aeb7fb692bbb72d69bebfa47279c1fd725e954" - integrity sha512-xcLxITLe2HYa1cnYnwCjkOO1PqUHQpozB8x9AR0OgWN2woOBi5kSDVxKfd0b7sb1hw5qFeJhXm9H1nu3xSfLeQ== - dependencies: - tslib "^2.0.0" - -aria-query@^4.2.2: - version "4.2.2" - resolved "https://registry.yarnpkg.com/aria-query/-/aria-query-4.2.2.tgz#0d2ca6c9aceb56b8977e9fed6aed7e15bbd2f83b" - integrity sha512-o/HelwhuKpTj/frsOsbNLNgnNGVIFsVP/SW2BSF14gVl7kAfMOJ6/8wUAUvG1R1NHKrfG+2sHZTu0yauT1qBrA== - dependencies: - "@babel/runtime" "^7.10.2" - "@babel/runtime-corejs3" "^7.10.2" - -aria-query@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/aria-query/-/aria-query-5.0.0.tgz#210c21aaf469613ee8c9a62c7f86525e058db52c" - integrity sha512-V+SM7AbUwJ+EBnB8+DXs0hPZHO0W6pqBcc0dW90OwtVG02PswOu/teuARoLQjdDOH+t9pJgGnW5/Qmouf3gPJg== - -array-buffer-byte-length@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/array-buffer-byte-length/-/array-buffer-byte-length-1.0.0.tgz#fabe8bc193fea865f317fe7807085ee0dee5aead" - integrity sha512-LPuwb2P+NrQw3XhxGc36+XSvuBPopovXYTR9Ew++Du9Yb/bx5AzBfrIsBoj0EZUifjQU+sHL21sseZ3jerWO/A== - dependencies: - call-bind "^1.0.2" - is-array-buffer "^3.0.1" - -array-includes@^3.1.3: - version "3.1.3" - resolved "https://registry.yarnpkg.com/array-includes/-/array-includes-3.1.3.tgz#c7f619b382ad2afaf5326cddfdc0afc61af7690a" - integrity sha512-gcem1KlBU7c9rB+Rq8/3PPKsK2kjqeEBa3bD5kkQo4nYlOHQCJqIJFqBXDEfwaRuYTT4E+FxA9xez7Gf/e3Q7A== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.3" - es-abstract "^1.18.0-next.2" - get-intrinsic "^1.1.1" - is-string "^1.0.5" - -array-includes@^3.1.4, array-includes@^3.1.5: - version "3.1.5" - resolved "https://registry.yarnpkg.com/array-includes/-/array-includes-3.1.5.tgz#2c320010db8d31031fd2a5f6b3bbd4b1aad31bdb" - integrity sha512-iSDYZMMyTPkiFasVqfuAQnWAYcvO/SeBSCGKePoEthjp4LEMTe4uLc7b025o4jAZpHhihh8xPo99TNWUWWkGDQ== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - es-abstract "^1.19.5" - get-intrinsic "^1.1.1" - is-string "^1.0.7" - -array-includes@^3.1.6: - version "3.1.6" - resolved "https://registry.yarnpkg.com/array-includes/-/array-includes-3.1.6.tgz#9e9e720e194f198266ba9e18c29e6a9b0e4b225f" - integrity sha512-sgTbLvL6cNnw24FnbaDyjmvddQ2ML8arZsgaJhoABMoplz/4QRhtrYS+alr1BUM1Bwp6dhx8vVCBSLG+StwOFw== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - es-abstract "^1.20.4" - get-intrinsic "^1.1.3" - is-string "^1.0.7" - -array-union@^1.0.1: - version "1.0.2" - resolved "https://registry.yarnpkg.com/array-union/-/array-union-1.0.2.tgz#9a34410e4f4e3da23dea375be5be70f24778ec39" - integrity sha1-mjRBDk9OPaI96jdb5b5w8kd47Dk= - dependencies: - array-uniq "^1.0.1" - -array-union@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/array-union/-/array-union-2.1.0.tgz#b798420adbeb1de828d84acd8a2e23d3efe85e8d" - integrity sha512-HGyxoOTYUyCM6stUe6EJgnd4EoewAI7zMdfqO+kGjnlZmBDz/cR5pf8r/cR4Wq60sL/p0IkcjUEEPwS3GFrIyw== - -array-uniq@^1.0.1: - version "1.0.3" - resolved "https://registry.yarnpkg.com/array-uniq/-/array-uniq-1.0.3.tgz#af6ac877a25cc7f74e058894753858dfdb24fdb6" - integrity sha1-r2rId6Jcx/dOBYiUdThY39sk/bY= - -array.prototype.flat@^1.3.1: - version "1.3.1" - resolved "https://registry.yarnpkg.com/array.prototype.flat/-/array.prototype.flat-1.3.1.tgz#ffc6576a7ca3efc2f46a143b9d1dda9b4b3cf5e2" - integrity sha512-roTU0KWIOmJ4DRLmwKd19Otg0/mT3qPNt0Qb3GWW8iObuZXxrjB/pzn0R3hqpRSWg4HCwqx+0vwOnWnvlOyeIA== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - es-abstract "^1.20.4" - es-shim-unscopables "^1.0.0" - -array.prototype.flatmap@^1.3.0: - version "1.3.0" - resolved "https://registry.yarnpkg.com/array.prototype.flatmap/-/array.prototype.flatmap-1.3.0.tgz#a7e8ed4225f4788a70cd910abcf0791e76a5534f" - integrity sha512-PZC9/8TKAIxcWKdyeb77EzULHPrIX/tIZebLJUQOMR1OwYosT8yggdfWScfTBCDj5utONvOuPQQumYsU2ULbkg== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.3" - es-abstract "^1.19.2" - es-shim-unscopables "^1.0.0" - -array.prototype.flatmap@^1.3.1: - version "1.3.1" - resolved "https://registry.yarnpkg.com/array.prototype.flatmap/-/array.prototype.flatmap-1.3.1.tgz#1aae7903c2100433cb8261cd4ed310aab5c4a183" - integrity sha512-8UGn9O1FDVvMNB0UlLv4voxRMze7+FpHyF5mSMRjWHUMlpoDViniy05870VlxhfgTnLbpuwTzvD76MTtWxB/mQ== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - es-abstract "^1.20.4" - es-shim-unscopables "^1.0.0" - -arrify@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/arrify/-/arrify-1.0.1.tgz#898508da2226f380df904728456849c1501a4b0d" - integrity sha1-iYUI2iIm84DfkEcoRWhJwVAaSw0= - -asap@~2.0.3: - version "2.0.6" - resolved "https://registry.yarnpkg.com/asap/-/asap-2.0.6.tgz#e50347611d7e690943208bbdafebcbc2fb866d46" - integrity sha512-BSHWgDSAiKs50o2Re8ppvp3seVHXSRM44cdSsT9FfNEUUZLOGWVCsiWaRPWM1Znn+mqZ1OfVZ3z3DWEzSp7hRA== - -ast-types-flow@^0.0.7: - version "0.0.7" - resolved "https://registry.yarnpkg.com/ast-types-flow/-/ast-types-flow-0.0.7.tgz#f70b735c6bca1a5c9c22d982c3e39e7feba3bdad" - integrity sha1-9wtzXGvKGlycItmCw+Oef+ujva0= - -astral-regex@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/astral-regex/-/astral-regex-2.0.0.tgz#483143c567aeed4785759c0865786dc77d7d2e31" - integrity sha512-Z7tMw1ytTXt5jqMcOP+OQteU1VuNK9Y02uuJtKQ1Sv69jXQKKg5cibLwGJow8yzZP+eAc18EmLGPal0bp36rvQ== - -asynckit@^0.4.0: - version "0.4.0" - resolved "https://registry.yarnpkg.com/asynckit/-/asynckit-0.4.0.tgz#c79ed97f7f34cb8f2ba1bc9790bcc366474b4b79" - integrity sha1-x57Zf380y48robyXkLzDZkdLS3k= - -atob@^2.1.2: - version "2.1.2" - resolved "https://registry.yarnpkg.com/atob/-/atob-2.1.2.tgz#6d9517eb9e030d2436666651e86bd9f6f13533c9" - integrity sha512-Wm6ukoaOGJi/73p/cl2GvLjTI5JM1k/O14isD73YML8StrH/7/lRFgmg8nICZgD3bZZvjwCGxtMOD3wWNAu8cg== - -available-typed-arrays@^1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/available-typed-arrays/-/available-typed-arrays-1.0.5.tgz#92f95616501069d07d10edb2fc37d3e1c65123b7" - integrity sha512-DMD0KiN46eipeziST1LPP/STfDU0sufISXmjSgvVsoU2tqxctQeASejWcfNtxYKqETM1UxQ8sp2OrSBWpHY6sw== - -axe-core@^4.3.5: - version "4.4.2" - resolved "https://registry.yarnpkg.com/axe-core/-/axe-core-4.4.2.tgz#dcf7fb6dea866166c3eab33d68208afe4d5f670c" - integrity sha512-LVAaGp/wkkgYJcjmHsoKx4juT1aQvJyPcW09MLCjVTh3V2cc6PnyempiLMNH5iMdfIX/zdbjUx2KDjMLCTdPeA== - -axios@^1.7.4: - version "1.7.4" - resolved "https://registry.yarnpkg.com/axios/-/axios-1.7.4.tgz#4c8ded1b43683c8dd362973c393f3ede24052aa2" - integrity sha512-DukmaFRnY6AzAALSH4J2M3k6PkaC+MfaAGdEERRWcC9q3/TWQwLpHR8ZRLKTdQ3aBDL64EdluRDjJqKw+BPZEw== - dependencies: - follow-redirects "^1.15.6" - form-data "^4.0.0" - proxy-from-env "^1.1.0" - -axobject-query@^2.2.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/axobject-query/-/axobject-query-2.2.0.tgz#943d47e10c0b704aa42275e20edf3722648989be" - integrity sha512-Td525n+iPOOyUQIeBfcASuG6uJsDOITl7Mds5gFyerkWiX7qhUTdYUBlSgNMyVqtSJqwpt1kXGLdUt6SykLMRA== - -babel-jest@^27.3.1: - version "27.3.1" - resolved "https://registry.yarnpkg.com/babel-jest/-/babel-jest-27.3.1.tgz#0636a3404c68e07001e434ac4956d82da8a80022" - integrity sha512-SjIF8hh/ir0peae2D6S6ZKRhUy7q/DnpH7k/V6fT4Bgs/LXXUztOpX4G2tCgq8mLo5HA9mN6NmlFMeYtKmIsTQ== - dependencies: - "@jest/transform" "^27.3.1" - "@jest/types" "^27.2.5" - "@types/babel__core" "^7.1.14" - babel-plugin-istanbul "^6.0.0" - babel-preset-jest "^27.2.0" - chalk "^4.0.0" - graceful-fs "^4.2.4" - slash "^3.0.0" - -babel-jest@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/babel-jest/-/babel-jest-27.5.1.tgz#a1bf8d61928edfefd21da27eb86a695bfd691444" - integrity sha512-cdQ5dXjGRd0IBRATiQ4mZGlGlRE8kJpjPOixdNRdT+m3UcNqmYWN6rK6nvtXYfY3D76cb8s/O1Ss8ea24PIwcg== - dependencies: - "@jest/transform" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/babel__core" "^7.1.14" - babel-plugin-istanbul "^6.1.1" - babel-preset-jest "^27.5.1" - chalk "^4.0.0" - graceful-fs "^4.2.9" - slash "^3.0.0" - -babel-loader@^9.1.0: - version "9.1.2" - resolved "https://registry.yarnpkg.com/babel-loader/-/babel-loader-9.1.2.tgz#a16a080de52d08854ee14570469905a5fc00d39c" - integrity sha512-mN14niXW43tddohGl8HPu5yfQq70iUThvFL/4QzESA7GcZoC0eVOhvWdQ8+3UlSjaDE9MVtsW9mxDY07W7VpVA== - dependencies: - find-cache-dir "^3.3.2" - schema-utils "^4.0.0" - -babel-plugin-istanbul@^6.0.0, babel-plugin-istanbul@^6.1.1: - version "6.1.1" - resolved "https://registry.yarnpkg.com/babel-plugin-istanbul/-/babel-plugin-istanbul-6.1.1.tgz#fa88ec59232fd9b4e36dbbc540a8ec9a9b47da73" - integrity sha512-Y1IQok9821cC9onCx5otgFfRm7Lm+I+wwxOx738M/WLPZ9Q42m4IG5W0FNX8WLL2gYMZo3JkuXIH2DOpWM+qwA== - dependencies: - "@babel/helper-plugin-utils" "^7.0.0" - "@istanbuljs/load-nyc-config" "^1.0.0" - "@istanbuljs/schema" "^0.1.2" - istanbul-lib-instrument "^5.0.4" - test-exclude "^6.0.0" - -babel-plugin-jest-hoist@^27.2.0: - version "27.2.0" - resolved "https://registry.yarnpkg.com/babel-plugin-jest-hoist/-/babel-plugin-jest-hoist-27.2.0.tgz#79f37d43f7e5c4fdc4b2ca3e10cc6cf545626277" - integrity sha512-TOux9khNKdi64mW+0OIhcmbAn75tTlzKhxmiNXevQaPbrBYK7YKjP1jl6NHTJ6XR5UgUrJbCnWlKVnJn29dfjw== - dependencies: - "@babel/template" "^7.3.3" - "@babel/types" "^7.3.3" - "@types/babel__core" "^7.0.0" - "@types/babel__traverse" "^7.0.6" - -babel-plugin-jest-hoist@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/babel-plugin-jest-hoist/-/babel-plugin-jest-hoist-27.5.1.tgz#9be98ecf28c331eb9f5df9c72d6f89deb8181c2e" - integrity sha512-50wCwD5EMNW4aRpOwtqzyZHIewTYNxLA4nhB+09d8BIssfNfzBRhkBIHiaPv1Si226TQSvp8gxAJm2iY2qs2hQ== - dependencies: - "@babel/template" "^7.3.3" - "@babel/types" "^7.3.3" - "@types/babel__core" "^7.0.0" - "@types/babel__traverse" "^7.0.6" - -babel-plugin-macros@^2.6.1: - version "2.8.0" - resolved "https://registry.yarnpkg.com/babel-plugin-macros/-/babel-plugin-macros-2.8.0.tgz#0f958a7cc6556b1e65344465d99111a1e5e10138" - integrity sha512-SEP5kJpfGYqYKpBrj5XU3ahw5p5GOHJ0U5ssOSQ/WBVdwkD2Dzlce95exQTs3jOVWPPKLBN2rlEWkCK7dSmLvg== - dependencies: - "@babel/runtime" "^7.7.2" - cosmiconfig "^6.0.0" - resolve "^1.12.0" - -babel-plugin-polyfill-corejs2@^0.4.10: - version "0.4.11" - resolved "https://registry.yarnpkg.com/babel-plugin-polyfill-corejs2/-/babel-plugin-polyfill-corejs2-0.4.11.tgz#30320dfe3ffe1a336c15afdcdafd6fd615b25e33" - integrity sha512-sMEJ27L0gRHShOh5G54uAAPaiCOygY/5ratXuiyb2G46FmlSpc9eFCzYVyDiPxfNbwzA7mYahmjQc5q+CZQ09Q== - dependencies: - "@babel/compat-data" "^7.22.6" - "@babel/helper-define-polyfill-provider" "^0.6.2" - semver "^6.3.1" - -babel-plugin-polyfill-corejs3@^0.10.1, babel-plugin-polyfill-corejs3@^0.10.4: - version "0.10.4" - resolved "https://registry.yarnpkg.com/babel-plugin-polyfill-corejs3/-/babel-plugin-polyfill-corejs3-0.10.4.tgz#789ac82405ad664c20476d0233b485281deb9c77" - integrity sha512-25J6I8NGfa5YkCDogHRID3fVCadIR8/pGl1/spvCkzb6lVn6SR3ojpx9nOn9iEBcUsjY24AmdKm5khcfKdylcg== - dependencies: - "@babel/helper-define-polyfill-provider" "^0.6.1" - core-js-compat "^3.36.1" - -babel-plugin-polyfill-regenerator@^0.6.1: - version "0.6.2" - resolved "https://registry.yarnpkg.com/babel-plugin-polyfill-regenerator/-/babel-plugin-polyfill-regenerator-0.6.2.tgz#addc47e240edd1da1058ebda03021f382bba785e" - integrity sha512-2R25rQZWP63nGwaAswvDazbPXfrM3HwVoBXK6HcqeKrSrL/JqcC/rDcf95l4r7LXLyxDXc8uQDa064GubtCABg== - dependencies: - "@babel/helper-define-polyfill-provider" "^0.6.2" - -babel-preset-current-node-syntax@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/babel-preset-current-node-syntax/-/babel-preset-current-node-syntax-1.0.1.tgz#b4399239b89b2a011f9ddbe3e4f401fc40cff73b" - integrity sha512-M7LQ0bxarkxQoN+vz5aJPsLBn77n8QgTFmo8WK0/44auK2xlCXrYcUxHFxgU7qW5Yzw/CjmLRK2uJzaCd7LvqQ== - dependencies: - "@babel/plugin-syntax-async-generators" "^7.8.4" - "@babel/plugin-syntax-bigint" "^7.8.3" - "@babel/plugin-syntax-class-properties" "^7.8.3" - "@babel/plugin-syntax-import-meta" "^7.8.3" - "@babel/plugin-syntax-json-strings" "^7.8.3" - "@babel/plugin-syntax-logical-assignment-operators" "^7.8.3" - "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" - "@babel/plugin-syntax-numeric-separator" "^7.8.3" - "@babel/plugin-syntax-object-rest-spread" "^7.8.3" - "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" - "@babel/plugin-syntax-optional-chaining" "^7.8.3" - "@babel/plugin-syntax-top-level-await" "^7.8.3" - -babel-preset-jest@^27.2.0: - version "27.2.0" - resolved "https://registry.yarnpkg.com/babel-preset-jest/-/babel-preset-jest-27.2.0.tgz#556bbbf340608fed5670ab0ea0c8ef2449fba885" - integrity sha512-z7MgQ3peBwN5L5aCqBKnF6iqdlvZvFUQynEhu0J+X9nHLU72jO3iY331lcYrg+AssJ8q7xsv5/3AICzVmJ/wvg== - dependencies: - babel-plugin-jest-hoist "^27.2.0" - babel-preset-current-node-syntax "^1.0.0" - -babel-preset-jest@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/babel-preset-jest/-/babel-preset-jest-27.5.1.tgz#91f10f58034cb7989cb4f962b69fa6eef6a6bc81" - integrity sha512-Nptf2FzlPCWYuJg41HBqXVT8ym6bXOevuCTbhxlUpjwtysGaIWFvDEjp4y+G7fl13FgOdjs7P/DmErqH7da0Ag== - dependencies: - babel-plugin-jest-hoist "^27.5.1" - babel-preset-current-node-syntax "^1.0.0" - -bail@^2.0.0: - version "2.0.2" - resolved "https://registry.yarnpkg.com/bail/-/bail-2.0.2.tgz#d26f5cd8fe5d6f832a31517b9f7c356040ba6d5d" - integrity sha512-0xO6mYd7JB2YesxDKplafRpsiOzPt9V02ddPCLbY1xYGPOX24NTyN50qnUxgCPcSoYMhKpAuBTjQoRZCAkUDRw== - -balanced-match@^1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/balanced-match/-/balanced-match-1.0.2.tgz#e83e3a7e3f300b34cb9d87f615fa0cbf357690ee" - integrity sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw== - -balanced-match@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/balanced-match/-/balanced-match-2.0.0.tgz#dc70f920d78db8b858535795867bf48f820633d9" - integrity sha512-1ugUSr8BHXRnK23KfuYS+gVMC3LB8QGH9W1iGtDPsNWoQbgtXSExkBu2aDR4epiGWZOjZsj6lDl/N/AqqTC3UA== - -base16@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/base16/-/base16-1.0.0.tgz#e297f60d7ec1014a7a971a39ebc8a98c0b681e70" - integrity sha512-pNdYkNPiJUnEhnfXV56+sQy8+AaPcG3POZAUnwr4EeqCUZFz4u2PePbo3e5Gj4ziYPCWGUZT9RHisvJKnwFuBQ== - -big-integer@^1.6.16: - version "1.6.51" - resolved "https://registry.yarnpkg.com/big-integer/-/big-integer-1.6.51.tgz#0df92a5d9880560d3ff2d5fd20245c889d130686" - integrity sha512-GPEid2Y9QU1Exl1rpO9B2IPJGHPSupF5GnVIP0blYvNOMer2bTvSWs1jGOUg04hTmu67nmLsQ9TBo1puaotBHg== - -big.js@^5.2.2: - version "5.2.2" - resolved "https://registry.yarnpkg.com/big.js/-/big.js-5.2.2.tgz#65f0af382f578bcdc742bd9c281e9cb2d7768328" - integrity sha512-vyL2OymJxmarO8gxMr0mhChsO9QGwhynfuu4+MHTAW6czfq9humCB7rKpUjDd9YUiDPU4mzpyupFSvOClAwbmQ== - -boolbase@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/boolbase/-/boolbase-1.0.0.tgz#68dff5fbe60c51eb37725ea9e3ed310dcc1e776e" - integrity sha1-aN/1++YMUes3cl6p4+0xDcwed24= - -bootstrap@^3.3: - version "3.4.1" - resolved "https://registry.yarnpkg.com/bootstrap/-/bootstrap-3.4.1.tgz#c3a347d419e289ad11f4033e3c4132b87c081d72" - integrity sha512-yN5oZVmRCwe5aKwzRj6736nSmKDX7pLYwsXiCj/EYmo16hODaBiT4En5btW/jhBF/seV+XMx3aYwukYC3A49DA== - -brace-expansion@^1.1.7: - version "1.1.11" - resolved "https://registry.yarnpkg.com/brace-expansion/-/brace-expansion-1.1.11.tgz#3c7fcbf529d87226f3d2f52b966ff5271eb441dd" - integrity sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA== - dependencies: - balanced-match "^1.0.0" - concat-map "0.0.1" - -brace-expansion@^2.0.1: - version "2.0.1" - resolved "https://registry.yarnpkg.com/brace-expansion/-/brace-expansion-2.0.1.tgz#1edc459e0f0c548486ecf9fc99f2221364b9a0ae" - integrity sha512-XnAIvQ8eM+kC6aULx6wuQiwVsnzsi9d3WxzV3FpWTGA19F621kwdbsAcFKXgKUHZWsy+mY6iL1sHTxWEFCytDA== - dependencies: - balanced-match "^1.0.0" - -braces@^3.0.3: - version "3.0.3" - resolved "https://registry.yarnpkg.com/braces/-/braces-3.0.3.tgz#490332f40919452272d55a8480adc0c441358789" - integrity sha512-yQbXgO/OSZVD2IsiLlro+7Hf6Q18EJrKSEsdoMzKePKXct3gvD8oLcOQdIzGupr5Fj+EDe8gO/lxc1BzfMpxvA== - dependencies: - fill-range "^7.1.1" - -broadcast-channel@^3.4.1: - version "3.7.0" - resolved "https://registry.yarnpkg.com/broadcast-channel/-/broadcast-channel-3.7.0.tgz#2dfa5c7b4289547ac3f6705f9c00af8723889937" - integrity sha512-cIAKJXAxGJceNZGTZSBzMxzyOn72cVgPnKx4dc6LRjQgbaJUQqhy5rzL3zbMxkMWsGKkv2hSFkPRMEXfoMZ2Mg== - dependencies: - "@babel/runtime" "^7.7.2" - detect-node "^2.1.0" - js-sha3 "0.8.0" - microseconds "0.2.0" - nano-time "1.0.0" - oblivious-set "1.0.0" - rimraf "3.0.2" - unload "2.2.0" - -browser-process-hrtime@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/browser-process-hrtime/-/browser-process-hrtime-1.0.0.tgz#3c9b4b7d782c8121e56f10106d84c0d0ffc94626" - integrity sha512-9o5UecI3GhkpM6DrXr69PblIuWxPKk9Y0jHBRhdocZ2y7YECBFCsHm79Pr3OyR2AvjhDkabFJaDJMYRazHgsow== - -browserslist@^4.0.0, browserslist@^4.16.6, browserslist@^4.17.5, browserslist@^4.20.3, browserslist@^4.21.10, browserslist@^4.22.2, browserslist@^4.23.0: - version "4.23.3" - resolved "https://registry.yarnpkg.com/browserslist/-/browserslist-4.23.3.tgz#debb029d3c93ebc97ffbc8d9cbb03403e227c800" - integrity sha512-btwCFJVjI4YWDNfau8RhZ+B1Q/VLoUITrm3RlP6y1tYGWIOa+InuYiRGXUBXo8nA1qKmHMyLB/iVQg5TT4eFoA== - dependencies: - caniuse-lite "^1.0.30001646" - electron-to-chromium "^1.5.4" - node-releases "^2.0.18" - update-browserslist-db "^1.1.0" - -bser@2.1.1: - version "2.1.1" - resolved "https://registry.yarnpkg.com/bser/-/bser-2.1.1.tgz#e6787da20ece9d07998533cfd9de6f5c38f4bc05" - integrity sha512-gQxTNE/GAfIIrmHLUE3oJyp5FO6HRBfhjnw4/wMmA63ZGDJnWBmgY/lyQBpnDUkGmAhbSe39tx2d/iTOAfglwQ== - dependencies: - node-int64 "^0.4.0" - -buffer-from@^1.0.0: - version "1.1.2" - resolved "https://registry.yarnpkg.com/buffer-from/-/buffer-from-1.1.2.tgz#2b146a6fd72e80b4f55d255f35ed59a3a9a41bd5" - integrity sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ== - -cacache@^15.0.5: - version "15.2.0" - resolved "https://registry.yarnpkg.com/cacache/-/cacache-15.2.0.tgz#73af75f77c58e72d8c630a7a2858cb18ef523389" - integrity sha512-uKoJSHmnrqXgthDFx/IU6ED/5xd+NNGe+Bb+kLZy7Ku4P+BaiWEUflAKPZ7eAzsYGcsAGASJZsybXp+quEcHTw== - dependencies: - "@npmcli/move-file" "^1.0.1" - chownr "^2.0.0" - fs-minipass "^2.0.0" - glob "^7.1.4" - infer-owner "^1.0.4" - lru-cache "^6.0.0" - minipass "^3.1.1" - minipass-collect "^1.0.2" - minipass-flush "^1.0.5" - minipass-pipeline "^1.2.2" - mkdirp "^1.0.3" - p-map "^4.0.0" - promise-inflight "^1.0.1" - rimraf "^3.0.2" - ssri "^8.0.1" - tar "^6.0.2" - unique-filename "^1.1.1" - -call-bind@^1.0.0, call-bind@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/call-bind/-/call-bind-1.0.2.tgz#b1d4e89e688119c3c9a903ad30abb2f6a919be3c" - integrity sha512-7O+FbCihrB5WGbFYesctwmTKae6rOiIzmz1icreWJ+0aA7LJfuqhEso2T9ncpcFtzMQtzXf2QGGueWJGTYsqrA== - dependencies: - function-bind "^1.1.1" - get-intrinsic "^1.0.2" - -call-me-maybe@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/call-me-maybe/-/call-me-maybe-1.0.1.tgz#26d208ea89e37b5cbde60250a15f031c16a4d66b" - integrity sha1-JtII6onje1y95gJQoV8DHBak1ms= - -callsites@^3.0.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/callsites/-/callsites-3.1.0.tgz#b3630abd8943432f54b3f0519238e33cd7df2f73" - integrity sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ== - -camelcase-keys@^7.0.0: - version "7.0.0" - resolved "https://registry.yarnpkg.com/camelcase-keys/-/camelcase-keys-7.0.0.tgz#40fcbe171f7432888369d0c871df7cfa5ce4f788" - integrity sha512-qlQlECgDl5Ev+gkvONaiD4X4TF2gyZKuLBvzx0zLo2UwAxmz3hJP/841aaMHTeH1T7v5HRwoRq91daulXoYWvg== - dependencies: - camelcase "^6.2.0" - map-obj "^4.1.0" - quick-lru "^5.1.1" - type-fest "^1.2.1" - -camelcase@^5.3.1: - version "5.3.1" - resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-5.3.1.tgz#e3c9b31569e106811df242f715725a1f4c494320" - integrity sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg== - -camelcase@^6.2.0: - version "6.2.0" - resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-6.2.0.tgz#924af881c9d525ac9d87f40d964e5cea982a1809" - integrity sha512-c7wVvbw3f37nuobQNtgsgG9POC9qMbNuMQmTCqZv23b6MIz0fcYpBiOlv9gEN/hdLdnZTDQhg6e9Dq5M1vKvfg== - -caniuse-api@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/caniuse-api/-/caniuse-api-3.0.0.tgz#5e4d90e2274961d46291997df599e3ed008ee4c0" - integrity sha512-bsTwuIg/BZZK/vreVTYYbSWoe2F+71P7K5QGEX+pT250DZbfU1MQ5prOKpPR+LL6uWKK3KMwMCAS74QB3Um1uw== - dependencies: - browserslist "^4.0.0" - caniuse-lite "^1.0.0" - lodash.memoize "^4.1.2" - lodash.uniq "^4.5.0" - -caniuse-lite@^1.0.0: - version "1.0.30001589" - resolved "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001589.tgz" - integrity sha512-vNQWS6kI+q6sBlHbh71IIeC+sRwK2N3EDySc/updIGhIee2x5z00J4c1242/5/d6EpEMdOnk/m+6tuk4/tcsqg== - -caniuse-lite@^1.0.30001646: - version "1.0.30001653" - resolved "https://registry.yarnpkg.com/caniuse-lite/-/caniuse-lite-1.0.30001653.tgz#b8af452f8f33b1c77f122780a4aecebea0caca56" - integrity sha512-XGWQVB8wFQ2+9NZwZ10GxTYC5hk0Fa+q8cSkr0tgvMhYhMHP/QC+WTgrePMDBWiWc/pV+1ik82Al20XOK25Gcw== - -ccount@^2.0.0: - version "2.0.1" - resolved "https://registry.yarnpkg.com/ccount/-/ccount-2.0.1.tgz#17a3bf82302e0870d6da43a01311a8bc02a3ecf5" - integrity sha512-eyrF0jiFpY+3drT6383f1qhkbGsLSifNAjA61IUjZjmLCWjItY6LB9ft9YhoDgwfmclB2zhu51Lc7+95b8NRAg== - -chakra-react-select@^4.0.0: - version "4.0.3" - resolved "https://registry.yarnpkg.com/chakra-react-select/-/chakra-react-select-4.0.3.tgz#6760a92ee0b814ec89181503dde796584360e03d" - integrity sha512-QEjySGsd666s0LSrLxpJiOv0mVFPVHVjPMcj3JRga3H/rHpUukZ6ydYX0uXl0WMZtUST7R9hcKNs0bzA6RTP8Q== - dependencies: - react-select "^5.3.2" - -chalk@^2.0.0, chalk@^2.4.2: - version "2.4.2" - resolved "https://registry.yarnpkg.com/chalk/-/chalk-2.4.2.tgz#cd42541677a54333cf541a49108c1432b44c9424" - integrity sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ== - dependencies: - ansi-styles "^3.2.1" - escape-string-regexp "^1.0.5" - supports-color "^5.3.0" - -chalk@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/chalk/-/chalk-3.0.0.tgz#3f73c2bf526591f574cc492c51e2456349f844e4" - integrity sha512-4D3B6Wf41KOYRFdszmDqMCGq5VV/uMAB273JILmO+3jAlh8X4qDtdtgCR3fxtbLEMzSx22QdhnDcJvu2u1fVwg== - dependencies: - ansi-styles "^4.1.0" - supports-color "^7.1.0" - -chalk@^4.0.0, chalk@^4.1.0: - version "4.1.1" - resolved "https://registry.yarnpkg.com/chalk/-/chalk-4.1.1.tgz#c80b3fab28bf6371e6863325eee67e618b77e6ad" - integrity sha512-diHzdDKxcU+bAsUboHLPEDQiw0qEe0qd7SYUn3HgcFlWgbDcfLGswOHYeGrHKzG9z6UYf01d9VFMfZxPM1xZSg== - dependencies: - ansi-styles "^4.1.0" - supports-color "^7.1.0" - -chalk@^5.0.1: - version "5.0.1" - resolved "https://registry.yarnpkg.com/chalk/-/chalk-5.0.1.tgz#ca57d71e82bb534a296df63bbacc4a1c22b2a4b6" - integrity sha512-Fo07WOYGqMfCWHOzSXOt2CxDbC6skS/jO9ynEcmpANMoPrD+W1r1K6Vx7iNm+AQmETU1Xr2t+n8nzkV9t6xh3w== - -char-regex@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/char-regex/-/char-regex-1.0.2.tgz#d744358226217f981ed58f479b1d6bcc29545dcf" - integrity sha512-kWWXztvZ5SBQV+eRgKFeh8q5sLuZY2+8WUIzlxWVTg+oGwY14qylx1KbKzHd8P6ZYkAg0xyIDU9JMHhyJMZ1jw== - -character-entities-legacy@^1.0.0: - version "1.1.4" - resolved "https://registry.yarnpkg.com/character-entities-legacy/-/character-entities-legacy-1.1.4.tgz#94bc1845dce70a5bb9d2ecc748725661293d8fc1" - integrity sha512-3Xnr+7ZFS1uxeiUDvV02wQ+QDbc55o97tIV5zHScSPJpcLm/r0DFPcoY3tYRp+VZukxuMeKgXYmsXQHO05zQeA== - -character-entities@^1.0.0: - version "1.2.4" - resolved "https://registry.yarnpkg.com/character-entities/-/character-entities-1.2.4.tgz#e12c3939b7eaf4e5b15e7ad4c5e28e1d48c5b16b" - integrity sha512-iBMyeEHxfVnIakwOuDXpVkc54HijNgCyQB2w0VfGQThle6NXn50zU6V/u+LDhxHcDUPojn6Kpga3PTAD8W1bQw== - -character-entities@^2.0.0: - version "2.0.2" - resolved "https://registry.yarnpkg.com/character-entities/-/character-entities-2.0.2.tgz#2d09c2e72cd9523076ccb21157dff66ad43fcc22" - integrity sha512-shx7oQ0Awen/BRIdkjkvz54PnEEI/EjwXDSIZp86/KKdbafHh1Df/RYGBhn4hbe2+uKC9FnT5UCEdyPz3ai9hQ== - -character-reference-invalid@^1.0.0: - version "1.1.4" - resolved "https://registry.yarnpkg.com/character-reference-invalid/-/character-reference-invalid-1.1.4.tgz#083329cda0eae272ab3dbbf37e9a382c13af1560" - integrity sha512-mKKUkUbhPpQlCOfIuZkvSEgktjPFIsZKRRbC6KWVEMvlzblj3i3asQv5ODsrwt0N3pHAEvjP8KTQPHkp0+6jOg== - -chownr@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/chownr/-/chownr-2.0.0.tgz#15bfbe53d2eab4cf70f18a8cd68ebe5b3cb1dece" - integrity sha512-bIomtDF5KGpdogkLd9VspvFzk9KfpyyGlS8YFVZl7TGPBHL5snIOnxeshwVgPteQ9b4Eydl+pVbIyE1DcvCWgQ== - -chrome-trace-event@^1.0.2: - version "1.0.3" - resolved "https://registry.yarnpkg.com/chrome-trace-event/-/chrome-trace-event-1.0.3.tgz#1015eced4741e15d06664a957dbbf50d041e26ac" - integrity sha512-p3KULyQg4S7NIHixdwbGX+nFHkoBiA4YQmyWtjb8XngSKV124nJmRysgAeujbUVb15vh+RvFUfCPqU7rXk+hZg== - -ci-info@^3.2.0: - version "3.2.0" - resolved "https://registry.yarnpkg.com/ci-info/-/ci-info-3.2.0.tgz#2876cb948a498797b5236f0095bc057d0dca38b6" - integrity sha512-dVqRX7fLUm8J6FgHJ418XuIgDLZDkYcDFTeL6TA2gt5WlIZUQrrH6EZrNClwT/H0FateUsZkGIOPRrLbP+PR9A== - -cjs-module-lexer@^1.0.0: - version "1.2.2" - resolved "https://registry.yarnpkg.com/cjs-module-lexer/-/cjs-module-lexer-1.2.2.tgz#9f84ba3244a512f3a54e5277e8eef4c489864e40" - integrity sha512-cOU9usZw8/dXIXKtwa8pM0OTJQuJkxMN6w30csNRUerHfeQ5R6U3kkU/FtJeIf3M202OHfY2U8ccInBG7/xogA== - -classcat@^5.0.3, classcat@^5.0.4: - version "5.0.4" - resolved "https://registry.yarnpkg.com/classcat/-/classcat-5.0.4.tgz#e12d1dfe6df6427f260f03b80dc63571a5107ba6" - integrity sha512-sbpkOw6z413p+HDGcBENe498WM9woqWHiJxCq7nvmxe9WmrUmqfAcxpIwAiMtM5Q3AhYkzXcNQHqsWq0mND51g== - -classnames@^2.3.1: - version "2.3.1" - resolved "https://registry.yarnpkg.com/classnames/-/classnames-2.3.1.tgz#dfcfa3891e306ec1dad105d0e88f4417b8535e8e" - integrity sha512-OlQdbZ7gLfGarSqxesMesDa5uz7KFbID8Kpq/SxIoNGDqY8lSYs0D+hhtBXhcdB3rcbXArFr7vlHheLk1voeNA== - -clean-stack@^2.0.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/clean-stack/-/clean-stack-2.2.0.tgz#ee8472dbb129e727b31e8a10a427dee9dfe4008b" - integrity sha512-4diC9HaTE+KRAMWhDhrGOECgWZxoevMc5TlkObMqNSsVU62PYzXZ/SMTjzyGAFF1YusgxGcSWTEXBhp0CPwQ1A== - -clean-webpack-plugin@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/clean-webpack-plugin/-/clean-webpack-plugin-3.0.0.tgz#a99d8ec34c1c628a4541567aa7b457446460c62b" - integrity sha512-MciirUH5r+cYLGCOL5JX/ZLzOZbVr1ot3Fw+KcvbhUb6PM+yycqd9ZhIlcigQ5gl+XhppNmw3bEFuaaMNyLj3A== - dependencies: - "@types/webpack" "^4.4.31" - del "^4.1.1" - -cli@~1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/cli/-/cli-1.0.1.tgz#22817534f24bfa4950c34d532d48ecbc621b8c14" - integrity sha1-IoF1NPJL+klQw01TLUjsvGIbjBQ= - dependencies: - exit "0.1.2" - glob "^7.1.1" - -cliui@^7.0.2: - version "7.0.4" - resolved "https://registry.yarnpkg.com/cliui/-/cliui-7.0.4.tgz#a0265ee655476fc807aea9df3df8df7783808b4f" - integrity sha512-OcRE68cOsVMXp1Yvonl/fzkQOyjLSu/8bhPDfQt0e0/Eb283TKP20Fs2MqoPsr9SwA595rRCA+QMzYc9nBP+JQ== - dependencies: - string-width "^4.2.0" - strip-ansi "^6.0.0" - wrap-ansi "^7.0.0" - -clone-deep@^4.0.1: - version "4.0.1" - resolved "https://registry.yarnpkg.com/clone-deep/-/clone-deep-4.0.1.tgz#c19fd9bdbbf85942b4fd979c84dcf7d5f07c2387" - integrity sha512-neHB9xuzh/wk0dIHweyAXv2aPGZIVk3pLMe+/RNzINf17fe0OG96QroktYAUm7SM1PBnzTabaLboqqxDyMU+SQ== - dependencies: - is-plain-object "^2.0.4" - kind-of "^6.0.2" - shallow-clone "^3.0.0" - -clsx@^1.1.0: - version "1.1.1" - resolved "https://registry.yarnpkg.com/clsx/-/clsx-1.1.1.tgz#98b3134f9abbdf23b2663491ace13c5c03a73188" - integrity sha512-6/bPho624p3S2pMyvP5kKBPXnI3ufHLObBFCfgx+LkeR5lg2XYy2hqZqUf45ypD8COn2bhgGJSUE+l5dhNBieA== - -co@^4.6.0: - version "4.6.0" - resolved "https://registry.yarnpkg.com/co/-/co-4.6.0.tgz#6ea6bdf3d853ae54ccb8e47bfa0bf3f9031fb184" - integrity sha1-bqa989hTrlTMuOR7+gvz+QMfsYQ= - -codemirror@^5.59.1: - version "5.61.1" - resolved "https://registry.yarnpkg.com/codemirror/-/codemirror-5.61.1.tgz#ccfc8a43b8fcfb8b12e8e75b5ffde48d541406e0" - integrity sha512-+D1NZjAucuzE93vJGbAaXzvoBHwp9nJZWWWF9utjv25+5AZUiah6CIlfb4ikG4MoDsFsCG8niiJH5++OO2LgIQ== - -collect-v8-coverage@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/collect-v8-coverage/-/collect-v8-coverage-1.0.1.tgz#cc2c8e94fc18bbdffe64d6534570c8a673b27f59" - integrity sha512-iBPtljfCNcTKNAto0KEtDfZ3qzjJvqE3aTGZsbhjSBlorqpXJlaWWtPO35D+ZImoC3KWejX64o+yPGxhWSTzfg== - -color-convert@^1.9.0: - version "1.9.3" - resolved "https://registry.yarnpkg.com/color-convert/-/color-convert-1.9.3.tgz#bb71850690e1f136567de629d2d5471deda4c1e8" - integrity sha512-QfAUtd+vFdAtFQcC8CCyYt1fYWxSqAiK2cSD6zDB8N3cpsEBAvRxp9zOGg6G/SHHJYAT88/az/IuDGALsNVbGg== - dependencies: - color-name "1.1.3" - -color-convert@^2.0.1: - version "2.0.1" - resolved "https://registry.yarnpkg.com/color-convert/-/color-convert-2.0.1.tgz#72d3a68d598c9bdb3af2ad1e84f21d896abd4de3" - integrity sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ== - dependencies: - color-name "~1.1.4" - -color-name@1.1.3: - version "1.1.3" - resolved "https://registry.yarnpkg.com/color-name/-/color-name-1.1.3.tgz#a7d0558bd89c42f795dd42328f740831ca53bc25" - integrity sha1-p9BVi9icQveV3UIyj3QIMcpTvCU= - -color-name@^1.0.0, color-name@^1.1.4, color-name@~1.1.4: - version "1.1.4" - resolved "https://registry.yarnpkg.com/color-name/-/color-name-1.1.4.tgz#c2a09a87acbde69543de6f63fa3995c826c536a2" - integrity sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA== - -color-string@^1.9.0: - version "1.9.1" - resolved "https://registry.yarnpkg.com/color-string/-/color-string-1.9.1.tgz#4467f9146f036f855b764dfb5bf8582bf342c7a4" - integrity sha512-shrVawQFojnZv6xM40anx4CkoDP+fZsw/ZerEMsW/pyzsRbElpsL/DBVW7q3ExxwusdNXI3lXpuhEZkzs8p5Eg== - dependencies: - color-name "^1.0.0" - simple-swizzle "^0.2.2" - -color2k@^2.0.0: - version "2.0.2" - resolved "https://registry.yarnpkg.com/color2k/-/color2k-2.0.2.tgz#ac2b4aea11c822a6bcb70c768b5a289f4fffcebb" - integrity sha512-kJhwH5nAwb34tmyuqq/lgjEKzlFXn1U99NlnB6Ws4qVaERcRUYeYP1cBw6BJ4vxaWStAUEef4WMr7WjOCnBt8w== - -color@^4.2.3: - version "4.2.3" - resolved "https://registry.yarnpkg.com/color/-/color-4.2.3.tgz#d781ecb5e57224ee43ea9627560107c0e0c6463a" - integrity sha512-1rXeuUUiGGrykh+CeBdu5Ie7OJwinCgQY0bc7GCRxy5xVHy+moaqkpL/jqQq0MtQOeYcrqEz4abc5f0KtU7W4A== - dependencies: - color-convert "^2.0.1" - color-string "^1.9.0" - -colord@^2.9.1, colord@^2.9.3: - version "2.9.3" - resolved "https://registry.yarnpkg.com/colord/-/colord-2.9.3.tgz#4f8ce919de456f1d5c1c368c307fe20f3e59fb43" - integrity sha512-jeC1axXpnb0/2nn/Y1LPuLdgXBLH7aDcHu4KEKfqw3CUhX7ZpfBSlPKyqXE6btIgEzfWtrX3/tyBCaCvXvMkOw== - -colorette@^1.2.0: - version "1.2.2" - resolved "https://registry.yarnpkg.com/colorette/-/colorette-1.2.2.tgz#cbcc79d5e99caea2dbf10eb3a26fd8b3e6acfa94" - integrity sha512-MKGMzyfeuutC/ZJ1cba9NqcNpfeqMUcYmyF1ZFY6/Cn7CNSAKx6a+s48sqLqyAiZuaP2TcqMhoo+dlwFnVxT9w== - -colorette@^2.0.14: - version "2.0.19" - resolved "https://registry.yarnpkg.com/colorette/-/colorette-2.0.19.tgz#cdf044f47ad41a0f4b56b3a0d5b4e6e1a2d5a798" - integrity sha512-3tlv/dIP7FWvj3BsbHrGLJ6l/oKh1O3TcgBqMn+yyCagOxc23fyzDS6HypQbgxWbkpDnf52p1LuR4eWDQ/K9WQ== - -combined-stream@^1.0.8: - version "1.0.8" - resolved "https://registry.yarnpkg.com/combined-stream/-/combined-stream-1.0.8.tgz#c3d45a8b34fd730631a110a8a2520682b31d5a7f" - integrity sha512-FQN4MRfuJeHf7cBbBMJFXhKSDq+2kAArBlmRBvcvFE5BB1HZKXtSFASDhdlz9zOYwxh8lDdnvmMOe/+5cdoEdg== - dependencies: - delayed-stream "~1.0.0" - -comma-separated-tokens@^1.0.0: - version "1.0.8" - resolved "https://registry.yarnpkg.com/comma-separated-tokens/-/comma-separated-tokens-1.0.8.tgz#632b80b6117867a158f1080ad498b2fbe7e3f5ea" - integrity sha512-GHuDRO12Sypu2cV70d1dkA2EUmXHgntrzbpvOB+Qy+49ypNfGgFQIC2fhhXbnyrJRynDCAARsT7Ou0M6hirpfw== - -comma-separated-tokens@^2.0.0: - version "2.0.3" - resolved "https://registry.yarnpkg.com/comma-separated-tokens/-/comma-separated-tokens-2.0.3.tgz#4e89c9458acb61bc8fef19f4529973b2392839ee" - integrity sha512-Fu4hJdvzeylCfQPp9SGWidpzrMs7tTrlu6Vb8XGaRGck8QSNZJJp538Wrb60Lax4fPwR64ViY468OIUTbRlGZg== - -commander@2, commander@^2.20.0: - version "2.20.3" - resolved "https://registry.yarnpkg.com/commander/-/commander-2.20.3.tgz#fd485e84c03eb4881c20722ba48035e8531aeb33" - integrity sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ== - -commander@^7.0.0, commander@^7.2.0: - version "7.2.0" - resolved "https://registry.yarnpkg.com/commander/-/commander-7.2.0.tgz#a36cb57d0b501ce108e4d20559a150a391d97ab7" - integrity sha512-QrWXB+ZQSVPmIWIhtEO9H+gwHaMGYiF5ChvoJ+K9ZGHG/sVsa6yiesAD1GC/x46sET00Xlwo1u49RVVVzvcSkw== - -commondir@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/commondir/-/commondir-1.0.1.tgz#ddd800da0c66127393cca5950ea968a3aaf1253b" - integrity sha1-3dgA2gxmEnOTzKWVDqloo6rxJTs= - -compute-scroll-into-view@1.0.14: - version "1.0.14" - resolved "https://registry.yarnpkg.com/compute-scroll-into-view/-/compute-scroll-into-view-1.0.14.tgz#80e3ebb25d6aa89f42e533956cb4b16a04cfe759" - integrity sha512-mKDjINe3tc6hGelUMNDzuhorIUZ7kS7BwyY0r2wQd2HOH2tRuJykiC06iSEX8y1TuhNzvz4GcJnK16mM2J1NMQ== - -concat-map@0.0.1: - version "0.0.1" - resolved "https://registry.yarnpkg.com/concat-map/-/concat-map-0.0.1.tgz#d8a96bd77fd68df7793a73036a3ba0d5405d477b" - integrity sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg== - -confusing-browser-globals@^1.0.10: - version "1.0.10" - resolved "https://registry.yarnpkg.com/confusing-browser-globals/-/confusing-browser-globals-1.0.10.tgz#30d1e7f3d1b882b25ec4933d1d1adac353d20a59" - integrity sha512-gNld/3lySHwuhaVluJUKLePYirM3QNCKzVxqAdhJII9/WXKVX5PURzMVJspS1jTslSqjeuG4KMVTSouit5YPHA== - -console-browserify@1.1.x: - version "1.1.0" - resolved "https://registry.yarnpkg.com/console-browserify/-/console-browserify-1.1.0.tgz#f0241c45730a9fc6323b206dbf38edc741d0bb10" - integrity sha1-8CQcRXMKn8YyOyBtvzjtx0HQuxA= - dependencies: - date-now "^0.1.4" - -convert-source-map@^1.4.0, convert-source-map@^1.5.0, convert-source-map@^1.6.0: - version "1.8.0" - resolved "https://registry.yarnpkg.com/convert-source-map/-/convert-source-map-1.8.0.tgz#f3373c32d21b4d780dd8004514684fb791ca4369" - integrity sha512-+OQdjP49zViI/6i7nIJpA8rAl4sV/JdPfU9nZs3VqOwGIgizICvuN2ru6fMd+4llL0tar18UYJXfZ/TWtmhUjA== - dependencies: - safe-buffer "~5.1.1" - -convert-source-map@^1.7.0: - version "1.7.0" - resolved "https://registry.yarnpkg.com/convert-source-map/-/convert-source-map-1.7.0.tgz#17a2cb882d7f77d3490585e2ce6c524424a3a442" - integrity sha512-4FJkXzKXEDB1snCFZlLP4gpC3JILicCpGbzG9f9G7tGqGCzETQ2hWPrcinA9oU4wtf2biUaEH5065UnMeR33oA== - dependencies: - safe-buffer "~5.1.1" - -convert-source-map@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/convert-source-map/-/convert-source-map-2.0.0.tgz#4b560f649fc4e918dd0ab75cf4961e8bc882d82a" - integrity sha512-Kvp459HrV2FEJ1CAsi1Ku+MY3kasH19TFykTz2xWmMeq6bk2NU3XXvfJ+Q61m0xktWwt+1HSYf3JZsTms3aRJg== - -copy-to-clipboard@3.3.1: - version "3.3.1" - resolved "https://registry.yarnpkg.com/copy-to-clipboard/-/copy-to-clipboard-3.3.1.tgz#115aa1a9998ffab6196f93076ad6da3b913662ae" - integrity sha512-i13qo6kIHTTpCm8/Wup+0b1mVWETvu2kIMzKoK8FpkLkFxlt0znUAHcMzox+T8sPlqtZXq3CulEjQHsYiGFJUw== - dependencies: - toggle-selection "^1.0.6" - -copy-webpack-plugin@^6.0.3: - version "6.4.1" - resolved "https://registry.yarnpkg.com/copy-webpack-plugin/-/copy-webpack-plugin-6.4.1.tgz#138cd9b436dbca0a6d071720d5414848992ec47e" - integrity sha512-MXyPCjdPVx5iiWyl40Va3JGh27bKzOTNY3NjUTrosD2q7dR/cLD0013uqJ3BpFbUjyONINjb6qI7nDIJujrMbA== - dependencies: - cacache "^15.0.5" - fast-glob "^3.2.4" - find-cache-dir "^3.3.1" - glob-parent "^5.1.1" - globby "^11.0.1" - loader-utils "^2.0.0" - normalize-path "^3.0.0" - p-limit "^3.0.2" - schema-utils "^3.0.0" - serialize-javascript "^5.0.1" - webpack-sources "^1.4.3" - -core-js-compat@^3.31.0, core-js-compat@^3.36.1: - version "3.37.1" - resolved "https://registry.yarnpkg.com/core-js-compat/-/core-js-compat-3.37.1.tgz#c844310c7852f4bdf49b8d339730b97e17ff09ee" - integrity sha512-9TNiImhKvQqSUkOvk/mMRZzOANTiEVC7WaBNhHcKM7x+/5E1l5NvsysR19zuDQScE8k+kfQXWRN3AtS/eOSHpg== - dependencies: - browserslist "^4.23.0" - -core-js-pure@^3.16.0: - version "3.18.0" - resolved "https://registry.yarnpkg.com/core-js-pure/-/core-js-pure-3.18.0.tgz#e5187347bae66448c9e2d67c01c34c4df3261dc5" - integrity sha512-ZnK+9vyuMhKulIGqT/7RHGRok8RtkHMEX/BGPHkHx+ouDkq+MUvf9mfIgdqhpmPDu8+V5UtRn/CbCRc9I4lX4w== - -core-util-is@~1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/core-util-is/-/core-util-is-1.0.2.tgz#b5fd54220aa2bc5ab57aab7140c940754503c1a7" - integrity sha1-tf1UIgqivFq1eqtxQMlAdUUDwac= - -cosmiconfig@^6.0.0: - version "6.0.0" - resolved "https://registry.yarnpkg.com/cosmiconfig/-/cosmiconfig-6.0.0.tgz#da4fee853c52f6b1e6935f41c1a2fc50bd4a9982" - integrity sha512-xb3ZL6+L8b9JLLCx3ZdoZy4+2ECphCMo2PwqgP1tlfVq6M6YReyzBJtvWWtbDSpNr9hn96pkCiZqUcFEc+54Qg== - dependencies: - "@types/parse-json" "^4.0.0" - import-fresh "^3.1.0" - parse-json "^5.0.0" - path-type "^4.0.0" - yaml "^1.7.2" - -cosmiconfig@^8.2.0: - version "8.2.0" - resolved "https://registry.yarnpkg.com/cosmiconfig/-/cosmiconfig-8.2.0.tgz#f7d17c56a590856cd1e7cee98734dca272b0d8fd" - integrity sha512-3rTMnFJA1tCOPwRxtgF4wd7Ab2qvDbL8jX+3smjIbS4HlZBagTlpERbdN7iAbWlrfxE3M8c27kTwTawQ7st+OQ== - dependencies: - import-fresh "^3.2.1" - js-yaml "^4.1.0" - parse-json "^5.0.0" - path-type "^4.0.0" - -cross-fetch@^3.1.5: - version "3.1.5" - resolved "https://registry.yarnpkg.com/cross-fetch/-/cross-fetch-3.1.5.tgz#e1389f44d9e7ba767907f7af8454787952ab534f" - integrity sha512-lvb1SBsI0Z7GDwmuid+mU3kWVBwTVUbe7S0H52yaaAdQOXq2YktTCZdlAcNKFzE6QtRz0snpw9bNiPeOIkkQvw== - dependencies: - node-fetch "2.6.7" - -cross-spawn@^7.0.2, cross-spawn@^7.0.3: - version "7.0.5" - resolved "https://registry.yarnpkg.com/cross-spawn/-/cross-spawn-7.0.5.tgz#910aac880ff5243da96b728bc6521a5f6c2f2f82" - integrity sha512-ZVJrKKYunU38/76t0RMOulHOnUcbU9GbpWKAOZ0mhjr7CX6FVrH+4FrAapSOekrgFQ3f/8gwMEuIft0aKq6Hug== - dependencies: - path-key "^3.1.0" - shebang-command "^2.0.0" - which "^2.0.1" - -css-box-model@1.2.1: - version "1.2.1" - resolved "https://registry.yarnpkg.com/css-box-model/-/css-box-model-1.2.1.tgz#59951d3b81fd6b2074a62d49444415b0d2b4d7c1" - integrity sha512-a7Vr4Q/kd/aw96bnJG332W9V9LkJO69JRcaCYDUqjp6/z0w6VcZjgAcTbgFxEPfBgdnAwlh3iwu+hLopa+flJw== - dependencies: - tiny-invariant "^1.0.6" - -css-declaration-sorter@^6.2.2: - version "6.3.0" - resolved "https://registry.yarnpkg.com/css-declaration-sorter/-/css-declaration-sorter-6.3.0.tgz#72ebd995c8f4532ff0036631f7365cce9759df14" - integrity sha512-OGT677UGHJTAVMRhPO+HJ4oKln3wkBTwtDFH0ojbqm+MJm6xuDMHp2nkhh/ThaBqq20IbraBQSWKfSLNHQO9Og== - -css-functions-list@^3.1.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/css-functions-list/-/css-functions-list-3.1.0.tgz#cf5b09f835ad91a00e5959bcfc627cd498e1321b" - integrity sha512-/9lCvYZaUbBGvYUgYGFJ4dcYiyqdhSjG7IPVluoV8A1ILjkF7ilmhp1OGUz8n+nmBcu0RNrQAzgD8B6FJbrt2w== - -css-loader@5.2.7: - version "5.2.7" - resolved "https://registry.yarnpkg.com/css-loader/-/css-loader-5.2.7.tgz#9b9f111edf6fb2be5dc62525644cbc9c232064ae" - integrity sha512-Q7mOvpBNBG7YrVGMxRxcBJZFL75o+cH2abNASdibkj/fffYD8qWbInZrD0S9ccI6vZclF3DsHE7njGlLtaHbhg== - dependencies: - icss-utils "^5.1.0" - loader-utils "^2.0.0" - postcss "^8.2.15" - postcss-modules-extract-imports "^3.0.0" - postcss-modules-local-by-default "^4.0.0" - postcss-modules-scope "^3.0.0" - postcss-modules-values "^4.0.0" - postcss-value-parser "^4.1.0" - schema-utils "^3.0.0" - semver "^7.3.5" - -css-minimizer-webpack-plugin@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/css-minimizer-webpack-plugin/-/css-minimizer-webpack-plugin-4.0.0.tgz#e11800388c19c2b7442c39cc78ac8ae3675c9605" - integrity sha512-7ZXXRzRHvofv3Uac5Y+RkWRNo0ZMlcg8e9/OtrqUYmwDWJo+qs67GvdeFrXLsFb7czKNwjQhPkM0avlIYl+1nA== - dependencies: - cssnano "^5.1.8" - jest-worker "^27.5.1" - postcss "^8.4.13" - schema-utils "^4.0.0" - serialize-javascript "^6.0.0" - source-map "^0.6.1" - -css-select@^4.1.3: - version "4.3.0" - resolved "https://registry.yarnpkg.com/css-select/-/css-select-4.3.0.tgz#db7129b2846662fd8628cfc496abb2b59e41529b" - integrity sha512-wPpOYtnsVontu2mODhA19JrqWxNsfdatRKd64kmpRbQgh1KtItko5sTnEpPdpSaJszTOhEMlF/RPz28qj4HqhQ== - dependencies: - boolbase "^1.0.0" - css-what "^6.0.1" - domhandler "^4.3.1" - domutils "^2.8.0" - nth-check "^2.0.1" - -css-tree@^1.1.2, css-tree@^1.1.3: - version "1.1.3" - resolved "https://registry.yarnpkg.com/css-tree/-/css-tree-1.1.3.tgz#eb4870fb6fd7707327ec95c2ff2ab09b5e8db91d" - integrity sha512-tRpdppF7TRazZrjJ6v3stzv93qxRcSsFmW6cX0Zm2NVKpxE1WV1HblnghVv9TreireHkqI/VDEsfolRF1p6y7Q== - dependencies: - mdn-data "2.0.14" - source-map "^0.6.1" - -css-tree@^2.3.1: - version "2.3.1" - resolved "https://registry.yarnpkg.com/css-tree/-/css-tree-2.3.1.tgz#10264ce1e5442e8572fc82fbe490644ff54b5c20" - integrity sha512-6Fv1DV/TYw//QF5IzQdqsNDjx/wc8TrMBZsqjL9eW01tWb7R7k/mq+/VXfJCl7SoD5emsJop9cOByJZfs8hYIw== - dependencies: - mdn-data "2.0.30" - source-map-js "^1.0.1" - -css-what@^6.0.1: - version "6.1.0" - resolved "https://registry.yarnpkg.com/css-what/-/css-what-6.1.0.tgz#fb5effcf76f1ddea2c81bdfaa4de44e79bac70f4" - integrity sha512-HTUrgRJ7r4dsZKU6GjmpfRK1O76h97Z8MfS1G0FozR+oF2kG6Vfe8JE6zwrkbxigziPHinCJ+gCPjA9EaBDtRw== - -css.escape@^1.5.1: - version "1.5.1" - resolved "https://registry.yarnpkg.com/css.escape/-/css.escape-1.5.1.tgz#42e27d4fa04ae32f931a4b4d4191fa9cddee97cb" - integrity sha1-QuJ9T6BK4y+TGktNQZH6nN3ul8s= - -css@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/css/-/css-3.0.0.tgz#4447a4d58fdd03367c516ca9f64ae365cee4aa5d" - integrity sha512-DG9pFfwOrzc+hawpmqX/dHYHJG+Bsdb0klhyi1sDneOgGOXy9wQIC8hzyVp1e4NRYDBdxcylvywPkkXCHAzTyQ== - dependencies: - inherits "^2.0.4" - source-map "^0.6.1" - source-map-resolve "^0.6.0" - -cssesc@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/cssesc/-/cssesc-3.0.0.tgz#37741919903b868565e1c09ea747445cd18983ee" - integrity sha512-/Tb/JcjK111nNScGob5MNtsntNM1aCNUDipB/TkwZFhyDrrE47SOx/18wF2bbjgc3ZzCSKW1T5nt5EbFoAz/Vg== - -cssfontparser@^1.2.1: - version "1.2.1" - resolved "https://registry.yarnpkg.com/cssfontparser/-/cssfontparser-1.2.1.tgz#f4022fc8f9700c68029d542084afbaf425a3f3e3" - integrity sha512-6tun4LoZnj7VN6YeegOVb67KBX/7JJsqvj+pv3ZA7F878/eN33AbGa5b/S/wXxS/tcp8nc40xRUrsPlxIyNUPg== - -cssnano-preset-default@^5.2.11: - version "5.2.11" - resolved "https://registry.yarnpkg.com/cssnano-preset-default/-/cssnano-preset-default-5.2.11.tgz#28350471bc1af9df14052472b61340347f453a53" - integrity sha512-4PadR1NtuaIK8MvLNuY7MznK4WJteldGlzCiMaaTiOUP+apeiIvUDIXykzUOoqgOOUAHrU64ncdD90NfZR3LSQ== - dependencies: - css-declaration-sorter "^6.2.2" - cssnano-utils "^3.1.0" - postcss-calc "^8.2.3" - postcss-colormin "^5.3.0" - postcss-convert-values "^5.1.2" - postcss-discard-comments "^5.1.2" - postcss-discard-duplicates "^5.1.0" - postcss-discard-empty "^5.1.1" - postcss-discard-overridden "^5.1.0" - postcss-merge-longhand "^5.1.5" - postcss-merge-rules "^5.1.2" - postcss-minify-font-values "^5.1.0" - postcss-minify-gradients "^5.1.1" - postcss-minify-params "^5.1.3" - postcss-minify-selectors "^5.2.1" - postcss-normalize-charset "^5.1.0" - postcss-normalize-display-values "^5.1.0" - postcss-normalize-positions "^5.1.0" - postcss-normalize-repeat-style "^5.1.0" - postcss-normalize-string "^5.1.0" - postcss-normalize-timing-functions "^5.1.0" - postcss-normalize-unicode "^5.1.0" - postcss-normalize-url "^5.1.0" - postcss-normalize-whitespace "^5.1.1" - postcss-ordered-values "^5.1.2" - postcss-reduce-initial "^5.1.0" - postcss-reduce-transforms "^5.1.0" - postcss-svgo "^5.1.0" - postcss-unique-selectors "^5.1.1" - -cssnano-utils@^3.1.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/cssnano-utils/-/cssnano-utils-3.1.0.tgz#95684d08c91511edfc70d2636338ca37ef3a6861" - integrity sha512-JQNR19/YZhz4psLX/rQ9M83e3z2Wf/HdJbryzte4a3NSuafyp9w/I4U+hx5C2S9g41qlstH7DEWnZaaj83OuEA== - -cssnano@^5.1.8: - version "5.1.11" - resolved "https://registry.yarnpkg.com/cssnano/-/cssnano-5.1.11.tgz#3bb003380718c7948ce3813493370e8946caf04b" - integrity sha512-2nx+O6LvewPo5EBtYrKc8762mMkZRk9cMGIOP4UlkmxHm7ObxH+zvsJJ+qLwPkUc4/yumL/qJkavYi9NlodWIQ== - dependencies: - cssnano-preset-default "^5.2.11" - lilconfig "^2.0.3" - yaml "^1.10.2" - -csso@^4.2.0: - version "4.2.0" - resolved "https://registry.yarnpkg.com/csso/-/csso-4.2.0.tgz#ea3a561346e8dc9f546d6febedd50187cf389529" - integrity sha512-wvlcdIbf6pwKEk7vHj8/Bkc0B4ylXZruLvOgs9doS5eOsOpuodOV2zJChSpkp+pRpYQLQMeF04nr3Z68Sta9jA== - dependencies: - css-tree "^1.1.2" - -cssom@^0.4.4: - version "0.4.4" - resolved "https://registry.yarnpkg.com/cssom/-/cssom-0.4.4.tgz#5a66cf93d2d0b661d80bf6a44fb65f5c2e4e0a10" - integrity sha512-p3pvU7r1MyyqbTk+WbNJIgJjG2VmTIaB10rI93LzVPrmDJKkzKYMtxxyAvQXR/NS6otuzveI7+7BBq3SjBS2mw== - -cssom@~0.3.6: - version "0.3.8" - resolved "https://registry.yarnpkg.com/cssom/-/cssom-0.3.8.tgz#9f1276f5b2b463f2114d3f2c75250af8c1a36f4a" - integrity sha512-b0tGHbfegbhPJpxpiBPU2sCkigAqtM9O121le6bbOlgyV+NyGyCmVfJ6QW9eRjz8CpNfWEOYBIMIGRYkLwsIYg== - -cssstyle@^2.3.0: - version "2.3.0" - resolved "https://registry.yarnpkg.com/cssstyle/-/cssstyle-2.3.0.tgz#ff665a0ddbdc31864b09647f34163443d90b0852" - integrity sha512-AZL67abkUzIuvcHqk7c09cezpGNcxUxU4Ioi/05xHk4DQeTkWmGYftIE6ctU6AEt+Gn4n1lDStOtj7FKycP71A== - dependencies: - cssom "~0.3.6" - -csstype@^3.0.11: - version "3.1.0" - resolved "https://registry.yarnpkg.com/csstype/-/csstype-3.1.0.tgz#4ddcac3718d787cf9df0d1b7d15033925c8f29f2" - integrity sha512-uX1KG+x9h5hIJsaKR9xHUeUraxf8IODOwq9JLNPq6BwB04a/xgpq3rcx47l5BZu5zBPlgD342tdke3Hom/nJRA== - -csstype@^3.0.2: - version "3.0.8" - resolved "https://registry.yarnpkg.com/csstype/-/csstype-3.0.8.tgz#d2266a792729fb227cd216fb572f43728e1ad340" - integrity sha512-jXKhWqXPmlUeoQnF/EhTtTl4C9SnrxSH/jZUih3jmO6lBKr99rP3/+FmrMj4EFpOXzMtXHAZkd3x0E6h6Fgflw== - -d3-array@1, d3-array@^1.1.1, d3-array@^1.2.0: - version "1.2.4" - resolved "https://registry.yarnpkg.com/d3-array/-/d3-array-1.2.4.tgz#635ce4d5eea759f6f605863dbcfc30edc737f71f" - integrity sha512-KHW6M86R+FUPYGb3R5XiYjXPq7VzwxZ22buHhAEVG5ztoEcZZMLov530mmccaqA1GghZArjQV46fuc8kUqhhHw== - -d3-array@2, d3-array@^2.3.0: - version "2.12.1" - resolved "https://registry.yarnpkg.com/d3-array/-/d3-array-2.12.1.tgz#e20b41aafcdffdf5d50928004ececf815a465e81" - integrity sha512-B0ErZK/66mHtEsR1TkPEEkwdy+WDesimkM5gpZr5Dsg54BiTA5RXtYW5qTLIAcekaS9xfZrzBLF/OAkB3Qn1YQ== - dependencies: - internmap "^1.0.0" - -d3-axis@1: - version "1.0.12" - resolved "https://registry.yarnpkg.com/d3-axis/-/d3-axis-1.0.12.tgz#cdf20ba210cfbb43795af33756886fb3638daac9" - integrity sha512-ejINPfPSNdGFKEOAtnBtdkpr24c4d4jsei6Lg98mxf424ivoDP2956/5HDpIAtmHo85lqT4pruy+zEgvRUBqaQ== - -d3-brush@1: - version "1.1.6" - resolved "https://registry.yarnpkg.com/d3-brush/-/d3-brush-1.1.6.tgz#b0a22c7372cabec128bdddf9bddc058592f89e9b" - integrity sha512-7RW+w7HfMCPyZLifTz/UnJmI5kdkXtpCbombUSs8xniAyo0vIbrDzDwUJB6eJOgl9u5DQOt2TQlYumxzD1SvYA== - dependencies: - d3-dispatch "1" - d3-drag "1" - d3-interpolate "1" - d3-selection "1" - d3-transition "1" - -d3-chord@1: - version "1.0.6" - resolved "https://registry.yarnpkg.com/d3-chord/-/d3-chord-1.0.6.tgz#309157e3f2db2c752f0280fedd35f2067ccbb15f" - integrity sha512-JXA2Dro1Fxw9rJe33Uv+Ckr5IrAa74TlfDEhE/jfLOaXegMQFQTAgAw9WnZL8+HxVBRXaRGCkrNU7pJeylRIuA== - dependencies: - d3-array "1" - d3-path "1" - -d3-collection@1, d3-collection@^1.0.4: - version "1.0.7" - resolved "https://registry.yarnpkg.com/d3-collection/-/d3-collection-1.0.7.tgz#349bd2aa9977db071091c13144d5e4f16b5b310e" - integrity sha512-ii0/r5f4sjKNTfh84Di+DpztYwqKhEyUlKoPrzUFfeSkWxjW49xU2QzO9qrPrNkpdI0XJkfzvmTu8V2Zylln6A== - -d3-color@1, "d3-color@1 - 2", "d3-color@1 - 3", d3-color@^3.1.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/d3-color/-/d3-color-3.1.0.tgz#395b2833dfac71507f12ac2f7af23bf819de24e2" - integrity sha512-zg/chbXyeBtMQ1LbD/WSoW2DpC3I0mpmPdW+ynRTj/x2DAWYrIY7qeZIHidozwV24m4iavr15lNwIwLxRmOxhA== - -d3-contour@1: - version "1.3.2" - resolved "https://registry.yarnpkg.com/d3-contour/-/d3-contour-1.3.2.tgz#652aacd500d2264cb3423cee10db69f6f59bead3" - integrity sha512-hoPp4K/rJCu0ladiH6zmJUEz6+u3lgR+GSm/QdM2BBvDraU39Vr7YdDCicJcxP1z8i9B/2dJLgDC1NcvlF8WCg== - dependencies: - d3-array "^1.1.1" - -d3-dispatch@1: - version "1.0.6" - resolved "https://registry.yarnpkg.com/d3-dispatch/-/d3-dispatch-1.0.6.tgz#00d37bcee4dd8cd97729dd893a0ac29caaba5d58" - integrity sha512-fVjoElzjhCEy+Hbn8KygnmMS7Or0a9sI2UzGwoB7cCtvI1XpVN9GpoYlnb3xt2YV66oXYb1fLJ8GMvP4hdU1RA== - -"d3-dispatch@1 - 3": - version "3.0.1" - resolved "https://registry.yarnpkg.com/d3-dispatch/-/d3-dispatch-3.0.1.tgz#5fc75284e9c2375c36c839411a0cf550cbfc4d5e" - integrity sha512-rzUyPU/S7rwUflMyLc1ETDeBj0NRuHKKAcvukozwhshr6g6c5d8zh4c2gQjY2bZ0dXeGLWc1PF174P2tVvKhfg== - -d3-drag@1: - version "1.2.5" - resolved "https://registry.yarnpkg.com/d3-drag/-/d3-drag-1.2.5.tgz#2537f451acd39d31406677b7dc77c82f7d988f70" - integrity sha512-rD1ohlkKQwMZYkQlYVCrSFxsWPzI97+W+PaEIBNTMxRuxz9RF0Hi5nJWHGVJ3Om9d2fRTe1yOBINJyy/ahV95w== - dependencies: - d3-dispatch "1" - d3-selection "1" - -"d3-drag@2 - 3", d3-drag@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/d3-drag/-/d3-drag-3.0.0.tgz#994aae9cd23c719f53b5e10e3a0a6108c69607ba" - integrity sha512-pWbUJLdETVA8lQNJecMxoXfH6x+mO2UQo8rSmZ+QqxcbyA3hfeprFgIT//HW2nlHChWeIIMwS2Fq+gEARkhTkg== - dependencies: - d3-dispatch "1 - 3" - d3-selection "3" - -d3-dsv@1: - version "1.2.0" - resolved "https://registry.yarnpkg.com/d3-dsv/-/d3-dsv-1.2.0.tgz#9d5f75c3a5f8abd611f74d3f5847b0d4338b885c" - integrity sha512-9yVlqvZcSOMhCYzniHE7EVUws7Fa1zgw+/EAV2BxJoG3ME19V6BQFBwI855XQDsxyOuG7NibqRMTtiF/Qup46g== - dependencies: - commander "2" - iconv-lite "0.4" - rw "1" - -d3-ease@1: - version "1.0.7" - resolved "https://registry.yarnpkg.com/d3-ease/-/d3-ease-1.0.7.tgz#9a834890ef8b8ae8c558b2fe55bd57f5993b85e2" - integrity sha512-lx14ZPYkhNx0s/2HX5sLFUI3mbasHjSSpwO/KaaNACweVwxUruKyWVcb293wMv1RqTPZyZ8kSZ2NogUZNcLOFQ== - -"d3-ease@1 - 3": - version "3.0.1" - resolved "https://registry.yarnpkg.com/d3-ease/-/d3-ease-3.0.1.tgz#9658ac38a2140d59d346160f1f6c30fda0bd12f4" - integrity sha512-wR/XK3D3XcLIZwpbvQwQ5fK+8Ykds1ip7A2Txe0yxncXSdq1L9skcG7blcedkOX+ZcgxGAmLX1FrRGbADwzi0w== - -d3-fetch@1: - version "1.2.0" - resolved "https://registry.yarnpkg.com/d3-fetch/-/d3-fetch-1.2.0.tgz#15ce2ecfc41b092b1db50abd2c552c2316cf7fc7" - integrity sha512-yC78NBVcd2zFAyR/HnUiBS7Lf6inSCoWcSxFfw8FYL7ydiqe80SazNwoffcqOfs95XaLo7yebsmQqDKSsXUtvA== - dependencies: - d3-dsv "1" - -d3-force@1: - version "1.2.1" - resolved "https://registry.yarnpkg.com/d3-force/-/d3-force-1.2.1.tgz#fd29a5d1ff181c9e7f0669e4bd72bdb0e914ec0b" - integrity sha512-HHvehyaiUlVo5CxBJ0yF/xny4xoaxFxDnBXNvNcfW9adORGZfyNF1dj6DGLKyk4Yh3brP/1h3rnDzdIAwL08zg== - dependencies: - d3-collection "1" - d3-dispatch "1" - d3-quadtree "1" - d3-timer "1" - -d3-format@1: - version "1.4.5" - resolved "https://registry.yarnpkg.com/d3-format/-/d3-format-1.4.5.tgz#374f2ba1320e3717eb74a9356c67daee17a7edb4" - integrity sha512-J0piedu6Z8iB6TbIGfZgDzfXxUFN3qQRMofy2oPdXzQibYGqPB/9iMcxr/TGalU+2RsyDO+U4f33id8tbnSRMQ== - -"d3-format@1 - 2": - version "2.0.0" - resolved "https://registry.yarnpkg.com/d3-format/-/d3-format-2.0.0.tgz#a10bcc0f986c372b729ba447382413aabf5b0767" - integrity sha512-Ab3S6XuE/Q+flY96HXT0jOXcM4EAClYFnRGY5zsjRGNy6qCYrQsMffs7cV5Q9xejb35zxW5hf/guKw34kvIKsA== - -d3-geo@1: - version "1.12.1" - resolved "https://registry.yarnpkg.com/d3-geo/-/d3-geo-1.12.1.tgz#7fc2ab7414b72e59fbcbd603e80d9adc029b035f" - integrity sha512-XG4d1c/UJSEX9NfU02KwBL6BYPj8YKHxgBEw5om2ZnTRSbIcego6dhHwcxuSR3clxh0EpE38os1DVPOmnYtTPg== - dependencies: - d3-array "1" - -d3-hierarchy@1: - version "1.1.9" - resolved "https://registry.yarnpkg.com/d3-hierarchy/-/d3-hierarchy-1.1.9.tgz#2f6bee24caaea43f8dc37545fa01628559647a83" - integrity sha512-j8tPxlqh1srJHAtxfvOUwKNYJkQuBFdM1+JAUfq6xqH5eAqf93L7oG1NVqDa4CpFZNvnNKtCYEUC8KY9yEn9lQ== - -d3-interpolate@1, d3-interpolate@^1.4.0: - version "1.4.0" - resolved "https://registry.yarnpkg.com/d3-interpolate/-/d3-interpolate-1.4.0.tgz#526e79e2d80daa383f9e0c1c1c7dcc0f0583e987" - integrity sha512-V9znK0zc3jOPV4VD2zZn0sDhZU3WAE2bmlxdIwwQPPzPjvyLkd8B3JUVdS1IDUFDkWZ72c9qnv1GK2ZagTZ8EA== - dependencies: - d3-color "1" - -"d3-interpolate@1 - 3": - version "3.0.1" - resolved "https://registry.yarnpkg.com/d3-interpolate/-/d3-interpolate-3.0.1.tgz#3c47aa5b32c5b3dfb56ef3fd4342078a632b400d" - integrity sha512-3bYs1rOD33uo8aqJfKP3JWPAibgw8Zm2+L9vBKEHJ2Rg+viTR7o5Mmv5mZcieN+FRYaAOWX5SJATX6k1PWz72g== - dependencies: - d3-color "1 - 3" - -"d3-interpolate@1.2.0 - 2": - version "2.0.1" - resolved "https://registry.yarnpkg.com/d3-interpolate/-/d3-interpolate-2.0.1.tgz#98be499cfb8a3b94d4ff616900501a64abc91163" - integrity sha512-c5UhwwTs/yybcmTpAVqwSFl6vrQ8JZJoT5F7xNFK9pymv5C0Ymcc9/LIJHtYIggg/yS9YHw8i8O8tgb9pupjeQ== - dependencies: - d3-color "1 - 2" - -d3-path@1, d3-path@^1.0.5: - version "1.0.9" - resolved "https://registry.yarnpkg.com/d3-path/-/d3-path-1.0.9.tgz#48c050bb1fe8c262493a8caf5524e3e9591701cf" - integrity sha512-VLaYcn81dtHVTjEHd8B+pbe9yHWpXKZUC87PzoFmsFrJqgFwDe/qxfp5MlfsfM1V5E/iVt0MmEbWQ7FVIXh/bg== - -"d3-path@1 - 2": - version "2.0.0" - resolved "https://registry.yarnpkg.com/d3-path/-/d3-path-2.0.0.tgz#55d86ac131a0548adae241eebfb56b4582dd09d8" - integrity sha512-ZwZQxKhBnv9yHaiWd6ZU4x5BtCQ7pXszEV9CU6kRgwIQVQGLMv1oiL4M+MK/n79sYzsj+gcgpPQSctJUsLN7fA== - -d3-polygon@1: - version "1.0.6" - resolved "https://registry.yarnpkg.com/d3-polygon/-/d3-polygon-1.0.6.tgz#0bf8cb8180a6dc107f518ddf7975e12abbfbd38e" - integrity sha512-k+RF7WvI08PC8reEoXa/w2nSg5AUMTi+peBD9cmFc+0ixHfbs4QmxxkarVal1IkVkgxVuk9JSHhJURHiyHKAuQ== - -d3-quadtree@1: - version "1.0.7" - resolved "https://registry.yarnpkg.com/d3-quadtree/-/d3-quadtree-1.0.7.tgz#ca8b84df7bb53763fe3c2f24bd435137f4e53135" - integrity sha512-RKPAeXnkC59IDGD0Wu5mANy0Q2V28L+fNe65pOCXVdVuTJS3WPKaJlFHer32Rbh9gIo9qMuJXio8ra4+YmIymA== - -d3-random@1: - version "1.1.2" - resolved "https://registry.yarnpkg.com/d3-random/-/d3-random-1.1.2.tgz#2833be7c124360bf9e2d3fd4f33847cfe6cab291" - integrity sha512-6AK5BNpIFqP+cx/sreKzNjWbwZQCSUatxq+pPRmFIQaWuoD+NrbVWw7YWpHiXpCQ/NanKdtGDuB+VQcZDaEmYQ== - -d3-scale-chromatic@1: - version "1.5.0" - resolved "https://registry.yarnpkg.com/d3-scale-chromatic/-/d3-scale-chromatic-1.5.0.tgz#54e333fc78212f439b14641fb55801dd81135a98" - integrity sha512-ACcL46DYImpRFMBcpk9HhtIyC7bTBR4fNOPxwVSl0LfulDAwyiHyPOTqcDG1+t5d4P9W7t/2NAuWu59aKko/cg== - dependencies: - d3-color "1" - d3-interpolate "1" - -d3-scale@2: - version "2.2.2" - resolved "https://registry.yarnpkg.com/d3-scale/-/d3-scale-2.2.2.tgz#4e880e0b2745acaaddd3ede26a9e908a9e17b81f" - integrity sha512-LbeEvGgIb8UMcAa0EATLNX0lelKWGYDQiPdHj+gLblGVhGLyNbaCn3EvrJf0A3Y/uOOU5aD6MTh5ZFCdEwGiCw== - dependencies: - d3-array "^1.2.0" - d3-collection "1" - d3-format "1" - d3-interpolate "1" - d3-time "1" - d3-time-format "2" - -d3-scale@^3.3.0: - version "3.3.0" - resolved "https://registry.yarnpkg.com/d3-scale/-/d3-scale-3.3.0.tgz#28c600b29f47e5b9cd2df9749c206727966203f3" - integrity sha512-1JGp44NQCt5d1g+Yy+GeOnZP7xHo0ii8zsQp6PGzd+C1/dl0KGsp9A7Mxwp+1D1o4unbTTxVdU/ZOIEBoeZPbQ== - dependencies: - d3-array "^2.3.0" - d3-format "1 - 2" - d3-interpolate "1.2.0 - 2" - d3-time "^2.1.1" - d3-time-format "2 - 3" - -d3-selection@1, d3-selection@^1.1.0, d3-selection@^1.3.0: - version "1.4.2" - resolved "https://registry.yarnpkg.com/d3-selection/-/d3-selection-1.4.2.tgz#dcaa49522c0dbf32d6c1858afc26b6094555bc5c" - integrity sha512-SJ0BqYihzOjDnnlfyeHT0e30k0K1+5sR3d5fNueCNeuhZTnGw4M4o8mqJchSwgKMXCNFo+e2VTChiSJ0vYtXkg== - -"d3-selection@2 - 3", d3-selection@3, d3-selection@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/d3-selection/-/d3-selection-3.0.0.tgz#c25338207efa72cc5b9bd1458a1a41901f1e1b31" - integrity sha512-fmTRWbNMmsmWq6xJV8D19U/gw/bwrHfNXxrIN+HfZgnzqTHp9jOmKMhsTUjXOJnZOdZY9Q28y4yebKzqDKlxlQ== - -d3-shape@1, d3-shape@^1.0.6, d3-shape@^1.2.0: - version "1.3.7" - resolved "https://registry.yarnpkg.com/d3-shape/-/d3-shape-1.3.7.tgz#df63801be07bc986bc54f63789b4fe502992b5d7" - integrity sha512-EUkvKjqPFUAZyOlhY5gzCxCeI0Aep04LwIRpsZ/mLFelJiUfnK56jo5JMDSE7yyP2kLSb6LtF+S5chMk7uqPqw== - dependencies: - d3-path "1" - -d3-shape@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/d3-shape/-/d3-shape-2.1.0.tgz#3b6a82ccafbc45de55b57fcf956c584ded3b666f" - integrity sha512-PnjUqfM2PpskbSLTJvAzp2Wv4CZsnAgTfcVRTwW03QR3MkXF8Uo7B1y/lWkAsmbKwuecto++4NlsYcvYpXpTHA== - dependencies: - d3-path "1 - 2" - -d3-time-format@2: - version "2.3.0" - resolved "https://registry.yarnpkg.com/d3-time-format/-/d3-time-format-2.3.0.tgz#107bdc028667788a8924ba040faf1fbccd5a7850" - integrity sha512-guv6b2H37s2Uq/GefleCDtbe0XZAuy7Wa49VGkPVPMfLL9qObgBST3lEHJBMUp8S7NdLQAGIvr2KXk8Hc98iKQ== - dependencies: - d3-time "1" - -"d3-time-format@2 - 3": - version "3.0.0" - resolved "https://registry.yarnpkg.com/d3-time-format/-/d3-time-format-3.0.0.tgz#df8056c83659e01f20ac5da5fdeae7c08d5f1bb6" - integrity sha512-UXJh6EKsHBTjopVqZBhFysQcoXSv/5yLONZvkQ5Kk3qbwiUYkdX17Xa1PT6U1ZWXGGfB1ey5L8dKMlFq2DO0Ag== - dependencies: - d3-time "1 - 2" - -d3-time@1: - version "1.1.0" - resolved "https://registry.yarnpkg.com/d3-time/-/d3-time-1.1.0.tgz#b1e19d307dae9c900b7e5b25ffc5dcc249a8a0f1" - integrity sha512-Xh0isrZ5rPYYdqhAVk8VLnMEidhz5aP7htAADH6MfzgmmicPkTo8LhkLxci61/lCB7n7UmE3bN0leRt+qvkLxA== - -"d3-time@1 - 2", d3-time@^2.1.1: - version "2.1.1" - resolved "https://registry.yarnpkg.com/d3-time/-/d3-time-2.1.1.tgz#e9d8a8a88691f4548e68ca085e5ff956724a6682" - integrity sha512-/eIQe/eR4kCQwq7yxi7z4c6qEXf2IYGcjoWB5OOQy4Tq9Uv39/947qlDcN2TLkiTzQWzvnsuYPB9TrWaNfipKQ== - dependencies: - d3-array "2" - -d3-timer@1: - version "1.0.10" - resolved "https://registry.yarnpkg.com/d3-timer/-/d3-timer-1.0.10.tgz#dfe76b8a91748831b13b6d9c793ffbd508dd9de5" - integrity sha512-B1JDm0XDaQC+uvo4DT79H0XmBskgS3l6Ve+1SBCfxgmtIb1AVrPIoqd+nPSv+loMX8szQ0sVUhGngL7D5QPiXw== - -"d3-timer@1 - 3": - version "3.0.1" - resolved "https://registry.yarnpkg.com/d3-timer/-/d3-timer-3.0.1.tgz#6284d2a2708285b1abb7e201eda4380af35e63b0" - integrity sha512-ndfJ/JxxMd3nw31uyKoY2naivF+r29V+Lc0svZxe1JvvIRmi8hUsrMvdOwgS1o6uBHmiz91geQ0ylPP0aj1VUA== - -d3-tip@^0.9.1: - version "0.9.1" - resolved "https://registry.yarnpkg.com/d3-tip/-/d3-tip-0.9.1.tgz#84e6d331c4e6650d80c5228a07e41820609ab64b" - integrity sha512-EVBfG9d+HnjIoyVXfhpytWxlF59JaobwizqMX9EBXtsFmJytjwHeYiUs74ldHQjE7S9vzfKTx2LCtvUrIbuFYg== - dependencies: - d3-collection "^1.0.4" - d3-selection "^1.3.0" - -d3-transition@1: - version "1.3.2" - resolved "https://registry.yarnpkg.com/d3-transition/-/d3-transition-1.3.2.tgz#a98ef2151be8d8600543434c1ca80140ae23b398" - integrity sha512-sc0gRU4PFqZ47lPVHloMn9tlPcv8jxgOQg+0zjhfZXMQuvppjG6YuwdMBE0TuqCZjeJkLecku/l9R0JPcRhaDA== - dependencies: - d3-color "1" - d3-dispatch "1" - d3-ease "1" - d3-interpolate "1" - d3-selection "^1.1.0" - d3-timer "1" - -"d3-transition@2 - 3": - version "3.0.1" - resolved "https://registry.yarnpkg.com/d3-transition/-/d3-transition-3.0.1.tgz#6869fdde1448868077fdd5989200cb61b2a1645f" - integrity sha512-ApKvfjsSR6tg06xrL434C0WydLr7JewBB3V+/39RMHsaXTOG0zmt/OAXeng5M5LBm0ojmxJrpomQVZ1aPvBL4w== - dependencies: - d3-color "1 - 3" - d3-dispatch "1 - 3" - d3-ease "1 - 3" - d3-interpolate "1 - 3" - d3-timer "1 - 3" - -d3-voronoi@1: - version "1.1.4" - resolved "https://registry.yarnpkg.com/d3-voronoi/-/d3-voronoi-1.1.4.tgz#dd3c78d7653d2bb359284ae478645d95944c8297" - integrity sha512-dArJ32hchFsrQ8uMiTBLq256MpnZjeuBtdHpaDlYuQyjU0CVzCJl/BVW+SkszaAeH95D/8gxqAhgx0ouAWAfRg== - -d3-zoom@1: - version "1.8.3" - resolved "https://registry.yarnpkg.com/d3-zoom/-/d3-zoom-1.8.3.tgz#b6a3dbe738c7763121cd05b8a7795ffe17f4fc0a" - integrity sha512-VoLXTK4wvy1a0JpH2Il+F2CiOhVu7VRXWF5M/LroMIh3/zBAC3WAt7QoIvPibOavVo20hN6/37vwAsdBejLyKQ== - dependencies: - d3-dispatch "1" - d3-drag "1" - d3-interpolate "1" - d3-selection "1" - d3-transition "1" - -d3-zoom@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/d3-zoom/-/d3-zoom-3.0.0.tgz#d13f4165c73217ffeaa54295cd6969b3e7aee8f3" - integrity sha512-b8AmV3kfQaqWAuacbPuNbL6vahnOJflOhexLzMMNLga62+/nh0JzvJ0aO/5a5MVgUFGS7Hu1P9P03o3fJkDCyw== - dependencies: - d3-dispatch "1 - 3" - d3-drag "2 - 3" - d3-interpolate "1 - 3" - d3-selection "2 - 3" - d3-transition "2 - 3" - -d3@^3.4.4: - version "3.5.17" - resolved "https://registry.yarnpkg.com/d3/-/d3-3.5.17.tgz#bc46748004378b21a360c9fc7cf5231790762fb8" - integrity sha512-yFk/2idb8OHPKkbAL8QaOaqENNoMhIaSHZerk3oQsECwkObkCpJyjYwCe+OHiq6UEdhe1m8ZGARRRO3ljFjlKg== - -d3@^5.14: - version "5.16.0" - resolved "https://registry.yarnpkg.com/d3/-/d3-5.16.0.tgz#9c5e8d3b56403c79d4ed42fbd62f6113f199c877" - integrity sha512-4PL5hHaHwX4m7Zr1UapXW23apo6pexCgdetdJ5kTmADpG/7T9Gkxw0M0tf/pjoB63ezCCm0u5UaFYy2aMt0Mcw== - dependencies: - d3-array "1" - d3-axis "1" - d3-brush "1" - d3-chord "1" - d3-collection "1" - d3-color "1" - d3-contour "1" - d3-dispatch "1" - d3-drag "1" - d3-dsv "1" - d3-ease "1" - d3-fetch "1" - d3-force "1" - d3-format "1" - d3-geo "1" - d3-hierarchy "1" - d3-interpolate "1" - d3-path "1" - d3-polygon "1" - d3-quadtree "1" - d3-random "1" - d3-scale "2" - d3-scale-chromatic "1" - d3-selection "1" - d3-shape "1" - d3-time "1" - d3-time-format "2" - d3-timer "1" - d3-transition "1" - d3-voronoi "1" - d3-zoom "1" - -dagre-d3@^0.6.4: - version "0.6.4" - resolved "https://registry.yarnpkg.com/dagre-d3/-/dagre-d3-0.6.4.tgz#0728d5ce7f177ca2337df141ceb60fbe6eeb7b29" - integrity sha512-e/6jXeCP7/ptlAM48clmX4xTZc5Ek6T6kagS7Oz2HrYSdqcLZFLqpAfh7ldbZRFfxCZVyh61NEPR08UQRVxJzQ== - dependencies: - d3 "^5.14" - dagre "^0.8.5" - graphlib "^2.1.8" - lodash "^4.17.15" - -dagre@^0.8.5: - version "0.8.5" - resolved "https://registry.yarnpkg.com/dagre/-/dagre-0.8.5.tgz#ba30b0055dac12b6c1fcc247817442777d06afee" - integrity sha512-/aTqmnRta7x7MCCpExk7HQL2O4owCT2h8NT//9I1OQ9vt29Pa0BzSAkR5lwFUcQ7491yVi/3CXU9jQ5o0Mn2Sw== - dependencies: - graphlib "^2.1.8" - lodash "^4.17.15" - -damerau-levenshtein@^1.0.7: - version "1.0.8" - resolved "https://registry.yarnpkg.com/damerau-levenshtein/-/damerau-levenshtein-1.0.8.tgz#b43d286ccbd36bc5b2f7ed41caf2d0aba1f8a6e7" - integrity sha512-sdQSFB7+llfUcQHUQO3+B8ERRj0Oa4w9POWMI/puGtuf7gFywGmkaLCElnudfTiKZV+NvHqL0ifzdrI8Ro7ESA== - -data-urls@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/data-urls/-/data-urls-2.0.0.tgz#156485a72963a970f5d5821aaf642bef2bf2db9b" - integrity sha512-X5eWTSXO/BJmpdIKCRuKUgSCgAN0OwliVK3yPKbwIWU1Tdw5BRajxlzMidvh+gwko9AfQ9zIj52pzF91Q3YAvQ== - dependencies: - abab "^2.0.3" - whatwg-mimetype "^2.3.0" - whatwg-url "^8.0.0" - -date-now@^0.1.4: - version "0.1.4" - resolved "https://registry.yarnpkg.com/date-now/-/date-now-0.1.4.tgz#eaf439fd4d4848ad74e5cc7dbef200672b9e345b" - integrity sha1-6vQ5/U1ISK105cx9vvIAZyueNFs= - -debug@4, debug@^4.0.0, debug@^4.1.0, debug@^4.1.1, debug@^4.3.2, debug@^4.3.4: - version "4.3.4" - resolved "https://registry.yarnpkg.com/debug/-/debug-4.3.4.tgz#1319f6579357f2338d3337d2cdd4914bb5dcc865" - integrity sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ== - dependencies: - ms "2.1.2" - -debug@^3.2.6, debug@^3.2.7: - version "3.2.7" - resolved "https://registry.yarnpkg.com/debug/-/debug-3.2.7.tgz#72580b7e9145fb39b6676f9c5e5fb100b934179a" - integrity sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ== - dependencies: - ms "^2.1.1" - -debug@^4.3.1: - version "4.3.5" - resolved "https://registry.yarnpkg.com/debug/-/debug-4.3.5.tgz#e83444eceb9fedd4a1da56d671ae2446a01a6e1e" - integrity sha512-pt0bNEmneDIvdL1Xsd9oDQ/wrQRkXDT4AUWlNZNPKvW5x/jyO9VFXkJUP07vQ2upmw5PlaITaPKc31jK13V+jg== - dependencies: - ms "2.1.2" - -decamelize-keys@^1.1.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/decamelize-keys/-/decamelize-keys-1.1.0.tgz#d171a87933252807eb3cb61dc1c1445d078df2d9" - integrity sha1-0XGoeTMlKAfrPLYdwcFEXQeN8tk= - dependencies: - decamelize "^1.1.0" - map-obj "^1.0.0" - -decamelize@^1.1.0: - version "1.2.0" - resolved "https://registry.yarnpkg.com/decamelize/-/decamelize-1.2.0.tgz#f6534d15148269b20352e7bee26f501f9a191290" - integrity sha1-9lNNFRSCabIDUue+4m9QH5oZEpA= - -decamelize@^5.0.0: - version "5.0.1" - resolved "https://registry.yarnpkg.com/decamelize/-/decamelize-5.0.1.tgz#db11a92e58c741ef339fb0a2868d8a06a9a7b1e9" - integrity sha512-VfxadyCECXgQlkoEAjeghAr5gY3Hf+IKjKb+X8tGVDtveCjN+USwprd2q3QXBR9T1+x2DG0XZF5/w+7HAtSaXA== - -decimal.js@^10.2.1: - version "10.3.1" - resolved "https://registry.yarnpkg.com/decimal.js/-/decimal.js-10.3.1.tgz#d8c3a444a9c6774ba60ca6ad7261c3a94fd5e783" - integrity sha512-V0pfhfr8suzyPGOx3nmq4aHqabehUZn6Ch9kyFpV79TGDTWFmHqUqXdabR7QHqxzrYolF4+tVmJhUG4OURg5dQ== - -decko@^1.2.0: - version "1.2.0" - resolved "https://registry.yarnpkg.com/decko/-/decko-1.2.0.tgz#fd43c735e967b8013306884a56fbe665996b6817" - integrity sha1-/UPHNelnuAEzBohKVvvmZZlraBc= - -decode-named-character-reference@^1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/decode-named-character-reference/-/decode-named-character-reference-1.0.2.tgz#daabac9690874c394c81e4162a0304b35d824f0e" - integrity sha512-O8x12RzrUF8xyVcY0KJowWsmaJxQbmy0/EtnNtHRpsOcT7dFk5W598coHqBVpmWo1oQQfsCqfCmkZN5DJrZVdg== - dependencies: - character-entities "^2.0.0" - -decode-uri-component@^0.2.0: - version "0.2.2" - resolved "https://registry.yarnpkg.com/decode-uri-component/-/decode-uri-component-0.2.2.tgz#e69dbe25d37941171dd540e024c444cd5188e1e9" - integrity sha512-FqUYQ+8o158GyGTrMFJms9qh3CqTKvAqgqsTnkLI8sKu0028orqBhxNMFkFen0zGyg6epACD32pjVk58ngIErQ== - -dedent@^0.7.0: - version "0.7.0" - resolved "https://registry.yarnpkg.com/dedent/-/dedent-0.7.0.tgz#2495ddbaf6eb874abb0e1be9df22d2e5a544326c" - integrity sha1-JJXduvbrh0q7Dhvp3yLS5aVEMmw= - -deep-is@^0.1.3, deep-is@~0.1.3: - version "0.1.4" - resolved "https://registry.yarnpkg.com/deep-is/-/deep-is-0.1.4.tgz#a6f2dce612fadd2ef1f519b73551f17e85199831" - integrity sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ== - -deepmerge@^4.2.2: - version "4.2.2" - resolved "https://registry.yarnpkg.com/deepmerge/-/deepmerge-4.2.2.tgz#44d2ea3679b8f4d4ffba33f03d865fc1e7bf4955" - integrity sha512-FJ3UgI4gIl+PHZm53knsuSFpE+nESMr7M4v9QcgB7S63Kj/6WqMiFQJpBBYz1Pt+66bZpP3Q7Lye0Oo9MPKEdg== - -define-properties@^1.1.3: - version "1.1.3" - resolved "https://registry.yarnpkg.com/define-properties/-/define-properties-1.1.3.tgz#cf88da6cbee26fe6db7094f61d870cbd84cee9f1" - integrity sha512-3MqfYKj2lLzdMSf8ZIZE/V+Zuy+BgD6f164e8K2w7dgnpKArBDerGYpM46IYYcjnkdPNMjPk9A6VFB8+3SKlXQ== - dependencies: - object-keys "^1.0.12" - -define-properties@^1.1.4: - version "1.1.4" - resolved "https://registry.yarnpkg.com/define-properties/-/define-properties-1.1.4.tgz#0b14d7bd7fbeb2f3572c3a7eda80ea5d57fb05b1" - integrity sha512-uckOqKcfaVvtBdsVkdPv3XjveQJsNQqmhXgRi8uhvWWuPYZCNlzT8qAyblUgNoXdHdjMTzAqeGjAoli8f+bzPA== - dependencies: - has-property-descriptors "^1.0.0" - object-keys "^1.1.1" - -del@^4.1.1: - version "4.1.1" - resolved "https://registry.yarnpkg.com/del/-/del-4.1.1.tgz#9e8f117222ea44a31ff3a156c049b99052a9f0b4" - integrity sha512-QwGuEUouP2kVwQenAsOof5Fv8K9t3D8Ca8NxcXKrIpEHjTXK5J2nXLdP+ALI1cgv8wj7KuwBhTwBkOZSJKM5XQ== - dependencies: - "@types/glob" "^7.1.1" - globby "^6.1.0" - is-path-cwd "^2.0.0" - is-path-in-cwd "^2.0.0" - p-map "^2.0.0" - pify "^4.0.1" - rimraf "^2.6.3" - -delayed-stream@~1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/delayed-stream/-/delayed-stream-1.0.0.tgz#df3ae199acadfb7d440aaae0b29e2272b24ec619" - integrity sha1-3zrhmayt+31ECqrgsp4icrJOxhk= - -dequal@^2.0.0: - version "2.0.3" - resolved "https://registry.yarnpkg.com/dequal/-/dequal-2.0.3.tgz#2644214f1997d39ed0ee0ece72335490a7ac67be" - integrity sha512-0je+qPKHEMohvfRTCEo3CrPG6cAzAYgmzKyxRiYSSDkS6eGJdyVJm7WaYA5ECaAD9wLB2T4EEeymA5aFVcYXCA== - -detect-newline@^3.0.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/detect-newline/-/detect-newline-3.1.0.tgz#576f5dfc63ae1a192ff192d8ad3af6308991b651" - integrity sha512-TLz+x/vEXm/Y7P7wn1EJFNLxYpUD4TgMosxY6fAVJUnJMbupHBOncxyWUG9OpTaH9EBD7uFI5LfEgmMOc54DsA== - -detect-node-es@^1.1.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/detect-node-es/-/detect-node-es-1.1.0.tgz#163acdf643330caa0b4cd7c21e7ee7755d6fa493" - integrity sha512-ypdmJU/TbBby2Dxibuv7ZLW3Bs1QEmM7nHjEANfohJLvE0XVujisn1qPJcZxg+qDucsr+bP6fLD1rPS3AhJ7EQ== - -detect-node@^2.0.4, detect-node@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/detect-node/-/detect-node-2.1.0.tgz#c9c70775a49c3d03bc2c06d9a73be550f978f8b1" - integrity sha512-T0NIuQpnTvFDATNuHN5roPwSBG83rFsuO+MXXH9/3N1eFbn4wcPjttvjMLEPWJ0RGUYgQE7cGgS3tNxbqCGM7g== - -diff-sequences@^27.0.6: - version "27.0.6" - resolved "https://registry.yarnpkg.com/diff-sequences/-/diff-sequences-27.0.6.tgz#3305cb2e55a033924054695cc66019fd7f8e5723" - integrity sha512-ag6wfpBFyNXZ0p8pcuIDS//D8H062ZQJ3fzYxjpmeKjnz8W4pekL3AI8VohmyZmsWW2PWaHgjsmqR6L13101VQ== - -diff-sequences@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/diff-sequences/-/diff-sequences-27.5.1.tgz#eaecc0d327fd68c8d9672a1e64ab8dccb2ef5327" - integrity sha512-k1gCAXAsNgLwEL+Y8Wvl+M6oEFj5bgazfZULpS5CneoPPXRaCCW7dm+q21Ky2VEE5X+VeRDBVg1Pcvvsr4TtNQ== - -diff@^5.0.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/diff/-/diff-5.1.0.tgz#bc52d298c5ea8df9194800224445ed43ffc87e40" - integrity sha512-D+mk+qE8VC/PAUrlAU34N+VfXev0ghe5ywmpqrawphmVZc1bEfn56uo9qpyGp1p4xpzOHkSW4ztBd6L7Xx4ACw== - -dir-glob@^3.0.1: - version "3.0.1" - resolved "https://registry.yarnpkg.com/dir-glob/-/dir-glob-3.0.1.tgz#56dbf73d992a4a93ba1584f4534063fd2e41717f" - integrity sha512-WkrWp9GR4KXfKGYzOLmTuGVi1UWFfws377n9cc55/tb6DuqyF6pcQ5AbiHEshaDpY9v6oaSr2XCDidGmMwdzIA== - dependencies: - path-type "^4.0.0" - -doctrine@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/doctrine/-/doctrine-2.1.0.tgz#5cd01fc101621b42c4cd7f5d1a66243716d3f39d" - integrity sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw== - dependencies: - esutils "^2.0.2" - -doctrine@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/doctrine/-/doctrine-3.0.0.tgz#addebead72a6574db783639dc87a121773973961" - integrity sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w== - dependencies: - esutils "^2.0.2" - -dom-accessibility-api@^0.5.6, dom-accessibility-api@^0.5.9: - version "0.5.10" - resolved "https://registry.yarnpkg.com/dom-accessibility-api/-/dom-accessibility-api-0.5.10.tgz#caa6d08f60388d0bb4539dd75fe458a9a1d0014c" - integrity sha512-Xu9mD0UjrJisTmv7lmVSDMagQcU9R5hwAbxsaAE/35XPnPLJobbuREfV/rraiSaEj/UOvgrzQs66zyTWTlyd+g== - -dom-helpers@^5.0.1: - version "5.2.1" - resolved "https://registry.yarnpkg.com/dom-helpers/-/dom-helpers-5.2.1.tgz#d9400536b2bf8225ad98fe052e029451ac40e902" - integrity sha512-nRCa7CK3VTrM2NmGkIy4cbK7IZlgBE/PYMn55rrXefr5xXDP0LdtfPnblFDoVdcAfslJ7or6iqAUnx0CCGIWQA== - dependencies: - "@babel/runtime" "^7.8.7" - csstype "^3.0.2" - -dom-serializer@0: - version "0.2.2" - resolved "https://registry.yarnpkg.com/dom-serializer/-/dom-serializer-0.2.2.tgz#1afb81f533717175d478655debc5e332d9f9bb51" - integrity sha512-2/xPb3ORsQ42nHYiSunXkDjPLBaEj/xTwUO4B7XCZQTRk7EBtTOPaygh10YAAh2OI1Qrp6NWfpAhzswj0ydt9g== - dependencies: - domelementtype "^2.0.1" - entities "^2.0.0" - -dom-serializer@^1.0.1: - version "1.3.2" - resolved "https://registry.yarnpkg.com/dom-serializer/-/dom-serializer-1.3.2.tgz#6206437d32ceefaec7161803230c7a20bc1b4d91" - integrity sha512-5c54Bk5Dw4qAxNOI1pFEizPSjVsx5+bpJKmL2kPn8JhBUq2q09tTCa3mjijun2NfK78NMouDYNMBkOrPZiS+ig== - dependencies: - domelementtype "^2.0.1" - domhandler "^4.2.0" - entities "^2.0.0" - -domelementtype@1: - version "1.3.1" - resolved "https://registry.yarnpkg.com/domelementtype/-/domelementtype-1.3.1.tgz#d048c44b37b0d10a7f2a3d5fee3f4333d790481f" - integrity sha512-BSKB+TSpMpFI/HOxCNr1O8aMOTZ8hT3pM3GQ0w/mWRmkhEDSFJkkyzz4XQsBV44BChwGkrDfMyjVD0eA2aFV3w== - -domelementtype@^2.0.1, domelementtype@^2.2.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/domelementtype/-/domelementtype-2.2.0.tgz#9a0b6c2782ed6a1c7323d42267183df9bd8b1d57" - integrity sha512-DtBMo82pv1dFtUmHyr48beiuq792Sxohr+8Hm9zoxklYPfa6n0Z3Byjj2IV7bmr2IyqClnqEQhfgHJJ5QF0R5A== - -domexception@^2.0.1: - version "2.0.1" - resolved "https://registry.yarnpkg.com/domexception/-/domexception-2.0.1.tgz#fb44aefba793e1574b0af6aed2801d057529f304" - integrity sha512-yxJ2mFy/sibVQlu5qHjOkf9J3K6zgmCxgJ94u2EdvDOV09H+32LtRswEcUsmUWN72pVLOEnTSRaIVVzVQgS0dg== - dependencies: - webidl-conversions "^5.0.0" - -domhandler@2.3: - version "2.3.0" - resolved "https://registry.yarnpkg.com/domhandler/-/domhandler-2.3.0.tgz#2de59a0822d5027fabff6f032c2b25a2a8abe738" - integrity sha1-LeWaCCLVAn+r/28DLCsloqir5zg= - dependencies: - domelementtype "1" - -domhandler@^4.0.0, domhandler@^4.2.0: - version "4.2.0" - resolved "https://registry.yarnpkg.com/domhandler/-/domhandler-4.2.0.tgz#f9768a5f034be60a89a27c2e4d0f74eba0d8b059" - integrity sha512-zk7sgt970kzPks2Bf+dwT/PLzghLnsivb9CcxkvR8Mzr66Olr0Ofd8neSbglHJHaHa2MadfoSdNlKYAaafmWfA== - dependencies: - domelementtype "^2.2.0" - -domhandler@^4.3.1: - version "4.3.1" - resolved "https://registry.yarnpkg.com/domhandler/-/domhandler-4.3.1.tgz#8d792033416f59d68bc03a5aa7b018c1ca89279c" - integrity sha512-GrwoxYN+uWlzO8uhUXRl0P+kHE4GtVPfYzVLcUxPL7KNdHKj66vvlhiweIHqYYXWlw+T8iLMp42Lm67ghw4WMQ== - dependencies: - domelementtype "^2.2.0" - -dompurify@^2.2.8: - version "2.5.6" - resolved "https://registry.yarnpkg.com/dompurify/-/dompurify-2.5.6.tgz#8402b501611eaa7fb3786072297fcbe2787f8592" - integrity sha512-zUTaUBO8pY4+iJMPE1B9XlO2tXVYIcEA4SNGtvDELzTSCQO7RzH+j7S180BmhmJId78lqGU2z19vgVx2Sxs/PQ== - -domutils@1.5: - version "1.5.1" - resolved "https://registry.yarnpkg.com/domutils/-/domutils-1.5.1.tgz#dcd8488a26f563d61079e48c9f7b7e32373682cf" - integrity sha1-3NhIiib1Y9YQeeSMn3t+Mjc2gs8= - dependencies: - dom-serializer "0" - domelementtype "1" - -domutils@^2.5.2: - version "2.7.0" - resolved "https://registry.yarnpkg.com/domutils/-/domutils-2.7.0.tgz#8ebaf0c41ebafcf55b0b72ec31c56323712c5442" - integrity sha512-8eaHa17IwJUPAiB+SoTYBo5mCdeMgdcAoXJ59m6DT1vw+5iLS3gNoqYaRowaBKtGVrOF1Jz4yDTgYKLK2kvfJg== - dependencies: - dom-serializer "^1.0.1" - domelementtype "^2.2.0" - domhandler "^4.2.0" - -domutils@^2.8.0: - version "2.8.0" - resolved "https://registry.yarnpkg.com/domutils/-/domutils-2.8.0.tgz#4437def5db6e2d1f5d6ee859bd95ca7d02048135" - integrity sha512-w96Cjofp72M5IIhpjgobBimYEfoPjx1Vx0BSX9P30WBdZW2WIKU0T1Bd0kz2eNZ9ikjKgHbEyKx8BB6H1L3h3A== - dependencies: - dom-serializer "^1.0.1" - domelementtype "^2.2.0" - domhandler "^4.2.0" - -echarts@^5.4.2: - version "5.4.2" - resolved "https://registry.yarnpkg.com/echarts/-/echarts-5.4.2.tgz#9f38781c9c6ae323e896956178f6956952c77a48" - integrity sha512-2W3vw3oI2tWJdyAz+b8DuWS0nfXtSDqlDmqgin/lfzbkB01cuMEN66KWBlmur3YMp5nEDEEt5s23pllnAzB4EA== - dependencies: - tslib "2.3.0" - zrender "5.4.3" - -electron-to-chromium@^1.5.4: - version "1.5.13" - resolved "https://registry.yarnpkg.com/electron-to-chromium/-/electron-to-chromium-1.5.13.tgz#1abf0410c5344b2b829b7247e031f02810d442e6" - integrity sha512-lbBcvtIJ4J6sS4tb5TLp1b4LyfCdMkwStzXPyAgVgTRAsep4bvrAGaBOP7ZJtQMNJpSQ9SqG4brWOroNaQtm7Q== - -elkjs@^0.7.1: - version "0.7.1" - resolved "https://registry.yarnpkg.com/elkjs/-/elkjs-0.7.1.tgz#4751c5e918a4988139baf7f214e010aea22de969" - integrity sha512-lD86RWdh480/UuRoHhRcnv2IMkIcK6yMDEuT8TPBIbO3db4HfnVF+1lgYdQi99Ck0yb+lg5Eb46JCHI5uOsmAw== - -emittery@^0.8.1: - version "0.8.1" - resolved "https://registry.yarnpkg.com/emittery/-/emittery-0.8.1.tgz#bb23cc86d03b30aa75a7f734819dee2e1ba70860" - integrity sha512-uDfvUjVrfGJJhymx/kz6prltenw1u7WrCg1oa94zYY8xxVpLLUu045LAT0dhDZdXG58/EpPL/5kA180fQ/qudg== - -emoji-regex@^8.0.0: - version "8.0.0" - resolved "https://registry.yarnpkg.com/emoji-regex/-/emoji-regex-8.0.0.tgz#e818fd69ce5ccfcb404594f842963bf53164cc37" - integrity sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A== - -emoji-regex@^9.2.2: - version "9.2.2" - resolved "https://registry.yarnpkg.com/emoji-regex/-/emoji-regex-9.2.2.tgz#840c8803b0d8047f4ff0cf963176b32d4ef3ed72" - integrity sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg== - -emojis-list@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/emojis-list/-/emojis-list-3.0.0.tgz#5570662046ad29e2e916e71aae260abdff4f6a78" - integrity sha512-/kyM18EfinwXZbno9FyUGeFh87KC8HRQBQGildHZbEuRyWFOmv1U10o9BBp8XVZDVNNuQKyIGIu5ZYAAXJ0V2Q== - -enhanced-resolve@^5.17.1: - version "5.17.1" - resolved "https://registry.yarnpkg.com/enhanced-resolve/-/enhanced-resolve-5.17.1.tgz#67bfbbcc2f81d511be77d686a90267ef7f898a15" - integrity sha512-LMHl3dXhTcfv8gM4kEzIUeTQ+7fpdA0l2tUf34BddXPkz2A5xJ5L/Pchd5BL6rdccM9QGvu0sWZzK1Z1t4wwyg== - dependencies: - graceful-fs "^4.2.4" - tapable "^2.2.0" - -entities@1.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/entities/-/entities-1.0.0.tgz#b2987aa3821347fcde642b24fdfc9e4fb712bf26" - integrity sha1-sph6o4ITR/zeZCsk/fyeT7cSvyY= - -entities@^2.0.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/entities/-/entities-2.2.0.tgz#098dc90ebb83d8dffa089d55256b351d34c4da55" - integrity sha512-p92if5Nz619I0w+akJrLZH0MX0Pb5DX39XOwQTtXSdQQOaYH03S1uIQp4mhOZtAXrxq4ViO67YTiLBo2638o9A== - -envinfo@^7.7.3: - version "7.8.1" - resolved "https://registry.yarnpkg.com/envinfo/-/envinfo-7.8.1.tgz#06377e3e5f4d379fea7ac592d5ad8927e0c4d475" - integrity sha512-/o+BXHmB7ocbHEAs6F2EnG0ogybVVUdkRunTT2glZU9XAaGmhqskrvKwqXuDfNjEO0LZKWdejEEpnq8aM0tOaw== - -eonasdan-bootstrap-datetimepicker@^4.17.47: - version "4.17.49" - resolved "https://registry.yarnpkg.com/eonasdan-bootstrap-datetimepicker/-/eonasdan-bootstrap-datetimepicker-4.17.49.tgz#5534ba581c1e7eb988dbf773e2fed8a7f48cc76a" - integrity sha512-7KZeDpkj+A6AtPR3XjX8gAnRPUkPSfW0OmMANG1dkUOPMtLSzbyoCjDIdEcfRtQPU5X0D9Gob7wWKn0h4QWy7A== - dependencies: - bootstrap "^3.3" - jquery "^3.5.1" - moment "^2.10" - moment-timezone "^0.4.0" - -error-ex@^1.3.1: - version "1.3.2" - resolved "https://registry.yarnpkg.com/error-ex/-/error-ex-1.3.2.tgz#b4ac40648107fdcdcfae242f428bea8a14d4f1bf" - integrity sha512-7dFHNmqeFSEt2ZBsCriorKnn3Z2pj+fd9kmI6QoWw4//DL+icEBfc0U7qJCisqrTsKTjw4fNFy2pW9OqStD84g== - dependencies: - is-arrayish "^0.2.1" - -es-abstract@^1.18.0-next.2: - version "1.18.3" - resolved "https://registry.yarnpkg.com/es-abstract/-/es-abstract-1.18.3.tgz#25c4c3380a27aa203c44b2b685bba94da31b63e0" - integrity sha512-nQIr12dxV7SSxE6r6f1l3DtAeEYdsGpps13dR0TwJg1S8gyp4ZPgy3FZcHBgbiQqnoqSTb+oC+kO4UQ0C/J8vw== - dependencies: - call-bind "^1.0.2" - es-to-primitive "^1.2.1" - function-bind "^1.1.1" - get-intrinsic "^1.1.1" - has "^1.0.3" - has-symbols "^1.0.2" - is-callable "^1.2.3" - is-negative-zero "^2.0.1" - is-regex "^1.1.3" - is-string "^1.0.6" - object-inspect "^1.10.3" - object-keys "^1.1.1" - object.assign "^4.1.2" - string.prototype.trimend "^1.0.4" - string.prototype.trimstart "^1.0.4" - unbox-primitive "^1.0.1" - -es-abstract@^1.19.0, es-abstract@^1.19.1, es-abstract@^1.19.2, es-abstract@^1.19.5: - version "1.20.1" - resolved "https://registry.yarnpkg.com/es-abstract/-/es-abstract-1.20.1.tgz#027292cd6ef44bd12b1913b828116f54787d1814" - integrity sha512-WEm2oBhfoI2sImeM4OF2zE2V3BYdSF+KnSi9Sidz51fQHd7+JuF8Xgcj9/0o+OWeIeIS/MiuNnlruQrJf16GQA== - dependencies: - call-bind "^1.0.2" - es-to-primitive "^1.2.1" - function-bind "^1.1.1" - function.prototype.name "^1.1.5" - get-intrinsic "^1.1.1" - get-symbol-description "^1.0.0" - has "^1.0.3" - has-property-descriptors "^1.0.0" - has-symbols "^1.0.3" - internal-slot "^1.0.3" - is-callable "^1.2.4" - is-negative-zero "^2.0.2" - is-regex "^1.1.4" - is-shared-array-buffer "^1.0.2" - is-string "^1.0.7" - is-weakref "^1.0.2" - object-inspect "^1.12.0" - object-keys "^1.1.1" - object.assign "^4.1.2" - regexp.prototype.flags "^1.4.3" - string.prototype.trimend "^1.0.5" - string.prototype.trimstart "^1.0.5" - unbox-primitive "^1.0.2" - -es-abstract@^1.20.4: - version "1.21.2" - resolved "https://registry.yarnpkg.com/es-abstract/-/es-abstract-1.21.2.tgz#a56b9695322c8a185dc25975aa3b8ec31d0e7eff" - integrity sha512-y/B5POM2iBnIxCiernH1G7rC9qQoM77lLIMQLuob0zhp8C56Po81+2Nj0WFKnd0pNReDTnkYryc+zhOzpEIROg== - dependencies: - array-buffer-byte-length "^1.0.0" - available-typed-arrays "^1.0.5" - call-bind "^1.0.2" - es-set-tostringtag "^2.0.1" - es-to-primitive "^1.2.1" - function.prototype.name "^1.1.5" - get-intrinsic "^1.2.0" - get-symbol-description "^1.0.0" - globalthis "^1.0.3" - gopd "^1.0.1" - has "^1.0.3" - has-property-descriptors "^1.0.0" - has-proto "^1.0.1" - has-symbols "^1.0.3" - internal-slot "^1.0.5" - is-array-buffer "^3.0.2" - is-callable "^1.2.7" - is-negative-zero "^2.0.2" - is-regex "^1.1.4" - is-shared-array-buffer "^1.0.2" - is-string "^1.0.7" - is-typed-array "^1.1.10" - is-weakref "^1.0.2" - object-inspect "^1.12.3" - object-keys "^1.1.1" - object.assign "^4.1.4" - regexp.prototype.flags "^1.4.3" - safe-regex-test "^1.0.0" - string.prototype.trim "^1.2.7" - string.prototype.trimend "^1.0.6" - string.prototype.trimstart "^1.0.6" - typed-array-length "^1.0.4" - unbox-primitive "^1.0.2" - which-typed-array "^1.1.9" - -es-module-lexer@^1.2.1: - version "1.5.4" - resolved "https://registry.yarnpkg.com/es-module-lexer/-/es-module-lexer-1.5.4.tgz#a8efec3a3da991e60efa6b633a7cad6ab8d26b78" - integrity sha512-MVNK56NiMrOwitFB7cqDwq0CQutbw+0BvLshJSse0MUNU+y1FC3bUS/AQg7oUng+/wKrrki7JfmwtVHkVfPLlw== - -es-set-tostringtag@^2.0.1: - version "2.0.1" - resolved "https://registry.yarnpkg.com/es-set-tostringtag/-/es-set-tostringtag-2.0.1.tgz#338d502f6f674301d710b80c8592de8a15f09cd8" - integrity sha512-g3OMbtlwY3QewlqAiMLI47KywjWZoEytKr8pf6iTC8uJq5bIAH52Z9pnQ8pVL6whrCto53JZDuUIsifGeLorTg== - dependencies: - get-intrinsic "^1.1.3" - has "^1.0.3" - has-tostringtag "^1.0.0" - -es-shim-unscopables@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/es-shim-unscopables/-/es-shim-unscopables-1.0.0.tgz#702e632193201e3edf8713635d083d378e510241" - integrity sha512-Jm6GPcCdC30eMLbZ2x8z2WuRwAws3zTBBKuusffYVUrNj/GVSUAZ+xKMaUpfNDR5IbyNA5LJbaecoUVbmUcB1w== - dependencies: - has "^1.0.3" - -es-to-primitive@^1.2.1: - version "1.2.1" - resolved "https://registry.yarnpkg.com/es-to-primitive/-/es-to-primitive-1.2.1.tgz#e55cd4c9cdc188bcefb03b366c736323fc5c898a" - integrity sha512-QCOllgZJtaUo9miYBcLChTUaHNjJF3PYs1VidD7AwiEj1kYxKeQTctLAezAOH5ZKRH0g2IgPn6KwB4IT8iRpvA== - dependencies: - is-callable "^1.1.4" - is-date-object "^1.0.1" - is-symbol "^1.0.2" - -es6-promise@^3.2.1: - version "3.3.1" - resolved "https://registry.yarnpkg.com/es6-promise/-/es6-promise-3.3.1.tgz#a08cdde84ccdbf34d027a1451bc91d4bcd28a613" - integrity sha1-oIzd6EzNvzTQJ6FFG8kdS80ophM= - -escalade@^3.1.1: - version "3.1.1" - resolved "https://registry.yarnpkg.com/escalade/-/escalade-3.1.1.tgz#d8cfdc7000965c5a0174b4a82eaa5c0552742e40" - integrity sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw== - -escalade@^3.1.2: - version "3.1.2" - resolved "https://registry.yarnpkg.com/escalade/-/escalade-3.1.2.tgz#54076e9ab29ea5bf3d8f1ed62acffbb88272df27" - integrity sha512-ErCHMCae19vR8vQGe50xIsVomy19rg6gFu3+r3jkEO46suLMWBksvVyoGgQV+jOfl84ZSOSlmv6Gxa89PmTGmA== - -escape-string-regexp@^1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz#1b61c0562190a8dff6ae3bb2cf0200ca130b86d4" - integrity sha1-G2HAViGQqN/2rjuyzwIAyhMLhtQ= - -escape-string-regexp@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz#a30304e99daa32e23b2fd20f51babd07cffca344" - integrity sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w== - -escape-string-regexp@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz#14ba83a5d373e3d311e5afca29cf5bfad965bf34" - integrity sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA== - -escape-string-regexp@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-5.0.0.tgz#4683126b500b61762f2dbebace1806e8be31b1c8" - integrity sha512-/veY75JbMK4j1yjvuUxuVsiS/hr/4iHs9FTT6cgTexxdE0Ly/glccBAkloH/DofkjRbZU3bnoj38mOmhkZ0lHw== - -escodegen@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/escodegen/-/escodegen-2.0.0.tgz#5e32b12833e8aa8fa35e1bf0befa89380484c7dd" - integrity sha512-mmHKys/C8BFUGI+MAWNcSYoORYLMdPzjrknd2Vc+bUsjN5bXcr8EhrNB+UTqfL1y3I9c4fw2ihgtMPQLBRiQxw== - dependencies: - esprima "^4.0.1" - estraverse "^5.2.0" - esutils "^2.0.2" - optionator "^0.8.1" - optionalDependencies: - source-map "~0.6.1" - -eslint-config-airbnb-base@^15.0.0: - version "15.0.0" - resolved "https://registry.yarnpkg.com/eslint-config-airbnb-base/-/eslint-config-airbnb-base-15.0.0.tgz#6b09add90ac79c2f8d723a2580e07f3925afd236" - integrity sha512-xaX3z4ZZIcFLvh2oUNvcX5oEofXda7giYmuplVxoOg5A7EXJMrUyqRgR+mhDhPK8LZ4PttFOBvCYDbX3sUoUig== - dependencies: - confusing-browser-globals "^1.0.10" - object.assign "^4.1.2" - object.entries "^1.1.5" - semver "^6.3.0" - -eslint-config-airbnb-typescript@^17.0.0: - version "17.0.0" - resolved "https://registry.yarnpkg.com/eslint-config-airbnb-typescript/-/eslint-config-airbnb-typescript-17.0.0.tgz#360dbcf810b26bbcf2ff716198465775f1c49a07" - integrity sha512-elNiuzD0kPAPTXjFWg+lE24nMdHMtuxgYoD30OyMD6yrW1AhFZPAg27VX7d3tzOErw+dgJTNWfRSDqEcXb4V0g== - dependencies: - eslint-config-airbnb-base "^15.0.0" - -eslint-config-airbnb@^19.0.4: - version "19.0.4" - resolved "https://registry.yarnpkg.com/eslint-config-airbnb/-/eslint-config-airbnb-19.0.4.tgz#84d4c3490ad70a0ffa571138ebcdea6ab085fdc3" - integrity sha512-T75QYQVQX57jiNgpF9r1KegMICE94VYwoFQyMGhrvc+lB8YF2E/M/PYDaQe1AJcWaEgqLE+ErXV1Og/+6Vyzew== - dependencies: - eslint-config-airbnb-base "^15.0.0" - object.assign "^4.1.2" - object.entries "^1.1.5" - -eslint-config-prettier@^8.6.0: - version "8.6.0" - resolved "https://registry.yarnpkg.com/eslint-config-prettier/-/eslint-config-prettier-8.6.0.tgz#dec1d29ab728f4fa63061774e1672ac4e363d207" - integrity sha512-bAF0eLpLVqP5oEVUFKpMA+NnRFICwn9X8B5jrR9FcqnYBuPbqWEjTEspPWMj5ye6czoSLDweCzSo3Ko7gGrZaA== - -eslint-import-resolver-node@^0.3.7: - version "0.3.7" - resolved "https://registry.yarnpkg.com/eslint-import-resolver-node/-/eslint-import-resolver-node-0.3.7.tgz#83b375187d412324a1963d84fa664377a23eb4d7" - integrity sha512-gozW2blMLJCeFpBwugLTGyvVjNoeo1knonXAcatC6bjPBZitotxdWf7Gimr25N4c0AAOo4eOUfaG82IJPDpqCA== - dependencies: - debug "^3.2.7" - is-core-module "^2.11.0" - resolve "^1.22.1" - -eslint-module-utils@^2.7.4: - version "2.7.4" - resolved "https://registry.yarnpkg.com/eslint-module-utils/-/eslint-module-utils-2.7.4.tgz#4f3e41116aaf13a20792261e61d3a2e7e0583974" - integrity sha512-j4GT+rqzCoRKHwURX7pddtIPGySnX9Si/cgMI5ztrcqOPtk5dDEeZ34CQVPphnqkJytlc97Vuk05Um2mJ3gEQA== - dependencies: - debug "^3.2.7" - -eslint-plugin-es@^3.0.0: - version "3.0.1" - resolved "https://registry.yarnpkg.com/eslint-plugin-es/-/eslint-plugin-es-3.0.1.tgz#75a7cdfdccddc0589934aeeb384175f221c57893" - integrity sha512-GUmAsJaN4Fc7Gbtl8uOBlayo2DqhwWvEzykMHSCZHU3XdJ+NSzzZcVhXh3VxX5icqQ+oQdIEawXX8xkR3mIFmQ== - dependencies: - eslint-utils "^2.0.0" - regexpp "^3.0.0" - -eslint-plugin-html@^6.0.2: - version "6.1.2" - resolved "https://registry.yarnpkg.com/eslint-plugin-html/-/eslint-plugin-html-6.1.2.tgz#fa26e4804428956c80e963b6499c192061c2daf3" - integrity sha512-bhBIRyZFqI4EoF12lGDHAmgfff8eLXx6R52/K3ESQhsxzCzIE6hdebS7Py651f7U3RBotqroUnC3L29bR7qJWQ== - dependencies: - htmlparser2 "^6.0.1" - -eslint-plugin-import@^2.27.5: - version "2.27.5" - resolved "https://registry.yarnpkg.com/eslint-plugin-import/-/eslint-plugin-import-2.27.5.tgz#876a6d03f52608a3e5bb439c2550588e51dd6c65" - integrity sha512-LmEt3GVofgiGuiE+ORpnvP+kAm3h6MLZJ4Q5HCyHADofsb4VzXFsRiWj3c0OFiV+3DWFh0qg3v9gcPlfc3zRow== - dependencies: - array-includes "^3.1.6" - array.prototype.flat "^1.3.1" - array.prototype.flatmap "^1.3.1" - debug "^3.2.7" - doctrine "^2.1.0" - eslint-import-resolver-node "^0.3.7" - eslint-module-utils "^2.7.4" - has "^1.0.3" - is-core-module "^2.11.0" - is-glob "^4.0.3" - minimatch "^3.1.2" - object.values "^1.1.6" - resolve "^1.22.1" - semver "^6.3.0" - tsconfig-paths "^3.14.1" - -eslint-plugin-jsx-a11y@^6.5.0: - version "6.5.1" - resolved "https://registry.yarnpkg.com/eslint-plugin-jsx-a11y/-/eslint-plugin-jsx-a11y-6.5.1.tgz#cdbf2df901040ca140b6ec14715c988889c2a6d8" - integrity sha512-sVCFKX9fllURnXT2JwLN5Qgo24Ug5NF6dxhkmxsMEUZhXRcGg+X3e1JbJ84YePQKBl5E0ZjAH5Q4rkdcGY99+g== - dependencies: - "@babel/runtime" "^7.16.3" - aria-query "^4.2.2" - array-includes "^3.1.4" - ast-types-flow "^0.0.7" - axe-core "^4.3.5" - axobject-query "^2.2.0" - damerau-levenshtein "^1.0.7" - emoji-regex "^9.2.2" - has "^1.0.3" - jsx-ast-utils "^3.2.1" - language-tags "^1.0.5" - minimatch "^3.0.4" - -eslint-plugin-node@^11.1.0: - version "11.1.0" - resolved "https://registry.yarnpkg.com/eslint-plugin-node/-/eslint-plugin-node-11.1.0.tgz#c95544416ee4ada26740a30474eefc5402dc671d" - integrity sha512-oUwtPJ1W0SKD0Tr+wqu92c5xuCeQqB3hSCHasn/ZgjFdA9iDGNkNf2Zi9ztY7X+hNuMib23LNGRm6+uN+KLE3g== - dependencies: - eslint-plugin-es "^3.0.0" - eslint-utils "^2.0.0" - ignore "^5.1.1" - minimatch "^3.0.4" - resolve "^1.10.1" - semver "^6.1.0" - -eslint-plugin-promise@^4.2.1: - version "4.3.1" - resolved "https://registry.yarnpkg.com/eslint-plugin-promise/-/eslint-plugin-promise-4.3.1.tgz#61485df2a359e03149fdafc0a68b0e030ad2ac45" - integrity sha512-bY2sGqyptzFBDLh/GMbAxfdJC+b0f23ME63FOE4+Jao0oZ3E1LEwFtWJX/1pGMJLiTtrSSern2CRM/g+dfc0eQ== - -eslint-plugin-react-hooks@^4.5.0: - version "4.6.0" - resolved "https://registry.yarnpkg.com/eslint-plugin-react-hooks/-/eslint-plugin-react-hooks-4.6.0.tgz#4c3e697ad95b77e93f8646aaa1630c1ba607edd3" - integrity sha512-oFc7Itz9Qxh2x4gNHStv3BqJq54ExXmfC+a1NjAta66IAN87Wu0R/QArgIS9qKzX3dXKPI9H5crl9QchNMY9+g== - -eslint-plugin-react@^7.30.0: - version "7.30.0" - resolved "https://registry.yarnpkg.com/eslint-plugin-react/-/eslint-plugin-react-7.30.0.tgz#8e7b1b2934b8426ac067a0febade1b13bd7064e3" - integrity sha512-RgwH7hjW48BleKsYyHK5vUAvxtE9SMPDKmcPRQgtRCYaZA0XQPt5FSkrU3nhz5ifzMZcA8opwmRJ2cmOO8tr5A== - dependencies: - array-includes "^3.1.5" - array.prototype.flatmap "^1.3.0" - doctrine "^2.1.0" - estraverse "^5.3.0" - jsx-ast-utils "^2.4.1 || ^3.0.0" - minimatch "^3.1.2" - object.entries "^1.1.5" - object.fromentries "^2.0.5" - object.hasown "^1.1.1" - object.values "^1.1.5" - prop-types "^15.8.1" - resolve "^2.0.0-next.3" - semver "^6.3.0" - string.prototype.matchall "^4.0.7" - -eslint-plugin-standard@^4.0.1: - version "4.1.0" - resolved "https://registry.yarnpkg.com/eslint-plugin-standard/-/eslint-plugin-standard-4.1.0.tgz#0c3bf3a67e853f8bbbc580fb4945fbf16f41b7c5" - integrity sha512-ZL7+QRixjTR6/528YNGyDotyffm5OQst/sGxKDwGb9Uqs4In5Egi4+jbobhqJoyoCM6/7v/1A5fhQ7ScMtDjaQ== - -eslint-scope@5.1.1, eslint-scope@^5.1.1: - version "5.1.1" - resolved "https://registry.yarnpkg.com/eslint-scope/-/eslint-scope-5.1.1.tgz#e786e59a66cb92b3f6c1fb0d508aab174848f48c" - integrity sha512-2NxwbF/hZ0KpepYN0cNbo+FN6XoK7GaHlQhgx/hIZl6Va0bF45RQOOwhLIy8lQDbuCiadSLCBnH2CFYquit5bw== - dependencies: - esrecurse "^4.3.0" - estraverse "^4.1.1" - -eslint-scope@^7.1.1: - version "7.1.1" - resolved "https://registry.yarnpkg.com/eslint-scope/-/eslint-scope-7.1.1.tgz#fff34894c2f65e5226d3041ac480b4513a163642" - integrity sha512-QKQM/UXpIiHcLqJ5AOyIW7XZmzjkzQXYE54n1++wb0u9V/abW3l9uQnxX8Z5Xd18xyKIMTUAyQ0k1e8pz6LUrw== - dependencies: - esrecurse "^4.3.0" - estraverse "^5.2.0" - -eslint-utils@^2.0.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/eslint-utils/-/eslint-utils-2.1.0.tgz#d2de5e03424e707dc10c74068ddedae708741b27" - integrity sha512-w94dQYoauyvlDc43XnGB8lU3Zt713vNChgt4EWwhXAP2XkBvndfxF0AgIqKOOasjPIPzj9JqgwkwbCYD0/V3Zg== - dependencies: - eslint-visitor-keys "^1.1.0" - -eslint-utils@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/eslint-utils/-/eslint-utils-3.0.0.tgz#8aebaface7345bb33559db0a1f13a1d2d48c3672" - integrity sha512-uuQC43IGctw68pJA1RgbQS8/NP7rch6Cwd4j3ZBtgo4/8Flj4eGE7ZYSZRN3iq5pVUv6GPdW5Z1RFleo84uLDA== - dependencies: - eslint-visitor-keys "^2.0.0" - -eslint-visitor-keys@^1.1.0: - version "1.3.0" - resolved "https://registry.yarnpkg.com/eslint-visitor-keys/-/eslint-visitor-keys-1.3.0.tgz#30ebd1ef7c2fdff01c3a4f151044af25fab0523e" - integrity sha512-6J72N8UNa462wa/KFODt/PJ3IU60SDpC3QXC1Hjc1BXXpfL2C9R5+AU7jhe0F6GREqVMh4Juu+NY7xn+6dipUQ== - -eslint-visitor-keys@^2.0.0, eslint-visitor-keys@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/eslint-visitor-keys/-/eslint-visitor-keys-2.1.0.tgz#f65328259305927392c938ed44eb0a5c9b2bd303" - integrity sha512-0rSmRBzXgDzIsD6mGdJgevzgezI534Cer5L/vyMX0kHzT/jiB43jRhd9YUlMGYLQy2zprNmoT8qasCGtY+QaKw== - -eslint-visitor-keys@^3.3.0: - version "3.3.0" - resolved "https://registry.yarnpkg.com/eslint-visitor-keys/-/eslint-visitor-keys-3.3.0.tgz#f6480fa6b1f30efe2d1968aa8ac745b862469826" - integrity sha512-mQ+suqKJVyeuwGYHAdjMFqjCyfl8+Ldnxuyp3ldiMBFKkvytrXUZWaiPCEav8qDHKty44bD+qV1IP4T+w+xXRA== - -eslint@^8.6.0: - version "8.17.0" - resolved "https://registry.yarnpkg.com/eslint/-/eslint-8.17.0.tgz#1cfc4b6b6912f77d24b874ca1506b0fe09328c21" - integrity sha512-gq0m0BTJfci60Fz4nczYxNAlED+sMcihltndR8t9t1evnU/azx53x3t2UHXC/uRjcbvRw/XctpaNygSTcQD+Iw== - dependencies: - "@eslint/eslintrc" "^1.3.0" - "@humanwhocodes/config-array" "^0.9.2" - ajv "^6.10.0" - chalk "^4.0.0" - cross-spawn "^7.0.2" - debug "^4.3.2" - doctrine "^3.0.0" - escape-string-regexp "^4.0.0" - eslint-scope "^7.1.1" - eslint-utils "^3.0.0" - eslint-visitor-keys "^3.3.0" - espree "^9.3.2" - esquery "^1.4.0" - esutils "^2.0.2" - fast-deep-equal "^3.1.3" - file-entry-cache "^6.0.1" - functional-red-black-tree "^1.0.1" - glob-parent "^6.0.1" - globals "^13.15.0" - ignore "^5.2.0" - import-fresh "^3.0.0" - imurmurhash "^0.1.4" - is-glob "^4.0.0" - js-yaml "^4.1.0" - json-stable-stringify-without-jsonify "^1.0.1" - levn "^0.4.1" - lodash.merge "^4.6.2" - minimatch "^3.1.2" - natural-compare "^1.4.0" - optionator "^0.9.1" - regexpp "^3.2.0" - strip-ansi "^6.0.1" - strip-json-comments "^3.1.0" - text-table "^0.2.0" - v8-compile-cache "^2.0.3" - -espree@^9.3.2: - version "9.3.2" - resolved "https://registry.yarnpkg.com/espree/-/espree-9.3.2.tgz#f58f77bd334731182801ced3380a8cc859091596" - integrity sha512-D211tC7ZwouTIuY5x9XnS0E9sWNChB7IYKX/Xp5eQj3nFXhqmiUDB9q27y76oFl8jTg3pXcQx/bpxMfs3CIZbA== - dependencies: - acorn "^8.7.1" - acorn-jsx "^5.3.2" - eslint-visitor-keys "^3.3.0" - -esprima@^4.0.0, esprima@^4.0.1: - version "4.0.1" - resolved "https://registry.yarnpkg.com/esprima/-/esprima-4.0.1.tgz#13b04cdb3e6c5d19df91ab6987a8695619b0aa71" - integrity sha512-eGuFFw7Upda+g4p+QHvnW0RyTX/SVeJBDM/gCtMARO0cLuT2HcEKnTPvhjV6aGeqrCB/sbNop0Kszm0jsaWU4A== - -esquery@^1.4.0: - version "1.4.0" - resolved "https://registry.yarnpkg.com/esquery/-/esquery-1.4.0.tgz#2148ffc38b82e8c7057dfed48425b3e61f0f24a5" - integrity sha512-cCDispWt5vHHtwMY2YrAQ4ibFkAL8RbH5YGBnZBc90MolvvfkkQcJro/aZiAQUlQ3qgrYS6D6v8Gc5G5CQsc9w== - dependencies: - estraverse "^5.1.0" - -esrecurse@^4.3.0: - version "4.3.0" - resolved "https://registry.yarnpkg.com/esrecurse/-/esrecurse-4.3.0.tgz#7ad7964d679abb28bee72cec63758b1c5d2c9921" - integrity sha512-KmfKL3b6G+RXvP8N1vr3Tq1kL/oCFgn2NYXEtqP8/L3pKapUA4G8cFVaoF3SU323CD4XypR/ffioHmkti6/Tag== - dependencies: - estraverse "^5.2.0" - -estraverse@^4.1.1: - version "4.3.0" - resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-4.3.0.tgz#398ad3f3c5a24948be7725e83d11a7de28cdbd1d" - integrity sha512-39nnKffWz8xN1BU/2c79n9nB9HDzo0niYUqx6xyqUnyoAnQyyWpOTdZEeiCch8BBu515t4wp9ZmgVfVhn9EBpw== - -estraverse@^5.1.0, estraverse@^5.3.0: - version "5.3.0" - resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-5.3.0.tgz#2eea5290702f26ab8fe5370370ff86c965d21123" - integrity sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA== - -estraverse@^5.2.0: - version "5.2.0" - resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-5.2.0.tgz#307df42547e6cc7324d3cf03c155d5cdb8c53880" - integrity sha512-BxbNGGNm0RyRYvUdHpIwv9IWzeM9XClbOxwoATuFdOE7ZE6wHL+HQ5T8hoPM+zHvmKzzsEqhgy0GrQ5X13afiQ== - -esutils@^2.0.2: - version "2.0.3" - resolved "https://registry.yarnpkg.com/esutils/-/esutils-2.0.3.tgz#74d2eb4de0b8da1293711910d50775b9b710ef64" - integrity sha512-kVscqXk4OCp68SZ0dkgEKVi6/8ij300KBWTJq32P/dYeWTSwK41WyTxalN1eRmA5Z9UU/LX9D7FWSmV9SAYx6g== - -eventemitter3@^4.0.7: - version "4.0.7" - resolved "https://registry.yarnpkg.com/eventemitter3/-/eventemitter3-4.0.7.tgz#2de9b68f6528d5644ef5c59526a1b4a07306169f" - integrity sha512-8guHBZCwKnFhYdHr2ysuRWErTwhoN2X8XELRlrRwpmfeY2jjuUN4taQMsULKUVo1K4DvZl+0pgfyoysHxvmvEw== - -events@^3.2.0: - version "3.3.0" - resolved "https://registry.yarnpkg.com/events/-/events-3.3.0.tgz#31a95ad0a924e2d2c419a813aeb2c4e878ea7400" - integrity sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q== - -execa@^5.0.0: - version "5.1.1" - resolved "https://registry.yarnpkg.com/execa/-/execa-5.1.1.tgz#f80ad9cbf4298f7bd1d4c9555c21e93741c411dd" - integrity sha512-8uSpZZocAZRBAPIEINJj3Lo9HyGitllczc27Eh5YYojjMFMn8yHMDMaUHE2Jqfq05D/wucwI4JGURyXt1vchyg== - dependencies: - cross-spawn "^7.0.3" - get-stream "^6.0.0" - human-signals "^2.1.0" - is-stream "^2.0.0" - merge-stream "^2.0.0" - npm-run-path "^4.0.1" - onetime "^5.1.2" - signal-exit "^3.0.3" - strip-final-newline "^2.0.0" - -exit@0.1.2, exit@0.1.x, exit@^0.1.2: - version "0.1.2" - resolved "https://registry.yarnpkg.com/exit/-/exit-0.1.2.tgz#0632638f8d877cc82107d30a0fff1a17cba1cd0c" - integrity sha1-BjJjj42HfMghB9MKD/8aF8uhzQw= - -expect@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/expect/-/expect-27.5.1.tgz#83ce59f1e5bdf5f9d2b94b61d2050db48f3fef74" - integrity sha512-E1q5hSUG2AmYQwQJ041nvgpkODHQvB+RKlB4IYdru6uJsyFTRyZAP463M+1lINorwbqAmUggi6+WwkD8lCS/Dw== - dependencies: - "@jest/types" "^27.5.1" - jest-get-type "^27.5.1" - jest-matcher-utils "^27.5.1" - jest-message-util "^27.5.1" - -extend@^3.0.0: - version "3.0.2" - resolved "https://registry.yarnpkg.com/extend/-/extend-3.0.2.tgz#f8b1136b4071fbd8eb140aff858b1019ec2915fa" - integrity sha512-fjquC59cD7CyW6urNXK0FBufkZcoiGG80wTuPujX590cB5Ttln20E2UB4S/WARVqhXffZl2LNgS+gQdPIIim/g== - -fast-deep-equal@^3.1.1, fast-deep-equal@^3.1.3: - version "3.1.3" - resolved "https://registry.yarnpkg.com/fast-deep-equal/-/fast-deep-equal-3.1.3.tgz#3a7d56b559d6cbc3eb512325244e619a65c6c525" - integrity sha512-f3qQ9oQy9j2AhBe/H9VC91wLmKBCCU/gDOnKNAYG5hswO7BLKj09Hc5HYNz9cGI++xlpDCIgDaitVs03ATR84Q== - -fast-glob@^3.2.4, fast-glob@^3.2.9, fast-glob@^3.3.0: - version "3.3.0" - resolved "https://registry.yarnpkg.com/fast-glob/-/fast-glob-3.3.0.tgz#7c40cb491e1e2ed5664749e87bfb516dbe8727c0" - integrity sha512-ChDuvbOypPuNjO8yIDf36x7BlZX1smcUMTTcyoIjycexOxd6DFsKsg21qVBzEmr3G7fUKIRy2/psii+CIUt7FA== - dependencies: - "@nodelib/fs.stat" "^2.0.2" - "@nodelib/fs.walk" "^1.2.3" - glob-parent "^5.1.2" - merge2 "^1.3.0" - micromatch "^4.0.4" - -fast-json-stable-stringify@^2.0.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/fast-json-stable-stringify/-/fast-json-stable-stringify-2.1.0.tgz#874bf69c6f404c2b5d99c481341399fd55892633" - integrity sha512-lhd/wF+Lk98HZoTCtlVraHtfh5XYijIjalXck7saUtuanSDyLMxnHhSXEDJqHxD7msR8D0uCmqlkwjCV8xvwHw== - -fast-levenshtein@^2.0.6, fast-levenshtein@~2.0.6: - version "2.0.6" - resolved "https://registry.yarnpkg.com/fast-levenshtein/-/fast-levenshtein-2.0.6.tgz#3d8a5c66883a16a30ca8643e851f19baa7797917" - integrity sha512-DCXu6Ifhqcks7TZKY3Hxp3y6qphY5SJZmrWMDrKcERSOXWQdMhU9Ig/PYrzyw/ul9jOIyh0N4M0tbC5hodg8dw== - -fast-safe-stringify@^2.0.7: - version "2.0.7" - resolved "https://registry.yarnpkg.com/fast-safe-stringify/-/fast-safe-stringify-2.0.7.tgz#124aa885899261f68aedb42a7c080de9da608743" - integrity sha512-Utm6CdzT+6xsDk2m8S6uL8VHxNwI6Jub+e9NYTcAms28T84pTa25GJQV9j0CY0N1rM8hK4x6grpF2BQf+2qwVA== - -fastest-levenshtein@^1.0.12, fastest-levenshtein@^1.0.16: - version "1.0.16" - resolved "https://registry.yarnpkg.com/fastest-levenshtein/-/fastest-levenshtein-1.0.16.tgz#210e61b6ff181de91ea9b3d1b84fdedd47e034e5" - integrity sha512-eRnCtTTtGZFpQCwhJiUOuxPQWRXVKYDn0b2PeHfXL6/Zi53SLAzAHfVhVWK2AryC/WH05kGfxhFIPvTF0SXQzg== - -fastq@^1.6.0: - version "1.11.0" - resolved "https://registry.yarnpkg.com/fastq/-/fastq-1.11.0.tgz#bb9fb955a07130a918eb63c1f5161cc32a5d0858" - integrity sha512-7Eczs8gIPDrVzT+EksYBcupqMyxSHXXrHOLRRxU2/DicV8789MRBRR8+Hc2uWzUupOs4YS4JzBmBxjjCVBxD/g== - dependencies: - reusify "^1.0.4" - -fault@^1.0.0: - version "1.0.4" - resolved "https://registry.yarnpkg.com/fault/-/fault-1.0.4.tgz#eafcfc0a6d214fc94601e170df29954a4f842f13" - integrity sha512-CJ0HCB5tL5fYTEA7ToAq5+kTwd++Borf1/bifxd9iT70QcXr4MRrO3Llf8Ifs70q+SJcGHFtnIE/Nw6giCtECA== - dependencies: - format "^0.2.0" - -fb-watchman@^2.0.0: - version "2.0.1" - resolved "https://registry.yarnpkg.com/fb-watchman/-/fb-watchman-2.0.1.tgz#fc84fb39d2709cf3ff6d743706157bb5708a8a85" - integrity sha512-DkPJKQeY6kKwmuMretBhr7G6Vodr7bFwDYTXIkfG1gjvNpaxBTQV3PbXg6bR1c1UP4jPOX0jHUbbHANL9vRjVg== - dependencies: - bser "2.1.1" - -fbemitter@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/fbemitter/-/fbemitter-3.0.0.tgz#00b2a1af5411254aab416cd75f9e6289bee4bff3" - integrity sha512-KWKaceCwKQU0+HPoop6gn4eOHk50bBv/VxjJtGMfwmJt3D29JpN4H4eisCtIPA+a8GVBam+ldMMpMjJUvpDyHw== - dependencies: - fbjs "^3.0.0" - -fbjs-css-vars@^1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/fbjs-css-vars/-/fbjs-css-vars-1.0.2.tgz#216551136ae02fe255932c3ec8775f18e2c078b8" - integrity sha512-b2XGFAFdWZWg0phtAWLHCk836A1Xann+I+Dgd3Gk64MHKZO44FfoD1KxyvbSh0qZsIoXQGGlVztIY+oitJPpRQ== - -fbjs@^3.0.0, fbjs@^3.0.1: - version "3.0.4" - resolved "https://registry.yarnpkg.com/fbjs/-/fbjs-3.0.4.tgz#e1871c6bd3083bac71ff2da868ad5067d37716c6" - integrity sha512-ucV0tDODnGV3JCnnkmoszb5lf4bNpzjv80K41wd4k798Etq+UYD0y0TIfalLjZoKgjive6/adkRnszwapiDgBQ== - dependencies: - cross-fetch "^3.1.5" - fbjs-css-vars "^1.0.0" - loose-envify "^1.0.0" - object-assign "^4.1.0" - promise "^7.1.1" - setimmediate "^1.0.5" - ua-parser-js "^0.7.30" - -file-entry-cache@^6.0.1: - version "6.0.1" - resolved "https://registry.yarnpkg.com/file-entry-cache/-/file-entry-cache-6.0.1.tgz#211b2dd9659cb0394b073e7323ac3c933d522027" - integrity sha512-7Gps/XWymbLk2QLYK4NzpMOrYjMhdIxXuIvy2QBsLE6ljuodKvdkWs/cpyJJ3CVIVpH0Oi1Hvg1ovbMzLdFBBg== - dependencies: - flat-cache "^3.0.4" - -file-loader@^6.0.0: - version "6.2.0" - resolved "https://registry.yarnpkg.com/file-loader/-/file-loader-6.2.0.tgz#baef7cf8e1840df325e4390b4484879480eebe4d" - integrity sha512-qo3glqyTa61Ytg4u73GultjHGjdRyig3tG6lPtyX/jOEJvHif9uB0/OCI2Kif6ctF3caQTW2G5gym21oAsI4pw== - dependencies: - loader-utils "^2.0.0" - schema-utils "^3.0.0" - -fill-range@^7.1.1: - version "7.1.1" - resolved "https://registry.yarnpkg.com/fill-range/-/fill-range-7.1.1.tgz#44265d3cac07e3ea7dc247516380643754a05292" - integrity sha512-YsGpe3WHLK8ZYi4tWDg2Jy3ebRz2rXowDxnld4bkQB00cc/1Zw9AWnC0i9ztDJitivtQvaI9KaLyKrc+hBW0yg== - dependencies: - to-regex-range "^5.0.1" - -find-cache-dir@^3.3.1: - version "3.3.1" - resolved "https://registry.yarnpkg.com/find-cache-dir/-/find-cache-dir-3.3.1.tgz#89b33fad4a4670daa94f855f7fbe31d6d84fe880" - integrity sha512-t2GDMt3oGC/v+BMwzmllWDuJF/xcDtE5j/fCGbqDD7OLuJkj0cfh1YSA5VKPvwMeLFLNDBkwOKZ2X85jGLVftQ== - dependencies: - commondir "^1.0.1" - make-dir "^3.0.2" - pkg-dir "^4.1.0" - -find-cache-dir@^3.3.2: - version "3.3.2" - resolved "https://registry.yarnpkg.com/find-cache-dir/-/find-cache-dir-3.3.2.tgz#b30c5b6eff0730731aea9bbd9dbecbd80256d64b" - integrity sha512-wXZV5emFEjrridIgED11OoUKLxiYjAcqot/NJdAkOhlJ+vGzwhOAfcG5OX1jP+S0PcjEn8bdMJv+g2jwQ3Onig== - dependencies: - commondir "^1.0.1" - make-dir "^3.0.2" - pkg-dir "^4.1.0" - -find-root@^1.1.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/find-root/-/find-root-1.1.0.tgz#abcfc8ba76f708c42a97b3d685b7e9450bfb9ce4" - integrity sha512-NKfW6bec6GfKc0SGx1e07QZY9PE99u0Bft/0rzSD5k3sO/vwkVUpDUKVm5Gpp5Ue3YfShPFTX2070tDs5kB9Ng== - -find-up@^4.0.0, find-up@^4.1.0: - version "4.1.0" - resolved "https://registry.yarnpkg.com/find-up/-/find-up-4.1.0.tgz#97afe7d6cdc0bc5928584b7c8d7b16e8a9aa5d19" - integrity sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw== - dependencies: - locate-path "^5.0.0" - path-exists "^4.0.0" - -find-up@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/find-up/-/find-up-5.0.0.tgz#4c92819ecb7083561e4f4a240a86be5198f536fc" - integrity sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng== - dependencies: - locate-path "^6.0.0" - path-exists "^4.0.0" - -flat-cache@^3.0.4: - version "3.0.4" - resolved "https://registry.yarnpkg.com/flat-cache/-/flat-cache-3.0.4.tgz#61b0338302b2fe9f957dcc32fc2a87f1c3048b11" - integrity sha512-dm9s5Pw7Jc0GvMYbshN6zchCA9RgQlzzEZX3vylR9IqFfS8XciblUXOKfW6SiuJ0e13eDYZoZV5wdrev7P3Nwg== - dependencies: - flatted "^3.1.0" - rimraf "^3.0.2" - -flatted@^3.1.0: - version "3.1.1" - resolved "https://registry.yarnpkg.com/flatted/-/flatted-3.1.1.tgz#c4b489e80096d9df1dfc97c79871aea7c617c469" - integrity sha512-zAoAQiudy+r5SvnSw3KJy5os/oRJYHzrzja/tBDqrZtNhUw8bt6y8OBzMWcjWr+8liV8Eb6yOhw8WZ7VFZ5ZzA== - -flux@^4.0.1: - version "4.0.3" - resolved "https://registry.yarnpkg.com/flux/-/flux-4.0.3.tgz#573b504a24982c4768fdfb59d8d2ea5637d72ee7" - integrity sha512-yKAbrp7JhZhj6uiT1FTuVMlIAT1J4jqEyBpFApi1kxpGZCvacMVc/t1pMQyotqHhAgvoE3bNvAykhCo2CLjnYw== - dependencies: - fbemitter "^3.0.0" - fbjs "^3.0.1" - -focus-lock@^0.11.6: - version "0.11.6" - resolved "https://registry.yarnpkg.com/focus-lock/-/focus-lock-0.11.6.tgz#e8821e21d218f03e100f7dc27b733f9c4f61e683" - integrity sha512-KSuV3ur4gf2KqMNoZx3nXNVhqCkn42GuTYCX4tXPEwf0MjpFQmNMiN6m7dXaUXgIoivL6/65agoUMg4RLS0Vbg== - dependencies: - tslib "^2.0.3" - -follow-redirects@^1.15.6: - version "1.15.6" - resolved "https://registry.yarnpkg.com/follow-redirects/-/follow-redirects-1.15.6.tgz#7f815c0cda4249c74ff09e95ef97c23b5fd0399b" - integrity sha512-wWN62YITEaOpSK584EZXJafH1AGpO8RVgElfkuXbTOrPX4fIfOyEpW/CsiNd8JdYrAoOvafRTOEnvsO++qCqFA== - -for-each@^0.3.3: - version "0.3.3" - resolved "https://registry.yarnpkg.com/for-each/-/for-each-0.3.3.tgz#69b447e88a0a5d32c3e7084f3f1710034b21376e" - integrity sha512-jqYfLp7mo9vIyQf8ykW2v7A+2N4QjeCeI5+Dz9XraiO1ign81wjiH7Fb9vSOWvQfNtmSa4H2RoQTrrXivdUZmw== - dependencies: - is-callable "^1.1.3" - -foreach@^2.0.4: - version "2.0.5" - resolved "https://registry.yarnpkg.com/foreach/-/foreach-2.0.5.tgz#0bee005018aeb260d0a3af3ae658dd0136ec1b99" - integrity sha1-C+4AUBiusmDQo6865ljdATbsG5k= - -form-data@^3.0.0: - version "3.0.1" - resolved "https://registry.yarnpkg.com/form-data/-/form-data-3.0.1.tgz#ebd53791b78356a99af9a300d4282c4d5eb9755f" - integrity sha512-RHkBKtLWUVwd7SqRIvCZMEvAMoGUp0XU+seQiZejj0COz3RI3hWP4sCv3gZWWLjJTd7rGwcsF5eKZGii0r/hbg== - dependencies: - asynckit "^0.4.0" - combined-stream "^1.0.8" - mime-types "^2.1.12" - -form-data@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/form-data/-/form-data-4.0.0.tgz#93919daeaf361ee529584b9b31664dc12c9fa452" - integrity sha512-ETEklSGi5t0QMZuiXoA/Q6vcnxcLQP5vdugSpuAyi6SVGi2clPPp+xgEhuMaHC+zGgn31Kd235W35f7Hykkaww== - dependencies: - asynckit "^0.4.0" - combined-stream "^1.0.8" - mime-types "^2.1.12" - -format@^0.2.0: - version "0.2.2" - resolved "https://registry.yarnpkg.com/format/-/format-0.2.2.tgz#d6170107e9efdc4ed30c9dc39016df942b5cb58b" - integrity sha512-wzsgA6WOq+09wrU1tsJ09udeR/YZRaeArL9e1wPbFg3GG2yDnC2ldKpxs4xunpFF9DgqCqOIra3bc1HWrJ37Ww== - -framer-motion@^6.0.0: - version "6.3.11" - resolved "https://registry.yarnpkg.com/framer-motion/-/framer-motion-6.3.11.tgz#c304ce9728601ad9377d47d5d9264e43d741d470" - integrity sha512-xQLk+ZSklNs5QNCUmdWPpKMOuWiB8ZETsvcIOWw8xvri9K3TamuifgCI/B6XpaEDR0/V2ZQF2Wm+gUAZrXo+rw== - dependencies: - framesync "6.0.1" - hey-listen "^1.0.8" - popmotion "11.0.3" - style-value-types "5.0.0" - tslib "^2.1.0" - optionalDependencies: - "@emotion/is-prop-valid" "^0.8.2" - -framesync@5.3.0: - version "5.3.0" - resolved "https://registry.yarnpkg.com/framesync/-/framesync-5.3.0.tgz#0ecfc955e8f5a6ddc8fdb0cc024070947e1a0d9b" - integrity sha512-oc5m68HDO/tuK2blj7ZcdEBRx3p1PjrgHazL8GYEpvULhrtGIFbQArN6cQS2QhW8mitffaB+VYzMjDqBxxQeoA== - dependencies: - tslib "^2.1.0" - -framesync@6.0.1: - version "6.0.1" - resolved "https://registry.yarnpkg.com/framesync/-/framesync-6.0.1.tgz#5e32fc01f1c42b39c654c35b16440e07a25d6f20" - integrity sha512-fUY88kXvGiIItgNC7wcTOl0SNRCVXMKSWW2Yzfmn7EKNc+MpCzcz9DhdHcdjbrtN3c6R4H5dTY2jiCpPdysEjA== - dependencies: - tslib "^2.1.0" - -fs-minipass@^2.0.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/fs-minipass/-/fs-minipass-2.1.0.tgz#7f5036fdbf12c63c169190cbe4199c852271f9fb" - integrity sha512-V/JgOLFCS+R6Vcq0slCuaeWEdNC3ouDlJMNIsacH2VtALiu9mV4LPrHc5cDl8k5aw6J8jwgWWpiTo5RYhmIzvg== - dependencies: - minipass "^3.0.0" - -fs.realpath@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/fs.realpath/-/fs.realpath-1.0.0.tgz#1504ad2523158caa40db4a2787cb01411994ea4f" - integrity sha1-FQStJSMVjKpA20onh8sBQRmU6k8= - -fsevents@^2.3.2: - version "2.3.2" - resolved "https://registry.yarnpkg.com/fsevents/-/fsevents-2.3.2.tgz#8a526f78b8fdf4623b709e0b975c52c24c02fd1a" - integrity sha512-xiqMQR4xAeHTuB9uWm+fFRcIOgKBMiOBP+eXiyT7jsgVCq1bkVygt00oASowB7EdtpOHaaPgKt812P9ab+DDKA== - -function-bind@^1.1.1: - version "1.1.1" - resolved "https://registry.yarnpkg.com/function-bind/-/function-bind-1.1.1.tgz#a56899d3ea3c9bab874bb9773b7c5ede92f4895d" - integrity sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A== - -function.prototype.name@^1.1.5: - version "1.1.5" - resolved "https://registry.yarnpkg.com/function.prototype.name/-/function.prototype.name-1.1.5.tgz#cce0505fe1ffb80503e6f9e46cc64e46a12a9621" - integrity sha512-uN7m/BzVKQnCUF/iW8jYea67v++2u7m5UgENbHRtdDVclOUP+FMPlCNdmk0h/ysGyo2tavMJEDqJAkJdRa1vMA== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.3" - es-abstract "^1.19.0" - functions-have-names "^1.2.2" - -functional-red-black-tree@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/functional-red-black-tree/-/functional-red-black-tree-1.0.1.tgz#1b0ab3bd553b2a0d6399d29c0e3ea0b252078327" - integrity sha512-dsKNQNdj6xA3T+QlADDA7mOSlX0qiMINjn0cgr+eGHGsbSHzTabcIogz2+p/iqP1Xs6EP/sS2SbqH+brGTbq0g== - -functions-have-names@^1.2.2: - version "1.2.3" - resolved "https://registry.yarnpkg.com/functions-have-names/-/functions-have-names-1.2.3.tgz#0404fe4ee2ba2f607f0e0ec3c80bae994133b834" - integrity sha512-xckBUXyTIqT97tq2x2AMb+g163b5JFysYk0x4qxNFwbfQkmNZoiRHb6sPzI9/QV33WeuvVYBUIiD4NzNIyqaRQ== - -gensync@^1.0.0-beta.2: - version "1.0.0-beta.2" - resolved "https://registry.yarnpkg.com/gensync/-/gensync-1.0.0-beta.2.tgz#32a6ee76c3d7f52d46b2b1ae5d93fea8580a25e0" - integrity sha512-3hN7NaskYvMDLQY55gnW3NQ+mesEAepTqlg+VEbj7zzqEMBVNhzcGYYeqFo/TlYz6eQiFcp1HcsCZO+nGgS8zg== - -get-caller-file@^2.0.5: - version "2.0.5" - resolved "https://registry.yarnpkg.com/get-caller-file/-/get-caller-file-2.0.5.tgz#4f94412a82db32f36e3b0b9741f8a97feb031f7e" - integrity sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg== - -get-intrinsic@^1.0.2, get-intrinsic@^1.1.0, get-intrinsic@^1.1.1: - version "1.1.1" - resolved "https://registry.yarnpkg.com/get-intrinsic/-/get-intrinsic-1.1.1.tgz#15f59f376f855c446963948f0d24cd3637b4abc6" - integrity sha512-kWZrnVM42QCiEA2Ig1bG8zjoIMOgxWwYCEeNdwY6Tv/cOSeGpcoX4pXHfKUxNKVoArnrEr2e9srnAxxGIraS9Q== - dependencies: - function-bind "^1.1.1" - has "^1.0.3" - has-symbols "^1.0.1" - -get-intrinsic@^1.1.3, get-intrinsic@^1.2.0: - version "1.2.0" - resolved "https://registry.yarnpkg.com/get-intrinsic/-/get-intrinsic-1.2.0.tgz#7ad1dc0535f3a2904bba075772763e5051f6d05f" - integrity sha512-L049y6nFOuom5wGyRc3/gdTLO94dySVKRACj1RmJZBQXlbTMhtNIgkWkUHq+jYmZvKf14EW1EoJnnjbmoHij0Q== - dependencies: - function-bind "^1.1.1" - has "^1.0.3" - has-symbols "^1.0.3" - -get-nonce@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/get-nonce/-/get-nonce-1.0.1.tgz#fdf3f0278073820d2ce9426c18f07481b1e0cdf3" - integrity sha512-FJhYRoDaiatfEkUK8HKlicmu/3SGFD51q3itKDGoSTysQJBnfOcxU5GxnhE1E6soB76MbT0MBtnKJuXyAx+96Q== - -get-npm-tarball-url@^2.0.1: - version "2.0.2" - resolved "https://registry.yarnpkg.com/get-npm-tarball-url/-/get-npm-tarball-url-2.0.2.tgz#1538165bdd19ad13d21ddff78e7a8ed57b782235" - integrity sha512-2dPhgT0K4pVyciTqdS0gr9nEwyCQwt9ql1/t5MCUMvcjWjAysjGJgT7Sx4n6oq3tFBjBN238mxX4RfTjT3838Q== - dependencies: - normalize-registry-url "^1.0.0" - -get-package-type@^0.1.0: - version "0.1.0" - resolved "https://registry.yarnpkg.com/get-package-type/-/get-package-type-0.1.0.tgz#8de2d803cff44df3bc6c456e6668b36c3926e11a" - integrity sha512-pjzuKtY64GYfWizNAJ0fr9VqttZkNiK2iS430LtIHzjBEr6bX8Am2zm4sW4Ro5wjWW5cAlRL1qAMTcXbjNAO2Q== - -get-stream@^6.0.0: - version "6.0.1" - resolved "https://registry.yarnpkg.com/get-stream/-/get-stream-6.0.1.tgz#a262d8eef67aced57c2852ad6167526a43cbf7b7" - integrity sha512-ts6Wi+2j3jQjqi70w5AlN8DFnkSwC+MqmxEzdEALB2qXZYV3X/b1CTfgPLGJNMeAWxdPfU8FO1ms3NUfaHCPYg== - -get-symbol-description@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/get-symbol-description/-/get-symbol-description-1.0.0.tgz#7fdb81c900101fbd564dd5f1a30af5aadc1e58d6" - integrity sha512-2EmdH1YvIQiZpltCNgkuiUnyukzxM/R6NDJX31Ke3BG1Nq5b0S2PhX59UKi9vZpPDQVdqn+1IcaAwnzTT5vCjw== - dependencies: - call-bind "^1.0.2" - get-intrinsic "^1.1.1" - -glob-parent@^5.1.1, glob-parent@^5.1.2: - version "5.1.2" - resolved "https://registry.yarnpkg.com/glob-parent/-/glob-parent-5.1.2.tgz#869832c58034fe68a4093c17dc15e8340d8401c4" - integrity sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow== - dependencies: - is-glob "^4.0.1" - -glob-parent@^6.0.1: - version "6.0.2" - resolved "https://registry.yarnpkg.com/glob-parent/-/glob-parent-6.0.2.tgz#6d237d99083950c79290f24c7642a3de9a28f9e3" - integrity sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A== - dependencies: - is-glob "^4.0.3" - -glob-to-regexp@^0.4.1: - version "0.4.1" - resolved "https://registry.yarnpkg.com/glob-to-regexp/-/glob-to-regexp-0.4.1.tgz#c75297087c851b9a578bd217dd59a92f59fe546e" - integrity sha512-lkX1HJXwyMcprw/5YUZc2s7DrpAiHB21/V+E1rHUrVNokkvB6bqMzT0VfV6/86ZNabt1k14YOIaT7nDvOX3Iiw== - -glob@^7.0.3, glob@^7.1.1, glob@^7.1.3, glob@^7.1.4: - version "7.1.7" - resolved "https://registry.yarnpkg.com/glob/-/glob-7.1.7.tgz#3b193e9233f01d42d0b3f78294bbeeb418f94a90" - integrity sha512-OvD9ENzPLbegENnYP5UUfJIirTg4+XwMWGaQfQTY0JenxNvvIKP3U3/tAQSPIu/lHxXYSZmpXlUHeqAIdKzBLQ== - dependencies: - fs.realpath "^1.0.0" - inflight "^1.0.4" - inherits "2" - minimatch "^3.0.4" - once "^1.3.0" - path-is-absolute "^1.0.0" - -glob@^7.1.2: - version "7.2.0" - resolved "https://registry.yarnpkg.com/glob/-/glob-7.2.0.tgz#d15535af7732e02e948f4c41628bd910293f6023" - integrity sha512-lmLf6gtyrPq8tTjSmrO94wBeQbFR3HbLHbuyD69wuyQkImp2hWqMGB47OX65FBkPffO641IP9jWa1z4ivqG26Q== - dependencies: - fs.realpath "^1.0.0" - inflight "^1.0.4" - inherits "2" - minimatch "^3.0.4" - once "^1.3.0" - path-is-absolute "^1.0.0" - -global-modules@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/global-modules/-/global-modules-2.0.0.tgz#997605ad2345f27f51539bea26574421215c7780" - integrity sha512-NGbfmJBp9x8IxyJSd1P+otYK8vonoJactOogrVfFRIAEY1ukil8RSKDz2Yo7wh1oihl51l/r6W4epkeKJHqL8A== - dependencies: - global-prefix "^3.0.0" - -global-prefix@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/global-prefix/-/global-prefix-3.0.0.tgz#fc85f73064df69f50421f47f883fe5b913ba9b97" - integrity sha512-awConJSVCHVGND6x3tmMaKcQvwXLhjdkmomy2W+Goaui8YPgYgXJZewhg3fWC+DlfqqQuWg8AwqjGTD2nAPVWg== - dependencies: - ini "^1.3.5" - kind-of "^6.0.2" - which "^1.3.1" - -globals@^11.1.0: - version "11.12.0" - resolved "https://registry.yarnpkg.com/globals/-/globals-11.12.0.tgz#ab8795338868a0babd8525758018c2a7eb95c42e" - integrity sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA== - -globals@^13.15.0: - version "13.15.0" - resolved "https://registry.yarnpkg.com/globals/-/globals-13.15.0.tgz#38113218c907d2f7e98658af246cef8b77e90bac" - integrity sha512-bpzcOlgDhMG070Av0Vy5Owklpv1I6+j96GhUI7Rh7IzDCKLzboflLrrfqMu8NquDbiR4EOQk7XzJwqVJxicxog== - dependencies: - type-fest "^0.20.2" - -globalthis@^1.0.3: - version "1.0.3" - resolved "https://registry.yarnpkg.com/globalthis/-/globalthis-1.0.3.tgz#5852882a52b80dc301b0660273e1ed082f0b6ccf" - integrity sha512-sFdI5LyBiNTHjRd7cGPWapiHWMOXKyuBNX/cWJ3NfzrZQVa8GI/8cofCl74AOVqq9W5kNmguTIzJ/1s2gyI9wA== - dependencies: - define-properties "^1.1.3" - -globalyzer@0.1.0: - version "0.1.0" - resolved "https://registry.yarnpkg.com/globalyzer/-/globalyzer-0.1.0.tgz#cb76da79555669a1519d5a8edf093afaa0bf1465" - integrity sha512-40oNTM9UfG6aBmuKxk/giHn5nQ8RVz/SS4Ir6zgzOv9/qC3kKZ9v4etGTcJbEl/NyVQH7FGU7d+X1egr57Md2Q== - -globby@^11.0.1, globby@^11.1.0: - version "11.1.0" - resolved "https://registry.yarnpkg.com/globby/-/globby-11.1.0.tgz#bd4be98bb042f83d796f7e3811991fbe82a0d34b" - integrity sha512-jhIXaOzy1sb8IyocaruWSn1TjmnBVs8Ayhcy83rmxNJ8q2uWKCAj3CnJY+KpGSXCueAPc0i05kVvVKtP1t9S3g== - dependencies: - array-union "^2.1.0" - dir-glob "^3.0.1" - fast-glob "^3.2.9" - ignore "^5.2.0" - merge2 "^1.4.1" - slash "^3.0.0" - -globby@^6.1.0: - version "6.1.0" - resolved "https://registry.yarnpkg.com/globby/-/globby-6.1.0.tgz#f5a6d70e8395e21c858fb0489d64df02424d506c" - integrity sha1-9abXDoOV4hyFj7BInWTfAkJNUGw= - dependencies: - array-union "^1.0.1" - glob "^7.0.3" - object-assign "^4.0.1" - pify "^2.0.0" - pinkie-promise "^2.0.0" - -globjoin@^0.1.4: - version "0.1.4" - resolved "https://registry.yarnpkg.com/globjoin/-/globjoin-0.1.4.tgz#2f4494ac8919e3767c5cbb691e9f463324285d43" - integrity sha1-L0SUrIkZ43Z8XLtpHp9GMyQoXUM= - -globrex@^0.1.2: - version "0.1.2" - resolved "https://registry.yarnpkg.com/globrex/-/globrex-0.1.2.tgz#dd5d9ec826232730cd6793a5e33a9302985e6098" - integrity sha512-uHJgbwAMwNFf5mLst7IWLNg14x1CkeqglJb/K3doi4dw6q2IvAAmM/Y81kevy83wP+Sst+nutFTYOGg3d1lsxg== - -gopd@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/gopd/-/gopd-1.0.1.tgz#29ff76de69dac7489b7c0918a5788e56477c332c" - integrity sha512-d65bNlIadxvpb/A2abVdlqKqV563juRnZ1Wtk6s1sIR8uNsXR70xqIzVqxVf1eTqDunwT2MkczEeaezCKTZhwA== - dependencies: - get-intrinsic "^1.1.3" - -graceful-fs@^4.1.2, graceful-fs@^4.2.11, graceful-fs@^4.2.4, graceful-fs@^4.2.9: - version "4.2.11" - resolved "https://registry.yarnpkg.com/graceful-fs/-/graceful-fs-4.2.11.tgz#4183e4e8bf08bb6e05bbb2f7d2e0c8f712ca40e3" - integrity sha512-RbJ5/jmFcNNCcDV5o9eTnBLJ/HszWV0P73bc+Ff4nS/rJj+YaS6IGyiOL0VoBYX+l1Wrl3k63h/KrH+nhJ0XvQ== - -graphlib@^2.1.8: - version "2.1.8" - resolved "https://registry.yarnpkg.com/graphlib/-/graphlib-2.1.8.tgz#5761d414737870084c92ec7b5dbcb0592c9d35da" - integrity sha512-jcLLfkpoVGmH7/InMC/1hIvOPSUh38oJtGhvrOFGzioE1DZ+0YW16RgmOJhHiuWTvGiJQ9Z1Ik43JvkRPRvE+A== - dependencies: - lodash "^4.17.15" - -hard-rejection@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/hard-rejection/-/hard-rejection-2.1.0.tgz#1c6eda5c1685c63942766d79bb40ae773cecd883" - integrity sha512-VIZB+ibDhx7ObhAe7OVtoEbuP4h/MuOTHJ+J8h/eBXotJYl0fBgR72xDFCKgIh22OJZIOVNxBMWuhAr10r8HdA== - -has-bigints@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/has-bigints/-/has-bigints-1.0.1.tgz#64fe6acb020673e3b78db035a5af69aa9d07b113" - integrity sha512-LSBS2LjbNBTf6287JEbEzvJgftkF5qFkmCo9hDRpAzKhUOlJ+hx8dd4USs00SgsUNwc4617J9ki5YtEClM2ffA== - -has-bigints@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/has-bigints/-/has-bigints-1.0.2.tgz#0871bd3e3d51626f6ca0966668ba35d5602d6eaa" - integrity sha512-tSvCKtBr9lkF0Ex0aQiP9N+OpV4zi2r/Nee5VkRDbaqv35RLYMzbwQfFSZZH0kR+Rd6302UJZ2p/bJCEoR3VoQ== - -has-flag@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/has-flag/-/has-flag-3.0.0.tgz#b5d454dc2199ae225699f3467e5a07f3b955bafd" - integrity sha1-tdRU3CGZriJWmfNGfloH87lVuv0= - -has-flag@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/has-flag/-/has-flag-4.0.0.tgz#944771fd9c81c81265c4d6941860da06bb59479b" - integrity sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ== - -has-property-descriptors@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/has-property-descriptors/-/has-property-descriptors-1.0.0.tgz#610708600606d36961ed04c196193b6a607fa861" - integrity sha512-62DVLZGoiEBDHQyqG4w9xCuZ7eJEwNmJRWw2VY84Oedb7WFcA27fiEVe8oUQx9hAUJ4ekurquucTGwsyO1XGdQ== - dependencies: - get-intrinsic "^1.1.1" - -has-proto@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/has-proto/-/has-proto-1.0.1.tgz#1885c1305538958aff469fef37937c22795408e0" - integrity sha512-7qE+iP+O+bgF9clE5+UoBFzE65mlBiVj3tKCrlNQ0Ogwm0BjpT/gK4SlLYDMybDh5I3TCTKnPPa0oMG7JDYrhg== - -has-symbols@^1.0.1, has-symbols@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/has-symbols/-/has-symbols-1.0.2.tgz#165d3070c00309752a1236a479331e3ac56f1423" - integrity sha512-chXa79rL/UC2KlX17jo3vRGz0azaWEx5tGqZg5pO3NUyEJVB17dMruQlzCCOfUvElghKcm5194+BCRvi2Rv/Gw== - -has-symbols@^1.0.3: - version "1.0.3" - resolved "https://registry.yarnpkg.com/has-symbols/-/has-symbols-1.0.3.tgz#bb7b2c4349251dce87b125f7bdf874aa7c8b39f8" - integrity sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A== - -has-tostringtag@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/has-tostringtag/-/has-tostringtag-1.0.0.tgz#7e133818a7d394734f941e73c3d3f9291e658b25" - integrity sha512-kFjcSNhnlGV1kyoGk7OXKSawH5JOb/LzUc5w9B02hOTO0dfFRjbHQKvg1d6cf3HbeUmtU9VbbV3qzZ2Teh97WQ== - dependencies: - has-symbols "^1.0.2" - -has@^1.0.3: - version "1.0.3" - resolved "https://registry.yarnpkg.com/has/-/has-1.0.3.tgz#722d7cbfc1f6aa8241f16dd814e011e1f41e8796" - integrity sha512-f2dvO0VU6Oej7RkWJGrehjbzMAjFp5/VKPp5tTpWIV4JHHZK1/BxbFRtf/siA2SWTe09caDmVtYYzWEIbBS4zw== - dependencies: - function-bind "^1.1.1" - -hast-util-parse-selector@^2.0.0: - version "2.2.5" - resolved "https://registry.yarnpkg.com/hast-util-parse-selector/-/hast-util-parse-selector-2.2.5.tgz#d57c23f4da16ae3c63b3b6ca4616683313499c3a" - integrity sha512-7j6mrk/qqkSehsM92wQjdIgWM2/BW61u/53G6xmC8i1OmEdKLHbk419QKQUjz6LglWsfqoiHmyMRkP1BGjecNQ== - -hast-util-whitespace@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/hast-util-whitespace/-/hast-util-whitespace-2.0.0.tgz#4fc1086467cc1ef5ba20673cb6b03cec3a970f1c" - integrity sha512-Pkw+xBHuV6xFeJprJe2BBEoDV+AvQySaz3pPDRUs5PNZEMQjpXJJueqrpcHIXxnWTcAGi/UOCgVShlkY6kLoqg== - -hastscript@^6.0.0: - version "6.0.0" - resolved "https://registry.yarnpkg.com/hastscript/-/hastscript-6.0.0.tgz#e8768d7eac56c3fdeac8a92830d58e811e5bf640" - integrity sha512-nDM6bvd7lIqDUiYEiu5Sl/+6ReP0BMk/2f4U/Rooccxkj0P5nm+acM5PrGJ/t5I8qPGiqZSE6hVAwZEdZIvP4w== - dependencies: - "@types/hast" "^2.0.0" - comma-separated-tokens "^1.0.0" - hast-util-parse-selector "^2.0.0" - property-information "^5.0.0" - space-separated-tokens "^1.0.0" - -hey-listen@^1.0.8: - version "1.0.8" - resolved "https://registry.yarnpkg.com/hey-listen/-/hey-listen-1.0.8.tgz#8e59561ff724908de1aa924ed6ecc84a56a9aa68" - integrity sha512-COpmrF2NOg4TBWUJ5UVyaCU2A88wEMkUPK4hNqyCkqHbxT92BbvfjoSozkAIIm6XhicGlJHhFdullInrdhwU8Q== - -highlight.js@^10.4.1, highlight.js@~10.7.0: - version "10.7.3" - resolved "https://registry.yarnpkg.com/highlight.js/-/highlight.js-10.7.3.tgz#697272e3991356e40c3cac566a74eef681756531" - integrity sha512-tzcUFauisWKNHaRkN4Wjl/ZA07gENAjFl3J/c480dprkGTg5EQstgaNFqBfUqCq54kZRIEcreTsAgF/m2quD7A== - -history@^5.2.0: - version "5.3.0" - resolved "https://registry.yarnpkg.com/history/-/history-5.3.0.tgz#1548abaa245ba47992f063a0783db91ef201c73b" - integrity sha512-ZqaKwjjrAYUYfLG+htGaIIZ4nioX2L70ZUMIFysS3xvBsSG4x/n1V6TXV3N8ZYNuFGlDirFg32T7B6WOUPDYcQ== - dependencies: - "@babel/runtime" "^7.7.6" - -hoist-non-react-statics@^3.3.1: - version "3.3.2" - resolved "https://registry.yarnpkg.com/hoist-non-react-statics/-/hoist-non-react-statics-3.3.2.tgz#ece0acaf71d62c2969c2ec59feff42a4b1a85b45" - integrity sha512-/gGivxi8JPKWNm/W0jSmzcMPpfpPLc3dY/6GxhX2hQ9iGj3aDfklV4ET7NjKpSinLpJ5vafa9iiGIEZg10SfBw== - dependencies: - react-is "^16.7.0" - -hosted-git-info@^4.0.1: - version "4.0.2" - resolved "https://registry.yarnpkg.com/hosted-git-info/-/hosted-git-info-4.0.2.tgz#5e425507eede4fea846b7262f0838456c4209961" - integrity sha512-c9OGXbZ3guC/xOlCg1Ci/VgWlwsqDv1yMQL1CWqXDL0hDjXuNcq0zuR4xqPSuasI3kqFDhqSyTjREz5gzq0fXg== - dependencies: - lru-cache "^6.0.0" - -html-encoding-sniffer@^2.0.1: - version "2.0.1" - resolved "https://registry.yarnpkg.com/html-encoding-sniffer/-/html-encoding-sniffer-2.0.1.tgz#42a6dc4fd33f00281176e8b23759ca4e4fa185f3" - integrity sha512-D5JbOMBIR/TVZkubHT+OyT2705QvogUW4IBn6nHd756OwieSF9aDYFj4dv6HHEVGYbHaLETa3WggZYWWMyy3ZQ== - dependencies: - whatwg-encoding "^1.0.5" - -html-escaper@^2.0.0: - version "2.0.2" - resolved "https://registry.yarnpkg.com/html-escaper/-/html-escaper-2.0.2.tgz#dfd60027da36a36dfcbe236262c00a5822681453" - integrity sha512-H2iMtd0I4Mt5eYiapRdIDjp+XzelXQ0tFE4JS7YFwFevXXMmOp9myNrUvCg0D6ws8iqkRPBfKHgbwig1SmlLfg== - -html-tags@^3.3.1: - version "3.3.1" - resolved "https://registry.yarnpkg.com/html-tags/-/html-tags-3.3.1.tgz#a04026a18c882e4bba8a01a3d39cfe465d40b5ce" - integrity sha512-ztqyC3kLto0e9WbNp0aeP+M3kTt+nbaIveGmUxAtZa+8iFgKLUOD4YKM5j+f3QD89bra7UeumolZHKuOXnTmeQ== - -htmlparser2@3.8.x: - version "3.8.3" - resolved "https://registry.yarnpkg.com/htmlparser2/-/htmlparser2-3.8.3.tgz#996c28b191516a8be86501a7d79757e5c70c1068" - integrity sha1-mWwosZFRaovoZQGn15dX5ccMEGg= - dependencies: - domelementtype "1" - domhandler "2.3" - domutils "1.5" - entities "1.0" - readable-stream "1.1" - -htmlparser2@^6.0.1: - version "6.1.0" - resolved "https://registry.yarnpkg.com/htmlparser2/-/htmlparser2-6.1.0.tgz#c4d762b6c3371a05dbe65e94ae43a9f845fb8fb7" - integrity sha512-gyyPk6rgonLFEDGoeRgQNaEUvdJ4ktTmmUh/h2t7s+M8oPpIPxgNACWa+6ESR57kXstwqPiCut0V8NRpcwgU7A== - dependencies: - domelementtype "^2.0.1" - domhandler "^4.0.0" - domutils "^2.5.2" - entities "^2.0.0" - -http-proxy-agent@^4.0.1: - version "4.0.1" - resolved "https://registry.yarnpkg.com/http-proxy-agent/-/http-proxy-agent-4.0.1.tgz#8a8c8ef7f5932ccf953c296ca8291b95aa74aa3a" - integrity sha512-k0zdNgqWTGA6aeIRVpvfVob4fL52dTfaehylg0Y4UvSySvOq/Y+BOyPrgpUrA7HylqvU8vIZGsRuXmspskV0Tg== - dependencies: - "@tootallnate/once" "1" - agent-base "6" - debug "4" - -http2-client@^1.2.5: - version "1.3.3" - resolved "https://registry.yarnpkg.com/http2-client/-/http2-client-1.3.3.tgz#90fc15d646cca86956b156d07c83947d57d659a9" - integrity sha512-nUxLymWQ9pzkzTmir24p2RtsgruLmhje7lH3hLX1IpwvyTg77fW+1brenPPP3USAR+rQ36p5sTA/x7sjCJVkAA== - -https-proxy-agent@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/https-proxy-agent/-/https-proxy-agent-5.0.0.tgz#e2a90542abb68a762e0a0850f6c9edadfd8506b2" - integrity sha512-EkYm5BcKUGiduxzSt3Eppko+PiNWNEpa4ySk9vTC6wDsQJW9rHSa+UhGNJoRYp7bz6Ht1eaRIa6QaJqO5rCFbA== - dependencies: - agent-base "6" - debug "4" - -human-signals@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/human-signals/-/human-signals-2.1.0.tgz#dc91fcba42e4d06e4abaed33b3e7a3c02f514ea0" - integrity sha512-B4FFZ6q/T2jhhksgkbEW3HBvWIfDW85snkQgawt07S7J5QXTk6BkNV+0yAeZrM5QpMAdYlocGoljn0sJ/WQkFw== - -iconv-lite@0.4, iconv-lite@0.4.24, iconv-lite@^0.4.4: - version "0.4.24" - resolved "https://registry.yarnpkg.com/iconv-lite/-/iconv-lite-0.4.24.tgz#2022b4b25fbddc21d2f524974a474aafe733908b" - integrity sha512-v3MXnZAcvnywkTUEZomIActle7RXXeedOR31wwl7VlyoXO4Qi9arvSenNQWne1TcRwhCL1HwLI21bEqdpj8/rA== - dependencies: - safer-buffer ">= 2.1.2 < 3" - -icss-utils@^5.0.0, icss-utils@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/icss-utils/-/icss-utils-5.1.0.tgz#c6be6858abd013d768e98366ae47e25d5887b1ae" - integrity sha512-soFhflCVWLfRNOPU3iv5Z9VUdT44xFRbzjLsEzSr5AQmgqPMTHdU3PMT1Cf1ssx8fLNJDA1juftYl+PUcv3MqA== - -ignore@^5.1.1, ignore@^5.2.0, ignore@^5.2.4: - version "5.2.4" - resolved "https://registry.yarnpkg.com/ignore/-/ignore-5.2.4.tgz#a291c0c6178ff1b960befe47fcdec301674a6324" - integrity sha512-MAb38BcSbH0eHNBxn7ql2NH/kX33OkB3lZ1BNdh7ENeRChHTYsTvWrMubiIAMNS2llXEEgZ1MUOBtXChP3kaFQ== - -import-fresh@^3.0.0, import-fresh@^3.1.0, import-fresh@^3.2.1: - version "3.3.0" - resolved "https://registry.yarnpkg.com/import-fresh/-/import-fresh-3.3.0.tgz#37162c25fcb9ebaa2e6e53d5b4d88ce17d9e0c2b" - integrity sha512-veYYhQa+D1QBKznvhUHxb8faxlrwUnxseDAbAp457E0wLNio2bOSKnjYDhMj+YiAq61xrMGhQk9iXVk5FzgQMw== - dependencies: - parent-module "^1.0.0" - resolve-from "^4.0.0" - -import-lazy@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/import-lazy/-/import-lazy-4.0.0.tgz#e8eb627483a0a43da3c03f3e35548be5cb0cc153" - integrity sha512-rKtvo6a868b5Hu3heneU+L4yEQ4jYKLtjpnPeUdK7h0yzXGmyBTypknlkCvHFBqfX9YlorEiMM6Dnq/5atfHkw== - -import-local@^3.0.2: - version "3.0.3" - resolved "https://registry.yarnpkg.com/import-local/-/import-local-3.0.3.tgz#4d51c2c495ca9393da259ec66b62e022920211e0" - integrity sha512-bE9iaUY3CXH8Cwfan/abDKAxe1KGT9kyGsBPqf6DMK/z0a2OzAsrukeYNgIH6cH5Xr452jb1TUL8rSfCLjZ9uA== - dependencies: - pkg-dir "^4.2.0" - resolve-cwd "^3.0.0" - -imports-loader@^1.1.0: - version "1.2.0" - resolved "https://registry.yarnpkg.com/imports-loader/-/imports-loader-1.2.0.tgz#b06823d0bb42e6f5ff89bc893829000eda46693f" - integrity sha512-zPvangKEgrrPeqeUqH0Uhc59YqK07JqZBi9a9cQ3v/EKUIqrbJHY4CvUrDus2lgQa5AmPyXuGrWP8JJTqzE5RQ== - dependencies: - loader-utils "^2.0.0" - schema-utils "^3.0.0" - source-map "^0.6.1" - strip-comments "^2.0.1" - -imurmurhash@^0.1.4: - version "0.1.4" - resolved "https://registry.yarnpkg.com/imurmurhash/-/imurmurhash-0.1.4.tgz#9218b9b2b928a238b13dc4fb6b6d576f231453ea" - integrity sha1-khi5srkoojixPcT7a21XbyMUU+o= - -indent-string@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/indent-string/-/indent-string-4.0.0.tgz#624f8f4497d619b2d9768531d58f4122854d7251" - integrity sha512-EdDDZu4A2OyIK7Lr/2zG+w5jmbuk1DVBnEwREQvBzspBJkCEbRa8GxU1lghYcaGJCnRWibjDXlq779X1/y5xwg== - -indent-string@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/indent-string/-/indent-string-5.0.0.tgz#4fd2980fccaf8622d14c64d694f4cf33c81951a5" - integrity sha512-m6FAo/spmsW2Ab2fU35JTYwtOKa2yAwXSwgjSv1TJzh4Mh7mC3lzAOVLBprb72XsTrgkEIsl7YrFNAiDiRhIGg== - -infer-owner@^1.0.4: - version "1.0.4" - resolved "https://registry.yarnpkg.com/infer-owner/-/infer-owner-1.0.4.tgz#c4cefcaa8e51051c2a40ba2ce8a3d27295af9467" - integrity sha512-IClj+Xz94+d7irH5qRyfJonOdfTzuDaifE6ZPWfx0N0+/ATZCbuTPq2prFl526urkQd90WyUKIh1DfBQ2hMz9A== - -inflight@^1.0.4: - version "1.0.6" - resolved "https://registry.yarnpkg.com/inflight/-/inflight-1.0.6.tgz#49bd6331d7d02d0c09bc910a1075ba8165b56df9" - integrity sha1-Sb1jMdfQLQwJvJEKEHW6gWW1bfk= - dependencies: - once "^1.3.0" - wrappy "1" - -inherits@2, inherits@^2.0.4, inherits@~2.0.1: - version "2.0.4" - resolved "https://registry.yarnpkg.com/inherits/-/inherits-2.0.4.tgz#0fa2c64f932917c3433a0ded55363aae37416b7c" - integrity sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ== - -ini@^1.3.5: - version "1.3.8" - resolved "https://registry.yarnpkg.com/ini/-/ini-1.3.8.tgz#a29da425b48806f34767a4efce397269af28432c" - integrity sha512-JV/yugV2uzW5iMRSiZAyDtQd+nxtUnjeLt0acNdw98kKLrvuRVyB80tsREOE7yvGVgalhZ6RNXCmEHkUKBKxew== - -inline-style-parser@0.1.1: - version "0.1.1" - resolved "https://registry.yarnpkg.com/inline-style-parser/-/inline-style-parser-0.1.1.tgz#ec8a3b429274e9c0a1f1c4ffa9453a7fef72cea1" - integrity sha512-7NXolsK4CAS5+xvdj5OMMbI962hU/wvwoxk+LWR9Ek9bVtyuuYScDN6eS0rUm6TxApFpw7CX1o4uJzcd4AyD3Q== - -internal-slot@^1.0.3: - version "1.0.3" - resolved "https://registry.yarnpkg.com/internal-slot/-/internal-slot-1.0.3.tgz#7347e307deeea2faac2ac6205d4bc7d34967f59c" - integrity sha512-O0DB1JC/sPyZl7cIo78n5dR7eUSwwpYPiXRhTzNxZVAMUuB8vlnRFyLxdrVToks6XPLVnFfbzaVd5WLjhgg+vA== - dependencies: - get-intrinsic "^1.1.0" - has "^1.0.3" - side-channel "^1.0.4" - -internal-slot@^1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/internal-slot/-/internal-slot-1.0.5.tgz#f2a2ee21f668f8627a4667f309dc0f4fb6674986" - integrity sha512-Y+R5hJrzs52QCG2laLn4udYVnxsfny9CpOhNhUvk/SSSVyF6T27FzRbF0sroPidSu3X8oEAkOn2K804mjpt6UQ== - dependencies: - get-intrinsic "^1.2.0" - has "^1.0.3" - side-channel "^1.0.4" - -internmap@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/internmap/-/internmap-1.0.1.tgz#0017cc8a3b99605f0302f2b198d272e015e5df95" - integrity sha512-lDB5YccMydFBtasVtxnZ3MRBHuaoE8GKsppq+EchKL2U4nK/DmEpPHNH8MZe5HkMtpSiTSOZwfN0tzYjO/lJEw== - -interpret@^2.2.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/interpret/-/interpret-2.2.0.tgz#1a78a0b5965c40a5416d007ad6f50ad27c417df9" - integrity sha512-Ju0Bz/cEia55xDwUWEa8+olFpCiQoypjnQySseKtmjNrnps3P+xfpUmGr90T7yjlVJmOtybRvPXhKMbHr+fWnw== - -invariant@^2.2.4: - version "2.2.4" - resolved "https://registry.yarnpkg.com/invariant/-/invariant-2.2.4.tgz#610f3c92c9359ce1db616e538008d23ff35158e6" - integrity sha512-phJfQVBuaJM5raOpJjSfkiD6BpbCE4Ns//LaXl6wGYtUBY83nWS6Rf9tXm2e8VaK60JEjYldbPif/A2B1C2gNA== - dependencies: - loose-envify "^1.0.0" - -is-alphabetical@^1.0.0: - version "1.0.4" - resolved "https://registry.yarnpkg.com/is-alphabetical/-/is-alphabetical-1.0.4.tgz#9e7d6b94916be22153745d184c298cbf986a686d" - integrity sha512-DwzsA04LQ10FHTZuL0/grVDk4rFoVH1pjAToYwBrHSxcrBIGQuXrQMtD5U1b0U2XVgKZCTLLP8u2Qxqhy3l2Vg== - -is-alphanumerical@^1.0.0: - version "1.0.4" - resolved "https://registry.yarnpkg.com/is-alphanumerical/-/is-alphanumerical-1.0.4.tgz#7eb9a2431f855f6b1ef1a78e326df515696c4dbf" - integrity sha512-UzoZUr+XfVz3t3v4KyGEniVL9BDRoQtY7tOyrRybkVNjDFWyo1yhXNGrrBTQxp3ib9BLAWs7k2YKBQsFRkZG9A== - dependencies: - is-alphabetical "^1.0.0" - is-decimal "^1.0.0" - -is-array-buffer@^3.0.1, is-array-buffer@^3.0.2: - version "3.0.2" - resolved "https://registry.yarnpkg.com/is-array-buffer/-/is-array-buffer-3.0.2.tgz#f2653ced8412081638ecb0ebbd0c41c6e0aecbbe" - integrity sha512-y+FyyR/w8vfIRq4eQcM1EYgSTnmHXPqaF+IgzgraytCFq5Xh8lllDVmAZolPJiZttZLeFSINPYMaEJ7/vWUa1w== - dependencies: - call-bind "^1.0.2" - get-intrinsic "^1.2.0" - is-typed-array "^1.1.10" - -is-arrayish@^0.2.1: - version "0.2.1" - resolved "https://registry.yarnpkg.com/is-arrayish/-/is-arrayish-0.2.1.tgz#77c99840527aa8ecb1a8ba697b80645a7a926a9d" - integrity sha1-d8mYQFJ6qOyxqLppe4BkWnqSap0= - -is-arrayish@^0.3.1: - version "0.3.2" - resolved "https://registry.yarnpkg.com/is-arrayish/-/is-arrayish-0.3.2.tgz#4574a2ae56f7ab206896fb431eaeed066fdf8f03" - integrity sha512-eVRqCvVlZbuw3GrM63ovNSNAeA1K16kaR/LRY/92w0zxQ5/1YzwblUX652i4Xs9RwAGjW9d9y6X88t8OaAJfWQ== - -is-bigint@^1.0.1: - version "1.0.2" - resolved "https://registry.yarnpkg.com/is-bigint/-/is-bigint-1.0.2.tgz#ffb381442503235ad245ea89e45b3dbff040ee5a" - integrity sha512-0JV5+SOCQkIdzjBK9buARcV804Ddu7A0Qet6sHi3FimE9ne6m4BGQZfRn+NZiXbBk4F4XmHfDZIipLj9pX8dSA== - -is-boolean-object@^1.1.0: - version "1.1.1" - resolved "https://registry.yarnpkg.com/is-boolean-object/-/is-boolean-object-1.1.1.tgz#3c0878f035cb821228d350d2e1e36719716a3de8" - integrity sha512-bXdQWkECBUIAcCkeH1unwJLIpZYaa5VvuygSyS/c2lf719mTKZDU5UdDRlpd01UjADgmW8RfqaP+mRaVPdr/Ng== - dependencies: - call-bind "^1.0.2" - -is-buffer@^2.0.0: - version "2.0.5" - resolved "https://registry.yarnpkg.com/is-buffer/-/is-buffer-2.0.5.tgz#ebc252e400d22ff8d77fa09888821a24a658c191" - integrity sha512-i2R6zNFDwgEHJyQUtJEk0XFi1i0dPFn/oqjK3/vPCcDeJvW5NQ83V8QbicfF1SupOaB0h8ntgBC2YiE7dfyctQ== - -is-callable@^1.1.3, is-callable@^1.2.7: - version "1.2.7" - resolved "https://registry.yarnpkg.com/is-callable/-/is-callable-1.2.7.tgz#3bc2a85ea742d9e36205dcacdd72ca1fdc51b055" - integrity sha512-1BC0BVFhS/p0qtw6enp8e+8OD0UrK0oFLztSjNzhcKA3WDuJxxAPXzPuPtKkjEY9UUoEWlX/8fgKeu2S8i9JTA== - -is-callable@^1.1.4, is-callable@^1.2.3: - version "1.2.3" - resolved "https://registry.yarnpkg.com/is-callable/-/is-callable-1.2.3.tgz#8b1e0500b73a1d76c70487636f368e519de8db8e" - integrity sha512-J1DcMe8UYTBSrKezuIUTUwjXsho29693unXM2YhJUTR2txK/eG47bvNa/wipPFmZFgr/N6f1GA66dv0mEyTIyQ== - -is-callable@^1.2.4: - version "1.2.4" - resolved "https://registry.yarnpkg.com/is-callable/-/is-callable-1.2.4.tgz#47301d58dd0259407865547853df6d61fe471945" - integrity sha512-nsuwtxZfMX67Oryl9LCQ+upnC0Z0BgpwntpS89m1H/TLF0zNfzfLMV/9Wa/6MZsj0acpEjAO0KF1xT6ZdLl95w== - -is-core-module@^2.11.0: - version "2.11.0" - resolved "https://registry.yarnpkg.com/is-core-module/-/is-core-module-2.11.0.tgz#ad4cb3e3863e814523c96f3f58d26cc570ff0144" - integrity sha512-RRjxlvLDkD1YJwDbroBHMb+cukurkDWNyHx7D3oNB5x9rb5ogcksMC5wHCadcXoo67gVr/+3GFySh3134zi6rw== - dependencies: - has "^1.0.3" - -is-core-module@^2.2.0: - version "2.4.0" - resolved "https://registry.yarnpkg.com/is-core-module/-/is-core-module-2.4.0.tgz#8e9fc8e15027b011418026e98f0e6f4d86305cc1" - integrity sha512-6A2fkfq1rfeQZjxrZJGerpLCTHRNEBiSgnu0+obeJpEPZRUooHgsizvzv0ZjJwOz3iWIHdJtVWJ/tmPr3D21/A== - dependencies: - has "^1.0.3" - -is-core-module@^2.5.0: - version "2.12.1" - resolved "https://registry.yarnpkg.com/is-core-module/-/is-core-module-2.12.1.tgz#0c0b6885b6f80011c71541ce15c8d66cf5a4f9fd" - integrity sha512-Q4ZuBAe2FUsKtyQJoQHlvP8OvBERxO3jEmy1I7hcRXcJBGGHFh/aJBswbXuS9sgrDH2QUO8ilkwNPHvHMd8clg== - dependencies: - has "^1.0.3" - -is-core-module@^2.8.1: - version "2.9.0" - resolved "https://registry.yarnpkg.com/is-core-module/-/is-core-module-2.9.0.tgz#e1c34429cd51c6dd9e09e0799e396e27b19a9c69" - integrity sha512-+5FPy5PnwmO3lvfMb0AsoPaBG+5KHUI0wYFXOtYPnVVVspTFUuMZNfNaNVRt3FZadstu2c8x23vykRW/NBoU6A== - dependencies: - has "^1.0.3" - -is-date-object@^1.0.1: - version "1.0.4" - resolved "https://registry.yarnpkg.com/is-date-object/-/is-date-object-1.0.4.tgz#550cfcc03afada05eea3dd30981c7b09551f73e5" - integrity sha512-/b4ZVsG7Z5XVtIxs/h9W8nvfLgSAyKYdtGWQLbqy6jA1icmgjf8WCoTKgeS4wy5tYaPePouzFMANbnj94c2Z+A== - -is-decimal@^1.0.0: - version "1.0.4" - resolved "https://registry.yarnpkg.com/is-decimal/-/is-decimal-1.0.4.tgz#65a3a5958a1c5b63a706e1b333d7cd9f630d3fa5" - integrity sha512-RGdriMmQQvZ2aqaQq3awNA6dCGtKpiDFcOzrTWrDAT2MiWrKQVPmxLGHl7Y2nNu6led0kEyoX0enY0qXYsv9zw== - -is-extglob@^2.1.1: - version "2.1.1" - resolved "https://registry.yarnpkg.com/is-extglob/-/is-extglob-2.1.1.tgz#a88c02535791f02ed37c76a1b9ea9773c833f8c2" - integrity sha1-qIwCU1eR8C7TfHahueqXc8gz+MI= - -is-fullwidth-code-point@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz#f116f8064fe90b3f7844a38997c0b75051269f1d" - integrity sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg== - -is-generator-fn@^2.0.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/is-generator-fn/-/is-generator-fn-2.1.0.tgz#7d140adc389aaf3011a8f2a2a4cfa6faadffb118" - integrity sha512-cTIB4yPYL/Grw0EaSzASzg6bBy9gqCofvWN8okThAYIxKJZC+udlRAmGbM0XLeniEJSs8uEgHPGuHSe1XsOLSQ== - -is-glob@^4.0.0: - version "4.0.1" - resolved "https://registry.yarnpkg.com/is-glob/-/is-glob-4.0.1.tgz#7567dbe9f2f5e2467bc77ab83c4a29482407a5dc" - integrity sha512-5G0tKtBTFImOqDnLB2hG6Bp2qcKEFduo4tZu9MT/H6NQv/ghhy30o55ufafxJ/LdH79LLs2Kfrn85TLKyA7BUg== - dependencies: - is-extglob "^2.1.1" - -is-glob@^4.0.1, is-glob@^4.0.3: - version "4.0.3" - resolved "https://registry.yarnpkg.com/is-glob/-/is-glob-4.0.3.tgz#64f61e42cbbb2eec2071a9dac0b28ba1e65d5084" - integrity sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg== - dependencies: - is-extglob "^2.1.1" - -is-hexadecimal@^1.0.0: - version "1.0.4" - resolved "https://registry.yarnpkg.com/is-hexadecimal/-/is-hexadecimal-1.0.4.tgz#cc35c97588da4bd49a8eedd6bc4082d44dcb23a7" - integrity sha512-gyPJuv83bHMpocVYoqof5VDiZveEoGoFL8m3BXNb2VW8Xs+rz9kqO8LOQ5DH6EsuvilT1ApazU0pyl+ytbPtlw== - -is-negative-zero@^2.0.1: - version "2.0.1" - resolved "https://registry.yarnpkg.com/is-negative-zero/-/is-negative-zero-2.0.1.tgz#3de746c18dda2319241a53675908d8f766f11c24" - integrity sha512-2z6JzQvZRa9A2Y7xC6dQQm4FSTSTNWjKIYYTt4246eMTJmIo0Q+ZyOsU66X8lxK1AbB92dFeglPLrhwpeRKO6w== - -is-negative-zero@^2.0.2: - version "2.0.2" - resolved "https://registry.yarnpkg.com/is-negative-zero/-/is-negative-zero-2.0.2.tgz#7bf6f03a28003b8b3965de3ac26f664d765f3150" - integrity sha512-dqJvarLawXsFbNDeJW7zAz8ItJ9cd28YufuuFzh0G8pNHjJMnY08Dv7sYX2uF5UpQOwieAeOExEYAWWfu7ZZUA== - -is-number-object@^1.0.4: - version "1.0.5" - resolved "https://registry.yarnpkg.com/is-number-object/-/is-number-object-1.0.5.tgz#6edfaeed7950cff19afedce9fbfca9ee6dd289eb" - integrity sha512-RU0lI/n95pMoUKu9v1BZP5MBcZuNSVJkMkAG2dJqC4z2GlkGUNeH68SuHuBKBD/XFe+LHZ+f9BKkLET60Niedw== - -is-number@^7.0.0: - version "7.0.0" - resolved "https://registry.yarnpkg.com/is-number/-/is-number-7.0.0.tgz#7535345b896734d5f80c4d06c50955527a14f12b" - integrity sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng== - -is-path-cwd@^2.0.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/is-path-cwd/-/is-path-cwd-2.2.0.tgz#67d43b82664a7b5191fd9119127eb300048a9fdb" - integrity sha512-w942bTcih8fdJPJmQHFzkS76NEP8Kzzvmw92cXsazb8intwLqPibPPdXf4ANdKV3rYMuuQYGIWtvz9JilB3NFQ== - -is-path-in-cwd@^2.0.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/is-path-in-cwd/-/is-path-in-cwd-2.1.0.tgz#bfe2dca26c69f397265a4009963602935a053acb" - integrity sha512-rNocXHgipO+rvnP6dk3zI20RpOtrAM/kzbB258Uw5BWr3TpXi861yzjo16Dn4hUox07iw5AyeMLHWsujkjzvRQ== - dependencies: - is-path-inside "^2.1.0" - -is-path-inside@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/is-path-inside/-/is-path-inside-2.1.0.tgz#7c9810587d659a40d27bcdb4d5616eab059494b2" - integrity sha512-wiyhTzfDWsvwAW53OBWF5zuvaOGlZ6PwYxAbPVDhpm+gM09xKQGjBq/8uYN12aDvMxnAnq3dxTyoSoRNmg5YFg== - dependencies: - path-is-inside "^1.0.2" - -is-plain-obj@^1.1.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/is-plain-obj/-/is-plain-obj-1.1.0.tgz#71a50c8429dfca773c92a390a4a03b39fcd51d3e" - integrity sha1-caUMhCnfync8kqOQpKA7OfzVHT4= - -is-plain-obj@^4.0.0: - version "4.1.0" - resolved "https://registry.yarnpkg.com/is-plain-obj/-/is-plain-obj-4.1.0.tgz#d65025edec3657ce032fd7db63c97883eaed71f0" - integrity sha512-+Pgi+vMuUNkJyExiMBt5IlFoMyKnr5zhJ4Uspz58WOhBF5QoIZkFyNHIbBAtHwzVAgk5RtndVNsDRN61/mmDqg== - -is-plain-object@^2.0.4: - version "2.0.4" - resolved "https://registry.yarnpkg.com/is-plain-object/-/is-plain-object-2.0.4.tgz#2c163b3fafb1b606d9d17928f05c2a1c38e07677" - integrity sha512-h5PpgXkWitc38BBMYawTYMWJHFZJVnBquFE57xFpjB8pJFiF6gZ+bU+WyI/yqXiFR5mdLsgYNaPe8uao6Uv9Og== - dependencies: - isobject "^3.0.1" - -is-plain-object@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/is-plain-object/-/is-plain-object-5.0.0.tgz#4427f50ab3429e9025ea7d52e9043a9ef4159344" - integrity sha512-VRSzKkbMm5jMDoKLbltAkFQ5Qr7VDiTFGXxYFXXowVj387GeGNOCsOH6Msy00SGZ3Fp84b1Naa1psqgcCIEP5Q== - -is-potential-custom-element-name@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/is-potential-custom-element-name/-/is-potential-custom-element-name-1.0.1.tgz#171ed6f19e3ac554394edf78caa05784a45bebb5" - integrity sha512-bCYeRA2rVibKZd+s2625gGnGF/t7DSqDs4dP7CrLA1m7jKWz6pps0LpYLJN8Q64HtmPKJ1hrN3nzPNKFEKOUiQ== - -is-regex@^1.1.3: - version "1.1.3" - resolved "https://registry.yarnpkg.com/is-regex/-/is-regex-1.1.3.tgz#d029f9aff6448b93ebbe3f33dac71511fdcbef9f" - integrity sha512-qSVXFz28HM7y+IWX6vLCsexdlvzT1PJNFSBuaQLQ5o0IEw8UDYW6/2+eCMVyIsbM8CNLX2a/QWmSpyxYEHY7CQ== - dependencies: - call-bind "^1.0.2" - has-symbols "^1.0.2" - -is-regex@^1.1.4: - version "1.1.4" - resolved "https://registry.yarnpkg.com/is-regex/-/is-regex-1.1.4.tgz#eef5663cd59fa4c0ae339505323df6854bb15958" - integrity sha512-kvRdxDsxZjhzUX07ZnLydzS1TU/TJlTUHHY4YLL87e37oUA49DfkLqgy+VjFocowy29cKvcSiu+kIv728jTTVg== - dependencies: - call-bind "^1.0.2" - has-tostringtag "^1.0.0" - -is-shared-array-buffer@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/is-shared-array-buffer/-/is-shared-array-buffer-1.0.2.tgz#8f259c573b60b6a32d4058a1a07430c0a7344c79" - integrity sha512-sqN2UDu1/0y6uvXyStCOzyhAjCSlHceFoMKJW8W9EU9cvic/QdsZ0kEU93HEy3IUEFZIiH/3w+AH/UQbPHNdhA== - dependencies: - call-bind "^1.0.2" - -is-stream@^2.0.0: - version "2.0.1" - resolved "https://registry.yarnpkg.com/is-stream/-/is-stream-2.0.1.tgz#fac1e3d53b97ad5a9d0ae9cef2389f5810a5c077" - integrity sha512-hFoiJiTl63nn+kstHGBtewWSKnQLpyb155KHheA1l39uvtO9nWIop1p3udqPcUd/xbF1VLMO4n7OI6p7RbngDg== - -is-string@^1.0.5, is-string@^1.0.6: - version "1.0.6" - resolved "https://registry.yarnpkg.com/is-string/-/is-string-1.0.6.tgz#3fe5d5992fb0d93404f32584d4b0179a71b54a5f" - integrity sha512-2gdzbKUuqtQ3lYNrUTQYoClPhm7oQu4UdpSZMp1/DGgkHBT8E2Z1l0yMdb6D4zNAxwDiMv8MdulKROJGNl0Q0w== - -is-string@^1.0.7: - version "1.0.7" - resolved "https://registry.yarnpkg.com/is-string/-/is-string-1.0.7.tgz#0dd12bf2006f255bb58f695110eff7491eebc0fd" - integrity sha512-tE2UXzivje6ofPW7l23cjDOMa09gb7xlAqG6jG5ej6uPV32TlWP3NKPigtaGeHNu9fohccRYvIiZMfOOnOYUtg== - dependencies: - has-tostringtag "^1.0.0" - -is-symbol@^1.0.2, is-symbol@^1.0.3: - version "1.0.4" - resolved "https://registry.yarnpkg.com/is-symbol/-/is-symbol-1.0.4.tgz#a6dac93b635b063ca6872236de88910a57af139c" - integrity sha512-C/CPBqKWnvdcxqIARxyOh4v1UUEOCHpgDa0WYgpKDFMszcrPcffg5uhwSgPCLD2WWxmq6isisz87tzT01tuGhg== - dependencies: - has-symbols "^1.0.2" - -is-typed-array@^1.1.10, is-typed-array@^1.1.9: - version "1.1.10" - resolved "https://registry.yarnpkg.com/is-typed-array/-/is-typed-array-1.1.10.tgz#36a5b5cb4189b575d1a3e4b08536bfb485801e3f" - integrity sha512-PJqgEHiWZvMpaFZ3uTc8kHPM4+4ADTlDniuQL7cU/UDA0Ql7F70yGfHph3cLNe+c9toaigv+DFzTJKhc2CtO6A== - dependencies: - available-typed-arrays "^1.0.5" - call-bind "^1.0.2" - for-each "^0.3.3" - gopd "^1.0.1" - has-tostringtag "^1.0.0" - -is-typedarray@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/is-typedarray/-/is-typedarray-1.0.0.tgz#e479c80858df0c1b11ddda6940f96011fcda4a9a" - integrity sha1-5HnICFjfDBsR3dppQPlgEfzaSpo= - -is-weakref@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/is-weakref/-/is-weakref-1.0.2.tgz#9529f383a9338205e89765e0392efc2f100f06f2" - integrity sha512-qctsuLZmIQ0+vSSMfoVvyFe2+GSEvnmZ2ezTup1SBse9+twCCeial6EEi3Nc2KFcf6+qz2FBPnjXsk8xhKSaPQ== - dependencies: - call-bind "^1.0.2" - -isarray@0.0.1: - version "0.0.1" - resolved "https://registry.yarnpkg.com/isarray/-/isarray-0.0.1.tgz#8a18acfca9a8f4177e09abfc6038939b05d1eedf" - integrity sha1-ihis/Kmo9Bd+Cav8YDiTmwXR7t8= - -isexe@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/isexe/-/isexe-2.0.0.tgz#e8fbf374dc556ff8947a10dcb0572d633f2cfa10" - integrity sha1-6PvzdNxVb/iUehDcsFctYz8s+hA= - -isobject@^3.0.1: - version "3.0.1" - resolved "https://registry.yarnpkg.com/isobject/-/isobject-3.0.1.tgz#4e431e92b11a9731636aa1f9c8d1ccbcfdab78df" - integrity sha1-TkMekrEalzFjaqH5yNHMvP2reN8= - -istanbul-lib-coverage@^3.0.0, istanbul-lib-coverage@^3.2.0: - version "3.2.0" - resolved "https://registry.yarnpkg.com/istanbul-lib-coverage/-/istanbul-lib-coverage-3.2.0.tgz#189e7909d0a39fa5a3dfad5b03f71947770191d3" - integrity sha512-eOeJ5BHCmHYvQK7xt9GkdHuzuCGS1Y6g9Gvnx3Ym33fz/HpLRYxiS0wHNr+m/MBC8B647Xt608vCDEvhl9c6Mw== - -istanbul-lib-instrument@^5.0.4, istanbul-lib-instrument@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/istanbul-lib-instrument/-/istanbul-lib-instrument-5.1.0.tgz#7b49198b657b27a730b8e9cb601f1e1bff24c59a" - integrity sha512-czwUz525rkOFDJxfKK6mYfIs9zBKILyrZQxjz3ABhjQXhbhFsSbo1HW/BFcsDnfJYJWA6thRR5/TUY2qs5W99Q== - dependencies: - "@babel/core" "^7.12.3" - "@babel/parser" "^7.14.7" - "@istanbuljs/schema" "^0.1.2" - istanbul-lib-coverage "^3.2.0" - semver "^6.3.0" - -istanbul-lib-report@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/istanbul-lib-report/-/istanbul-lib-report-3.0.0.tgz#7518fe52ea44de372f460a76b5ecda9ffb73d8a6" - integrity sha512-wcdi+uAKzfiGT2abPpKZ0hSU1rGQjUQnLvtY5MpQ7QCTahD3VODhcu4wcfY1YtkGaDD5yuydOLINXsfbus9ROw== - dependencies: - istanbul-lib-coverage "^3.0.0" - make-dir "^3.0.0" - supports-color "^7.1.0" - -istanbul-lib-source-maps@^4.0.0: - version "4.0.1" - resolved "https://registry.yarnpkg.com/istanbul-lib-source-maps/-/istanbul-lib-source-maps-4.0.1.tgz#895f3a709fcfba34c6de5a42939022f3e4358551" - integrity sha512-n3s8EwkdFIJCG3BPKBYvskgXGoy88ARzvegkitk60NxRdwltLOTaH7CUiMRXvwYorl0Q712iEjcWB+fK/MrWVw== - dependencies: - debug "^4.1.1" - istanbul-lib-coverage "^3.0.0" - source-map "^0.6.1" - -istanbul-reports@^3.1.3: - version "3.1.4" - resolved "https://registry.yarnpkg.com/istanbul-reports/-/istanbul-reports-3.1.4.tgz#1b6f068ecbc6c331040aab5741991273e609e40c" - integrity sha512-r1/DshN4KSE7xWEknZLLLLDn5CJybV3nw01VTkp6D5jzLuELlcbudfj/eSQFvrKsJuTVCGnePO7ho82Nw9zzfw== - dependencies: - html-escaper "^2.0.0" - istanbul-lib-report "^3.0.0" - -jest-canvas-mock@^2.5.1: - version "2.5.1" - resolved "https://registry.yarnpkg.com/jest-canvas-mock/-/jest-canvas-mock-2.5.1.tgz#81509af658ef485e9a1bf39c64e06761517bdbcb" - integrity sha512-IVnRiz+v4EYn3ydM/pBo8GW/J+nU/Hg5gHBQQOUQhdRyNfvHnabB8ReqARLO0p+kvQghqr4V0tA92CF3JcUSRg== - dependencies: - cssfontparser "^1.2.1" - moo-color "^1.0.2" - -jest-changed-files@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-changed-files/-/jest-changed-files-27.5.1.tgz#a348aed00ec9bf671cc58a66fcbe7c3dfd6a68f5" - integrity sha512-buBLMiByfWGCoMsLLzGUUSpAmIAGnbR2KJoMN10ziLhOLvP4e0SlypHnAel8iqQXTrcbmfEY9sSqae5sgUsTvw== - dependencies: - "@jest/types" "^27.5.1" - execa "^5.0.0" - throat "^6.0.1" - -jest-circus@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-circus/-/jest-circus-27.5.1.tgz#37a5a4459b7bf4406e53d637b49d22c65d125ecc" - integrity sha512-D95R7x5UtlMA5iBYsOHFFbMD/GVA4R/Kdq15f7xYWUfWHBto9NYRsOvnSauTgdF+ogCpJ4tyKOXhUifxS65gdw== - dependencies: - "@jest/environment" "^27.5.1" - "@jest/test-result" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/node" "*" - chalk "^4.0.0" - co "^4.6.0" - dedent "^0.7.0" - expect "^27.5.1" - is-generator-fn "^2.0.0" - jest-each "^27.5.1" - jest-matcher-utils "^27.5.1" - jest-message-util "^27.5.1" - jest-runtime "^27.5.1" - jest-snapshot "^27.5.1" - jest-util "^27.5.1" - pretty-format "^27.5.1" - slash "^3.0.0" - stack-utils "^2.0.3" - throat "^6.0.1" - -jest-cli@^27.3.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-cli/-/jest-cli-27.5.1.tgz#278794a6e6458ea8029547e6c6cbf673bd30b145" - integrity sha512-Hc6HOOwYq4/74/c62dEE3r5elx8wjYqxY0r0G/nFrLDPMFRu6RA/u8qINOIkvhxG7mMQ5EJsOGfRpI8L6eFUVw== - dependencies: - "@jest/core" "^27.5.1" - "@jest/test-result" "^27.5.1" - "@jest/types" "^27.5.1" - chalk "^4.0.0" - exit "^0.1.2" - graceful-fs "^4.2.9" - import-local "^3.0.2" - jest-config "^27.5.1" - jest-util "^27.5.1" - jest-validate "^27.5.1" - prompts "^2.0.1" - yargs "^16.2.0" - -jest-config@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-config/-/jest-config-27.5.1.tgz#5c387de33dca3f99ad6357ddeccd91bf3a0e4a41" - integrity sha512-5sAsjm6tGdsVbW9ahcChPAFCk4IlkQUknH5AvKjuLTSlcO/wCZKyFdn7Rg0EkC+OGgWODEy2hDpWB1PgzH0JNA== - dependencies: - "@babel/core" "^7.8.0" - "@jest/test-sequencer" "^27.5.1" - "@jest/types" "^27.5.1" - babel-jest "^27.5.1" - chalk "^4.0.0" - ci-info "^3.2.0" - deepmerge "^4.2.2" - glob "^7.1.1" - graceful-fs "^4.2.9" - jest-circus "^27.5.1" - jest-environment-jsdom "^27.5.1" - jest-environment-node "^27.5.1" - jest-get-type "^27.5.1" - jest-jasmine2 "^27.5.1" - jest-regex-util "^27.5.1" - jest-resolve "^27.5.1" - jest-runner "^27.5.1" - jest-util "^27.5.1" - jest-validate "^27.5.1" - micromatch "^4.0.4" - parse-json "^5.2.0" - pretty-format "^27.5.1" - slash "^3.0.0" - strip-json-comments "^3.1.1" - -jest-diff@^27.0.0: - version "27.3.1" - resolved "https://registry.yarnpkg.com/jest-diff/-/jest-diff-27.3.1.tgz#d2775fea15411f5f5aeda2a5e02c2f36440f6d55" - integrity sha512-PCeuAH4AWUo2O5+ksW4pL9v5xJAcIKPUPfIhZBcG1RKv/0+dvaWTQK1Nrau8d67dp65fOqbeMdoil+6PedyEPQ== - dependencies: - chalk "^4.0.0" - diff-sequences "^27.0.6" - jest-get-type "^27.3.1" - pretty-format "^27.3.1" - -jest-diff@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-diff/-/jest-diff-27.5.1.tgz#a07f5011ac9e6643cf8a95a462b7b1ecf6680def" - integrity sha512-m0NvkX55LDt9T4mctTEgnZk3fmEg3NRYutvMPWM/0iPnkFj2wIeF45O1718cMSOFO1vINkqmxqD8vE37uTEbqw== - dependencies: - chalk "^4.0.0" - diff-sequences "^27.5.1" - jest-get-type "^27.5.1" - pretty-format "^27.5.1" - -jest-docblock@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-docblock/-/jest-docblock-27.5.1.tgz#14092f364a42c6108d42c33c8cf30e058e25f6c0" - integrity sha512-rl7hlABeTsRYxKiUfpHrQrG4e2obOiTQWfMEH3PxPjOtdsfLQO4ReWSZaQ7DETm4xu07rl4q/h4zcKXyU0/OzQ== - dependencies: - detect-newline "^3.0.0" - -jest-each@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-each/-/jest-each-27.5.1.tgz#5bc87016f45ed9507fed6e4702a5b468a5b2c44e" - integrity sha512-1Ff6p+FbhT/bXQnEouYy00bkNSY7OUpfIcmdl8vZ31A1UUaurOLPA8a8BbJOF2RDUElwJhmeaV7LnagI+5UwNQ== - dependencies: - "@jest/types" "^27.5.1" - chalk "^4.0.0" - jest-get-type "^27.5.1" - jest-util "^27.5.1" - pretty-format "^27.5.1" - -jest-environment-jsdom@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-environment-jsdom/-/jest-environment-jsdom-27.5.1.tgz#ea9ccd1fc610209655a77898f86b2b559516a546" - integrity sha512-TFBvkTC1Hnnnrka/fUb56atfDtJ9VMZ94JkjTbggl1PEpwrYtUBKMezB3inLmWqQsXYLcMwNoDQwoBTAvFfsfw== - dependencies: - "@jest/environment" "^27.5.1" - "@jest/fake-timers" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/node" "*" - jest-mock "^27.5.1" - jest-util "^27.5.1" - jsdom "^16.6.0" - -jest-environment-node@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-environment-node/-/jest-environment-node-27.5.1.tgz#dedc2cfe52fab6b8f5714b4808aefa85357a365e" - integrity sha512-Jt4ZUnxdOsTGwSRAfKEnE6BcwsSPNOijjwifq5sDFSA2kesnXTvNqKHYgM0hDq3549Uf/KzdXNYn4wMZJPlFLw== - dependencies: - "@jest/environment" "^27.5.1" - "@jest/fake-timers" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/node" "*" - jest-mock "^27.5.1" - jest-util "^27.5.1" - -jest-get-type@^27.3.1: - version "27.3.1" - resolved "https://registry.yarnpkg.com/jest-get-type/-/jest-get-type-27.3.1.tgz#a8a2b0a12b50169773099eee60a0e6dd11423eff" - integrity sha512-+Ilqi8hgHSAdhlQ3s12CAVNd8H96ZkQBfYoXmArzZnOfAtVAJEiPDBirjByEblvG/4LPJmkL+nBqPO3A1YJAEg== - -jest-get-type@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-get-type/-/jest-get-type-27.5.1.tgz#3cd613c507b0f7ace013df407a1c1cd578bcb4f1" - integrity sha512-2KY95ksYSaK7DMBWQn6dQz3kqAf3BB64y2udeG+hv4KfSOb9qwcYQstTJc1KCbsix+wLZWZYN8t7nwX3GOBLRw== - -jest-haste-map@^27.3.1: - version "27.3.1" - resolved "https://registry.yarnpkg.com/jest-haste-map/-/jest-haste-map-27.3.1.tgz#7656fbd64bf48bda904e759fc9d93e2c807353ee" - integrity sha512-lYfNZIzwPccDJZIyk9Iz5iQMM/MH56NIIcGj7AFU1YyA4ewWFBl8z+YPJuSCRML/ee2cCt2y3W4K3VXPT6Nhzg== - dependencies: - "@jest/types" "^27.2.5" - "@types/graceful-fs" "^4.1.2" - "@types/node" "*" - anymatch "^3.0.3" - fb-watchman "^2.0.0" - graceful-fs "^4.2.4" - jest-regex-util "^27.0.6" - jest-serializer "^27.0.6" - jest-util "^27.3.1" - jest-worker "^27.3.1" - micromatch "^4.0.4" - walker "^1.0.7" - optionalDependencies: - fsevents "^2.3.2" - -jest-haste-map@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-haste-map/-/jest-haste-map-27.5.1.tgz#9fd8bd7e7b4fa502d9c6164c5640512b4e811e7f" - integrity sha512-7GgkZ4Fw4NFbMSDSpZwXeBiIbx+t/46nJ2QitkOjvwPYyZmqttu2TDSimMHP1EkPOi4xUZAN1doE5Vd25H4Jng== - dependencies: - "@jest/types" "^27.5.1" - "@types/graceful-fs" "^4.1.2" - "@types/node" "*" - anymatch "^3.0.3" - fb-watchman "^2.0.0" - graceful-fs "^4.2.9" - jest-regex-util "^27.5.1" - jest-serializer "^27.5.1" - jest-util "^27.5.1" - jest-worker "^27.5.1" - micromatch "^4.0.4" - walker "^1.0.7" - optionalDependencies: - fsevents "^2.3.2" - -jest-jasmine2@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-jasmine2/-/jest-jasmine2-27.5.1.tgz#a037b0034ef49a9f3d71c4375a796f3b230d1ac4" - integrity sha512-jtq7VVyG8SqAorDpApwiJJImd0V2wv1xzdheGHRGyuT7gZm6gG47QEskOlzsN1PG/6WNaCo5pmwMHDf3AkG2pQ== - dependencies: - "@jest/environment" "^27.5.1" - "@jest/source-map" "^27.5.1" - "@jest/test-result" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/node" "*" - chalk "^4.0.0" - co "^4.6.0" - expect "^27.5.1" - is-generator-fn "^2.0.0" - jest-each "^27.5.1" - jest-matcher-utils "^27.5.1" - jest-message-util "^27.5.1" - jest-runtime "^27.5.1" - jest-snapshot "^27.5.1" - jest-util "^27.5.1" - pretty-format "^27.5.1" - throat "^6.0.1" - -jest-leak-detector@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-leak-detector/-/jest-leak-detector-27.5.1.tgz#6ec9d54c3579dd6e3e66d70e3498adf80fde3fb8" - integrity sha512-POXfWAMvfU6WMUXftV4HolnJfnPOGEu10fscNCA76KBpRRhcMN2c8d3iT2pxQS3HLbA+5X4sOUPzYO2NUyIlHQ== - dependencies: - jest-get-type "^27.5.1" - pretty-format "^27.5.1" - -jest-matcher-utils@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-matcher-utils/-/jest-matcher-utils-27.5.1.tgz#9c0cdbda8245bc22d2331729d1091308b40cf8ab" - integrity sha512-z2uTx/T6LBaCoNWNFWwChLBKYxTMcGBRjAt+2SbP929/Fflb9aa5LGma654Rz8z9HLxsrUaYzxE9T/EFIL/PAw== - dependencies: - chalk "^4.0.0" - jest-diff "^27.5.1" - jest-get-type "^27.5.1" - pretty-format "^27.5.1" - -jest-message-util@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-message-util/-/jest-message-util-27.5.1.tgz#bdda72806da10d9ed6425e12afff38cd1458b6cf" - integrity sha512-rMyFe1+jnyAAf+NHwTclDz0eAaLkVDdKVHHBFWsBWHnnh5YeJMNWWsv7AbFYXfK3oTqvL7VTWkhNLu1jX24D+g== - dependencies: - "@babel/code-frame" "^7.12.13" - "@jest/types" "^27.5.1" - "@types/stack-utils" "^2.0.0" - chalk "^4.0.0" - graceful-fs "^4.2.9" - micromatch "^4.0.4" - pretty-format "^27.5.1" - slash "^3.0.0" - stack-utils "^2.0.3" - -jest-mock@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-mock/-/jest-mock-27.5.1.tgz#19948336d49ef4d9c52021d34ac7b5f36ff967d6" - integrity sha512-K4jKbY1d4ENhbrG2zuPWaQBvDly+iZ2yAW+T1fATN78hc0sInwn7wZB8XtlNnvHug5RMwV897Xm4LqmPM4e2Og== - dependencies: - "@jest/types" "^27.5.1" - "@types/node" "*" - -jest-pnp-resolver@^1.2.2: - version "1.2.2" - resolved "https://registry.yarnpkg.com/jest-pnp-resolver/-/jest-pnp-resolver-1.2.2.tgz#b704ac0ae028a89108a4d040b3f919dfddc8e33c" - integrity sha512-olV41bKSMm8BdnuMsewT4jqlZ8+3TCARAXjZGT9jcoSnrfUnRCqnMoF9XEeoWjbzObpqF9dRhHQj0Xb9QdF6/w== - -jest-regex-util@^27.0.6: - version "27.0.6" - resolved "https://registry.yarnpkg.com/jest-regex-util/-/jest-regex-util-27.0.6.tgz#02e112082935ae949ce5d13b2675db3d8c87d9c5" - integrity sha512-SUhPzBsGa1IKm8hx2F4NfTGGp+r7BXJ4CulsZ1k2kI+mGLG+lxGrs76veN2LF/aUdGosJBzKgXmNCw+BzFqBDQ== - -jest-regex-util@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-regex-util/-/jest-regex-util-27.5.1.tgz#4da143f7e9fd1e542d4aa69617b38e4a78365b95" - integrity sha512-4bfKq2zie+x16okqDXjXn9ql2B0dScQu+vcwe4TvFVhkVyuWLqpZrZtXxLLWoXYgn0E87I6r6GRYHF7wFZBUvg== - -jest-resolve-dependencies@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-resolve-dependencies/-/jest-resolve-dependencies-27.5.1.tgz#d811ecc8305e731cc86dd79741ee98fed06f1da8" - integrity sha512-QQOOdY4PE39iawDn5rzbIePNigfe5B9Z91GDD1ae/xNDlu9kaat8QQ5EKnNmVWPV54hUdxCVwwj6YMgR2O7IOg== - dependencies: - "@jest/types" "^27.5.1" - jest-regex-util "^27.5.1" - jest-snapshot "^27.5.1" - -jest-resolve@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-resolve/-/jest-resolve-27.5.1.tgz#a2f1c5a0796ec18fe9eb1536ac3814c23617b384" - integrity sha512-FFDy8/9E6CV83IMbDpcjOhumAQPDyETnU2KZ1O98DwTnz8AOBsW/Xv3GySr1mOZdItLR+zDZ7I/UdTFbgSOVCw== - dependencies: - "@jest/types" "^27.5.1" - chalk "^4.0.0" - graceful-fs "^4.2.9" - jest-haste-map "^27.5.1" - jest-pnp-resolver "^1.2.2" - jest-util "^27.5.1" - jest-validate "^27.5.1" - resolve "^1.20.0" - resolve.exports "^1.1.0" - slash "^3.0.0" - -jest-runner@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-runner/-/jest-runner-27.5.1.tgz#071b27c1fa30d90540805c5645a0ec167c7b62e5" - integrity sha512-g4NPsM4mFCOwFKXO4p/H/kWGdJp9V8kURY2lX8Me2drgXqG7rrZAx5kv+5H7wtt/cdFIjhqYx1HrlqWHaOvDaQ== - dependencies: - "@jest/console" "^27.5.1" - "@jest/environment" "^27.5.1" - "@jest/test-result" "^27.5.1" - "@jest/transform" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/node" "*" - chalk "^4.0.0" - emittery "^0.8.1" - graceful-fs "^4.2.9" - jest-docblock "^27.5.1" - jest-environment-jsdom "^27.5.1" - jest-environment-node "^27.5.1" - jest-haste-map "^27.5.1" - jest-leak-detector "^27.5.1" - jest-message-util "^27.5.1" - jest-resolve "^27.5.1" - jest-runtime "^27.5.1" - jest-util "^27.5.1" - jest-worker "^27.5.1" - source-map-support "^0.5.6" - throat "^6.0.1" - -jest-runtime@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-runtime/-/jest-runtime-27.5.1.tgz#4896003d7a334f7e8e4a53ba93fb9bcd3db0a1af" - integrity sha512-o7gxw3Gf+H2IGt8fv0RiyE1+r83FJBRruoA+FXrlHw6xEyBsU8ugA6IPfTdVyA0w8HClpbK+DGJxH59UrNMx8A== - dependencies: - "@jest/environment" "^27.5.1" - "@jest/fake-timers" "^27.5.1" - "@jest/globals" "^27.5.1" - "@jest/source-map" "^27.5.1" - "@jest/test-result" "^27.5.1" - "@jest/transform" "^27.5.1" - "@jest/types" "^27.5.1" - chalk "^4.0.0" - cjs-module-lexer "^1.0.0" - collect-v8-coverage "^1.0.0" - execa "^5.0.0" - glob "^7.1.3" - graceful-fs "^4.2.9" - jest-haste-map "^27.5.1" - jest-message-util "^27.5.1" - jest-mock "^27.5.1" - jest-regex-util "^27.5.1" - jest-resolve "^27.5.1" - jest-snapshot "^27.5.1" - jest-util "^27.5.1" - slash "^3.0.0" - strip-bom "^4.0.0" - -jest-serializer@^27.0.6: - version "27.0.6" - resolved "https://registry.yarnpkg.com/jest-serializer/-/jest-serializer-27.0.6.tgz#93a6c74e0132b81a2d54623251c46c498bb5bec1" - integrity sha512-PtGdVK9EGC7dsaziskfqaAPib6wTViY3G8E5wz9tLVPhHyiDNTZn/xjZ4khAw+09QkoOVpn7vF5nPSN6dtBexA== - dependencies: - "@types/node" "*" - graceful-fs "^4.2.4" - -jest-serializer@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-serializer/-/jest-serializer-27.5.1.tgz#81438410a30ea66fd57ff730835123dea1fb1f64" - integrity sha512-jZCyo6iIxO1aqUxpuBlwTDMkzOAJS4a3eYz3YzgxxVQFwLeSA7Jfq5cbqCY+JLvTDrWirgusI/0KwxKMgrdf7w== - dependencies: - "@types/node" "*" - graceful-fs "^4.2.9" - -jest-snapshot@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-snapshot/-/jest-snapshot-27.5.1.tgz#b668d50d23d38054a51b42c4039cab59ae6eb6a1" - integrity sha512-yYykXI5a0I31xX67mgeLw1DZ0bJB+gpq5IpSuCAoyDi0+BhgU/RIrL+RTzDmkNTchvDFWKP8lp+w/42Z3us5sA== - dependencies: - "@babel/core" "^7.7.2" - "@babel/generator" "^7.7.2" - "@babel/plugin-syntax-typescript" "^7.7.2" - "@babel/traverse" "^7.7.2" - "@babel/types" "^7.0.0" - "@jest/transform" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/babel__traverse" "^7.0.4" - "@types/prettier" "^2.1.5" - babel-preset-current-node-syntax "^1.0.0" - chalk "^4.0.0" - expect "^27.5.1" - graceful-fs "^4.2.9" - jest-diff "^27.5.1" - jest-get-type "^27.5.1" - jest-haste-map "^27.5.1" - jest-matcher-utils "^27.5.1" - jest-message-util "^27.5.1" - jest-util "^27.5.1" - natural-compare "^1.4.0" - pretty-format "^27.5.1" - semver "^7.3.2" - -jest-util@^27.3.1: - version "27.3.1" - resolved "https://registry.yarnpkg.com/jest-util/-/jest-util-27.3.1.tgz#a58cdc7b6c8a560caac9ed6bdfc4e4ff23f80429" - integrity sha512-8fg+ifEH3GDryLQf/eKZck1DEs2YuVPBCMOaHQxVVLmQwl/CDhWzrvChTX4efLZxGrw+AA0mSXv78cyytBt/uw== - dependencies: - "@jest/types" "^27.2.5" - "@types/node" "*" - chalk "^4.0.0" - ci-info "^3.2.0" - graceful-fs "^4.2.4" - picomatch "^2.2.3" - -jest-util@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-util/-/jest-util-27.5.1.tgz#3ba9771e8e31a0b85da48fe0b0891fb86c01c2f9" - integrity sha512-Kv2o/8jNvX1MQ0KGtw480E/w4fBCDOnH6+6DmeKi6LZUIlKA5kwY0YNdlzaWTiVgxqAqik11QyxDOKk543aKXw== - dependencies: - "@jest/types" "^27.5.1" - "@types/node" "*" - chalk "^4.0.0" - ci-info "^3.2.0" - graceful-fs "^4.2.9" - picomatch "^2.2.3" - -jest-validate@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-validate/-/jest-validate-27.5.1.tgz#9197d54dc0bdb52260b8db40b46ae668e04df067" - integrity sha512-thkNli0LYTmOI1tDB3FI1S1RTp/Bqyd9pTarJwL87OIBFuqEb5Apv5EaApEudYg4g86e3CT6kM0RowkhtEnCBQ== - dependencies: - "@jest/types" "^27.5.1" - camelcase "^6.2.0" - chalk "^4.0.0" - jest-get-type "^27.5.1" - leven "^3.1.0" - pretty-format "^27.5.1" - -jest-watcher@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-watcher/-/jest-watcher-27.5.1.tgz#71bd85fb9bde3a2c2ec4dc353437971c43c642a2" - integrity sha512-z676SuD6Z8o8qbmEGhoEUFOM1+jfEiL3DXHK/xgEiG2EyNYfFG60jluWcupY6dATjfEsKQuibReS1djInQnoVw== - dependencies: - "@jest/test-result" "^27.5.1" - "@jest/types" "^27.5.1" - "@types/node" "*" - ansi-escapes "^4.2.1" - chalk "^4.0.0" - jest-util "^27.5.1" - string-length "^4.0.1" - -jest-worker@^26.5.0: - version "26.6.2" - resolved "https://registry.yarnpkg.com/jest-worker/-/jest-worker-26.6.2.tgz#7f72cbc4d643c365e27b9fd775f9d0eaa9c7a8ed" - integrity sha512-KWYVV1c4i+jbMpaBC+U++4Va0cp8OisU185o73T1vo99hqi7w8tSJfUXYswwqqrjzwxa6KpRK54WhPvwf5w6PQ== - dependencies: - "@types/node" "*" - merge-stream "^2.0.0" - supports-color "^7.0.0" - -jest-worker@^27.3.1: - version "27.3.1" - resolved "https://registry.yarnpkg.com/jest-worker/-/jest-worker-27.3.1.tgz#0def7feae5b8042be38479799aeb7b5facac24b2" - integrity sha512-ks3WCzsiZaOPJl/oMsDjaf0TRiSv7ctNgs0FqRr2nARsovz6AWWy4oLElwcquGSz692DzgZQrCLScPNs5YlC4g== - dependencies: - "@types/node" "*" - merge-stream "^2.0.0" - supports-color "^8.0.0" - -jest-worker@^27.4.5, jest-worker@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/jest-worker/-/jest-worker-27.5.1.tgz#8d146f0900e8973b106b6f73cc1e9a8cb86f8db0" - integrity sha512-7vuh85V5cdDofPyxn58nrPjBktZo0u9x1g8WtjQol+jZDaE+fhN+cIvTj11GndBnMnyfrUOG1sZQxCdjKh+DKg== - dependencies: - "@types/node" "*" - merge-stream "^2.0.0" - supports-color "^8.0.0" - -jest@^27.3.1: - version "27.3.1" - resolved "https://registry.yarnpkg.com/jest/-/jest-27.3.1.tgz#b5bab64e8f56b6f7e275ba1836898b0d9f1e5c8a" - integrity sha512-U2AX0AgQGd5EzMsiZpYt8HyZ+nSVIh5ujQ9CPp9EQZJMjXIiSZpJNweZl0swatKRoqHWgGKM3zaSwm4Zaz87ng== - dependencies: - "@jest/core" "^27.3.1" - import-local "^3.0.2" - jest-cli "^27.3.1" - -jquery-ui@1.13.3: - version "1.13.3" - resolved "https://registry.yarnpkg.com/jquery-ui/-/jquery-ui-1.13.3.tgz#d9f5292b2857fa1f2fdbbe8f2e66081664eb9bc5" - integrity sha512-D2YJfswSJRh/B8M/zCowDpNFfwsDmtfnMPwjJTyvl+CBqzpYwQ+gFYIbUUlzijy/Qvoy30H1YhoSui4MNYpRwA== - dependencies: - jquery ">=1.8.0 <4.0.0" - -"jquery@>=1.8.0 <4.0.0": - version "3.7.1" - resolved "https://registry.yarnpkg.com/jquery/-/jquery-3.7.1.tgz#083ef98927c9a6a74d05a6af02806566d16274de" - integrity sha512-m4avr8yL8kmFN8psrbFFFmB/If14iN5o9nw/NgnnM+kybDJpRsAynV2BsfpTYrTRysYUdADVD7CkUUizgkpLfg== - -jquery@>=3.5.0, jquery@^3.5.1: - version "3.6.0" - resolved "https://registry.yarnpkg.com/jquery/-/jquery-3.6.0.tgz#c72a09f15c1bdce142f49dbf1170bdf8adac2470" - integrity sha512-JVzAR/AjBvVt2BmYhxRCSYysDsPcssdmTFnzyLEts9qNwmjmu4JTAMYubEfwVOSwpQ1I1sKKFcxhZCI2buerfw== - -js-levenshtein@^1.1.6: - version "1.1.6" - resolved "https://registry.yarnpkg.com/js-levenshtein/-/js-levenshtein-1.1.6.tgz#c6cee58eb3550372df8deb85fad5ce66ce01d59d" - integrity sha512-X2BB11YZtrRqY4EnQcLX5Rh373zbK4alC1FW7D7MBhL2gtcC17cTnr6DmfHZeS0s2rTHjUTMMHfG7gO8SSdw+g== - -js-sha3@0.8.0: - version "0.8.0" - resolved "https://registry.yarnpkg.com/js-sha3/-/js-sha3-0.8.0.tgz#b9b7a5da73afad7dedd0f8c463954cbde6818840" - integrity sha512-gF1cRrHhIzNfToc802P800N8PpXS+evLLXfsVpowqmAFR9uwbi89WvXg2QspOmXL8QL86J4T1EpFu+yUkwJY3Q== - -"js-tokens@^3.0.0 || ^4.0.0", js-tokens@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/js-tokens/-/js-tokens-4.0.0.tgz#19203fb59991df98e3a287050d4647cdeaf32499" - integrity sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ== - -js-yaml@^3.13.1: - version "3.14.1" - resolved "https://registry.yarnpkg.com/js-yaml/-/js-yaml-3.14.1.tgz#dae812fdb3825fa306609a8717383c50c36a0537" - integrity sha512-okMH7OXXJ7YrN9Ok3/SXrnu4iX9yOk+25nqX4imS2npuvTYDmo/QEZoqwZkYaIDk3jVvBOTOIEgEhaLOynBS9g== - dependencies: - argparse "^1.0.7" - esprima "^4.0.0" - -js-yaml@^4.1.0: - version "4.1.0" - resolved "https://registry.yarnpkg.com/js-yaml/-/js-yaml-4.1.0.tgz#c1fb65f8f5017901cdd2c951864ba18458a10602" - integrity sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA== - dependencies: - argparse "^2.0.1" - -jsdom@^16.6.0: - version "16.7.0" - resolved "https://registry.yarnpkg.com/jsdom/-/jsdom-16.7.0.tgz#918ae71965424b197c819f8183a754e18977b710" - integrity sha512-u9Smc2G1USStM+s/x1ru5Sxrl6mPYCbByG1U/hUmqaVsm4tbNyS7CicOSRyuGQYZhTu0h84qkZZQ/I+dzizSVw== - dependencies: - abab "^2.0.5" - acorn "^8.2.4" - acorn-globals "^6.0.0" - cssom "^0.4.4" - cssstyle "^2.3.0" - data-urls "^2.0.0" - decimal.js "^10.2.1" - domexception "^2.0.1" - escodegen "^2.0.0" - form-data "^3.0.0" - html-encoding-sniffer "^2.0.1" - http-proxy-agent "^4.0.1" - https-proxy-agent "^5.0.0" - is-potential-custom-element-name "^1.0.1" - nwsapi "^2.2.0" - parse5 "6.0.1" - saxes "^5.0.1" - symbol-tree "^3.2.4" - tough-cookie "^4.0.0" - w3c-hr-time "^1.0.2" - w3c-xmlserializer "^2.0.0" - webidl-conversions "^6.1.0" - whatwg-encoding "^1.0.5" - whatwg-mimetype "^2.3.0" - whatwg-url "^8.5.0" - ws "^7.4.6" - xml-name-validator "^3.0.0" - -jsesc@^2.5.1: - version "2.5.2" - resolved "https://registry.yarnpkg.com/jsesc/-/jsesc-2.5.2.tgz#80564d2e483dacf6e8ef209650a67df3f0c283a4" - integrity sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA== - -jsesc@~0.5.0: - version "0.5.0" - resolved "https://registry.yarnpkg.com/jsesc/-/jsesc-0.5.0.tgz#e7dee66e35d6fc16f710fe91d5cf69f70f08911d" - integrity sha1-597mbjXW/Bb3EP6R1c9p9w8IkR0= - -jshint@^2.13.4: - version "2.13.4" - resolved "https://registry.yarnpkg.com/jshint/-/jshint-2.13.4.tgz#cee025a57c3f393d5455532d9ec7ccb018f890db" - integrity sha512-HO3bosL84b2qWqI0q+kpT/OpRJwo0R4ivgmxaO848+bo10rc50SkPnrtwSFXttW0ym4np8jbJvLwk5NziB7jIw== - dependencies: - cli "~1.0.0" - console-browserify "1.1.x" - exit "0.1.x" - htmlparser2 "3.8.x" - lodash "~4.17.21" - minimatch "~3.0.2" - strip-json-comments "1.0.x" - -json-parse-even-better-errors@^2.3.0, json-parse-even-better-errors@^2.3.1: - version "2.3.1" - resolved "https://registry.yarnpkg.com/json-parse-even-better-errors/-/json-parse-even-better-errors-2.3.1.tgz#7c47805a94319928e05777405dc12e1f7a4ee02d" - integrity sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w== - -json-pointer@0.6.2, json-pointer@^0.6.2: - version "0.6.2" - resolved "https://registry.yarnpkg.com/json-pointer/-/json-pointer-0.6.2.tgz#f97bd7550be5e9ea901f8c9264c9d436a22a93cd" - integrity sha512-vLWcKbOaXlO+jvRy4qNd+TI1QUPZzfJj1tpJ3vAXDych5XJf93ftpUKe5pKCrzyIIwgBJcOcCVRUfqQP25afBw== - dependencies: - foreach "^2.0.4" - -json-schema-traverse@^0.4.1: - version "0.4.1" - resolved "https://registry.yarnpkg.com/json-schema-traverse/-/json-schema-traverse-0.4.1.tgz#69f6a87d9513ab8bb8fe63bdb0979c448e684660" - integrity sha512-xbbCH5dCYU5T8LcEhhuh7HJ88HXuW3qsI3Y0zOZFKfZEHcpWiHU/Jxzk629Brsab/mMiHQti9wMP+845RPe3Vg== - -json-schema-traverse@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/json-schema-traverse/-/json-schema-traverse-1.0.0.tgz#ae7bcb3656ab77a73ba5c49bf654f38e6b6860e2" - integrity sha512-NM8/P9n3XjXhIZn1lLhkFaACTOURQXjWhV4BA/RnOv8xvgqtqpAX9IO4mRQxSx1Rlo4tqzeqb0sOlruaOy3dug== - -json-stable-stringify-without-jsonify@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/json-stable-stringify-without-jsonify/-/json-stable-stringify-without-jsonify-1.0.1.tgz#9db7b59496ad3f3cfef30a75142d2d930ad72651" - integrity sha512-Bdboy+l7tA3OGW6FjyFHWkP5LuByj1Tk33Ljyq0axyzdk9//JSi2u3fP1QSmd1KNwq6VOKYGlAu87CisVir6Pw== - -json-to-pretty-yaml@^1.2.2: - version "1.2.2" - resolved "https://registry.yarnpkg.com/json-to-pretty-yaml/-/json-to-pretty-yaml-1.2.2.tgz#f4cd0bd0a5e8fe1df25aaf5ba118b099fd992d5b" - integrity sha512-rvm6hunfCcqegwYaG5T4yKJWxc9FXFgBVrcTZ4XfSVRwa5HA/Xs+vB/Eo9treYYHCeNM0nrSUr82V/M31Urc7A== - dependencies: - remedial "^1.0.7" - remove-trailing-spaces "^1.0.6" - -json5@>=2.2.2, json5@^1.0.2, json5@^2.1.2, json5@^2.2.3: - version "2.2.3" - resolved "https://registry.yarnpkg.com/json5/-/json5-2.2.3.tgz#78cd6f1a19bdc12b73db5ad0c61efd66c1e29283" - integrity sha512-XmOWe7eyHYH14cLdVPoyg+GOH3rYX++KpzrylJwSW98t3Nk+U8XOl8FWKOgwtzdb8lXGf6zYwDUzeHMWfxasyg== - -"jsx-ast-utils@^2.4.1 || ^3.0.0": - version "3.2.1" - resolved "https://registry.yarnpkg.com/jsx-ast-utils/-/jsx-ast-utils-3.2.1.tgz#720b97bfe7d901b927d87c3773637ae8ea48781b" - integrity sha512-uP5vu8xfy2F9A6LGC22KO7e2/vGTS1MhP+18f++ZNlf0Ohaxbc9nIEwHAsejlJKyzfZzU5UIhe5ItYkitcZnZA== - dependencies: - array-includes "^3.1.3" - object.assign "^4.1.2" - -jsx-ast-utils@^3.2.1: - version "3.3.0" - resolved "https://registry.yarnpkg.com/jsx-ast-utils/-/jsx-ast-utils-3.3.0.tgz#e624f259143b9062c92b6413ff92a164c80d3ccb" - integrity sha512-XzO9luP6L0xkxwhIJMTJQpZo/eeN60K08jHdexfD569AGxeNug6UketeHXEhROoM8aR7EcUoOQmIhcJQjcuq8Q== - dependencies: - array-includes "^3.1.4" - object.assign "^4.1.2" - -kind-of@^6.0.2, kind-of@^6.0.3: - version "6.0.3" - resolved "https://registry.yarnpkg.com/kind-of/-/kind-of-6.0.3.tgz#07c05034a6c349fa06e24fa35aa76db4580ce4dd" - integrity sha512-dcS1ul+9tmeD95T+x28/ehLgd9mENa3LsvDTtzm3vyBEO7RPptvAD+t44WVXaUjTBRcrpFeFlC8WCruUR456hw== - -kleur@^3.0.3: - version "3.0.3" - resolved "https://registry.yarnpkg.com/kleur/-/kleur-3.0.3.tgz#a79c9ecc86ee1ce3fa6206d1216c501f147fc07e" - integrity sha512-eTIzlVOSUR+JxdDFepEYcBMtZ9Qqdef+rnzWdRZuMbOywu5tO2w2N7rqjoANZ5k9vywhL6Br1VRjUIgTQx4E8w== - -kleur@^4.0.3: - version "4.1.5" - resolved "https://registry.yarnpkg.com/kleur/-/kleur-4.1.5.tgz#95106101795f7050c6c650f350c683febddb1780" - integrity sha512-o+NO+8WrRiQEE4/7nwRJhN1HWpVmJm511pBHUxPLtp0BUISzlBplORYSmTclCnJvQq2tKu/sgl3xVpkc7ZWuQQ== - -known-css-properties@^0.27.0: - version "0.27.0" - resolved "https://registry.yarnpkg.com/known-css-properties/-/known-css-properties-0.27.0.tgz#82a9358dda5fe7f7bd12b5e7142c0a205393c0c5" - integrity sha512-uMCj6+hZYDoffuvAJjFAPz56E9uoowFHmTkqRtRq5WyC5Q6Cu/fTZKNQpX/RbzChBYLLl3lo8CjFZBAZXq9qFg== - -language-subtag-registry@~0.3.2: - version "0.3.21" - resolved "https://registry.yarnpkg.com/language-subtag-registry/-/language-subtag-registry-0.3.21.tgz#04ac218bea46f04cb039084602c6da9e788dd45a" - integrity sha512-L0IqwlIXjilBVVYKFT37X9Ih11Um5NEl9cbJIuU/SwP/zEEAbBPOnEeeuxVMf45ydWQRDQN3Nqc96OgbH1K+Pg== - -language-tags@^1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/language-tags/-/language-tags-1.0.5.tgz#d321dbc4da30ba8bf3024e040fa5c14661f9193a" - integrity sha1-0yHbxNowuovzAk4ED6XBRmH5GTo= - dependencies: - language-subtag-registry "~0.3.2" - -leven@^3.1.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/leven/-/leven-3.1.0.tgz#77891de834064cccba82ae7842bb6b14a13ed7f2" - integrity sha512-qsda+H8jTaUaN/x5vzW2rzc+8Rw4TAQ/4KjB46IwK5VH+IlVeeeje/EoZRpiXvIqjFgK84QffqPztGI3VBLG1A== - -levn@^0.4.1: - version "0.4.1" - resolved "https://registry.yarnpkg.com/levn/-/levn-0.4.1.tgz#ae4562c007473b932a6200d403268dd2fffc6ade" - integrity sha512-+bT2uH4E5LGE7h/n3evcS/sQlJXCpIp6ym8OWJ5eV6+67Dsql/LaaT7qJBAt2rzfoa/5QBGBhxDix1dMt2kQKQ== - dependencies: - prelude-ls "^1.2.1" - type-check "~0.4.0" - -levn@~0.3.0: - version "0.3.0" - resolved "https://registry.yarnpkg.com/levn/-/levn-0.3.0.tgz#3b09924edf9f083c0490fdd4c0bc4421e04764ee" - integrity sha1-OwmSTt+fCDwEkP3UwLxEIeBHZO4= - dependencies: - prelude-ls "~1.1.2" - type-check "~0.3.2" - -lilconfig@^2.0.3: - version "2.0.5" - resolved "https://registry.yarnpkg.com/lilconfig/-/lilconfig-2.0.5.tgz#19e57fd06ccc3848fd1891655b5a447092225b25" - integrity sha512-xaYmXZtTHPAw5m+xLN8ab9C+3a8YmV3asNSPOATITbtwrfbwaLJj8h66H1WMIpALCkqsIzK3h7oQ+PdX+LQ9Eg== - -lines-and-columns@^1.1.6: - version "1.1.6" - resolved "https://registry.yarnpkg.com/lines-and-columns/-/lines-and-columns-1.1.6.tgz#1c00c743b433cd0a4e80758f7b64a57440d9ff00" - integrity sha1-HADHQ7QzzQpOgHWPe2SldEDZ/wA= - -loader-runner@^4.2.0: - version "4.3.0" - resolved "https://registry.yarnpkg.com/loader-runner/-/loader-runner-4.3.0.tgz#c1b4a163b99f614830353b16755e7149ac2314e1" - integrity sha512-3R/1M+yS3j5ou80Me59j7F9IMs4PXs3VqRrm0TU3AbKPxlmpoY1TNscJV/oGJXo8qCatFGTfDbY6W6ipGOYXfg== - -loader-utils@^2.0.0: - version "2.0.4" - resolved "https://registry.yarnpkg.com/loader-utils/-/loader-utils-2.0.4.tgz#8b5cb38b5c34a9a018ee1fc0e6a066d1dfcc528c" - integrity sha512-xXqpXoINfFhgua9xiqD8fPFHgkoq1mmmpE92WlDbm9rNRd/EbRb+Gqf908T2DMfuHjjJlksiK2RbHVOdD/MqSw== - dependencies: - big.js "^5.2.2" - emojis-list "^3.0.0" - json5 "^2.1.2" - -locate-path@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/locate-path/-/locate-path-5.0.0.tgz#1afba396afd676a6d42504d0a67a3a7eb9f62aa0" - integrity sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g== - dependencies: - p-locate "^4.1.0" - -locate-path@^6.0.0: - version "6.0.0" - resolved "https://registry.yarnpkg.com/locate-path/-/locate-path-6.0.0.tgz#55321eb309febbc59c4801d931a72452a681d286" - integrity sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw== - dependencies: - p-locate "^5.0.0" - -lodash.curry@^4.0.1: - version "4.1.1" - resolved "https://registry.yarnpkg.com/lodash.curry/-/lodash.curry-4.1.1.tgz#248e36072ede906501d75966200a86dab8b23170" - integrity sha512-/u14pXGviLaweY5JI0IUzgzF2J6Ne8INyzAZjImcryjgkZ+ebruBxy2/JaOOkTqScddcYtakjhSaeemV8lR0tA== - -lodash.debounce@^4.0.8: - version "4.0.8" - resolved "https://registry.yarnpkg.com/lodash.debounce/-/lodash.debounce-4.0.8.tgz#82d79bff30a67c4005ffd5e2515300ad9ca4d7af" - integrity sha1-gteb/zCmfEAF/9XiUVMArZyk168= - -lodash.difference@^4.5.0: - version "4.5.0" - resolved "https://registry.yarnpkg.com/lodash.difference/-/lodash.difference-4.5.0.tgz#9ccb4e505d486b91651345772885a2df27fd017c" - integrity sha1-nMtOUF1Ia5FlE0V3KIWi3yf9AXw= - -lodash.flow@^3.3.0: - version "3.5.0" - resolved "https://registry.yarnpkg.com/lodash.flow/-/lodash.flow-3.5.0.tgz#87bf40292b8cf83e4e8ce1a3ae4209e20071675a" - integrity sha512-ff3BX/tSioo+XojX4MOsOMhJw0nZoUEF011LX8g8d3gvjVbxd89cCio4BCXronjxcTUIJUoqKEUA+n4CqvvRPw== - -lodash.isequal@^4.5.0: - version "4.5.0" - resolved "https://registry.yarnpkg.com/lodash.isequal/-/lodash.isequal-4.5.0.tgz#415c4478f2bcc30120c22ce10ed3226f7d3e18e0" - integrity sha1-QVxEePK8wwEgwizhDtMib30+GOA= - -lodash.memoize@^4.1.2: - version "4.1.2" - resolved "https://registry.yarnpkg.com/lodash.memoize/-/lodash.memoize-4.1.2.tgz#bcc6c49a42a2840ed997f323eada5ecd182e0bfe" - integrity sha512-t7j+NzmgnQzTAYXcsHYLgimltOV1MXHtlOWf6GjL9Kj8GK5FInw5JotxvbOs+IvV1/Dzo04/fCGfLVs7aXb4Ag== - -lodash.merge@^4.6.2: - version "4.6.2" - resolved "https://registry.yarnpkg.com/lodash.merge/-/lodash.merge-4.6.2.tgz#558aa53b43b661e1925a0afdfa36a9a1085fe57a" - integrity sha512-0KpjqXRVvrYyCsX1swR/XTK0va6VQkQM6MNo7PqW77ByjAhoARA8EfrP1N4+KlKj8YS0ZUCtRT/YUuhyYDujIQ== - -lodash.mergewith@4.6.2: - version "4.6.2" - resolved "https://registry.yarnpkg.com/lodash.mergewith/-/lodash.mergewith-4.6.2.tgz#617121f89ac55f59047c7aec1ccd6654c6590f55" - integrity sha512-GK3g5RPZWTRSeLSpgP8Xhra+pnjBC56q9FZYe1d5RN3TJ35dbkGy3YqBSMbyCrlbi+CM9Z3Jk5yTL7RCsqboyQ== - -lodash.truncate@^4.4.2: - version "4.4.2" - resolved "https://registry.yarnpkg.com/lodash.truncate/-/lodash.truncate-4.4.2.tgz#5a350da0b1113b837ecfffd5812cbe58d6eae193" - integrity sha1-WjUNoLERO4N+z//VgSy+WNbq4ZM= - -lodash.uniq@^4.5.0: - version "4.5.0" - resolved "https://registry.yarnpkg.com/lodash.uniq/-/lodash.uniq-4.5.0.tgz#d0225373aeb652adc1bc82e4945339a842754773" - integrity sha512-xfBaXQd9ryd9dlSDvnvI0lvxfLJlYAZzXomUYzLKtUeOQvOP5piqAWuGtrhWeqaXK9hhoM/iyJc5AV+XfsX3HQ== - -lodash@^4.17.15, lodash@^4.17.20, lodash@^4.17.21, lodash@^4.7.0, lodash@~4.17.21: - version "4.17.21" - resolved "https://registry.yarnpkg.com/lodash/-/lodash-4.17.21.tgz#679591c564c3bffaae8454cf0b3df370c3d6911c" - integrity sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg== - -longest-streak@^3.0.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/longest-streak/-/longest-streak-3.1.0.tgz#62fa67cd958742a1574af9f39866364102d90cd4" - integrity sha512-9Ri+o0JYgehTaVBBDoMqIl8GXtbWg711O3srftcHhZ0dqnETqLaoIK0x17fUw9rFSlK/0NlsKe0Ahhyl5pXE2g== - -loose-envify@^1.0.0, loose-envify@^1.1.0, loose-envify@^1.4.0: - version "1.4.0" - resolved "https://registry.yarnpkg.com/loose-envify/-/loose-envify-1.4.0.tgz#71ee51fa7be4caec1a63839f7e682d8132d30caf" - integrity sha512-lyuxPGr/Wfhrlem2CL/UcnUc1zcqKAImBDzukY7Y5F/yQiNdko6+fRLevlw1HgMySw7f611UIY408EtxRSoK3Q== - dependencies: - js-tokens "^3.0.0 || ^4.0.0" - -lowlight@^1.17.0: - version "1.20.0" - resolved "https://registry.yarnpkg.com/lowlight/-/lowlight-1.20.0.tgz#ddb197d33462ad0d93bf19d17b6c301aa3941888" - integrity sha512-8Ktj+prEb1RoCPkEOrPMYUN/nCggB7qAWe3a7OpMjWQkh3l2RD5wKRQ+o8Q8YuI9RG/xs95waaI/E6ym/7NsTw== - dependencies: - fault "^1.0.0" - highlight.js "~10.7.0" - -lru-cache@^5.1.1: - version "5.1.1" - resolved "https://registry.yarnpkg.com/lru-cache/-/lru-cache-5.1.1.tgz#1da27e6710271947695daf6848e847f01d84b920" - integrity sha512-KpNARQA3Iwv+jTA0utUVVbrh+Jlrr1Fv0e56GGzAFOXN7dk/FviaDW8LHmK52DlcH4WP2n6gI8vN1aesBFgo9w== - dependencies: - yallist "^3.0.2" - -lru-cache@^6.0.0: - version "6.0.0" - resolved "https://registry.yarnpkg.com/lru-cache/-/lru-cache-6.0.0.tgz#6d6fe6570ebd96aaf90fcad1dafa3b2566db3a94" - integrity sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA== - dependencies: - yallist "^4.0.0" - -lunr@^2.3.9: - version "2.3.9" - resolved "https://registry.yarnpkg.com/lunr/-/lunr-2.3.9.tgz#18b123142832337dd6e964df1a5a7707b25d35e1" - integrity sha512-zTU3DaZaF3Rt9rhN3uBMGQD3dD2/vFQqnvZCDv4dl5iOzq2IZQqTxu90r4E5J+nP70J3ilqVCrbho2eWaeW8Ow== - -lz-string@^1.4.4: - version "1.4.4" - resolved "https://registry.yarnpkg.com/lz-string/-/lz-string-1.4.4.tgz#c0d8eaf36059f705796e1e344811cf4c498d3a26" - integrity sha1-wNjq82BZ9wV5bh40SBHPTEmNOiY= - -make-dir@^3.0.0, make-dir@^3.0.2: - version "3.1.0" - resolved "https://registry.yarnpkg.com/make-dir/-/make-dir-3.1.0.tgz#415e967046b3a7f1d185277d84aa58203726a13f" - integrity sha512-g3FeP20LNwhALb/6Cz6Dd4F2ngze0jz7tbzrD2wAV+o9FeNHe4rL+yK2md0J/fiSf1sa1ADhXqi5+oVwOM/eGw== - dependencies: - semver "^6.0.0" - -makeerror@1.0.12: - version "1.0.12" - resolved "https://registry.yarnpkg.com/makeerror/-/makeerror-1.0.12.tgz#3e5dd2079a82e812e983cc6610c4a2cb0eaa801a" - integrity sha512-JmqCvUhmt43madlpFzG4BQzG2Z3m6tvQDNKdClZnO3VbIudJYmxsT0FNJMeiB2+JTSlTQTSbU8QdesVmwJcmLg== - dependencies: - tmpl "1.0.5" - -map-obj@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/map-obj/-/map-obj-1.0.1.tgz#d933ceb9205d82bdcf4886f6742bdc2b4dea146d" - integrity sha1-2TPOuSBdgr3PSIb2dCvcK03qFG0= - -map-obj@^4.1.0: - version "4.3.0" - resolved "https://registry.yarnpkg.com/map-obj/-/map-obj-4.3.0.tgz#9304f906e93faae70880da102a9f1df0ea8bb05a" - integrity sha512-hdN1wVrZbb29eBGiGjJbeP8JbKjq1urkHJ/LIP/NY48MZ1QVXUsQBV1G1zvYFHn1XE06cwjBsOI2K3Ulnj1YXQ== - -mark.js@^8.11.1: - version "8.11.1" - resolved "https://registry.yarnpkg.com/mark.js/-/mark.js-8.11.1.tgz#180f1f9ebef8b0e638e4166ad52db879beb2ffc5" - integrity sha1-GA8fnr74sOY45BZq1S24eb6y/8U= - -markdown-table@^3.0.0: - version "3.0.3" - resolved "https://registry.yarnpkg.com/markdown-table/-/markdown-table-3.0.3.tgz#e6331d30e493127e031dd385488b5bd326e4a6bd" - integrity sha512-Z1NL3Tb1M9wH4XESsCDEksWoKTdlUafKc4pt0GRwjUyXaCFZ+dc3g2erqB6zm3szA2IUSi7VnPI+o/9jnxh9hw== - -marked@^4.0.15: - version "4.0.17" - resolved "https://registry.yarnpkg.com/marked/-/marked-4.0.17.tgz#1186193d85bb7882159cdcfc57d1dfccaffb3fe9" - integrity sha512-Wfk0ATOK5iPxM4ptrORkFemqroz0ZDxp5MWfYA7H/F+wO17NRWV5Ypxi6p3g2Xmw2bKeiYOl6oVnLHKxBA0VhA== - -match-sorter@^6.0.2: - version "6.3.1" - resolved "https://registry.yarnpkg.com/match-sorter/-/match-sorter-6.3.1.tgz#98cc37fda756093424ddf3cbc62bfe9c75b92bda" - integrity sha512-mxybbo3pPNuA+ZuCUhm5bwNkXrJTbsk5VWbR5wiwz/GC6LIiegBGn2w3O08UG/jdbYLinw51fSQ5xNU1U3MgBw== - dependencies: - "@babel/runtime" "^7.12.5" - remove-accents "0.4.2" - -mathml-tag-names@^2.1.3: - version "2.1.3" - resolved "https://registry.yarnpkg.com/mathml-tag-names/-/mathml-tag-names-2.1.3.tgz#4ddadd67308e780cf16a47685878ee27b736a0a3" - integrity sha512-APMBEanjybaPzUrfqU0IMU5I0AswKMH7k8OTLs0vvV4KZpExkTkY87nR/zpbuTPj+gARop7aGUbl11pnDfW6xg== - -mdast-util-definitions@^5.0.0: - version "5.1.1" - resolved "https://registry.yarnpkg.com/mdast-util-definitions/-/mdast-util-definitions-5.1.1.tgz#2c1d684b28e53f84938bb06317944bee8efa79db" - integrity sha512-rQ+Gv7mHttxHOBx2dkF4HWTg+EE+UR78ptQWDylzPKaQuVGdG4HIoY3SrS/pCp80nZ04greFvXbVFHT+uf0JVQ== - dependencies: - "@types/mdast" "^3.0.0" - "@types/unist" "^2.0.0" - unist-util-visit "^4.0.0" - -mdast-util-find-and-replace@^2.0.0: - version "2.2.1" - resolved "https://registry.yarnpkg.com/mdast-util-find-and-replace/-/mdast-util-find-and-replace-2.2.1.tgz#249901ef43c5f41d6e8a8d446b3b63b17e592d7c" - integrity sha512-SobxkQXFAdd4b5WmEakmkVoh18icjQRxGy5OWTCzgsLRm1Fu/KCtwD1HIQSsmq5ZRjVH0Ehwg6/Fn3xIUk+nKw== - dependencies: - escape-string-regexp "^5.0.0" - unist-util-is "^5.0.0" - unist-util-visit-parents "^5.0.0" - -mdast-util-from-markdown@^1.0.0: - version "1.2.0" - resolved "https://registry.yarnpkg.com/mdast-util-from-markdown/-/mdast-util-from-markdown-1.2.0.tgz#84df2924ccc6c995dec1e2368b2b208ad0a76268" - integrity sha512-iZJyyvKD1+K7QX1b5jXdE7Sc5dtoTry1vzV28UZZe8Z1xVnB/czKntJ7ZAkG0tANqRnBF6p3p7GpU1y19DTf2Q== - dependencies: - "@types/mdast" "^3.0.0" - "@types/unist" "^2.0.0" - decode-named-character-reference "^1.0.0" - mdast-util-to-string "^3.1.0" - micromark "^3.0.0" - micromark-util-decode-numeric-character-reference "^1.0.0" - micromark-util-decode-string "^1.0.0" - micromark-util-normalize-identifier "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - unist-util-stringify-position "^3.0.0" - uvu "^0.5.0" - -mdast-util-gfm-autolink-literal@^1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/mdast-util-gfm-autolink-literal/-/mdast-util-gfm-autolink-literal-1.0.2.tgz#4032dcbaddaef7d4f2f3768ed830475bb22d3970" - integrity sha512-FzopkOd4xTTBeGXhXSBU0OCDDh5lUj2rd+HQqG92Ld+jL4lpUfgX2AT2OHAVP9aEeDKp7G92fuooSZcYJA3cRg== - dependencies: - "@types/mdast" "^3.0.0" - ccount "^2.0.0" - mdast-util-find-and-replace "^2.0.0" - micromark-util-character "^1.0.0" - -mdast-util-gfm-footnote@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/mdast-util-gfm-footnote/-/mdast-util-gfm-footnote-1.0.1.tgz#11d2d40a1a673a399c459e467fa85e00223191fe" - integrity sha512-p+PrYlkw9DeCRkTVw1duWqPRHX6Ywh2BNKJQcZbCwAuP/59B0Lk9kakuAd7KbQprVO4GzdW8eS5++A9PUSqIyw== - dependencies: - "@types/mdast" "^3.0.0" - mdast-util-to-markdown "^1.3.0" - micromark-util-normalize-identifier "^1.0.0" - -mdast-util-gfm-strikethrough@^1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/mdast-util-gfm-strikethrough/-/mdast-util-gfm-strikethrough-1.0.2.tgz#6b4fa4ae37d449ccb988192ac0afbb2710ffcefd" - integrity sha512-T/4DVHXcujH6jx1yqpcAYYwd+z5lAYMw4Ls6yhTfbMMtCt0PHY4gEfhW9+lKsLBtyhUGKRIzcUA2FATVqnvPDA== - dependencies: - "@types/mdast" "^3.0.0" - mdast-util-to-markdown "^1.3.0" - -mdast-util-gfm-table@^1.0.0: - version "1.0.6" - resolved "https://registry.yarnpkg.com/mdast-util-gfm-table/-/mdast-util-gfm-table-1.0.6.tgz#184e900979fe790745fc3dabf77a4114595fcd7f" - integrity sha512-uHR+fqFq3IvB3Rd4+kzXW8dmpxUhvgCQZep6KdjsLK4O6meK5dYZEayLtIxNus1XO3gfjfcIFe8a7L0HZRGgag== - dependencies: - "@types/mdast" "^3.0.0" - markdown-table "^3.0.0" - mdast-util-from-markdown "^1.0.0" - mdast-util-to-markdown "^1.3.0" - -mdast-util-gfm-task-list-item@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/mdast-util-gfm-task-list-item/-/mdast-util-gfm-task-list-item-1.0.1.tgz#6f35f09c6e2bcbe88af62fdea02ac199cc802c5c" - integrity sha512-KZ4KLmPdABXOsfnM6JHUIjxEvcx2ulk656Z/4Balw071/5qgnhz+H1uGtf2zIGnrnvDC8xR4Fj9uKbjAFGNIeA== - dependencies: - "@types/mdast" "^3.0.0" - mdast-util-to-markdown "^1.3.0" - -mdast-util-gfm@^2.0.0: - version "2.0.1" - resolved "https://registry.yarnpkg.com/mdast-util-gfm/-/mdast-util-gfm-2.0.1.tgz#16fcf70110ae689a06d77e8f4e346223b64a0ea6" - integrity sha512-42yHBbfWIFisaAfV1eixlabbsa6q7vHeSPY+cg+BBjX51M8xhgMacqH9g6TftB/9+YkcI0ooV4ncfrJslzm/RQ== - dependencies: - mdast-util-from-markdown "^1.0.0" - mdast-util-gfm-autolink-literal "^1.0.0" - mdast-util-gfm-footnote "^1.0.0" - mdast-util-gfm-strikethrough "^1.0.0" - mdast-util-gfm-table "^1.0.0" - mdast-util-gfm-task-list-item "^1.0.0" - mdast-util-to-markdown "^1.0.0" - -mdast-util-to-hast@^12.1.0: - version "12.2.4" - resolved "https://registry.yarnpkg.com/mdast-util-to-hast/-/mdast-util-to-hast-12.2.4.tgz#34c1ef2b6cf01c27b3e3504e2c977c76f722e7e1" - integrity sha512-a21xoxSef1l8VhHxS1Dnyioz6grrJkoaCUgGzMD/7dWHvboYX3VW53esRUfB5tgTyz4Yos1n25SPcj35dJqmAg== - dependencies: - "@types/hast" "^2.0.0" - "@types/mdast" "^3.0.0" - mdast-util-definitions "^5.0.0" - micromark-util-sanitize-uri "^1.1.0" - trim-lines "^3.0.0" - unist-builder "^3.0.0" - unist-util-generated "^2.0.0" - unist-util-position "^4.0.0" - unist-util-visit "^4.0.0" - -mdast-util-to-markdown@^1.0.0, mdast-util-to-markdown@^1.3.0: - version "1.3.0" - resolved "https://registry.yarnpkg.com/mdast-util-to-markdown/-/mdast-util-to-markdown-1.3.0.tgz#38b6cdc8dc417de642a469c4fc2abdf8c931bd1e" - integrity sha512-6tUSs4r+KK4JGTTiQ7FfHmVOaDrLQJPmpjD6wPMlHGUVXoG9Vjc3jIeP+uyBWRf8clwB2blM+W7+KrlMYQnftA== - dependencies: - "@types/mdast" "^3.0.0" - "@types/unist" "^2.0.0" - longest-streak "^3.0.0" - mdast-util-to-string "^3.0.0" - micromark-util-decode-string "^1.0.0" - unist-util-visit "^4.0.0" - zwitch "^2.0.0" - -mdast-util-to-string@^3.0.0, mdast-util-to-string@^3.1.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/mdast-util-to-string/-/mdast-util-to-string-3.1.0.tgz#56c506d065fbf769515235e577b5a261552d56e9" - integrity sha512-n4Vypz/DZgwo0iMHLQL49dJzlp7YtAJP+N07MZHpjPf/5XJuHUWstviF4Mn2jEiR/GNmtnRRqnwsXExk3igfFA== - -mdn-data@2.0.14: - version "2.0.14" - resolved "https://registry.yarnpkg.com/mdn-data/-/mdn-data-2.0.14.tgz#7113fc4281917d63ce29b43446f701e68c25ba50" - integrity sha512-dn6wd0uw5GsdswPFfsgMp5NSB0/aDe6fK94YJV/AJDYXL6HVLWBsxeq7js7Ad+mU2K9LAlwpk6kN2D5mwCPVow== - -mdn-data@2.0.30: - version "2.0.30" - resolved "https://registry.yarnpkg.com/mdn-data/-/mdn-data-2.0.30.tgz#ce4df6f80af6cfbe218ecd5c552ba13c4dfa08cc" - integrity sha512-GaqWWShW4kv/G9IEucWScBx9G1/vsFZZJUO+tD26M8J8z3Kw5RDQjaoZe03YAClgeS/SWPOcb4nkFBTEi5DUEA== - -memoize-one@^5.0.0: - version "5.2.1" - resolved "https://registry.yarnpkg.com/memoize-one/-/memoize-one-5.2.1.tgz#8337aa3c4335581839ec01c3d594090cebe8f00e" - integrity sha512-zYiwtZUcYyXKo/np96AGZAckk+FWWsUdJ3cHGGmld7+AhvcWmQyGCYUh1hc4Q/pkOhb65dQR/pqCyK0cOaHz4Q== - -meow@^10.1.5: - version "10.1.5" - resolved "https://registry.yarnpkg.com/meow/-/meow-10.1.5.tgz#be52a1d87b5f5698602b0f32875ee5940904aa7f" - integrity sha512-/d+PQ4GKmGvM9Bee/DPa8z3mXs/pkvJE2KEThngVNOqtmljC6K7NMPxtc2JeZYTmpWb9k/TmxjeL18ez3h7vCw== - dependencies: - "@types/minimist" "^1.2.2" - camelcase-keys "^7.0.0" - decamelize "^5.0.0" - decamelize-keys "^1.1.0" - hard-rejection "^2.1.0" - minimist-options "4.1.0" - normalize-package-data "^3.0.2" - read-pkg-up "^8.0.0" - redent "^4.0.0" - trim-newlines "^4.0.2" - type-fest "^1.2.2" - yargs-parser "^20.2.9" - -merge-stream@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/merge-stream/-/merge-stream-2.0.0.tgz#52823629a14dd00c9770fb6ad47dc6310f2c1f60" - integrity sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w== - -merge2@^1.3.0, merge2@^1.4.1: - version "1.4.1" - resolved "https://registry.yarnpkg.com/merge2/-/merge2-1.4.1.tgz#4368892f885e907455a6fd7dc55c0c9d404990ae" - integrity sha512-8q7VEgMJW4J8tcfVPy8g09NcQwZdbwFEqhe/WZkoIzjn/3TGDwtOCYtXGxA3O8tPzpczCCDgv+P2P5y00ZJOOg== - -micromark-core-commonmark@^1.0.0, micromark-core-commonmark@^1.0.1: - version "1.0.6" - resolved "https://registry.yarnpkg.com/micromark-core-commonmark/-/micromark-core-commonmark-1.0.6.tgz#edff4c72e5993d93724a3c206970f5a15b0585ad" - integrity sha512-K+PkJTxqjFfSNkfAhp4GB+cZPfQd6dxtTXnf+RjZOV7T4EEXnvgzOcnp+eSTmpGk9d1S9sL6/lqrgSNn/s0HZA== - dependencies: - decode-named-character-reference "^1.0.0" - micromark-factory-destination "^1.0.0" - micromark-factory-label "^1.0.0" - micromark-factory-space "^1.0.0" - micromark-factory-title "^1.0.0" - micromark-factory-whitespace "^1.0.0" - micromark-util-character "^1.0.0" - micromark-util-chunked "^1.0.0" - micromark-util-classify-character "^1.0.0" - micromark-util-html-tag-name "^1.0.0" - micromark-util-normalize-identifier "^1.0.0" - micromark-util-resolve-all "^1.0.0" - micromark-util-subtokenize "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.1" - uvu "^0.5.0" - -micromark-extension-gfm-autolink-literal@^1.0.0: - version "1.0.3" - resolved "https://registry.yarnpkg.com/micromark-extension-gfm-autolink-literal/-/micromark-extension-gfm-autolink-literal-1.0.3.tgz#dc589f9c37eaff31a175bab49f12290edcf96058" - integrity sha512-i3dmvU0htawfWED8aHMMAzAVp/F0Z+0bPh3YrbTPPL1v4YAlCZpy5rBO5p0LPYiZo0zFVkoYh7vDU7yQSiCMjg== - dependencies: - micromark-util-character "^1.0.0" - micromark-util-sanitize-uri "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - uvu "^0.5.0" - -micromark-extension-gfm-footnote@^1.0.0: - version "1.0.4" - resolved "https://registry.yarnpkg.com/micromark-extension-gfm-footnote/-/micromark-extension-gfm-footnote-1.0.4.tgz#cbfd8873b983e820c494498c6dac0105920818d5" - integrity sha512-E/fmPmDqLiMUP8mLJ8NbJWJ4bTw6tS+FEQS8CcuDtZpILuOb2kjLqPEeAePF1djXROHXChM/wPJw0iS4kHCcIg== - dependencies: - micromark-core-commonmark "^1.0.0" - micromark-factory-space "^1.0.0" - micromark-util-character "^1.0.0" - micromark-util-normalize-identifier "^1.0.0" - micromark-util-sanitize-uri "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - uvu "^0.5.0" - -micromark-extension-gfm-strikethrough@^1.0.0: - version "1.0.4" - resolved "https://registry.yarnpkg.com/micromark-extension-gfm-strikethrough/-/micromark-extension-gfm-strikethrough-1.0.4.tgz#162232c284ffbedd8c74e59c1525bda217295e18" - integrity sha512-/vjHU/lalmjZCT5xt7CcHVJGq8sYRm80z24qAKXzaHzem/xsDYb2yLL+NNVbYvmpLx3O7SYPuGL5pzusL9CLIQ== - dependencies: - micromark-util-chunked "^1.0.0" - micromark-util-classify-character "^1.0.0" - micromark-util-resolve-all "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - uvu "^0.5.0" - -micromark-extension-gfm-table@^1.0.0: - version "1.0.5" - resolved "https://registry.yarnpkg.com/micromark-extension-gfm-table/-/micromark-extension-gfm-table-1.0.5.tgz#7b708b728f8dc4d95d486b9e7a2262f9cddbcbb4" - integrity sha512-xAZ8J1X9W9K3JTJTUL7G6wSKhp2ZYHrFk5qJgY/4B33scJzE2kpfRL6oiw/veJTbt7jiM/1rngLlOKPWr1G+vg== - dependencies: - micromark-factory-space "^1.0.0" - micromark-util-character "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - uvu "^0.5.0" - -micromark-extension-gfm-tagfilter@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/micromark-extension-gfm-tagfilter/-/micromark-extension-gfm-tagfilter-1.0.1.tgz#fb2e303f7daf616db428bb6a26e18fda14a90a4d" - integrity sha512-Ty6psLAcAjboRa/UKUbbUcwjVAv5plxmpUTy2XC/3nJFL37eHej8jrHrRzkqcpipJliuBH30DTs7+3wqNcQUVA== - dependencies: - micromark-util-types "^1.0.0" - -micromark-extension-gfm-task-list-item@^1.0.0: - version "1.0.3" - resolved "https://registry.yarnpkg.com/micromark-extension-gfm-task-list-item/-/micromark-extension-gfm-task-list-item-1.0.3.tgz#7683641df5d4a09795f353574d7f7f66e47b7fc4" - integrity sha512-PpysK2S1Q/5VXi72IIapbi/jliaiOFzv7THH4amwXeYXLq3l1uo8/2Be0Ac1rEwK20MQEsGH2ltAZLNY2KI/0Q== - dependencies: - micromark-factory-space "^1.0.0" - micromark-util-character "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - uvu "^0.5.0" - -micromark-extension-gfm@^2.0.0: - version "2.0.1" - resolved "https://registry.yarnpkg.com/micromark-extension-gfm/-/micromark-extension-gfm-2.0.1.tgz#40f3209216127a96297c54c67f5edc7ef2d1a2a2" - integrity sha512-p2sGjajLa0iYiGQdT0oelahRYtMWvLjy8J9LOCxzIQsllMCGLbsLW+Nc+N4vi02jcRJvedVJ68cjelKIO6bpDA== - dependencies: - micromark-extension-gfm-autolink-literal "^1.0.0" - micromark-extension-gfm-footnote "^1.0.0" - micromark-extension-gfm-strikethrough "^1.0.0" - micromark-extension-gfm-table "^1.0.0" - micromark-extension-gfm-tagfilter "^1.0.0" - micromark-extension-gfm-task-list-item "^1.0.0" - micromark-util-combine-extensions "^1.0.0" - micromark-util-types "^1.0.0" - -micromark-factory-destination@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/micromark-factory-destination/-/micromark-factory-destination-1.0.0.tgz#fef1cb59ad4997c496f887b6977aa3034a5a277e" - integrity sha512-eUBA7Rs1/xtTVun9TmV3gjfPz2wEwgK5R5xcbIM5ZYAtvGF6JkyaDsj0agx8urXnO31tEO6Ug83iVH3tdedLnw== - dependencies: - micromark-util-character "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - -micromark-factory-label@^1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/micromark-factory-label/-/micromark-factory-label-1.0.2.tgz#6be2551fa8d13542fcbbac478258fb7a20047137" - integrity sha512-CTIwxlOnU7dEshXDQ+dsr2n+yxpP0+fn271pu0bwDIS8uqfFcumXpj5mLn3hSC8iw2MUr6Gx8EcKng1dD7i6hg== - dependencies: - micromark-util-character "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - uvu "^0.5.0" - -micromark-factory-space@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/micromark-factory-space/-/micromark-factory-space-1.0.0.tgz#cebff49968f2b9616c0fcb239e96685cb9497633" - integrity sha512-qUmqs4kj9a5yBnk3JMLyjtWYN6Mzfcx8uJfi5XAveBniDevmZasdGBba5b4QsvRcAkmvGo5ACmSUmyGiKTLZew== - dependencies: - micromark-util-character "^1.0.0" - micromark-util-types "^1.0.0" - -micromark-factory-title@^1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/micromark-factory-title/-/micromark-factory-title-1.0.2.tgz#7e09287c3748ff1693930f176e1c4a328382494f" - integrity sha512-zily+Nr4yFqgMGRKLpTVsNl5L4PMu485fGFDOQJQBl2NFpjGte1e86zC0da93wf97jrc4+2G2GQudFMHn3IX+A== - dependencies: - micromark-factory-space "^1.0.0" - micromark-util-character "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - uvu "^0.5.0" - -micromark-factory-whitespace@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/micromark-factory-whitespace/-/micromark-factory-whitespace-1.0.0.tgz#e991e043ad376c1ba52f4e49858ce0794678621c" - integrity sha512-Qx7uEyahU1lt1RnsECBiuEbfr9INjQTGa6Err+gF3g0Tx4YEviPbqqGKNv/NrBaE7dVHdn1bVZKM/n5I/Bak7A== - dependencies: - micromark-factory-space "^1.0.0" - micromark-util-character "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - -micromark-util-character@^1.0.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/micromark-util-character/-/micromark-util-character-1.1.0.tgz#d97c54d5742a0d9611a68ca0cd4124331f264d86" - integrity sha512-agJ5B3unGNJ9rJvADMJ5ZiYjBRyDpzKAOk01Kpi1TKhlT1APx3XZk6eN7RtSz1erbWHC2L8T3xLZ81wdtGRZzg== - dependencies: - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - -micromark-util-chunked@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/micromark-util-chunked/-/micromark-util-chunked-1.0.0.tgz#5b40d83f3d53b84c4c6bce30ed4257e9a4c79d06" - integrity sha512-5e8xTis5tEZKgesfbQMKRCyzvffRRUX+lK/y+DvsMFdabAicPkkZV6gO+FEWi9RfuKKoxxPwNL+dFF0SMImc1g== - dependencies: - micromark-util-symbol "^1.0.0" - -micromark-util-classify-character@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/micromark-util-classify-character/-/micromark-util-classify-character-1.0.0.tgz#cbd7b447cb79ee6997dd274a46fc4eb806460a20" - integrity sha512-F8oW2KKrQRb3vS5ud5HIqBVkCqQi224Nm55o5wYLzY/9PwHGXC01tr3d7+TqHHz6zrKQ72Okwtvm/xQm6OVNZA== - dependencies: - micromark-util-character "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - -micromark-util-combine-extensions@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/micromark-util-combine-extensions/-/micromark-util-combine-extensions-1.0.0.tgz#91418e1e74fb893e3628b8d496085639124ff3d5" - integrity sha512-J8H058vFBdo/6+AsjHp2NF7AJ02SZtWaVUjsayNFeAiydTxUwViQPxN0Hf8dp4FmCQi0UUFovFsEyRSUmFH3MA== - dependencies: - micromark-util-chunked "^1.0.0" - micromark-util-types "^1.0.0" - -micromark-util-decode-numeric-character-reference@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/micromark-util-decode-numeric-character-reference/-/micromark-util-decode-numeric-character-reference-1.0.0.tgz#dcc85f13b5bd93ff8d2868c3dba28039d490b946" - integrity sha512-OzO9AI5VUtrTD7KSdagf4MWgHMtET17Ua1fIpXTpuhclCqD8egFWo85GxSGvxgkGS74bEahvtM0WP0HjvV0e4w== - dependencies: - micromark-util-symbol "^1.0.0" - -micromark-util-decode-string@^1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/micromark-util-decode-string/-/micromark-util-decode-string-1.0.2.tgz#942252ab7a76dec2dbf089cc32505ee2bc3acf02" - integrity sha512-DLT5Ho02qr6QWVNYbRZ3RYOSSWWFuH3tJexd3dgN1odEuPNxCngTCXJum7+ViRAd9BbdxCvMToPOD/IvVhzG6Q== - dependencies: - decode-named-character-reference "^1.0.0" - micromark-util-character "^1.0.0" - micromark-util-decode-numeric-character-reference "^1.0.0" - micromark-util-symbol "^1.0.0" - -micromark-util-encode@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/micromark-util-encode/-/micromark-util-encode-1.0.1.tgz#2c1c22d3800870ad770ece5686ebca5920353383" - integrity sha512-U2s5YdnAYexjKDel31SVMPbfi+eF8y1U4pfiRW/Y8EFVCy/vgxk/2wWTxzcqE71LHtCuCzlBDRU2a5CQ5j+mQA== - -micromark-util-html-tag-name@^1.0.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/micromark-util-html-tag-name/-/micromark-util-html-tag-name-1.1.0.tgz#eb227118befd51f48858e879b7a419fc0df20497" - integrity sha512-BKlClMmYROy9UiV03SwNmckkjn8QHVaWkqoAqzivabvdGcwNGMMMH/5szAnywmsTBUzDsU57/mFi0sp4BQO6dA== - -micromark-util-normalize-identifier@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/micromark-util-normalize-identifier/-/micromark-util-normalize-identifier-1.0.0.tgz#4a3539cb8db954bbec5203952bfe8cedadae7828" - integrity sha512-yg+zrL14bBTFrQ7n35CmByWUTFsgst5JhA4gJYoty4Dqzj4Z4Fr/DHekSS5aLfH9bdlfnSvKAWsAgJhIbogyBg== - dependencies: - micromark-util-symbol "^1.0.0" - -micromark-util-resolve-all@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/micromark-util-resolve-all/-/micromark-util-resolve-all-1.0.0.tgz#a7c363f49a0162e931960c44f3127ab58f031d88" - integrity sha512-CB/AGk98u50k42kvgaMM94wzBqozSzDDaonKU7P7jwQIuH2RU0TeBqGYJz2WY1UdihhjweivStrJ2JdkdEmcfw== - dependencies: - micromark-util-types "^1.0.0" - -micromark-util-sanitize-uri@^1.0.0, micromark-util-sanitize-uri@^1.1.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/micromark-util-sanitize-uri/-/micromark-util-sanitize-uri-1.1.0.tgz#f12e07a85106b902645e0364feb07cf253a85aee" - integrity sha512-RoxtuSCX6sUNtxhbmsEFQfWzs8VN7cTctmBPvYivo98xb/kDEoTCtJQX5wyzIYEmk/lvNFTat4hL8oW0KndFpg== - dependencies: - micromark-util-character "^1.0.0" - micromark-util-encode "^1.0.0" - micromark-util-symbol "^1.0.0" - -micromark-util-subtokenize@^1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/micromark-util-subtokenize/-/micromark-util-subtokenize-1.0.2.tgz#ff6f1af6ac836f8bfdbf9b02f40431760ad89105" - integrity sha512-d90uqCnXp/cy4G881Ub4psE57Sf8YD0pim9QdjCRNjfas2M1u6Lbt+XZK9gnHL2XFhnozZiEdCa9CNfXSfQ6xA== - dependencies: - micromark-util-chunked "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.0" - uvu "^0.5.0" - -micromark-util-symbol@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/micromark-util-symbol/-/micromark-util-symbol-1.0.1.tgz#b90344db62042ce454f351cf0bebcc0a6da4920e" - integrity sha512-oKDEMK2u5qqAptasDAwWDXq0tG9AssVwAx3E9bBF3t/shRIGsWIRG+cGafs2p/SnDSOecnt6hZPCE2o6lHfFmQ== - -micromark-util-types@^1.0.0, micromark-util-types@^1.0.1: - version "1.0.2" - resolved "https://registry.yarnpkg.com/micromark-util-types/-/micromark-util-types-1.0.2.tgz#f4220fdb319205812f99c40f8c87a9be83eded20" - integrity sha512-DCfg/T8fcrhrRKTPjRrw/5LLvdGV7BHySf/1LOZx7TzWZdYRjogNtyNq885z3nNallwr3QUKARjqvHqX1/7t+w== - -micromark@^3.0.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/micromark/-/micromark-3.1.0.tgz#eeba0fe0ac1c9aaef675157b52c166f125e89f62" - integrity sha512-6Mj0yHLdUZjHnOPgr5xfWIMqMWS12zDN6iws9SLuSz76W8jTtAv24MN4/CL7gJrl5vtxGInkkqDv/JIoRsQOvA== - dependencies: - "@types/debug" "^4.0.0" - debug "^4.0.0" - decode-named-character-reference "^1.0.0" - micromark-core-commonmark "^1.0.1" - micromark-factory-space "^1.0.0" - micromark-util-character "^1.0.0" - micromark-util-chunked "^1.0.0" - micromark-util-combine-extensions "^1.0.0" - micromark-util-decode-numeric-character-reference "^1.0.0" - micromark-util-encode "^1.0.0" - micromark-util-normalize-identifier "^1.0.0" - micromark-util-resolve-all "^1.0.0" - micromark-util-sanitize-uri "^1.0.0" - micromark-util-subtokenize "^1.0.0" - micromark-util-symbol "^1.0.0" - micromark-util-types "^1.0.1" - uvu "^0.5.0" - -micromatch@^4.0.4, micromatch@^4.0.5: - version "4.0.8" - resolved "https://registry.yarnpkg.com/micromatch/-/micromatch-4.0.8.tgz#d66fa18f3a47076789320b9b1af32bd86d9fa202" - integrity sha512-PXwfBhYu0hBCPw8Dn0E+WDYb7af3dSLVWKi3HGv84IdF4TyFoC0ysxFd0Goxw7nSv4T/PzEJQxsYsEiFCKo2BA== - dependencies: - braces "^3.0.3" - picomatch "^2.3.1" - -microseconds@0.2.0: - version "0.2.0" - resolved "https://registry.yarnpkg.com/microseconds/-/microseconds-0.2.0.tgz#233b25f50c62a65d861f978a4a4f8ec18797dc39" - integrity sha512-n7DHHMjR1avBbSpsTBj6fmMGh2AGrifVV4e+WYc3Q9lO+xnSZ3NyhcBND3vzzatt05LFhoKFRxrIyklmLlUtyA== - -mime-db@1.52.0: - version "1.52.0" - resolved "https://registry.yarnpkg.com/mime-db/-/mime-db-1.52.0.tgz#bbabcdc02859f4987301c856e3387ce5ec43bf70" - integrity sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg== - -mime-types@^2.1.12, mime-types@^2.1.26, mime-types@^2.1.27: - version "2.1.35" - resolved "https://registry.yarnpkg.com/mime-types/-/mime-types-2.1.35.tgz#381a871b62a734450660ae3deee44813f70d959a" - integrity sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw== - dependencies: - mime-db "1.52.0" - -mime@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/mime/-/mime-3.0.0.tgz#b374550dca3a0c18443b0c950a6a58f1931cf7a7" - integrity sha512-jSCU7/VB1loIWBZe14aEYHU/+1UMEHoaO7qxCOVJOw9GgH72VAWppxNcjU+x9a2k3GSIBXNKxXQFqRvvZ7vr3A== - -mimic-fn@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/mimic-fn/-/mimic-fn-2.1.0.tgz#7ed2c2ccccaf84d3ffcb7a69b57711fc2083401b" - integrity sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg== - -min-indent@^1.0.0, min-indent@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/min-indent/-/min-indent-1.0.1.tgz#a63f681673b30571fbe8bc25686ae746eefa9869" - integrity sha512-I9jwMn07Sy/IwOj3zVkVik2JTvgpaykDZEigL6Rx6N9LbMywwUSMtxET+7lVoDLLd3O3IXwJwvuuns8UB/HeAg== - -mini-css-extract-plugin@^1.6.2: - version "1.6.2" - resolved "https://registry.yarnpkg.com/mini-css-extract-plugin/-/mini-css-extract-plugin-1.6.2.tgz#83172b4fd812f8fc4a09d6f6d16f924f53990ca8" - integrity sha512-WhDvO3SjGm40oV5y26GjMJYjd2UMqrLAGKy5YS2/3QKJy2F7jgynuHTir/tgUUOiNQu5saXHdc8reo7YuhhT4Q== - dependencies: - loader-utils "^2.0.0" - schema-utils "^3.0.0" - webpack-sources "^1.1.0" - -minimatch@^3.0.4, minimatch@^3.1.2: - version "3.1.2" - resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-3.1.2.tgz#19cd194bfd3e428f049a70817c038d89ab4be35b" - integrity sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw== - dependencies: - brace-expansion "^1.1.7" - -minimatch@^5.0.1: - version "5.1.0" - resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-5.1.0.tgz#1717b464f4971b144f6aabe8f2d0b8e4511e09c7" - integrity sha512-9TPBGGak4nHfGZsPBohm9AWg6NoT7QTCehS3BIJABslyZbzxfV78QM2Y6+i741OPZIafFAaiiEMh5OyIrJPgtg== - dependencies: - brace-expansion "^2.0.1" - -minimatch@~3.0.2: - version "3.0.8" - resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-3.0.8.tgz#5e6a59bd11e2ab0de1cfb843eb2d82e546c321c1" - integrity sha512-6FsRAQsxQ61mw+qP1ZzbL9Bc78x2p5OqNgNpnoAFLTrX8n5Kxph0CsnhmKKNXTWjXqU5L0pGPR7hYk+XWZr60Q== - dependencies: - brace-expansion "^1.1.7" - -minimist-options@4.1.0: - version "4.1.0" - resolved "https://registry.yarnpkg.com/minimist-options/-/minimist-options-4.1.0.tgz#c0655713c53a8a2ebd77ffa247d342c40f010619" - integrity sha512-Q4r8ghd80yhO/0j1O3B2BjweX3fiHg9cdOwjJd2J76Q135c+NDxGCqdYKQ1SKBuFfgWbAUzBfvYjPUEeNgqN1A== - dependencies: - arrify "^1.0.1" - is-plain-obj "^1.1.0" - kind-of "^6.0.3" - -minimist@^1.2.6: - version "1.2.7" - resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.7.tgz#daa1c4d91f507390437c6a8bc01078e7000c4d18" - integrity sha512-bzfL1YUZsP41gmu/qjrEk0Q6i2ix/cVeAhbCbqH9u3zYutS1cLg00qhrD0M2MVdCcx4Sc0UpP2eBWo9rotpq6g== - -minipass-collect@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/minipass-collect/-/minipass-collect-1.0.2.tgz#22b813bf745dc6edba2576b940022ad6edc8c617" - integrity sha512-6T6lH0H8OG9kITm/Jm6tdooIbogG9e0tLgpY6mphXSm/A9u8Nq1ryBG+Qspiub9LjWlBPsPS3tWQ/Botq4FdxA== - dependencies: - minipass "^3.0.0" - -minipass-flush@^1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/minipass-flush/-/minipass-flush-1.0.5.tgz#82e7135d7e89a50ffe64610a787953c4c4cbb373" - integrity sha512-JmQSYYpPUqX5Jyn1mXaRwOda1uQ8HP5KAT/oDSLCzt1BYRhQU0/hDtsB1ufZfEEzMZ9aAVmsBw8+FWsIXlClWw== - dependencies: - minipass "^3.0.0" - -minipass-pipeline@^1.2.2: - version "1.2.4" - resolved "https://registry.yarnpkg.com/minipass-pipeline/-/minipass-pipeline-1.2.4.tgz#68472f79711c084657c067c5c6ad93cddea8214c" - integrity sha512-xuIq7cIOt09RPRJ19gdi4b+RiNvDFYe5JH+ggNvBqGqpQXcru3PcRmOZuHBKWK1Txf9+cQ+HMVN4d6z46LZP7A== - dependencies: - minipass "^3.0.0" - -minipass@^3.0.0, minipass@^3.1.1: - version "3.1.3" - resolved "https://registry.yarnpkg.com/minipass/-/minipass-3.1.3.tgz#7d42ff1f39635482e15f9cdb53184deebd5815fd" - integrity sha512-Mgd2GdMVzY+x3IJ+oHnVM+KG3lA5c8tnabyJKmHSaG2kAGpudxuOf8ToDkhumF7UzME7DecbQE9uOZhNm7PuJg== - dependencies: - yallist "^4.0.0" - -minipass@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/minipass/-/minipass-5.0.0.tgz#3e9788ffb90b694a5d0ec94479a45b5d8738133d" - integrity sha512-3FnjYuehv9k6ovOEbyOswadCDPX1piCfhV8ncmYtHOjuPwylVWsghTLo7rabjC3Rx5xD4HDx8Wm1xnMF7S5qFQ== - -minizlib@^2.1.1: - version "2.1.2" - resolved "https://registry.yarnpkg.com/minizlib/-/minizlib-2.1.2.tgz#e90d3466ba209b932451508a11ce3d3632145931" - integrity sha512-bAxsR8BVfj60DWXHE3u30oHzfl4G7khkSuPW+qvpd7jFRHm7dLxOjUk1EHACJ/hxLY8phGJ0YhYHZo7jil7Qdg== - dependencies: - minipass "^3.0.0" - yallist "^4.0.0" - -mkdirp@^1.0.3, mkdirp@^1.0.4: - version "1.0.4" - resolved "https://registry.yarnpkg.com/mkdirp/-/mkdirp-1.0.4.tgz#3eb5ed62622756d79a5f0e2a221dfebad75c2f7e" - integrity sha512-vVqVZQyf3WLx2Shd0qJ9xuvqgAyKPLAiqITEtqW0oIUjzo3PePDd6fW9iFz30ef7Ysp/oiWqbhszeGWW2T6Gzw== - -mobx-react-lite@^3.2.0: - version "3.2.0" - resolved "https://registry.yarnpkg.com/mobx-react-lite/-/mobx-react-lite-3.2.0.tgz#331d7365a6b053378dfe9c087315b4e41c5df69f" - integrity sha512-q5+UHIqYCOpBoFm/PElDuOhbcatvTllgRp3M1s+Hp5j0Z6XNgDbgqxawJ0ZAUEyKM8X1zs70PCuhAIzX1f4Q/g== - -mobx-react@^7.2.0: - version "7.2.0" - resolved "https://registry.yarnpkg.com/mobx-react/-/mobx-react-7.2.0.tgz#241e925e963bb83a31d269f65f9f379e37ecbaeb" - integrity sha512-KHUjZ3HBmZlNnPd1M82jcdVsQRDlfym38zJhZEs33VxyVQTvL77hODCArq6+C1P1k/6erEeo2R7rpE7ZeOL7dg== - dependencies: - mobx-react-lite "^3.2.0" - -moment-locales-webpack-plugin@^1.2.0: - version "1.2.0" - resolved "https://registry.yarnpkg.com/moment-locales-webpack-plugin/-/moment-locales-webpack-plugin-1.2.0.tgz#9af83876a44053706b868ceece5119584d10d7aa" - integrity sha512-QAi5v0OlPUP7GXviKMtxnpBAo8WmTHrUNN7iciAhNOEAd9evCOvuN0g1N7ThIg3q11GLCkjY1zQ2saRcf/43nQ== - dependencies: - lodash.difference "^4.5.0" - -moment-timezone@>=0.5.35, moment-timezone@^0.4.0: - version "0.5.45" - resolved "https://registry.yarnpkg.com/moment-timezone/-/moment-timezone-0.5.45.tgz#cb685acd56bac10e69d93c536366eb65aa6bcf5c" - integrity sha512-HIWmqA86KcmCAhnMAN0wuDOARV/525R2+lOLotuGFzn4HO+FH+/645z2wx0Dt3iDv6/p61SIvKnDstISainhLQ== - dependencies: - moment "^2.29.4" - -moment-timezone@^0.5.43: - version "0.5.43" - resolved "https://registry.yarnpkg.com/moment-timezone/-/moment-timezone-0.5.43.tgz#3dd7f3d0c67f78c23cd1906b9b2137a09b3c4790" - integrity sha512-72j3aNyuIsDxdF1i7CEgV2FfxM1r6aaqJyLB2vwb33mXYyoyLly+F1zbWqhA3/bVIoJ4szlUoMbUnVdid32NUQ== - dependencies: - moment "^2.29.4" - -moment@^2.10, moment@^2.29.4: - version "2.29.4" - resolved "https://registry.yarnpkg.com/moment/-/moment-2.29.4.tgz#3dbe052889fe7c1b2ed966fcb3a77328964ef108" - integrity sha512-5LC9SOxjSc2HF6vO2CyuTDNivEdoz2IvyJJGj6X8DJ0eFyfszE0QiEd+iXmBvUP3WHxSjFH/vIsA0EN00cgr8w== - -moo-color@^1.0.2: - version "1.0.3" - resolved "https://registry.yarnpkg.com/moo-color/-/moo-color-1.0.3.tgz#d56435f8359c8284d83ac58016df7427febece74" - integrity sha512-i/+ZKXMDf6aqYtBhuOcej71YSlbjT3wCO/4H1j8rPvxDJEifdwgg5MaFyu6iYAT8GBZJg2z0dkgK4YMzvURALQ== - dependencies: - color-name "^1.1.4" - -mri@^1.1.0: - version "1.2.0" - resolved "https://registry.yarnpkg.com/mri/-/mri-1.2.0.tgz#6721480fec2a11a4889861115a48b6cbe7cc8f0b" - integrity sha512-tzzskb3bG8LvYGFF/mDTpq3jpI6Q9wc3LEmBaghu+DdCssd1FakN7Bc0hVNmEyGq1bq3RgfkCb3cmQLpNPOroA== - -ms@2.1.2: - version "2.1.2" - resolved "https://registry.yarnpkg.com/ms/-/ms-2.1.2.tgz#d09d1f357b443f493382a8eb3ccd183872ae6009" - integrity sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w== - -ms@^2.1.1: - version "2.1.3" - resolved "https://registry.yarnpkg.com/ms/-/ms-2.1.3.tgz#574c8138ce1d2b5861f0b44579dbadd60c6615b2" - integrity sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA== - -nano-time@1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/nano-time/-/nano-time-1.0.0.tgz#b0554f69ad89e22d0907f7a12b0993a5d96137ef" - integrity sha1-sFVPaa2J4i0JB/ehKwmTpdlhN+8= - dependencies: - big-integer "^1.6.16" - -nanoid@^3.3.7: - version "3.3.8" - resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.3.8.tgz#b1be3030bee36aaff18bacb375e5cce521684baf" - integrity sha512-WNLf5Sd8oZxOm+TzppcYk8gVOgP+l58xNy58D0nbUnOxOWRWvlcCV4kUF7ltmI6PsrLl/BgKEyS4mqsGChFN0w== - -natural-compare@^1.4.0: - version "1.4.0" - resolved "https://registry.yarnpkg.com/natural-compare/-/natural-compare-1.4.0.tgz#4abebfeed7541f2c27acfb29bdbbd15c8d5ba4f7" - integrity sha1-Sr6/7tdUHywnrPspvbvRXI1bpPc= - -needle@^2.2.4: - version "2.9.1" - resolved "https://registry.yarnpkg.com/needle/-/needle-2.9.1.tgz#22d1dffbe3490c2b83e301f7709b6736cd8f2684" - integrity sha512-6R9fqJ5Zcmf+uYaFgdIHmLwNldn5HbK8L5ybn7Uz+ylX/rnOsSp1AHcvQSrCaFN+qNM1wpymHqD7mVasEOlHGQ== - dependencies: - debug "^3.2.6" - iconv-lite "^0.4.4" - sax "^1.2.4" - -neo-async@^2.6.2: - version "2.6.2" - resolved "https://registry.yarnpkg.com/neo-async/-/neo-async-2.6.2.tgz#b4aafb93e3aeb2d8174ca53cf163ab7d7308305f" - integrity sha512-Yd3UES5mWCSqR+qNT93S3UoYUkqAZ9lLg8a7g9rimsWmYGK8cVToA4/sF3RrshdyV3sAGMXVUmpMYOw+dLpOuw== - -node-fetch-h2@^2.3.0: - version "2.3.0" - resolved "https://registry.yarnpkg.com/node-fetch-h2/-/node-fetch-h2-2.3.0.tgz#c6188325f9bd3d834020bf0f2d6dc17ced2241ac" - integrity sha512-ofRW94Ab0T4AOh5Fk8t0h8OBWrmjb0SSB20xh1H8YnPV9EJ+f5AMoYSUQ2zgJ4Iq2HAK0I2l5/Nequ8YzFS3Hg== - dependencies: - http2-client "^1.2.5" - -node-fetch@2.6.7, node-fetch@^2.6.1: - version "2.6.7" - resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.6.7.tgz#24de9fba827e3b4ae44dc8b20256a379160052ad" - integrity sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ== - dependencies: - whatwg-url "^5.0.0" - -node-int64@^0.4.0: - version "0.4.0" - resolved "https://registry.yarnpkg.com/node-int64/-/node-int64-0.4.0.tgz#87a9065cdb355d3182d8f94ce11188b825c68a3b" - integrity sha1-h6kGXNs1XTGC2PlM4RGIuCXGijs= - -node-modules-regexp@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/node-modules-regexp/-/node-modules-regexp-1.0.0.tgz#8d9dbe28964a4ac5712e9131642107c71e90ec40" - integrity sha1-jZ2+KJZKSsVxLpExZCEHxx6Q7EA= - -node-readfiles@^0.2.0: - version "0.2.0" - resolved "https://registry.yarnpkg.com/node-readfiles/-/node-readfiles-0.2.0.tgz#dbbd4af12134e2e635c245ef93ffcf6f60673a5d" - integrity sha1-271K8SE04uY1wkXvk//Pb2BnOl0= - dependencies: - es6-promise "^3.2.1" - -node-releases@^2.0.18: - version "2.0.18" - resolved "https://registry.yarnpkg.com/node-releases/-/node-releases-2.0.18.tgz#f010e8d35e2fe8d6b2944f03f70213ecedc4ca3f" - integrity sha512-d9VeXT4SJ7ZeOqGX6R5EM022wpL+eWPooLI+5UpWn2jCT1aosUQEhQP214x33Wkwx3JQMvIm+tIoVOdodFS40g== - -normalize-package-data@^3.0.2: - version "3.0.3" - resolved "https://registry.yarnpkg.com/normalize-package-data/-/normalize-package-data-3.0.3.tgz#dbcc3e2da59509a0983422884cd172eefdfa525e" - integrity sha512-p2W1sgqij3zMMyRC067Dg16bfzVH+w7hyegmpIvZ4JNjqtGOVAIvLmjBx3yP7YTe9vKJgkoNOPjwQGogDoMXFA== - dependencies: - hosted-git-info "^4.0.1" - is-core-module "^2.5.0" - semver "^7.3.4" - validate-npm-package-license "^3.0.1" - -normalize-path@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/normalize-path/-/normalize-path-3.0.0.tgz#0dcd69ff23a1c9b11fd0978316644a0388216a65" - integrity sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA== - -normalize-registry-url@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/normalize-registry-url/-/normalize-registry-url-1.0.0.tgz#f75d2c48373da780c76f1f0eeb6382c06e784d13" - integrity sha512-0v6T4851b72ykk5zEtFoN4QX/Fqyk7pouIj9xZyAvAe9jlDhAwT4z6FlwsoQCHjeuK2EGUoAwy/F4y4B1uZq9A== - -normalize-url@^6.0.1: - version "6.1.0" - resolved "https://registry.yarnpkg.com/normalize-url/-/normalize-url-6.1.0.tgz#40d0885b535deffe3f3147bec877d05fe4c5668a" - integrity sha512-DlL+XwOy3NxAQ8xuC0okPgK46iuVNAK01YN7RueYBqqFeGsBjV9XmCAzAdgt+667bCl5kPh9EqKKDwnaPG1I7A== - -npm-run-path@^4.0.1: - version "4.0.1" - resolved "https://registry.yarnpkg.com/npm-run-path/-/npm-run-path-4.0.1.tgz#b7ecd1e5ed53da8e37a55e1c2269e0b97ed748ea" - integrity sha512-S48WzZW777zhNIrn7gxOlISNAqi9ZC/uQFnRdbeIHhZhCA6UqpkOT8T1G7BvfdgP4Er8gF4sUbaS0i7QvIfCWw== - dependencies: - path-key "^3.0.0" - -nth-check@^2.0.1: - version "2.1.1" - resolved "https://registry.yarnpkg.com/nth-check/-/nth-check-2.1.1.tgz#c9eab428effce36cd6b92c924bdb000ef1f1ed1d" - integrity sha512-lqjrjmaOoAnWfMmBPL+XNnynZh2+swxiX3WUE0s4yEHI6m+AwrK2UZOimIRl3X/4QctVqS8AiZjFqyOGrMXb/w== - dependencies: - boolbase "^1.0.0" - -nwsapi@^2.2.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/nwsapi/-/nwsapi-2.2.0.tgz#204879a9e3d068ff2a55139c2c772780681a38b7" - integrity sha512-h2AatdwYH+JHiZpv7pt/gSX1XoRGb7L/qSIeuqA6GwYoF9w1vP1cw42TO0aI2pNyshRK5893hNSl+1//vHK7hQ== - -oas-kit-common@^1.0.8: - version "1.0.8" - resolved "https://registry.yarnpkg.com/oas-kit-common/-/oas-kit-common-1.0.8.tgz#6d8cacf6e9097967a4c7ea8bcbcbd77018e1f535" - integrity sha512-pJTS2+T0oGIwgjGpw7sIRU8RQMcUoKCDWFLdBqKB2BNmGpbBMH2sdqAaOXUg8OzonZHU0L7vfJu1mJFEiYDWOQ== - dependencies: - fast-safe-stringify "^2.0.7" - -oas-linter@^3.2.2: - version "3.2.2" - resolved "https://registry.yarnpkg.com/oas-linter/-/oas-linter-3.2.2.tgz#ab6a33736313490659035ca6802dc4b35d48aa1e" - integrity sha512-KEGjPDVoU5K6swgo9hJVA/qYGlwfbFx+Kg2QB/kd7rzV5N8N5Mg6PlsoCMohVnQmo+pzJap/F610qTodKzecGQ== - dependencies: - "@exodus/schemasafe" "^1.0.0-rc.2" - should "^13.2.1" - yaml "^1.10.0" - -oas-resolver@^2.5.5: - version "2.5.5" - resolved "https://registry.yarnpkg.com/oas-resolver/-/oas-resolver-2.5.5.tgz#12304c85b7eea840bf7fb51ea85b132592a104f3" - integrity sha512-1po1gzIlTXQqyVNtLFWJuzDm4xxhMCJ8QcP3OarKDO8aJ8AmCtQ67XZ1X+nBbHH4CjTcEsIab1qX5+GIU4f2Gg== - dependencies: - node-fetch-h2 "^2.3.0" - oas-kit-common "^1.0.8" - reftools "^1.1.8" - yaml "^1.10.0" - yargs "^17.0.1" - -oas-schema-walker@^1.1.5: - version "1.1.5" - resolved "https://registry.yarnpkg.com/oas-schema-walker/-/oas-schema-walker-1.1.5.tgz#74c3cd47b70ff8e0b19adada14455b5d3ac38a22" - integrity sha512-2yucenq1a9YPmeNExoUa9Qwrt9RFkjqaMAA1X+U7sbb0AqBeTIdMHky9SQQ6iN94bO5NW0W4TRYXerG+BdAvAQ== - -oas-validator@^5.0.6: - version "5.0.6" - resolved "https://registry.yarnpkg.com/oas-validator/-/oas-validator-5.0.6.tgz#419ff4c14b9b16ca2052a31e81ee93efb7492978" - integrity sha512-bI+gyr3MiG/4Q5Ibvg0R77skVWS882gFMkxwB1p6qY7Rc4p7EoDezFVfondjYhJDPDnB1ZD7Aqj7AWROAsMBZg== - dependencies: - call-me-maybe "^1.0.1" - oas-kit-common "^1.0.8" - oas-linter "^3.2.2" - oas-resolver "^2.5.5" - oas-schema-walker "^1.1.5" - reftools "^1.1.8" - should "^13.2.1" - yaml "^1.10.0" - -object-assign@^4.0.1, object-assign@^4.1.0, object-assign@^4.1.1: - version "4.1.1" - resolved "https://registry.yarnpkg.com/object-assign/-/object-assign-4.1.1.tgz#2109adc7965887cfc05cbbd442cac8bfbb360863" - integrity sha512-rJgTQnkUnH1sFw8yT6VSU3zD3sWmu6sZhIseY8VX+GRu3P6F7Fu+JNDoXfklElbLJSnc3FUQHVe4cU5hj+BcUg== - -object-inspect@^1.10.3: - version "1.10.3" - resolved "https://registry.yarnpkg.com/object-inspect/-/object-inspect-1.10.3.tgz#c2aa7d2d09f50c99375704f7a0adf24c5782d369" - integrity sha512-e5mCJlSH7poANfC8z8S9s9S2IN5/4Zb3aZ33f5s8YqoazCFzNLloLU8r5VCG+G7WoqLvAAZoVMcy3tp/3X0Plw== - -object-inspect@^1.12.0: - version "1.12.2" - resolved "https://registry.yarnpkg.com/object-inspect/-/object-inspect-1.12.2.tgz#c0641f26394532f28ab8d796ab954e43c009a8ea" - integrity sha512-z+cPxW0QGUp0mcqcsgQyLVRDoXFQbXOwBaqyF7VIgI4TWNQsDHrBpUQslRmIfAoYWdYzs6UlKJtB2XJpTaNSpQ== - -object-inspect@^1.12.3: - version "1.12.3" - resolved "https://registry.yarnpkg.com/object-inspect/-/object-inspect-1.12.3.tgz#ba62dffd67ee256c8c086dfae69e016cd1f198b9" - integrity sha512-geUvdk7c+eizMNUDkRpW1wJwgfOiOeHbxBR/hLXK1aT6zmVSO0jsQcs7fj6MGw89jC/cjGfLcNOrtMYtGqm81g== - -object-inspect@^1.9.0: - version "1.11.0" - resolved "https://registry.yarnpkg.com/object-inspect/-/object-inspect-1.11.0.tgz#9dceb146cedd4148a0d9e51ab88d34cf509922b1" - integrity sha512-jp7ikS6Sd3GxQfZJPyH3cjcbJF6GZPClgdV+EFygjFLQ5FmW/dRUnTd9PQ9k0JhoNDabWFbpF1yCdSWCC6gexg== - -object-keys@^1.0.12, object-keys@^1.1.1: - version "1.1.1" - resolved "https://registry.yarnpkg.com/object-keys/-/object-keys-1.1.1.tgz#1c47f272df277f3b1daf061677d9c82e2322c60e" - integrity sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA== - -object.assign@^4.1.2: - version "4.1.2" - resolved "https://registry.yarnpkg.com/object.assign/-/object.assign-4.1.2.tgz#0ed54a342eceb37b38ff76eb831a0e788cb63940" - integrity sha512-ixT2L5THXsApyiUPYKmW+2EHpXXe5Ii3M+f4e+aJFAHao5amFRW6J0OO6c/LU8Be47utCx2GL89hxGB6XSmKuQ== - dependencies: - call-bind "^1.0.0" - define-properties "^1.1.3" - has-symbols "^1.0.1" - object-keys "^1.1.1" - -object.assign@^4.1.4: - version "4.1.4" - resolved "https://registry.yarnpkg.com/object.assign/-/object.assign-4.1.4.tgz#9673c7c7c351ab8c4d0b516f4343ebf4dfb7799f" - integrity sha512-1mxKf0e58bvyjSCtKYY4sRe9itRk3PJpquJOjeIkz885CczcI4IvJJDLPS72oowuSh+pBxUFROpX+TU++hxhZQ== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - has-symbols "^1.0.3" - object-keys "^1.1.1" - -object.entries@^1.1.5: - version "1.1.5" - resolved "https://registry.yarnpkg.com/object.entries/-/object.entries-1.1.5.tgz#e1acdd17c4de2cd96d5a08487cfb9db84d881861" - integrity sha512-TyxmjUoZggd4OrrU1W66FMDG6CuqJxsFvymeyXI51+vQLN67zYfZseptRge703kKQdo4uccgAKebXFcRCzk4+g== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.3" - es-abstract "^1.19.1" - -object.fromentries@^2.0.5: - version "2.0.5" - resolved "https://registry.yarnpkg.com/object.fromentries/-/object.fromentries-2.0.5.tgz#7b37b205109c21e741e605727fe8b0ad5fa08251" - integrity sha512-CAyG5mWQRRiBU57Re4FKoTBjXfDoNwdFVH2Y1tS9PqCsfUTymAohOkEMSG3aRNKmv4lV3O7p1et7c187q6bynw== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.3" - es-abstract "^1.19.1" - -object.hasown@^1.1.1: - version "1.1.1" - resolved "https://registry.yarnpkg.com/object.hasown/-/object.hasown-1.1.1.tgz#ad1eecc60d03f49460600430d97f23882cf592a3" - integrity sha512-LYLe4tivNQzq4JdaWW6WO3HMZZJWzkkH8fnI6EebWl0VZth2wL2Lovm74ep2/gZzlaTdV62JZHEqHQ2yVn8Q/A== - dependencies: - define-properties "^1.1.4" - es-abstract "^1.19.5" - -object.values@^1.1.5: - version "1.1.5" - resolved "https://registry.yarnpkg.com/object.values/-/object.values-1.1.5.tgz#959f63e3ce9ef108720333082131e4a459b716ac" - integrity sha512-QUZRW0ilQ3PnPpbNtgdNV1PDbEqLIiSFB3l+EnGtBQ/8SUTLj1PZwtQHABZtLgwpJZTSZhuGLOGk57Drx2IvYg== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.3" - es-abstract "^1.19.1" - -object.values@^1.1.6: - version "1.1.6" - resolved "https://registry.yarnpkg.com/object.values/-/object.values-1.1.6.tgz#4abbaa71eba47d63589d402856f908243eea9b1d" - integrity sha512-FVVTkD1vENCsAcwNs9k6jea2uHC/X0+JcjG8YA60FN5CMaJmG95wT9jek/xX9nornqGRrBkKtzuAu2wuHpKqvw== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - es-abstract "^1.20.4" - -oblivious-set@1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/oblivious-set/-/oblivious-set-1.0.0.tgz#c8316f2c2fb6ff7b11b6158db3234c49f733c566" - integrity sha512-z+pI07qxo4c2CulUHCDf9lcqDlMSo72N/4rLUpRXf6fu+q8vjt8y0xS+Tlf8NTJDdTXHbdeO1n3MlbctwEoXZw== - -once@^1.3.0: - version "1.4.0" - resolved "https://registry.yarnpkg.com/once/-/once-1.4.0.tgz#583b1aa775961d4b113ac17d9c50baef9dd76bd1" - integrity sha1-WDsap3WWHUsROsF9nFC6753Xa9E= - dependencies: - wrappy "1" - -onetime@^5.1.2: - version "5.1.2" - resolved "https://registry.yarnpkg.com/onetime/-/onetime-5.1.2.tgz#d0e96ebb56b07476df1dd9c4806e5237985ca45e" - integrity sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg== - dependencies: - mimic-fn "^2.1.0" - -openapi-sampler@^1.3.0: - version "1.3.0" - resolved "https://registry.yarnpkg.com/openapi-sampler/-/openapi-sampler-1.3.0.tgz#5b99ceb4156b00d2aa3f860e52ccb768a5695793" - integrity sha512-2QfjK1oM9Sv0q82Ae1RrUe3yfFmAyjF548+6eAeb+h/cL1Uj51TW4UezraBEvwEdzoBgfo4AaTLVFGTKj+yYDw== - dependencies: - "@types/json-schema" "^7.0.7" - json-pointer "0.6.2" - -openapi-typescript@^5.4.1: - version "5.4.1" - resolved "https://registry.yarnpkg.com/openapi-typescript/-/openapi-typescript-5.4.1.tgz#38b4b45244acc1361f3c444537833a9e9cb03bf6" - integrity sha512-AGB2QiZPz4rE7zIwV3dRHtoUC/CWHhUjuzGXvtmMQN2AFV8xCTLKcZUHLcdPQmt/83i22nRE7+TxXOXkK+gf4Q== - dependencies: - js-yaml "^4.1.0" - mime "^3.0.0" - prettier "^2.6.2" - tiny-glob "^0.2.9" - undici "^5.4.0" - yargs-parser "^21.0.1" - -optionator@^0.8.1: - version "0.8.3" - resolved "https://registry.yarnpkg.com/optionator/-/optionator-0.8.3.tgz#84fa1d036fe9d3c7e21d99884b601167ec8fb495" - integrity sha512-+IW9pACdk3XWmmTXG8m3upGUJst5XRGzxMRjXzAuJ1XnIFNvfhjjIuYkDvysnPQ7qzqVzLt78BCruntqRhWQbA== - dependencies: - deep-is "~0.1.3" - fast-levenshtein "~2.0.6" - levn "~0.3.0" - prelude-ls "~1.1.2" - type-check "~0.3.2" - word-wrap "~1.2.3" - -optionator@^0.9.1: - version "0.9.1" - resolved "https://registry.yarnpkg.com/optionator/-/optionator-0.9.1.tgz#4f236a6373dae0566a6d43e1326674f50c291499" - integrity sha512-74RlY5FCnhq4jRxVUPKDaRwrVNXMqsGsiW6AJw4XK8hmtm10wC0ypZBLw5IIp85NZMr91+qd1RvvENwg7jjRFw== - dependencies: - deep-is "^0.1.3" - fast-levenshtein "^2.0.6" - levn "^0.4.1" - prelude-ls "^1.2.1" - type-check "^0.4.0" - word-wrap "^1.2.3" - -p-limit@^2.2.0: - version "2.3.0" - resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-2.3.0.tgz#3dd33c647a214fdfffd835933eb086da0dc21db1" - integrity sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w== - dependencies: - p-try "^2.0.0" - -p-limit@^3.0.2: - version "3.1.0" - resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-3.1.0.tgz#e1daccbe78d0d1388ca18c64fea38e3e57e3706b" - integrity sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ== - dependencies: - yocto-queue "^0.1.0" - -p-locate@^4.1.0: - version "4.1.0" - resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-4.1.0.tgz#a3428bb7088b3a60292f66919278b7c297ad4f07" - integrity sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A== - dependencies: - p-limit "^2.2.0" - -p-locate@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-5.0.0.tgz#83c8315c6785005e3bd021839411c9e110e6d834" - integrity sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw== - dependencies: - p-limit "^3.0.2" - -p-map@^2.0.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/p-map/-/p-map-2.1.0.tgz#310928feef9c9ecc65b68b17693018a665cea175" - integrity sha512-y3b8Kpd8OAN444hxfBbFfj1FY/RjtTd8tzYwhUqNYXx0fXx2iX4maP4Qr6qhIKbQXI02wTLAda4fYUbDagTUFw== - -p-map@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/p-map/-/p-map-4.0.0.tgz#bb2f95a5eda2ec168ec9274e06a747c3e2904d2b" - integrity sha512-/bjOqmgETBYB5BoEeGVea8dmvHb2m9GLy1E9W43yeyfP6QQCZGFNa+XRceJEuDB6zqr+gKpIAmlLebMpykw/MQ== - dependencies: - aggregate-error "^3.0.0" - -p-try@^2.0.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/p-try/-/p-try-2.2.0.tgz#cb2868540e313d61de58fafbe35ce9004d5540e6" - integrity sha512-R4nPAVTAU0B9D35/Gk3uJf/7XYbQcyohSKdvAxIRSNghFl4e71hVoGnBNQz9cWaXxO2I10KTC+3jMdvvoKw6dQ== - -parent-module@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/parent-module/-/parent-module-1.0.1.tgz#691d2709e78c79fae3a156622452d00762caaaa2" - integrity sha512-GQ2EWRpQV8/o+Aw8YqtfZZPfNRWZYkbidE9k5rpl/hC3vtHHBfGm2Ifi6qWV+coDGkrUKZAxE3Lot5kcsRlh+g== - dependencies: - callsites "^3.0.0" - -parse-entities@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/parse-entities/-/parse-entities-2.0.0.tgz#53c6eb5b9314a1f4ec99fa0fdf7ce01ecda0cbe8" - integrity sha512-kkywGpCcRYhqQIchaWqZ875wzpS/bMKhz5HnN3p7wveJTkTtyAB/AlnS0f8DFSqYW1T82t6yEAkEcB+A1I3MbQ== - dependencies: - character-entities "^1.0.0" - character-entities-legacy "^1.0.0" - character-reference-invalid "^1.0.0" - is-alphanumerical "^1.0.0" - is-decimal "^1.0.0" - is-hexadecimal "^1.0.0" - -parse-json@^5.0.0, parse-json@^5.2.0: - version "5.2.0" - resolved "https://registry.yarnpkg.com/parse-json/-/parse-json-5.2.0.tgz#c76fc66dee54231c962b22bcc8a72cf2f99753cd" - integrity sha512-ayCKvm/phCGxOkYRSCM82iDwct8/EonSEgCSxWxD7ve6jHggsFl4fZVQBPRNgQoKiuV/odhFrGzQXZwbifC8Rg== - dependencies: - "@babel/code-frame" "^7.0.0" - error-ex "^1.3.1" - json-parse-even-better-errors "^2.3.0" - lines-and-columns "^1.1.6" - -parse5@6.0.1: - version "6.0.1" - resolved "https://registry.yarnpkg.com/parse5/-/parse5-6.0.1.tgz#e1a1c085c569b3dc08321184f19a39cc27f7c30b" - integrity sha512-Ofn/CTFzRGTTxwpNEs9PP93gXShHcTq255nzRYSKe8AkVpZY7e1fpmTfOyoIvjP5HG7Z2ZM7VS9PPhQGW2pOpw== - -path-browserify@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/path-browserify/-/path-browserify-1.0.1.tgz#d98454a9c3753d5790860f16f68867b9e46be1fd" - integrity sha512-b7uo2UCUOYZcnF/3ID0lulOJi/bafxa1xPe7ZPsammBSpjSWQkjNxlt635YGS2MiR9GjvuXCtz2emr3jbsz98g== - -path-exists@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/path-exists/-/path-exists-4.0.0.tgz#513bdbe2d3b95d7762e8c1137efa195c6c61b5b3" - integrity sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w== - -path-is-absolute@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/path-is-absolute/-/path-is-absolute-1.0.1.tgz#174b9268735534ffbc7ace6bf53a5a9e1b5c5f5f" - integrity sha1-F0uSaHNVNP+8es5r9TpanhtcX18= - -path-is-inside@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/path-is-inside/-/path-is-inside-1.0.2.tgz#365417dede44430d1c11af61027facf074bdfc53" - integrity sha1-NlQX3t5EQw0cEa9hAn+s8HS9/FM= - -path-key@^3.0.0, path-key@^3.1.0: - version "3.1.1" - resolved "https://registry.yarnpkg.com/path-key/-/path-key-3.1.1.tgz#581f6ade658cbba65a0d3380de7753295054f375" - integrity sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q== - -path-parse@^1.0.6, path-parse@^1.0.7: - version "1.0.7" - resolved "https://registry.yarnpkg.com/path-parse/-/path-parse-1.0.7.tgz#fbc114b60ca42b30d9daf5858e4bd68bbedb6735" - integrity sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw== - -path-type@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/path-type/-/path-type-4.0.0.tgz#84ed01c0a7ba380afe09d90a8c180dcd9d03043b" - integrity sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw== - -perfect-scrollbar@^1.5.1: - version "1.5.1" - resolved "https://registry.yarnpkg.com/perfect-scrollbar/-/perfect-scrollbar-1.5.1.tgz#8ee5b3ca06ce9c3f7338fd4ab67a55248a6cf3be" - integrity sha512-MrSImINnIh3Tm1hdPT6bji6fmIeRorVEegQvyUnhqko2hDGTHhmjPefHXfxG/Jb8xVbfCwgmUIlIajERGXjVXQ== - -picocolors@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/picocolors/-/picocolors-1.0.0.tgz#cb5bdc74ff3f51892236eaf79d68bc44564ab81c" - integrity sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ== - -picocolors@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/picocolors/-/picocolors-1.0.1.tgz#a8ad579b571952f0e5d25892de5445bcfe25aaa1" - integrity sha512-anP1Z8qwhkbmu7MFP5iTt+wQKXgwzf7zTyGlcdzabySa9vd0Xt392U0rVmz9poOaBj0uHJKyyo9/upk0HrEQew== - -picomatch@^2.0.4, picomatch@^2.2.3: - version "2.3.0" - resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.3.0.tgz#f1f061de8f6a4bf022892e2d128234fb98302972" - integrity sha512-lY1Q/PiJGC2zOv/z391WOTD+Z02bCgsFfvxoXXf6h7kv9o+WmsmzYqrAwY63sNgOxE4xEdq0WyUnXfKeBrSvYw== - -picomatch@^2.3.1: - version "2.3.1" - resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.3.1.tgz#3ba3833733646d9d3e4995946c1365a67fb07a42" - integrity sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA== - -pify@^2.0.0: - version "2.3.0" - resolved "https://registry.yarnpkg.com/pify/-/pify-2.3.0.tgz#ed141a6ac043a849ea588498e7dca8b15330e90c" - integrity sha1-7RQaasBDqEnqWISY59yosVMw6Qw= - -pify@^4.0.1: - version "4.0.1" - resolved "https://registry.yarnpkg.com/pify/-/pify-4.0.1.tgz#4b2cd25c50d598735c50292224fd8c6df41e3231" - integrity sha512-uB80kBFb/tfd68bVleG9T5GGsGPjJrLAUpR5PZIrhBnIaRTQRjqdJSsIKkOP6OAIFbj7GOrcudc5pNjZ+geV2g== - -pinkie-promise@^2.0.0: - version "2.0.1" - resolved "https://registry.yarnpkg.com/pinkie-promise/-/pinkie-promise-2.0.1.tgz#2135d6dfa7a358c069ac9b178776288228450ffa" - integrity sha1-ITXW36ejWMBprJsXh3YogihFD/o= - dependencies: - pinkie "^2.0.0" - -pinkie@^2.0.0: - version "2.0.4" - resolved "https://registry.yarnpkg.com/pinkie/-/pinkie-2.0.4.tgz#72556b80cfa0d48a974e80e77248e80ed4f7f870" - integrity sha1-clVrgM+g1IqXToDnckjoDtT3+HA= - -pirates@^4.0.1: - version "4.0.1" - resolved "https://registry.yarnpkg.com/pirates/-/pirates-4.0.1.tgz#643a92caf894566f91b2b986d2c66950a8e2fb87" - integrity sha512-WuNqLTbMI3tmfef2TKxlQmAiLHKtFhlsCZnPIpuv2Ow0RDVO8lfy1Opf4NUzlMXLjPl+Men7AuVdX6TA+s+uGA== - dependencies: - node-modules-regexp "^1.0.0" - -pirates@^4.0.4: - version "4.0.5" - resolved "https://registry.yarnpkg.com/pirates/-/pirates-4.0.5.tgz#feec352ea5c3268fb23a37c702ab1699f35a5f3b" - integrity sha512-8V9+HQPupnaXMA23c5hvl69zXvTwTzyAYasnkb0Tts4XvO4CliqONMOnvlq26rkhLC3nWDFBJf73LU1e1VZLaQ== - -pkg-dir@^4.1.0, pkg-dir@^4.2.0: - version "4.2.0" - resolved "https://registry.yarnpkg.com/pkg-dir/-/pkg-dir-4.2.0.tgz#f099133df7ede422e81d1d8448270eeb3e4261f3" - integrity sha512-HRDzbaKjC+AOWVXxAU/x54COGeIv9eb+6CkDSQoNTt4XyWoIJvuPsXizxu/Fr23EiekbtZwmh1IcIG/l/a10GQ== - dependencies: - find-up "^4.0.0" - -pluralize@^8.0.0: - version "8.0.0" - resolved "https://registry.yarnpkg.com/pluralize/-/pluralize-8.0.0.tgz#1a6fa16a38d12a1901e0320fa017051c539ce3b1" - integrity sha512-Nc3IT5yHzflTfbjgqWcCPpo7DaKy4FnpB0l/zCAW0Tc7jxAiuqSxHasntB3D7887LSrA93kDJ9IXovxJYxyLCA== - -polished@^4.1.3: - version "4.1.3" - resolved "https://registry.yarnpkg.com/polished/-/polished-4.1.3.tgz#7a3abf2972364e7d97770b827eec9a9e64002cfc" - integrity sha512-ocPAcVBUOryJEKe0z2KLd1l9EBa1r5mSwlKpExmrLzsnIzJo4axsoU9O2BjOTkDGDT4mZ0WFE5XKTlR3nLnZOA== - dependencies: - "@babel/runtime" "^7.14.0" - -popmotion@11.0.3: - version "11.0.3" - resolved "https://registry.yarnpkg.com/popmotion/-/popmotion-11.0.3.tgz#565c5f6590bbcddab7a33a074bb2ba97e24b0cc9" - integrity sha512-Y55FLdj3UxkR7Vl3s7Qr4e9m0onSnP8W7d/xQLsoJM40vs6UKHFdygs6SWryasTZYqugMjm3BepCF4CWXDiHgA== - dependencies: - framesync "6.0.1" - hey-listen "^1.0.8" - style-value-types "5.0.0" - tslib "^2.1.0" - -postcss-calc@^8.2.3: - version "8.2.4" - resolved "https://registry.yarnpkg.com/postcss-calc/-/postcss-calc-8.2.4.tgz#77b9c29bfcbe8a07ff6693dc87050828889739a5" - integrity sha512-SmWMSJmB8MRnnULldx0lQIyhSNvuDl9HfrZkaqqE/WHAhToYsAvDq+yAsA/kIyINDszOp3Rh0GFoNuH5Ypsm3Q== - dependencies: - postcss-selector-parser "^6.0.9" - postcss-value-parser "^4.2.0" - -postcss-colormin@^5.3.0: - version "5.3.0" - resolved "https://registry.yarnpkg.com/postcss-colormin/-/postcss-colormin-5.3.0.tgz#3cee9e5ca62b2c27e84fce63affc0cfb5901956a" - integrity sha512-WdDO4gOFG2Z8n4P8TWBpshnL3JpmNmJwdnfP2gbk2qBA8PWwOYcmjmI/t3CmMeL72a7Hkd+x/Mg9O2/0rD54Pg== - dependencies: - browserslist "^4.16.6" - caniuse-api "^3.0.0" - colord "^2.9.1" - postcss-value-parser "^4.2.0" - -postcss-convert-values@^5.1.2: - version "5.1.2" - resolved "https://registry.yarnpkg.com/postcss-convert-values/-/postcss-convert-values-5.1.2.tgz#31586df4e184c2e8890e8b34a0b9355313f503ab" - integrity sha512-c6Hzc4GAv95B7suy4udszX9Zy4ETyMCgFPUDtWjdFTKH1SE9eFY/jEpHSwTH1QPuwxHpWslhckUQWbNRM4ho5g== - dependencies: - browserslist "^4.20.3" - postcss-value-parser "^4.2.0" - -postcss-discard-comments@^5.1.2: - version "5.1.2" - resolved "https://registry.yarnpkg.com/postcss-discard-comments/-/postcss-discard-comments-5.1.2.tgz#8df5e81d2925af2780075840c1526f0660e53696" - integrity sha512-+L8208OVbHVF2UQf1iDmRcbdjJkuBF6IS29yBDSiWUIzpYaAhtNl6JYnYm12FnkeCwQqF5LeklOu6rAqgfBZqQ== - -postcss-discard-duplicates@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-discard-duplicates/-/postcss-discard-duplicates-5.1.0.tgz#9eb4fe8456706a4eebd6d3b7b777d07bad03e848" - integrity sha512-zmX3IoSI2aoenxHV6C7plngHWWhUOV3sP1T8y2ifzxzbtnuhk1EdPwm0S1bIUNaJ2eNbWeGLEwzw8huPD67aQw== - -postcss-discard-empty@^5.1.1: - version "5.1.1" - resolved "https://registry.yarnpkg.com/postcss-discard-empty/-/postcss-discard-empty-5.1.1.tgz#e57762343ff7f503fe53fca553d18d7f0c369c6c" - integrity sha512-zPz4WljiSuLWsI0ir4Mcnr4qQQ5e1Ukc3i7UfE2XcrwKK2LIPIqE5jxMRxO6GbI3cv//ztXDsXwEWT3BHOGh3A== - -postcss-discard-overridden@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-discard-overridden/-/postcss-discard-overridden-5.1.0.tgz#7e8c5b53325747e9d90131bb88635282fb4a276e" - integrity sha512-21nOL7RqWR1kasIVdKs8HNqQJhFxLsyRfAnUDm4Fe4t4mCWL9OJiHvlHPjcd8zc5Myu89b/7wZDnOSjFgeWRtw== - -postcss-merge-longhand@^5.1.5: - version "5.1.5" - resolved "https://registry.yarnpkg.com/postcss-merge-longhand/-/postcss-merge-longhand-5.1.5.tgz#b0e03bee3b964336f5f33c4fc8eacae608e91c05" - integrity sha512-NOG1grw9wIO+60arKa2YYsrbgvP6tp+jqc7+ZD5/MalIw234ooH2C6KlR6FEn4yle7GqZoBxSK1mLBE9KPur6w== - dependencies: - postcss-value-parser "^4.2.0" - stylehacks "^5.1.0" - -postcss-merge-rules@^5.1.2: - version "5.1.2" - resolved "https://registry.yarnpkg.com/postcss-merge-rules/-/postcss-merge-rules-5.1.2.tgz#7049a14d4211045412116d79b751def4484473a5" - integrity sha512-zKMUlnw+zYCWoPN6yhPjtcEdlJaMUZ0WyVcxTAmw3lkkN/NDMRkOkiuctQEoWAOvH7twaxUUdvBWl0d4+hifRQ== - dependencies: - browserslist "^4.16.6" - caniuse-api "^3.0.0" - cssnano-utils "^3.1.0" - postcss-selector-parser "^6.0.5" - -postcss-minify-font-values@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-minify-font-values/-/postcss-minify-font-values-5.1.0.tgz#f1df0014a726083d260d3bd85d7385fb89d1f01b" - integrity sha512-el3mYTgx13ZAPPirSVsHqFzl+BBBDrXvbySvPGFnQcTI4iNslrPaFq4muTkLZmKlGk4gyFAYUBMH30+HurREyA== - dependencies: - postcss-value-parser "^4.2.0" - -postcss-minify-gradients@^5.1.1: - version "5.1.1" - resolved "https://registry.yarnpkg.com/postcss-minify-gradients/-/postcss-minify-gradients-5.1.1.tgz#f1fe1b4f498134a5068240c2f25d46fcd236ba2c" - integrity sha512-VGvXMTpCEo4qHTNSa9A0a3D+dxGFZCYwR6Jokk+/3oB6flu2/PnPXAh2x7x52EkY5xlIHLm+Le8tJxe/7TNhzw== - dependencies: - colord "^2.9.1" - cssnano-utils "^3.1.0" - postcss-value-parser "^4.2.0" - -postcss-minify-params@^5.1.3: - version "5.1.3" - resolved "https://registry.yarnpkg.com/postcss-minify-params/-/postcss-minify-params-5.1.3.tgz#ac41a6465be2db735099bbd1798d85079a6dc1f9" - integrity sha512-bkzpWcjykkqIujNL+EVEPOlLYi/eZ050oImVtHU7b4lFS82jPnsCb44gvC6pxaNt38Els3jWYDHTjHKf0koTgg== - dependencies: - browserslist "^4.16.6" - cssnano-utils "^3.1.0" - postcss-value-parser "^4.2.0" - -postcss-minify-selectors@^5.2.1: - version "5.2.1" - resolved "https://registry.yarnpkg.com/postcss-minify-selectors/-/postcss-minify-selectors-5.2.1.tgz#d4e7e6b46147b8117ea9325a915a801d5fe656c6" - integrity sha512-nPJu7OjZJTsVUmPdm2TcaiohIwxP+v8ha9NehQ2ye9szv4orirRU3SDdtUmKH+10nzn0bAyOXZ0UEr7OpvLehg== - dependencies: - postcss-selector-parser "^6.0.5" - -postcss-modules-extract-imports@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/postcss-modules-extract-imports/-/postcss-modules-extract-imports-3.0.0.tgz#cda1f047c0ae80c97dbe28c3e76a43b88025741d" - integrity sha512-bdHleFnP3kZ4NYDhuGlVK+CMrQ/pqUm8bx/oGL93K6gVwiclvX5x0n76fYMKuIGKzlABOy13zsvqjb0f92TEXw== - -postcss-modules-local-by-default@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/postcss-modules-local-by-default/-/postcss-modules-local-by-default-4.0.0.tgz#ebbb54fae1598eecfdf691a02b3ff3b390a5a51c" - integrity sha512-sT7ihtmGSF9yhm6ggikHdV0hlziDTX7oFoXtuVWeDd3hHObNkcHRo9V3yg7vCAY7cONyxJC/XXCmmiHHcvX7bQ== - dependencies: - icss-utils "^5.0.0" - postcss-selector-parser "^6.0.2" - postcss-value-parser "^4.1.0" - -postcss-modules-scope@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/postcss-modules-scope/-/postcss-modules-scope-3.0.0.tgz#9ef3151456d3bbfa120ca44898dfca6f2fa01f06" - integrity sha512-hncihwFA2yPath8oZ15PZqvWGkWf+XUfQgUGamS4LqoP1anQLOsOJw0vr7J7IwLpoY9fatA2qiGUGmuZL0Iqlg== - dependencies: - postcss-selector-parser "^6.0.4" - -postcss-modules-values@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/postcss-modules-values/-/postcss-modules-values-4.0.0.tgz#d7c5e7e68c3bb3c9b27cbf48ca0bb3ffb4602c9c" - integrity sha512-RDxHkAiEGI78gS2ofyvCsu7iycRv7oqw5xMWn9iMoR0N/7mf9D50ecQqUo5BZ9Zh2vH4bCUR/ktCqbB9m8vJjQ== - dependencies: - icss-utils "^5.0.0" - -postcss-normalize-charset@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-normalize-charset/-/postcss-normalize-charset-5.1.0.tgz#9302de0b29094b52c259e9b2cf8dc0879879f0ed" - integrity sha512-mSgUJ+pd/ldRGVx26p2wz9dNZ7ji6Pn8VWBajMXFf8jk7vUoSrZ2lt/wZR7DtlZYKesmZI680qjr2CeFF2fbUg== - -postcss-normalize-display-values@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-normalize-display-values/-/postcss-normalize-display-values-5.1.0.tgz#72abbae58081960e9edd7200fcf21ab8325c3da8" - integrity sha512-WP4KIM4o2dazQXWmFaqMmcvsKmhdINFblgSeRgn8BJ6vxaMyaJkwAzpPpuvSIoG/rmX3M+IrRZEz2H0glrQNEA== - dependencies: - postcss-value-parser "^4.2.0" - -postcss-normalize-positions@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-normalize-positions/-/postcss-normalize-positions-5.1.0.tgz#902a7cb97cf0b9e8b1b654d4a43d451e48966458" - integrity sha512-8gmItgA4H5xiUxgN/3TVvXRoJxkAWLW6f/KKhdsH03atg0cB8ilXnrB5PpSshwVu/dD2ZsRFQcR1OEmSBDAgcQ== - dependencies: - postcss-value-parser "^4.2.0" - -postcss-normalize-repeat-style@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-normalize-repeat-style/-/postcss-normalize-repeat-style-5.1.0.tgz#f6d6fd5a54f51a741cc84a37f7459e60ef7a6398" - integrity sha512-IR3uBjc+7mcWGL6CtniKNQ4Rr5fTxwkaDHwMBDGGs1x9IVRkYIT/M4NelZWkAOBdV6v3Z9S46zqaKGlyzHSchw== - dependencies: - postcss-value-parser "^4.2.0" - -postcss-normalize-string@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-normalize-string/-/postcss-normalize-string-5.1.0.tgz#411961169e07308c82c1f8c55f3e8a337757e228" - integrity sha512-oYiIJOf4T9T1N4i+abeIc7Vgm/xPCGih4bZz5Nm0/ARVJ7K6xrDlLwvwqOydvyL3RHNf8qZk6vo3aatiw/go3w== - dependencies: - postcss-value-parser "^4.2.0" - -postcss-normalize-timing-functions@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-normalize-timing-functions/-/postcss-normalize-timing-functions-5.1.0.tgz#d5614410f8f0b2388e9f240aa6011ba6f52dafbb" - integrity sha512-DOEkzJ4SAXv5xkHl0Wa9cZLF3WCBhF3o1SKVxKQAa+0pYKlueTpCgvkFAHfk+Y64ezX9+nITGrDZeVGgITJXjg== - dependencies: - postcss-value-parser "^4.2.0" - -postcss-normalize-unicode@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-normalize-unicode/-/postcss-normalize-unicode-5.1.0.tgz#3d23aede35e160089a285e27bf715de11dc9db75" - integrity sha512-J6M3MizAAZ2dOdSjy2caayJLQT8E8K9XjLce8AUQMwOrCvjCHv24aLC/Lps1R1ylOfol5VIDMaM/Lo9NGlk1SQ== - dependencies: - browserslist "^4.16.6" - postcss-value-parser "^4.2.0" - -postcss-normalize-url@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-normalize-url/-/postcss-normalize-url-5.1.0.tgz#ed9d88ca82e21abef99f743457d3729a042adcdc" - integrity sha512-5upGeDO+PVthOxSmds43ZeMeZfKH+/DKgGRD7TElkkyS46JXAUhMzIKiCa7BabPeIy3AQcTkXwVVN7DbqsiCew== - dependencies: - normalize-url "^6.0.1" - postcss-value-parser "^4.2.0" - -postcss-normalize-whitespace@^5.1.1: - version "5.1.1" - resolved "https://registry.yarnpkg.com/postcss-normalize-whitespace/-/postcss-normalize-whitespace-5.1.1.tgz#08a1a0d1ffa17a7cc6efe1e6c9da969cc4493cfa" - integrity sha512-83ZJ4t3NUDETIHTa3uEg6asWjSBYL5EdkVB0sDncx9ERzOKBVJIUeDO9RyA9Zwtig8El1d79HBp0JEi8wvGQnA== - dependencies: - postcss-value-parser "^4.2.0" - -postcss-ordered-values@^5.1.2: - version "5.1.2" - resolved "https://registry.yarnpkg.com/postcss-ordered-values/-/postcss-ordered-values-5.1.2.tgz#daffacd4abf327d52d5ac570b59dfbcf4b836614" - integrity sha512-wr2avRbW4HS2XE2ZCqpfp4N/tDC6GZKZ+SVP8UBTOVS8QWrc4TD8MYrebJrvVVlGPKszmiSCzue43NDiVtgDmg== - dependencies: - cssnano-utils "^3.1.0" - postcss-value-parser "^4.2.0" - -postcss-reduce-initial@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-reduce-initial/-/postcss-reduce-initial-5.1.0.tgz#fc31659ea6e85c492fb2a7b545370c215822c5d6" - integrity sha512-5OgTUviz0aeH6MtBjHfbr57tml13PuedK/Ecg8szzd4XRMbYxH4572JFG067z+FqBIf6Zp/d+0581glkvvWMFw== - dependencies: - browserslist "^4.16.6" - caniuse-api "^3.0.0" - -postcss-reduce-transforms@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-reduce-transforms/-/postcss-reduce-transforms-5.1.0.tgz#333b70e7758b802f3dd0ddfe98bb1ccfef96b6e9" - integrity sha512-2fbdbmgir5AvpW9RLtdONx1QoYG2/EtqpNQbFASDlixBbAYuTcJ0dECwlqNqH7VbaUnEnh8SrxOe2sRIn24XyQ== - dependencies: - postcss-value-parser "^4.2.0" - -postcss-resolve-nested-selector@^0.1.1: - version "0.1.1" - resolved "https://registry.yarnpkg.com/postcss-resolve-nested-selector/-/postcss-resolve-nested-selector-0.1.1.tgz#29ccbc7c37dedfac304e9fff0bf1596b3f6a0e4e" - integrity sha1-Kcy8fDfe36wwTp//C/FZaz9qDk4= - -postcss-safe-parser@^6.0.0: - version "6.0.0" - resolved "https://registry.yarnpkg.com/postcss-safe-parser/-/postcss-safe-parser-6.0.0.tgz#bb4c29894171a94bc5c996b9a30317ef402adaa1" - integrity sha512-FARHN8pwH+WiS2OPCxJI8FuRJpTVnn6ZNFiqAM2aeW2LwTHWWmWgIyKC6cUo0L8aeKiF/14MNvnpls6R2PBeMQ== - -postcss-selector-parser@^6.0.13, postcss-selector-parser@^6.0.2, postcss-selector-parser@^6.0.4, postcss-selector-parser@^6.0.5, postcss-selector-parser@^6.0.9: - version "6.0.13" - resolved "https://registry.yarnpkg.com/postcss-selector-parser/-/postcss-selector-parser-6.0.13.tgz#d05d8d76b1e8e173257ef9d60b706a8e5e99bf1b" - integrity sha512-EaV1Gl4mUEV4ddhDnv/xtj7sxwrwxdetHdWUGnT4VJQf+4d05v6lHYZr8N573k5Z0BViss7BDhfWtKS3+sfAqQ== - dependencies: - cssesc "^3.0.0" - util-deprecate "^1.0.2" - -postcss-svgo@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/postcss-svgo/-/postcss-svgo-5.1.0.tgz#0a317400ced789f233a28826e77523f15857d80d" - integrity sha512-D75KsH1zm5ZrHyxPakAxJWtkyXew5qwS70v56exwvw542d9CRtTo78K0WeFxZB4G7JXKKMbEZtZayTGdIky/eA== - dependencies: - postcss-value-parser "^4.2.0" - svgo "^2.7.0" - -postcss-unique-selectors@^5.1.1: - version "5.1.1" - resolved "https://registry.yarnpkg.com/postcss-unique-selectors/-/postcss-unique-selectors-5.1.1.tgz#a9f273d1eacd09e9aa6088f4b0507b18b1b541b6" - integrity sha512-5JiODlELrz8L2HwxfPnhOWZYWDxVHWL83ufOv84NrcgipI7TaeRsatAhK4Tr2/ZiYldpK/wBvw5BD3qfaK96GA== - dependencies: - postcss-selector-parser "^6.0.5" - -postcss-value-parser@^4.1.0, postcss-value-parser@^4.2.0: - version "4.2.0" - resolved "https://registry.yarnpkg.com/postcss-value-parser/-/postcss-value-parser-4.2.0.tgz#723c09920836ba6d3e5af019f92bc0971c02e514" - integrity sha512-1NNCs6uurfkVbeXG4S8JFT9t19m45ICnif8zWLd5oPSZ50QnwMfK+H3jv408d4jw/7Bttv5axS5IiHoLaVNHeQ== - -postcss@^8.2.15, postcss@^8.4.13, postcss@^8.4.24: - version "8.4.33" - resolved "https://registry.yarnpkg.com/postcss/-/postcss-8.4.33.tgz#1378e859c9f69bf6f638b990a0212f43e2aaa742" - integrity sha512-Kkpbhhdjw2qQs2O2DGX+8m5OVqEcbB9HRBvuYM9pgrjEFUg30A9LmXNlTAUj4S9kgtGyrMbTzVjH7E+s5Re2yg== - dependencies: - nanoid "^3.3.7" - picocolors "^1.0.0" - source-map-js "^1.0.2" - -prelude-ls@^1.2.1: - version "1.2.1" - resolved "https://registry.yarnpkg.com/prelude-ls/-/prelude-ls-1.2.1.tgz#debc6489d7a6e6b0e7611888cec880337d316396" - integrity sha512-vkcDPrRZo1QZLbn5RLGPpg/WmIQ65qoWWhcGKf/b5eplkkarX0m9z8ppCat4mlOqUsWpyNuYgO3VRyrYHSzX5g== - -prelude-ls@~1.1.2: - version "1.1.2" - resolved "https://registry.yarnpkg.com/prelude-ls/-/prelude-ls-1.1.2.tgz#21932a549f5e52ffd9a827f570e04be62a97da54" - integrity sha1-IZMqVJ9eUv/ZqCf1cOBL5iqX2lQ= - -prettier@^2.6.2: - version "2.7.1" - resolved "https://registry.yarnpkg.com/prettier/-/prettier-2.7.1.tgz#e235806850d057f97bb08368a4f7d899f7760c64" - integrity sha512-ujppO+MkdPqoVINuDFDRLClm7D78qbDt0/NR+wp5FqEZOoTNAjPHWj17QRhu7geIHJfcNhRk1XVQmF8Bp3ye+g== - -prettier@^2.8.4: - version "2.8.4" - resolved "https://registry.yarnpkg.com/prettier/-/prettier-2.8.4.tgz#34dd2595629bfbb79d344ac4a91ff948694463c3" - integrity sha512-vIS4Rlc2FNh0BySk3Wkd6xmwxB0FpOndW5fisM5H8hsZSxU2VWVB5CWIkIjWvrHjIhxk2g3bfMKM87zNTrZddw== - -pretty-format@^27.0.0, pretty-format@^27.0.2, pretty-format@^27.3.1: - version "27.3.1" - resolved "https://registry.yarnpkg.com/pretty-format/-/pretty-format-27.3.1.tgz#7e9486365ccdd4a502061fa761d3ab9ca1b78df5" - integrity sha512-DR/c+pvFc52nLimLROYjnXPtolawm+uWDxr4FjuLDLUn+ktWnSN851KoHwHzzqq6rfCOjkzN8FLgDrSub6UDuA== - dependencies: - "@jest/types" "^27.2.5" - ansi-regex "^5.0.1" - ansi-styles "^5.0.0" - react-is "^17.0.1" - -pretty-format@^27.5.1: - version "27.5.1" - resolved "https://registry.yarnpkg.com/pretty-format/-/pretty-format-27.5.1.tgz#2181879fdea51a7a5851fb39d920faa63f01d88e" - integrity sha512-Qb1gy5OrP5+zDf2Bvnzdl3jsTf1qXVMazbvCoKhtKqVs4/YK4ozX4gKQJJVyNe+cajNPn0KoC0MC3FUmaHWEmQ== - dependencies: - ansi-regex "^5.0.1" - ansi-styles "^5.0.0" - react-is "^17.0.1" - -prismjs@^1.27.0: - version "1.28.0" - resolved "https://registry.yarnpkg.com/prismjs/-/prismjs-1.28.0.tgz#0d8f561fa0f7cf6ebca901747828b149147044b6" - integrity sha512-8aaXdYvl1F7iC7Xm1spqSaY/OJBpYW3v+KJ+F17iYxvdc8sfjW194COK5wVhMZX45tGteiBQgdvD/nhxcRwylw== - -prismjs@~1.27.0: - version "1.27.0" - resolved "https://registry.yarnpkg.com/prismjs/-/prismjs-1.27.0.tgz#bb6ee3138a0b438a3653dd4d6ce0cc6510a45057" - integrity sha512-t13BGPUlFDR7wRB5kQDG4jjl7XeuH6jbJGt11JHPL96qwsEHNX2+68tFXqc1/k+/jALsbSWJKUOT/hcYAZ5LkA== - -promise-inflight@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/promise-inflight/-/promise-inflight-1.0.1.tgz#98472870bf228132fcbdd868129bad12c3c029e3" - integrity sha1-mEcocL8igTL8vdhoEputEsPAKeM= - -promise@^7.1.1: - version "7.3.1" - resolved "https://registry.yarnpkg.com/promise/-/promise-7.3.1.tgz#064b72602b18f90f29192b8b1bc418ffd1ebd3bf" - integrity sha512-nolQXZ/4L+bP/UGlkfaIujX9BKxGwmQ9OT4mOt5yvy8iK1h3wqTEJCijzGANTCCl9nWjY41juyAn2K3Q1hLLTg== - dependencies: - asap "~2.0.3" - -prompts@^2.0.1: - version "2.4.2" - resolved "https://registry.yarnpkg.com/prompts/-/prompts-2.4.2.tgz#7b57e73b3a48029ad10ebd44f74b01722a4cb069" - integrity sha512-NxNv/kLguCA7p3jE8oL2aEBsrJWgAakBpgmgK6lpPWV+WuOmY6r2/zbAVnP+T8bQlA0nzHXSJSJW0Hq7ylaD2Q== - dependencies: - kleur "^3.0.3" - sisteransi "^1.0.5" - -prop-types@^15.0.0, prop-types@^15.5.10, prop-types@^15.6.0, prop-types@^15.6.2, prop-types@^15.7.2, prop-types@^15.8.1: - version "15.8.1" - resolved "https://registry.yarnpkg.com/prop-types/-/prop-types-15.8.1.tgz#67d87bf1a694f48435cf332c24af10214a3140b5" - integrity sha512-oj87CgZICdulUohogVAR7AjlC0327U4el4L6eAvOqCeudMDVU0NThNaV+b9Df4dXgSP1gXMTnPdhfe/2qDH5cg== - dependencies: - loose-envify "^1.4.0" - object-assign "^4.1.1" - react-is "^16.13.1" - -prop-types@^15.5.0: - version "15.7.2" - resolved "https://registry.yarnpkg.com/prop-types/-/prop-types-15.7.2.tgz#52c41e75b8c87e72b9d9360e0206b99dcbffa6c5" - integrity sha512-8QQikdH7//R2vurIJSutZ1smHYTcLpRWEOlHnzcWHmBYrOGUysKwSsrC89BCiFj3CbrfJ/nXFdJepOVrY1GCHQ== - dependencies: - loose-envify "^1.4.0" - object-assign "^4.1.1" - react-is "^16.8.1" - -property-information@^5.0.0: - version "5.6.0" - resolved "https://registry.yarnpkg.com/property-information/-/property-information-5.6.0.tgz#61675545fb23002f245c6540ec46077d4da3ed69" - integrity sha512-YUHSPk+A30YPv+0Qf8i9Mbfe/C0hdPXk1s1jPVToV8pk8BQtpw10ct89Eo7OWkutrwqvT0eicAxlOg3dOAu8JA== - dependencies: - xtend "^4.0.0" - -property-information@^6.0.0: - version "6.2.0" - resolved "https://registry.yarnpkg.com/property-information/-/property-information-6.2.0.tgz#b74f522c31c097b5149e3c3cb8d7f3defd986a1d" - integrity sha512-kma4U7AFCTwpqq5twzC1YVIDXSqg6qQK6JN0smOw8fgRy1OkMi0CYSzFmsy6dnqSenamAtj0CyXMUJ1Mf6oROg== - -proxy-from-env@^1.1.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/proxy-from-env/-/proxy-from-env-1.1.0.tgz#e102f16ca355424865755d2c9e8ea4f24d58c3e2" - integrity sha512-D+zkORCbA9f1tdWRK0RaCR3GPv50cMxcrz4X8k5LTSUD1Dkw47mKJEZQNunItRTkWwgtaUSo1RVFRIG9ZXiFYg== - -psl@^1.1.33: - version "1.8.0" - resolved "https://registry.yarnpkg.com/psl/-/psl-1.8.0.tgz#9326f8bcfb013adcc005fdff056acce020e51c24" - integrity sha512-RIdOzyoavK+hA18OGGWDqUTsCLhtA7IcZ/6NCs4fFJaHBDab+pDDmDIByWFRQJq2Cd7r1OoQxBGKOaztq+hjIQ== - -punycode@^2.1.0, punycode@^2.1.1: - version "2.1.1" - resolved "https://registry.yarnpkg.com/punycode/-/punycode-2.1.1.tgz#b58b010ac40c22c5657616c8d2c2c02c7bf479ec" - integrity sha512-XRsRjdf+j5ml+y/6GKHPZbrF/8p2Yga0JPtdqTIY2Xe5ohJPD9saDJJLPvp9+NSBprVvevdXZybnj2cv8OEd0A== - -pure-color@^1.2.0: - version "1.3.0" - resolved "https://registry.yarnpkg.com/pure-color/-/pure-color-1.3.0.tgz#1fe064fb0ac851f0de61320a8bf796836422f33e" - integrity sha512-QFADYnsVoBMw1srW7OVKEYjG+MbIa49s54w1MA1EDY6r2r/sTcKKYqRX1f4GYvnXP7eN/Pe9HFcX+hwzmrXRHA== - -querystringify@^2.1.1: - version "2.2.0" - resolved "https://registry.yarnpkg.com/querystringify/-/querystringify-2.2.0.tgz#3345941b4153cb9d082d8eee4cda2016a9aef7f6" - integrity sha512-FIqgj2EUvTa7R50u0rGsyTftzjYmv/a3hO345bZNrqabNqjtgiDMgmo4mkUjd+nzU5oF3dClKqFIPUKybUyqoQ== - -queue-microtask@^1.2.2: - version "1.2.3" - resolved "https://registry.yarnpkg.com/queue-microtask/-/queue-microtask-1.2.3.tgz#4929228bbc724dfac43e0efb058caf7b6cfb6243" - integrity sha512-NuaNSa6flKT5JaSYQzJok04JzTL1CA6aGhv5rfLW3PgqA+M2ChpZQnAC8h8i4ZFkBS8X5RqkDBHA7r4hej3K9A== - -quick-lru@^5.1.1: - version "5.1.1" - resolved "https://registry.yarnpkg.com/quick-lru/-/quick-lru-5.1.1.tgz#366493e6b3e42a3a6885e2e99d18f80fb7a8c932" - integrity sha512-WuyALRjWPDGtt/wzJiadO5AXY+8hZ80hVpe6MyivgraREW751X3SbhRvG3eLKOYN+8VEvqLcf3wdnt44Z4S4SA== - -randombytes@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/randombytes/-/randombytes-2.1.0.tgz#df6f84372f0270dc65cdf6291349ab7a473d4f2a" - integrity sha512-vYl3iOX+4CKUWuxGi9Ukhie6fsqXqS9FE2Zaic4tNFD2N2QQaXOMFbuKK4QmDHC0JO6B1Zp41J0LpT0oR68amQ== - dependencies: - safe-buffer "^5.1.0" - -react-base16-styling@^0.6.0: - version "0.6.0" - resolved "https://registry.yarnpkg.com/react-base16-styling/-/react-base16-styling-0.6.0.tgz#ef2156d66cf4139695c8a167886cb69ea660792c" - integrity sha512-yvh/7CArceR/jNATXOKDlvTnPKPmGZz7zsenQ3jUwLzHkNUR0CvY3yGYJbWJ/nnxsL8Sgmt5cO3/SILVuPO6TQ== - dependencies: - base16 "^1.0.0" - lodash.curry "^4.0.1" - lodash.flow "^3.3.0" - pure-color "^1.2.0" - -react-clientside-effect@^1.2.6: - version "1.2.6" - resolved "https://registry.yarnpkg.com/react-clientside-effect/-/react-clientside-effect-1.2.6.tgz#29f9b14e944a376b03fb650eed2a754dd128ea3a" - integrity sha512-XGGGRQAKY+q25Lz9a/4EPqom7WRjz3z9R2k4jhVKA/puQFH/5Nt27vFZYql4m4NVNdUvX8PS3O7r/Zzm7cjUlg== - dependencies: - "@babel/runtime" "^7.12.13" - -react-dom@^18.0.0: - version "18.1.0" - resolved "https://registry.yarnpkg.com/react-dom/-/react-dom-18.1.0.tgz#7f6dd84b706408adde05e1df575b3a024d7e8a2f" - integrity sha512-fU1Txz7Budmvamp7bshe4Zi32d0ll7ect+ccxNu9FlObT605GOEB8BfO4tmRJ39R5Zj831VCpvQ05QPBW5yb+w== - dependencies: - loose-envify "^1.1.0" - scheduler "^0.22.0" - -react-fast-compare@3.2.0: - version "3.2.0" - resolved "https://registry.yarnpkg.com/react-fast-compare/-/react-fast-compare-3.2.0.tgz#641a9da81b6a6320f270e89724fb45a0b39e43bb" - integrity sha512-rtGImPZ0YyLrscKI9xTpV8psd6I8VAtjKCzQDlzyDvqJA8XOW78TXYQwNRNd8g8JZnDu8q9Fu/1v4HPAVwVdHA== - -react-focus-lock@^2.9.1: - version "2.9.4" - resolved "https://registry.yarnpkg.com/react-focus-lock/-/react-focus-lock-2.9.4.tgz#4753f6dcd167c39050c9d84f9c63c71b3ff8462e" - integrity sha512-7pEdXyMseqm3kVjhdVH18sovparAzLg5h6WvIx7/Ck3ekjhrrDMEegHSa3swwC8wgfdd7DIdUVRGeiHT9/7Sgg== - dependencies: - "@babel/runtime" "^7.0.0" - focus-lock "^0.11.6" - prop-types "^15.6.2" - react-clientside-effect "^1.2.6" - use-callback-ref "^1.3.0" - use-sidecar "^1.1.2" - -react-icons@^5.2.1: - version "5.2.1" - resolved "https://registry.yarnpkg.com/react-icons/-/react-icons-5.2.1.tgz#28c2040917b2a2eda639b0f797bff1888e018e4a" - integrity sha512-zdbW5GstTzXaVKvGSyTaBalt7HSfuK5ovrzlpyiWHAFXndXTdd/1hdDHI4xBM1Mn7YriT6aqESucFl9kEXzrdw== - -react-is@^16.13.1, react-is@^16.7.0, react-is@^16.8.1: - version "16.13.1" - resolved "https://registry.yarnpkg.com/react-is/-/react-is-16.13.1.tgz#789729a4dc36de2999dc156dd6c1d9c18cea56a4" - integrity sha512-24e6ynE2H+OKt4kqsOvNd8kBpV65zoxbA4BVsEOB3ARVWQki/DHzaUoC5KuON/BiccDaCCTZBuOcfZs70kR8bQ== - -react-is@^17.0.1: - version "17.0.2" - resolved "https://registry.yarnpkg.com/react-is/-/react-is-17.0.2.tgz#e691d4a8e9c789365655539ab372762b0efb54f0" - integrity sha512-w2GsyukL62IJnlaff/nRegPQR94C/XXamvMWmSHRJ4y7Ts/4ocGRmTHvOs8PSE6pB3dWOrD/nueuU5sduBsQ4w== - -react-is@^18.0.0: - version "18.2.0" - resolved "https://registry.yarnpkg.com/react-is/-/react-is-18.2.0.tgz#199431eeaaa2e09f86427efbb4f1473edb47609b" - integrity sha512-xWGDIW6x921xtzPkhiULtthJHoJvBbF3q26fzloPCK0hsvxtPVelvftw3zjbHWSkR2km9Z+4uxbDDK/6Zw9B8w== - -react-json-view@^1.21.3: - version "1.21.3" - resolved "https://registry.yarnpkg.com/react-json-view/-/react-json-view-1.21.3.tgz#f184209ee8f1bf374fb0c41b0813cff54549c475" - integrity sha512-13p8IREj9/x/Ye4WI/JpjhoIwuzEgUAtgJZNBJckfzJt1qyh24BdTm6UQNGnyTq9dapQdrqvquZTo3dz1X6Cjw== - dependencies: - flux "^4.0.1" - react-base16-styling "^0.6.0" - react-lifecycles-compat "^3.0.4" - react-textarea-autosize "^8.3.2" - -react-lifecycles-compat@^3.0.4: - version "3.0.4" - resolved "https://registry.yarnpkg.com/react-lifecycles-compat/-/react-lifecycles-compat-3.0.4.tgz#4f1a273afdfc8f3488a8c516bfda78f872352362" - integrity sha512-fBASbA6LnOU9dOU2eW7aQ8xmYBSXUIWr+UmF9b1efZBazGNO+rcXT/icdKnYm2pTwcRylVUYwW7H1PHfLekVzA== - -react-markdown@^8.0.4: - version "8.0.4" - resolved "https://registry.yarnpkg.com/react-markdown/-/react-markdown-8.0.4.tgz#b5ff1f0f29ead71a7a6f98815eb1a70bcc2a036e" - integrity sha512-2oxHa6oDxc1apg/Gnc1Goh06t3B617xeywqI/92wmDV9FELI6ayRkwge7w7DoEqM0gRpZGTNU6xQG+YpJISnVg== - dependencies: - "@types/hast" "^2.0.0" - "@types/prop-types" "^15.0.0" - "@types/unist" "^2.0.0" - comma-separated-tokens "^2.0.0" - hast-util-whitespace "^2.0.0" - prop-types "^15.0.0" - property-information "^6.0.0" - react-is "^18.0.0" - remark-parse "^10.0.0" - remark-rehype "^10.0.0" - space-separated-tokens "^2.0.0" - style-to-object "^0.3.0" - unified "^10.0.0" - unist-util-visit "^4.0.0" - vfile "^5.0.0" - -react-query@^3.39.1: - version "3.39.1" - resolved "https://registry.yarnpkg.com/react-query/-/react-query-3.39.1.tgz#3876c0fdac7a3b5a84e195534e5fa8fbdd628847" - integrity sha512-qYKT1bavdDiQZbngWZyPotlBVzcBjDYEJg5RQLBa++5Ix5jjfbEYJmHSZRZD+USVHUSvl/ey9Hu+QfF1QAK80A== - dependencies: - "@babel/runtime" "^7.5.5" - broadcast-channel "^3.4.1" - match-sorter "^6.0.2" - -react-remove-scroll-bar@^2.3.3: - version "2.3.3" - resolved "https://registry.yarnpkg.com/react-remove-scroll-bar/-/react-remove-scroll-bar-2.3.3.tgz#e291f71b1bb30f5f67f023765b7435f4b2b2cd94" - integrity sha512-i9GMNWwpz8XpUpQ6QlevUtFjHGqnPG4Hxs+wlIJntu/xcsZVEpJcIV71K3ZkqNy2q3GfgvkD7y6t/Sv8ofYSbw== - dependencies: - react-style-singleton "^2.2.1" - tslib "^2.0.0" - -react-remove-scroll@^2.5.4: - version "2.5.5" - resolved "https://registry.yarnpkg.com/react-remove-scroll/-/react-remove-scroll-2.5.5.tgz#1e31a1260df08887a8a0e46d09271b52b3a37e77" - integrity sha512-ImKhrzJJsyXJfBZ4bzu8Bwpka14c/fQt0k+cyFp/PBhTfyDnU5hjOtM4AG/0AMyy8oKzOTR0lDgJIM7pYXI0kw== - dependencies: - react-remove-scroll-bar "^2.3.3" - react-style-singleton "^2.2.1" - tslib "^2.1.0" - use-callback-ref "^1.3.0" - use-sidecar "^1.1.2" - -react-router-dom@^6.3.0: - version "6.3.0" - resolved "https://registry.yarnpkg.com/react-router-dom/-/react-router-dom-6.3.0.tgz#a0216da813454e521905b5fa55e0e5176123f43d" - integrity sha512-uaJj7LKytRxZNQV8+RbzJWnJ8K2nPsOOEuX7aQstlMZKQT0164C+X2w6bnkqU3sjtLvpd5ojrezAyfZ1+0sStw== - dependencies: - history "^5.2.0" - react-router "6.3.0" - -react-router@6.3.0: - version "6.3.0" - resolved "https://registry.yarnpkg.com/react-router/-/react-router-6.3.0.tgz#3970cc64b4cb4eae0c1ea5203a80334fdd175557" - integrity sha512-7Wh1DzVQ+tlFjkeo+ujvjSqSJmkt1+8JO+T5xklPlgrh70y7ogx75ODRW0ThWhY7S+6yEDks8TYrtQe/aoboBQ== - dependencies: - history "^5.2.0" - -react-select@^5.3.2: - version "5.3.2" - resolved "https://registry.yarnpkg.com/react-select/-/react-select-5.3.2.tgz#ecee0d5c59ed4acb7f567f7de3c75a488d93dacb" - integrity sha512-W6Irh7U6Ha7p5uQQ2ZnemoCQ8mcfgOtHfw3wuMzG6FAu0P+CYicgofSLOq97BhjMx8jS+h+wwWdCBeVVZ9VqlQ== - dependencies: - "@babel/runtime" "^7.12.0" - "@emotion/cache" "^11.4.0" - "@emotion/react" "^11.8.1" - "@types/react-transition-group" "^4.4.0" - memoize-one "^5.0.0" - prop-types "^15.6.0" - react-transition-group "^4.3.0" - -react-style-singleton@^2.2.1: - version "2.2.1" - resolved "https://registry.yarnpkg.com/react-style-singleton/-/react-style-singleton-2.2.1.tgz#f99e420492b2d8f34d38308ff660b60d0b1205b4" - integrity sha512-ZWj0fHEMyWkHzKYUr2Bs/4zU6XLmq9HsgBURm7g5pAVfyn49DgUiNgY2d4lXRlYSiCif9YBGpQleewkcqddc7g== - dependencies: - get-nonce "^1.0.0" - invariant "^2.2.4" - tslib "^2.0.0" - -react-syntax-highlighter@^15.5.0: - version "15.5.0" - resolved "https://registry.yarnpkg.com/react-syntax-highlighter/-/react-syntax-highlighter-15.5.0.tgz#4b3eccc2325fa2ec8eff1e2d6c18fa4a9e07ab20" - integrity sha512-+zq2myprEnQmH5yw6Gqc8lD55QHnpKaU8TOcFeC/Lg/MQSs8UknEA0JC4nTZGFAXC2J2Hyj/ijJ7NlabyPi2gg== - dependencies: - "@babel/runtime" "^7.3.1" - highlight.js "^10.4.1" - lowlight "^1.17.0" - prismjs "^1.27.0" - refractor "^3.6.0" - -react-table@^7.8.0: - version "7.8.0" - resolved "https://registry.yarnpkg.com/react-table/-/react-table-7.8.0.tgz#07858c01c1718c09f7f1aed7034fcfd7bda907d2" - integrity sha512-hNaz4ygkZO4bESeFfnfOft73iBUj8K5oKi1EcSHPAibEydfsX2MyU6Z8KCr3mv3C9Kqqh71U+DhZkFvibbnPbA== - -react-tabs@^3.2.2: - version "3.2.2" - resolved "https://registry.yarnpkg.com/react-tabs/-/react-tabs-3.2.2.tgz#07bdc3cdb17bdffedd02627f32a93cd4b3d6e4d0" - integrity sha512-/o52eGKxFHRa+ssuTEgSM8qORnV4+k7ibW+aNQzKe+5gifeVz8nLxCrsI9xdRhfb0wCLdgIambIpb1qCxaMN+A== - dependencies: - clsx "^1.1.0" - prop-types "^15.5.0" - -react-textarea-autosize@^8.3.2, react-textarea-autosize@^8.3.4: - version "8.3.4" - resolved "https://registry.yarnpkg.com/react-textarea-autosize/-/react-textarea-autosize-8.3.4.tgz#270a343de7ad350534141b02c9cb78903e553524" - integrity sha512-CdtmP8Dc19xL8/R6sWvtknD/eCXkQr30dtvC4VmGInhRsfF8X/ihXCq6+9l9qbxmKRiq407/7z5fxE7cVWQNgQ== - dependencies: - "@babel/runtime" "^7.10.2" - use-composed-ref "^1.3.0" - use-latest "^1.2.1" - -react-transition-group@^4.3.0: - version "4.4.2" - resolved "https://registry.yarnpkg.com/react-transition-group/-/react-transition-group-4.4.2.tgz#8b59a56f09ced7b55cbd53c36768b922890d5470" - integrity sha512-/RNYfRAMlZwDSr6z4zNKV6xu53/e2BuaBbGhbyYIXTrmgu/bGHzmqOs7mJSJBHy9Ud+ApHx3QjrkKSp1pxvlFg== - dependencies: - "@babel/runtime" "^7.5.5" - dom-helpers "^5.0.1" - loose-envify "^1.4.0" - prop-types "^15.6.2" - -react@^18.0.0: - version "18.1.0" - resolved "https://registry.yarnpkg.com/react/-/react-18.1.0.tgz#6f8620382decb17fdc5cc223a115e2adbf104890" - integrity sha512-4oL8ivCz5ZEPyclFQXaNksK3adutVS8l2xzZU0cqEFrE9Sb7fC0EFK5uEk74wIreL1DERyjvsU915j1pcT2uEQ== - dependencies: - loose-envify "^1.1.0" - -reactflow@^11.7.4: - version "11.7.4" - resolved "https://registry.yarnpkg.com/reactflow/-/reactflow-11.7.4.tgz#b00159c3471d007bc4865b23005c636b1f08ab26" - integrity sha512-QI6+oc1Ft6oFeLSdHlp+SmgymbI5Tm49wj5JyE84O4A54yN/ImfYaBhLit9Cmfzxn9Tz6tDqmGMGbk4bdtB8/w== - dependencies: - "@reactflow/background" "11.2.4" - "@reactflow/controls" "11.1.15" - "@reactflow/core" "11.7.4" - "@reactflow/minimap" "11.5.4" - "@reactflow/node-resizer" "2.1.1" - "@reactflow/node-toolbar" "1.2.3" - -read-pkg-up@^8.0.0: - version "8.0.0" - resolved "https://registry.yarnpkg.com/read-pkg-up/-/read-pkg-up-8.0.0.tgz#72f595b65e66110f43b052dd9af4de6b10534670" - integrity sha512-snVCqPczksT0HS2EC+SxUndvSzn6LRCwpfSvLrIfR5BKDQQZMaI6jPRC9dYvYFDRAuFEAnkwww8kBBNE/3VvzQ== - dependencies: - find-up "^5.0.0" - read-pkg "^6.0.0" - type-fest "^1.0.1" - -read-pkg@^6.0.0: - version "6.0.0" - resolved "https://registry.yarnpkg.com/read-pkg/-/read-pkg-6.0.0.tgz#a67a7d6a1c2b0c3cd6aa2ea521f40c458a4a504c" - integrity sha512-X1Fu3dPuk/8ZLsMhEj5f4wFAF0DWoK7qhGJvgaijocXxBmSToKfbFtqbxMO7bVjNA1dmE5huAzjXj/ey86iw9Q== - dependencies: - "@types/normalize-package-data" "^2.4.0" - normalize-package-data "^3.0.2" - parse-json "^5.2.0" - type-fest "^1.0.1" - -readable-stream@1.1: - version "1.1.13" - resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-1.1.13.tgz#f6eef764f514c89e2b9e23146a75ba106756d23e" - integrity sha1-9u73ZPUUyJ4rniMUanW6EGdW0j4= - dependencies: - core-util-is "~1.0.0" - inherits "~2.0.1" - isarray "0.0.1" - string_decoder "~0.10.x" - -rechoir@^0.7.0: - version "0.7.1" - resolved "https://registry.yarnpkg.com/rechoir/-/rechoir-0.7.1.tgz#9478a96a1ca135b5e88fc027f03ee92d6c645686" - integrity sha512-/njmZ8s1wVeR6pjTZ+0nCnv8SpZNRMT2D1RLOJQESlYFDBvwpTA4KWJpZ+sBJ4+vhjILRcK7JIFdGCdxEAAitg== - dependencies: - resolve "^1.9.0" - -redent@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/redent/-/redent-3.0.0.tgz#e557b7998316bb53c9f1f56fa626352c6963059f" - integrity sha512-6tDA8g98We0zd0GvVeMT9arEOnTw9qM03L9cJXaCjrip1OO764RDBLBfrB4cwzNGDj5OA5ioymC9GkizgWJDUg== - dependencies: - indent-string "^4.0.0" - strip-indent "^3.0.0" - -redent@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/redent/-/redent-4.0.0.tgz#0c0ba7caabb24257ab3bb7a4fd95dd1d5c5681f9" - integrity sha512-tYkDkVVtYkSVhuQ4zBgfvciymHaeuel+zFKXShfDnFP5SyVEP7qo70Rf1jTOTCx3vGNAbnEi/xFkcfQVMIBWag== - dependencies: - indent-string "^5.0.0" - strip-indent "^4.0.0" - -redoc@^2.0.0-rc.72: - version "2.0.0-rc.72" - resolved "https://registry.yarnpkg.com/redoc/-/redoc-2.0.0-rc.72.tgz#9eee22104d652b4a90e19ca50009b0b623a7b5b3" - integrity sha512-IX/WvVh4N3zwo4sAjnQFz6ffIUd6G47hcflxPtrpxblJaeOy0MBSzzY8f179WjssWPYcSmmndP5v0hgEXFiimg== - dependencies: - "@redocly/openapi-core" "^1.0.0-beta.97" - classnames "^2.3.1" - decko "^1.2.0" - dompurify "^2.2.8" - eventemitter3 "^4.0.7" - json-pointer "^0.6.2" - lunr "^2.3.9" - mark.js "^8.11.1" - marked "^4.0.15" - mobx-react "^7.2.0" - openapi-sampler "^1.3.0" - path-browserify "^1.0.1" - perfect-scrollbar "^1.5.1" - polished "^4.1.3" - prismjs "^1.27.0" - prop-types "^15.7.2" - react-tabs "^3.2.2" - slugify "~1.4.7" - stickyfill "^1.1.1" - style-loader "^3.3.1" - swagger2openapi "^7.0.6" - url-template "^2.0.8" - -refractor@^3.6.0: - version "3.6.0" - resolved "https://registry.yarnpkg.com/refractor/-/refractor-3.6.0.tgz#ac318f5a0715ead790fcfb0c71f4dd83d977935a" - integrity sha512-MY9W41IOWxxk31o+YvFCNyNzdkc9M20NoZK5vq6jkv4I/uh2zkWcfudj0Q1fovjUQJrNewS9NMzeTtqPf+n5EA== - dependencies: - hastscript "^6.0.0" - parse-entities "^2.0.0" - prismjs "~1.27.0" - -reftools@^1.1.8: - version "1.1.8" - resolved "https://registry.yarnpkg.com/reftools/-/reftools-1.1.8.tgz#cc08fd67eb913d779fd330657d010cc080c7d643" - integrity sha512-Yvz9NH8uFHzD/AXX82Li1GdAP6FzDBxEZw+njerNBBQv/XHihqsWAjNfXtaq4QD2l4TEZVnp4UbktdYSegAM3g== - -regenerate-unicode-properties@^10.1.0: - version "10.1.1" - resolved "https://registry.yarnpkg.com/regenerate-unicode-properties/-/regenerate-unicode-properties-10.1.1.tgz#6b0e05489d9076b04c436f318d9b067bba459480" - integrity sha512-X007RyZLsCJVVrjgEFVpLUTZwyOZk3oiL75ZcuYjlIWd6rNJtOjkBwQc5AsRrpbKVkxN6sklw/k/9m2jJYOf8Q== - dependencies: - regenerate "^1.4.2" - -regenerate@^1.4.2: - version "1.4.2" - resolved "https://registry.yarnpkg.com/regenerate/-/regenerate-1.4.2.tgz#b9346d8827e8f5a32f7ba29637d398b69014848a" - integrity sha512-zrceR/XhGYU/d/opr2EKO7aRHUeiBI8qjtfHqADTwZd6Szfy16la6kqD0MIUs5z5hx6AaKa+PixpPrR289+I0A== - -regenerator-runtime@^0.13.11: - version "0.13.11" - resolved "https://registry.yarnpkg.com/regenerator-runtime/-/regenerator-runtime-0.13.11.tgz#f6dca3e7ceec20590d07ada785636a90cdca17f9" - integrity sha512-kY1AZVr2Ra+t+piVaJ4gxaFaReZVH40AKNo7UCX6W+dEwBo/2oZJzqfuN1qLq1oL45o56cPaTXELwrTh8Fpggg== - -regenerator-runtime@^0.13.4: - version "0.13.9" - resolved "https://registry.yarnpkg.com/regenerator-runtime/-/regenerator-runtime-0.13.9.tgz#8925742a98ffd90814988d7566ad30ca3b263b52" - integrity sha512-p3VT+cOEgxFsRRA9X4lkI1E+k2/CtnKtU4gcxyaCUreilL/vqI6CdZ3wxVUx3UOUg+gnUOQQcRI7BmSI656MYA== - -regenerator-transform@^0.15.2: - version "0.15.2" - resolved "https://registry.yarnpkg.com/regenerator-transform/-/regenerator-transform-0.15.2.tgz#5bbae58b522098ebdf09bca2f83838929001c7a4" - integrity sha512-hfMp2BoF0qOk3uc5V20ALGDS2ddjQaLrdl7xrGXvAIow7qeWRM2VA2HuCHkUKk9slq3VwEwLNK3DFBqDfPGYtg== - dependencies: - "@babel/runtime" "^7.8.4" - -regexp.prototype.flags@^1.4.1, regexp.prototype.flags@^1.4.3: - version "1.4.3" - resolved "https://registry.yarnpkg.com/regexp.prototype.flags/-/regexp.prototype.flags-1.4.3.tgz#87cab30f80f66660181a3bb7bf5981a872b367ac" - integrity sha512-fjggEOO3slI6Wvgjwflkc4NFRCTZAu5CnNfBd5qOMYhWdn67nJBBu34/TkD++eeFmd8C9r9jfXJ27+nSiRkSUA== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.3" - functions-have-names "^1.2.2" - -regexpp@^3.0.0, regexpp@^3.2.0: - version "3.2.0" - resolved "https://registry.yarnpkg.com/regexpp/-/regexpp-3.2.0.tgz#0425a2768d8f23bad70ca4b90461fa2f1213e1b2" - integrity sha512-pq2bWo9mVD43nbts2wGv17XLiNLya+GklZ8kaDLV2Z08gDCsGpnKn9BFMepvWuHCbyVvY7J5o5+BVvoQbmlJLg== - -regexpu-core@^5.3.1: - version "5.3.2" - resolved "https://registry.yarnpkg.com/regexpu-core/-/regexpu-core-5.3.2.tgz#11a2b06884f3527aec3e93dbbf4a3b958a95546b" - integrity sha512-RAM5FlZz+Lhmo7db9L298p2vHP5ZywrVXmVXpmAD9GuL5MPH6t9ROw1iA/wfHkQ76Qe7AaPF0nGuim96/IrQMQ== - dependencies: - "@babel/regjsgen" "^0.8.0" - regenerate "^1.4.2" - regenerate-unicode-properties "^10.1.0" - regjsparser "^0.9.1" - unicode-match-property-ecmascript "^2.0.0" - unicode-match-property-value-ecmascript "^2.1.0" - -regjsparser@^0.9.1: - version "0.9.1" - resolved "https://registry.yarnpkg.com/regjsparser/-/regjsparser-0.9.1.tgz#272d05aa10c7c1f67095b1ff0addae8442fc5709" - integrity sha512-dQUtn90WanSNl+7mQKcXAgZxvUe7Z0SqXlgzv0za4LwiUhyzBC58yQO3liFoUgu8GiJVInAhJjkj1N0EtQ5nkQ== - dependencies: - jsesc "~0.5.0" - -remark-gfm@^3.0.1: - version "3.0.1" - resolved "https://registry.yarnpkg.com/remark-gfm/-/remark-gfm-3.0.1.tgz#0b180f095e3036545e9dddac0e8df3fa5cfee54f" - integrity sha512-lEFDoi2PICJyNrACFOfDD3JlLkuSbOa5Wd8EPt06HUdptv8Gn0bxYTdbU/XXQ3swAPkEaGxxPN9cbnMHvVu1Ig== - dependencies: - "@types/mdast" "^3.0.0" - mdast-util-gfm "^2.0.0" - micromark-extension-gfm "^2.0.0" - unified "^10.0.0" - -remark-parse@^10.0.0: - version "10.0.1" - resolved "https://registry.yarnpkg.com/remark-parse/-/remark-parse-10.0.1.tgz#6f60ae53edbf0cf38ea223fe643db64d112e0775" - integrity sha512-1fUyHr2jLsVOkhbvPRBJ5zTKZZyD6yZzYaWCS6BPBdQ8vEMBCH+9zNCDA6tET/zHCi/jLqjCWtlJZUPk+DbnFw== - dependencies: - "@types/mdast" "^3.0.0" - mdast-util-from-markdown "^1.0.0" - unified "^10.0.0" - -remark-rehype@^10.0.0: - version "10.1.0" - resolved "https://registry.yarnpkg.com/remark-rehype/-/remark-rehype-10.1.0.tgz#32dc99d2034c27ecaf2e0150d22a6dcccd9a6279" - integrity sha512-EFmR5zppdBp0WQeDVZ/b66CWJipB2q2VLNFMabzDSGR66Z2fQii83G5gTBbgGEnEEA0QRussvrFHxk1HWGJskw== - dependencies: - "@types/hast" "^2.0.0" - "@types/mdast" "^3.0.0" - mdast-util-to-hast "^12.1.0" - unified "^10.0.0" - -remedial@^1.0.7: - version "1.0.8" - resolved "https://registry.yarnpkg.com/remedial/-/remedial-1.0.8.tgz#a5e4fd52a0e4956adbaf62da63a5a46a78c578a0" - integrity sha512-/62tYiOe6DzS5BqVsNpH/nkGlX45C/Sp6V+NtiN6JQNS1Viay7cWkazmRkrQrdFj2eshDe96SIQNIoMxqhzBOg== - -remove-accents@0.4.2: - version "0.4.2" - resolved "https://registry.yarnpkg.com/remove-accents/-/remove-accents-0.4.2.tgz#0a43d3aaae1e80db919e07ae254b285d9e1c7bb5" - integrity sha1-CkPTqq4egNuRngeuJUsoXZ4ce7U= - -remove-trailing-spaces@^1.0.6: - version "1.0.8" - resolved "https://registry.yarnpkg.com/remove-trailing-spaces/-/remove-trailing-spaces-1.0.8.tgz#4354d22f3236374702f58ee373168f6d6887ada7" - integrity sha512-O3vsMYfWighyFbTd8hk8VaSj9UAGENxAtX+//ugIst2RMk5e03h6RoIS+0ylsFxY1gvmPuAY/PO4It+gPEeySA== - -require-directory@^2.1.1: - version "2.1.1" - resolved "https://registry.yarnpkg.com/require-directory/-/require-directory-2.1.1.tgz#8c64ad5fd30dab1c976e2344ffe7f792a6a6df42" - integrity sha1-jGStX9MNqxyXbiNE/+f3kqam30I= - -require-from-string@^2.0.2: - version "2.0.2" - resolved "https://registry.yarnpkg.com/require-from-string/-/require-from-string-2.0.2.tgz#89a7fdd938261267318eafe14f9c32e598c36909" - integrity sha512-Xf0nWe6RseziFMu+Ap9biiUbmplq6S9/p+7w7YXP/JBHhrUDDUhwa+vANyubuqfZWTveU//DYVGsDG7RKL/vEw== - -requires-port@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/requires-port/-/requires-port-1.0.0.tgz#925d2601d39ac485e091cf0da5c6e694dc3dcaff" - integrity sha512-KigOCHcocU3XODJxsu8i/j8T9tzT4adHiecwORRQ0ZZFcp7ahwXuRU1m+yuO90C5ZUyGeGfocHDI14M3L3yDAQ== - -resolve-cwd@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/resolve-cwd/-/resolve-cwd-3.0.0.tgz#0f0075f1bb2544766cf73ba6a6e2adfebcb13f2d" - integrity sha512-OrZaX2Mb+rJCpH/6CpSqt9xFVpN++x01XnN2ie9g6P5/3xelLAkXWVADpdz1IHD/KFfEXyE6V0U01OQ3UO2rEg== - dependencies: - resolve-from "^5.0.0" - -resolve-from@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/resolve-from/-/resolve-from-4.0.0.tgz#4abcd852ad32dd7baabfe9b40e00a36db5f392e6" - integrity sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g== - -resolve-from@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/resolve-from/-/resolve-from-5.0.0.tgz#c35225843df8f776df21c57557bc087e9dfdfc69" - integrity sha512-qYg9KP24dD5qka9J47d0aVky0N+b4fTU89LN9iDnjB5waksiC49rvMB0PrUJQGoTmH50XPiqOvAjDfaijGxYZw== - -resolve.exports@^1.1.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/resolve.exports/-/resolve.exports-1.1.0.tgz#5ce842b94b05146c0e03076985d1d0e7e48c90c9" - integrity sha512-J1l+Zxxp4XK3LUDZ9m60LRJF/mAe4z6a4xyabPHk7pvK5t35dACV32iIjJDFeWZFfZlO29w6SZ67knR0tHzJtQ== - -resolve@^1.10.1, resolve@^1.12.0, resolve@^1.14.2, resolve@^1.20.0: - version "1.20.0" - resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.20.0.tgz#629a013fb3f70755d6f0b7935cc1c2c5378b1975" - integrity sha512-wENBPt4ySzg4ybFQW2TT1zMQucPK95HSh/nq2CFTZVOGut2+pQvSsgtda4d26YrYcr067wjbmzOG8byDPBX63A== - dependencies: - is-core-module "^2.2.0" - path-parse "^1.0.6" - -resolve@^1.22.1: - version "1.22.2" - resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.22.2.tgz#0ed0943d4e301867955766c9f3e1ae6d01c6845f" - integrity sha512-Sb+mjNHOULsBv818T40qSPeRiuWLyaGMa5ewydRLFimneixmVy2zdivRl+AF6jaYPC8ERxGDmFSiqui6SfPd+g== - dependencies: - is-core-module "^2.11.0" - path-parse "^1.0.7" - supports-preserve-symlinks-flag "^1.0.0" - -resolve@^1.9.0: - version "1.22.0" - resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.22.0.tgz#5e0b8c67c15df57a89bdbabe603a002f21731198" - integrity sha512-Hhtrw0nLeSrFQ7phPp4OOcVjLPIeMnRlr5mcnVuMe7M/7eBn98A3hmFRLoFo3DLZkivSYwhRUJTyPyWAk56WLw== - dependencies: - is-core-module "^2.8.1" - path-parse "^1.0.7" - supports-preserve-symlinks-flag "^1.0.0" - -resolve@^2.0.0-next.3: - version "2.0.0-next.3" - resolved "https://registry.yarnpkg.com/resolve/-/resolve-2.0.0-next.3.tgz#d41016293d4a8586a39ca5d9b5f15cbea1f55e46" - integrity sha512-W8LucSynKUIDu9ylraa7ueVZ7hc0uAgJBxVsQSKOXOyle8a93qXhcz+XAXZ8bIq2d6i4Ehddn6Evt+0/UwKk6Q== - dependencies: - is-core-module "^2.2.0" - path-parse "^1.0.6" - -reusify@^1.0.4: - version "1.0.4" - resolved "https://registry.yarnpkg.com/reusify/-/reusify-1.0.4.tgz#90da382b1e126efc02146e90845a88db12925d76" - integrity sha512-U9nH88a3fc/ekCF1l0/UP1IosiuIjyTh7hBvXVMHYgVcfGvt897Xguj2UOLDeI5BG2m7/uwyaLVT6fbtCwTyzw== - -rimraf@3.0.2, rimraf@^3.0.0, rimraf@^3.0.2: - version "3.0.2" - resolved "https://registry.yarnpkg.com/rimraf/-/rimraf-3.0.2.tgz#f1a5402ba6220ad52cc1282bac1ae3aa49fd061a" - integrity sha512-JZkJMZkAGFFPP2YqXZXPbMlMBgsxzE8ILs4lMIX/2o0L9UBw9O/Y3o6wFw/i9YLapcUJWwqbi3kdxIPdC62TIA== - dependencies: - glob "^7.1.3" - -rimraf@^2.6.3: - version "2.7.1" - resolved "https://registry.yarnpkg.com/rimraf/-/rimraf-2.7.1.tgz#35797f13a7fdadc566142c29d4f07ccad483e3ec" - integrity sha512-uWjbaKIK3T1OSVptzX7Nl6PvQ3qAGtKEtVRjRuazjfL3Bx5eI409VZSqgND+4UNnmzLVdPj9FqFJNPqBZFve4w== - dependencies: - glob "^7.1.3" - -run-parallel@^1.1.9: - version "1.2.0" - resolved "https://registry.yarnpkg.com/run-parallel/-/run-parallel-1.2.0.tgz#66d1368da7bdf921eb9d95bd1a9229e7f21a43ee" - integrity sha512-5l4VyZR86LZ/lDxZTR6jqL8AFE2S0IFLMP26AbjsLVADxHdhB/c0GUsH+y39UfCi3dzz8OlQuPmnaJOMoDHQBA== - dependencies: - queue-microtask "^1.2.2" - -rw@1: - version "1.3.3" - resolved "https://registry.yarnpkg.com/rw/-/rw-1.3.3.tgz#3f862dfa91ab766b14885ef4d01124bfda074fb4" - integrity sha512-PdhdWy89SiZogBLaw42zdeqtRJ//zFd2PgQavcICDUgJT5oW10QCRKbJ6bg4r0/UY2M6BWd5tkxuGFRvCkgfHQ== - -sade@^1.7.3: - version "1.8.1" - resolved "https://registry.yarnpkg.com/sade/-/sade-1.8.1.tgz#0a78e81d658d394887be57d2a409bf703a3b2701" - integrity sha512-xal3CZX1Xlo/k4ApwCFrHVACi9fBqJ7V+mwhBsuf/1IOKbBy098Fex+Wa/5QMubw09pSZ/u8EY8PWgevJsXp1A== - dependencies: - mri "^1.1.0" - -safe-buffer@^5.1.0: - version "5.2.1" - resolved "https://registry.yarnpkg.com/safe-buffer/-/safe-buffer-5.2.1.tgz#1eaf9fa9bdb1fdd4ec75f58f9cdb4e6b7827eec6" - integrity sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ== - -safe-buffer@~5.1.1: - version "5.1.2" - resolved "https://registry.yarnpkg.com/safe-buffer/-/safe-buffer-5.1.2.tgz#991ec69d296e0313747d59bdfd2b745c35f8828d" - integrity sha512-Gd2UZBJDkXlY7GbJxfsE8/nvKkUEU1G38c1siN6QP6a9PT9MmHB8GnpscSmMJSoF8LOIrt8ud/wPtojys4G6+g== - -safe-regex-test@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/safe-regex-test/-/safe-regex-test-1.0.0.tgz#793b874d524eb3640d1873aad03596db2d4f2295" - integrity sha512-JBUUzyOgEwXQY1NuPtvcj/qcBDbDmEvWufhlnXZIm75DEHp+afM1r1ujJpJsV/gSM4t59tpDyPi1sd6ZaPFfsA== - dependencies: - call-bind "^1.0.2" - get-intrinsic "^1.1.3" - is-regex "^1.1.4" - -"safer-buffer@>= 2.1.2 < 3": - version "2.1.2" - resolved "https://registry.yarnpkg.com/safer-buffer/-/safer-buffer-2.1.2.tgz#44fa161b0187b9549dd84bb91802f9bd8385cd6a" - integrity sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg== - -sax@^1.2.4: - version "1.2.4" - resolved "https://registry.yarnpkg.com/sax/-/sax-1.2.4.tgz#2816234e2378bddc4e5354fab5caa895df7100d9" - integrity sha512-NqVDv9TpANUjFm0N8uM5GxL36UgKi9/atZw+x7YFnQ8ckwFGKrl4xX4yWtrey3UJm5nP1kUbnYgLopqWNSRhWw== - -saxes@^5.0.1: - version "5.0.1" - resolved "https://registry.yarnpkg.com/saxes/-/saxes-5.0.1.tgz#eebab953fa3b7608dbe94e5dadb15c888fa6696d" - integrity sha512-5LBh1Tls8c9xgGjw3QrMwETmTMVk0oFgvrFSvWx62llR2hcEInrKNZ2GZCCuuy2lvWrdl5jhbpeqc5hRYKFOcw== - dependencies: - xmlchars "^2.2.0" - -scheduler@^0.22.0: - version "0.22.0" - resolved "https://registry.yarnpkg.com/scheduler/-/scheduler-0.22.0.tgz#83a5d63594edf074add9a7198b1bae76c3db01b8" - integrity sha512-6QAm1BgQI88NPYymgGQLCZgvep4FyePDWFpXVK+zNSUgHwlqpJy8VEh8Et0KxTACS4VWwMousBElAZOH9nkkoQ== - dependencies: - loose-envify "^1.1.0" - -schema-utils@^2.6.5, schema-utils@^2.7.0: - version "2.7.1" - resolved "https://registry.yarnpkg.com/schema-utils/-/schema-utils-2.7.1.tgz#1ca4f32d1b24c590c203b8e7a50bf0ea4cd394d7" - integrity sha512-SHiNtMOUGWBQJwzISiVYKu82GiV4QYGePp3odlY1tuKO7gPtphAT5R/py0fA6xtbgLL/RvtJZnU9b8s0F1q0Xg== - dependencies: - "@types/json-schema" "^7.0.5" - ajv "^6.12.4" - ajv-keywords "^3.5.2" - -schema-utils@^3.0.0, schema-utils@^3.1.1, schema-utils@^3.2.0: - version "3.3.0" - resolved "https://registry.yarnpkg.com/schema-utils/-/schema-utils-3.3.0.tgz#f50a88877c3c01652a15b622ae9e9795df7a60fe" - integrity sha512-pN/yOAvcC+5rQ5nERGuwrjLlYvLTbCibnZ1I7B1LaiAz9BRBlE9GMgE/eqV30P7aJQUf7Ddimy/RsbYO/GrVGg== - dependencies: - "@types/json-schema" "^7.0.8" - ajv "^6.12.5" - ajv-keywords "^3.5.2" - -schema-utils@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/schema-utils/-/schema-utils-4.0.0.tgz#60331e9e3ae78ec5d16353c467c34b3a0a1d3df7" - integrity sha512-1edyXKgh6XnJsJSQ8mKWXnN/BVaIbFMLpouRUrXgVq7WYne5kw3MW7UPhO44uRXQSIpTSXoJbmrR2X0w9kUTyg== - dependencies: - "@types/json-schema" "^7.0.9" - ajv "^8.8.0" - ajv-formats "^2.1.1" - ajv-keywords "^5.0.0" - -semver@^6.0.0, semver@^6.1.0, semver@^6.3.0, semver@^6.3.1: - version "6.3.1" - resolved "https://registry.yarnpkg.com/semver/-/semver-6.3.1.tgz#556d2ef8689146e46dcea4bfdd095f3434dffcb4" - integrity sha512-BR7VvDCVHO+q2xBEWskxS6DJE1qRnb7DxzUrogb71CWoSficBxYsiAGd+Kl0mmq/MprG9yArRkyrQxTO6XjMzA== - -semver@^7.3.2, semver@^7.3.4, semver@^7.3.5, semver@^7.3.7: - version "7.6.2" - resolved "https://registry.yarnpkg.com/semver/-/semver-7.6.2.tgz#1e3b34759f896e8f14d6134732ce798aeb0c6e13" - integrity sha512-FNAIBWCx9qcRhoHcgcJ0gvU7SN1lYU2ZXuSfl04bSC5OpvDHFyJCjdNHomPXxjQlCBU67YW64PzY7/VIEH7F2w== - -serialize-javascript@^5.0.1: - version "5.0.1" - resolved "https://registry.yarnpkg.com/serialize-javascript/-/serialize-javascript-5.0.1.tgz#7886ec848049a462467a97d3d918ebb2aaf934f4" - integrity sha512-SaaNal9imEO737H2c05Og0/8LUXG7EnsZyMa8MzkmuHoELfT6txuj0cMqRj6zfPKnmQ1yasR4PCJc8x+M4JSPA== - dependencies: - randombytes "^2.1.0" - -serialize-javascript@^6.0.0: - version "6.0.0" - resolved "https://registry.yarnpkg.com/serialize-javascript/-/serialize-javascript-6.0.0.tgz#efae5d88f45d7924141da8b5c3a7a7e663fefeb8" - integrity sha512-Qr3TosvguFt8ePWqsvRfrKyQXIiW+nGbYpy8XK24NQHE83caxWt+mIymTT19DGFbNWNLfEwsrkSmN64lVWB9ag== - dependencies: - randombytes "^2.1.0" - -serialize-javascript@^6.0.1: - version "6.0.2" - resolved "https://registry.yarnpkg.com/serialize-javascript/-/serialize-javascript-6.0.2.tgz#defa1e055c83bf6d59ea805d8da862254eb6a6c2" - integrity sha512-Saa1xPByTTq2gdeFZYLLo+RFE35NHZkAbqZeWNd3BpzppeVisAqpDjcp8dyf6uIvEqJRd46jemmyA4iFIeVk8g== - dependencies: - randombytes "^2.1.0" - -setimmediate@^1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/setimmediate/-/setimmediate-1.0.5.tgz#290cbb232e306942d7d7ea9b83732ab7856f8285" - integrity sha512-MATJdZp8sLqDl/68LfQmbP8zKPLQNV6BIZoIgrscFDQ+RsvK/BxeDQOgyxKKoh0y/8h3BqVFnCqQ/gd+reiIXA== - -shallow-clone@^3.0.0: - version "3.0.1" - resolved "https://registry.yarnpkg.com/shallow-clone/-/shallow-clone-3.0.1.tgz#8f2981ad92531f55035b01fb230769a40e02efa3" - integrity sha512-/6KqX+GVUdqPuPPd2LxDDxzX6CAbjJehAAOKlNpqqUpAqPM6HeL8f+o3a+JsyGjn2lv0WY8UsTgUJjU9Ok55NA== - dependencies: - kind-of "^6.0.2" - -shebang-command@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/shebang-command/-/shebang-command-2.0.0.tgz#ccd0af4f8835fbdc265b82461aaf0c36663f34ea" - integrity sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA== - dependencies: - shebang-regex "^3.0.0" - -shebang-regex@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/shebang-regex/-/shebang-regex-3.0.0.tgz#ae16f1644d873ecad843b0307b143362d4c42172" - integrity sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A== - -should-equal@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/should-equal/-/should-equal-2.0.0.tgz#6072cf83047360867e68e98b09d71143d04ee0c3" - integrity sha512-ZP36TMrK9euEuWQYBig9W55WPC7uo37qzAEmbjHz4gfyuXrEUgF8cUvQVO+w+d3OMfPvSRQJ22lSm8MQJ43LTA== - dependencies: - should-type "^1.4.0" - -should-format@^3.0.3: - version "3.0.3" - resolved "https://registry.yarnpkg.com/should-format/-/should-format-3.0.3.tgz#9bfc8f74fa39205c53d38c34d717303e277124f1" - integrity sha1-m/yPdPo5IFxT04w01xcwPidxJPE= - dependencies: - should-type "^1.3.0" - should-type-adaptors "^1.0.1" - -should-type-adaptors@^1.0.1: - version "1.1.0" - resolved "https://registry.yarnpkg.com/should-type-adaptors/-/should-type-adaptors-1.1.0.tgz#401e7f33b5533033944d5cd8bf2b65027792e27a" - integrity sha512-JA4hdoLnN+kebEp2Vs8eBe9g7uy0zbRo+RMcU0EsNy+R+k049Ki+N5tT5Jagst2g7EAja+euFuoXFCa8vIklfA== - dependencies: - should-type "^1.3.0" - should-util "^1.0.0" - -should-type@^1.3.0, should-type@^1.4.0: - version "1.4.0" - resolved "https://registry.yarnpkg.com/should-type/-/should-type-1.4.0.tgz#0756d8ce846dfd09843a6947719dfa0d4cff5cf3" - integrity sha1-B1bYzoRt/QmEOmlHcZ36DUz/XPM= - -should-util@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/should-util/-/should-util-1.0.1.tgz#fb0d71338f532a3a149213639e2d32cbea8bcb28" - integrity sha512-oXF8tfxx5cDk8r2kYqlkUJzZpDBqVY/II2WhvU0n9Y3XYvAYRmeaf1PvvIvTgPnv4KJ+ES5M0PyDq5Jp+Ygy2g== - -should@^13.2.1: - version "13.2.3" - resolved "https://registry.yarnpkg.com/should/-/should-13.2.3.tgz#96d8e5acf3e97b49d89b51feaa5ae8d07ef58f10" - integrity sha512-ggLesLtu2xp+ZxI+ysJTmNjh2U0TsC+rQ/pfED9bUZZ4DKefP27D+7YJVVTvKsmjLpIi9jAa7itwDGkDDmt1GQ== - dependencies: - should-equal "^2.0.0" - should-format "^3.0.3" - should-type "^1.4.0" - should-type-adaptors "^1.0.1" - should-util "^1.0.0" - -side-channel@^1.0.4: - version "1.0.4" - resolved "https://registry.yarnpkg.com/side-channel/-/side-channel-1.0.4.tgz#efce5c8fdc104ee751b25c58d4290011fa5ea2cf" - integrity sha512-q5XPytqFEIKHkGdiMIrY10mvLRvnQh42/+GoBlFW3b2LXLE2xxJpZFdm94we0BaoV3RwJyGqg5wS7epxTv0Zvw== - dependencies: - call-bind "^1.0.0" - get-intrinsic "^1.0.2" - object-inspect "^1.9.0" - -signal-exit@^3.0.2: - version "3.0.3" - resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.3.tgz#a1410c2edd8f077b08b4e253c8eacfcaf057461c" - integrity sha512-VUJ49FC8U1OxwZLxIbTTrDvLnf/6TDgxZcK8wxR8zs13xpx7xbG60ndBlhNrFi2EMuFRoeDoJO7wthSLq42EjA== - -signal-exit@^3.0.3: - version "3.0.5" - resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.5.tgz#9e3e8cc0c75a99472b44321033a7702e7738252f" - integrity sha512-KWcOiKeQj6ZyXx7zq4YxSMgHRlod4czeBQZrPb8OKcohcqAXShm7E20kEMle9WBt26hFcAf0qLOcp5zmY7kOqQ== - -signal-exit@^4.0.1: - version "4.0.2" - resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-4.0.2.tgz#ff55bb1d9ff2114c13b400688fa544ac63c36967" - integrity sha512-MY2/qGx4enyjprQnFaZsHib3Yadh3IXyV2C321GY0pjGfVBu4un0uDJkwgdxqO+Rdx8JMT8IfJIRwbYVz3Ob3Q== - -simple-swizzle@^0.2.2: - version "0.2.2" - resolved "https://registry.yarnpkg.com/simple-swizzle/-/simple-swizzle-0.2.2.tgz#a4da6b635ffcccca33f70d17cb92592de95e557a" - integrity sha512-JA//kQgZtbuY83m+xT+tXJkmJncGMTFT+C+g2h2R9uxkYIrE2yy9sgmcLhCnw57/WSD+Eh3J97FPEDFnbXnDUg== - dependencies: - is-arrayish "^0.3.1" - -sisteransi@^1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/sisteransi/-/sisteransi-1.0.5.tgz#134d681297756437cc05ca01370d3a7a571075ed" - integrity sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg== - -slash@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/slash/-/slash-3.0.0.tgz#6539be870c165adbd5240220dbe361f1bc4d4634" - integrity sha512-g9Q1haeby36OSStwb4ntCGGGaKsaVSjQ68fBxoQcutl5fS1vuY18H3wSt3jFyFtrkx+Kz0V1G85A4MyAdDMi2Q== - -slice-ansi@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/slice-ansi/-/slice-ansi-4.0.0.tgz#500e8dd0fd55b05815086255b3195adf2a45fe6b" - integrity sha512-qMCMfhY040cVHT43K9BFygqYbUPFZKHOg7K73mtTWJRb8pyP3fzf4Ixd5SzdEJQ6MRUg/WBnOLxghZtKKurENQ== - dependencies: - ansi-styles "^4.0.0" - astral-regex "^2.0.0" - is-fullwidth-code-point "^3.0.0" - -slugify@~1.4.7: - version "1.4.7" - resolved "https://registry.yarnpkg.com/slugify/-/slugify-1.4.7.tgz#e42359d505afd84a44513280868e31202a79a628" - integrity sha512-tf+h5W1IrjNm/9rKKj0JU2MDMruiopx0jjVA5zCdBtcGjfp0+c5rHw/zADLC3IeKlGHtVbHtpfzvYA0OYT+HKg== - -source-list-map@^2.0.0, source-list-map@^2.0.1: - version "2.0.1" - resolved "https://registry.yarnpkg.com/source-list-map/-/source-list-map-2.0.1.tgz#3993bd873bfc48479cca9ea3a547835c7c154b34" - integrity sha512-qnQ7gVMxGNxsiL4lEuJwe/To8UnK7fAnmbGEEH8RpLouuKbeEm0lhbQVFIrNSuB+G7tVrAlVsZgETT5nljf+Iw== - -source-map-js@^1.0.1, source-map-js@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/source-map-js/-/source-map-js-1.0.2.tgz#adbc361d9c62df380125e7f161f71c826f1e490c" - integrity sha512-R0XvVJ9WusLiqTCEiGCmICCMplcCkIwwR11mOSD9CR5u+IXYdiseeEuXCVAjS54zqwkLcPNnmU4OeJ6tUrWhDw== - -source-map-resolve@^0.6.0: - version "0.6.0" - resolved "https://registry.yarnpkg.com/source-map-resolve/-/source-map-resolve-0.6.0.tgz#3d9df87e236b53f16d01e58150fc7711138e5ed2" - integrity sha512-KXBr9d/fO/bWo97NXsPIAW1bFSBOuCnjbNTBMO7N59hsv5i9yzRDfcYwwt0l04+VqnKC+EwzvJZIP/qkuMgR/w== - dependencies: - atob "^2.1.2" - decode-uri-component "^0.2.0" - -source-map-support@^0.5.6, source-map-support@~0.5.20: - version "0.5.21" - resolved "https://registry.yarnpkg.com/source-map-support/-/source-map-support-0.5.21.tgz#04fe7c7f9e1ed2d662233c28cb2b35b9f63f6e4f" - integrity sha512-uBHU3L3czsIyYXKX88fdrGovxdSCoTGDRZ6SYXtSRxLZUzHg5P/66Ht6uoUlHu9EZod+inXhKo3qQgwXUT/y1w== - dependencies: - buffer-from "^1.0.0" - source-map "^0.6.0" - -source-map@^0.5.0, source-map@^0.5.7: - version "0.5.7" - resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.5.7.tgz#8a039d2d1021d22d1ea14c80d8ea468ba2ef3fcc" - integrity sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w= - -source-map@^0.6.0, source-map@^0.6.1, source-map@~0.6.1: - version "0.6.1" - resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.6.1.tgz#74722af32e9614e9c287a8d0bbde48b5e2f1a263" - integrity sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g== - -source-map@^0.7.3: - version "0.7.3" - resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.7.3.tgz#5302f8169031735226544092e64981f751750383" - integrity sha512-CkCj6giN3S+n9qrYiBTX5gystlENnRW5jZeNLHpe6aue+SrHcG5VYwujhW9s4dY31mEGsxBDrHR6oI69fTXsaQ== - -space-separated-tokens@^1.0.0: - version "1.1.5" - resolved "https://registry.yarnpkg.com/space-separated-tokens/-/space-separated-tokens-1.1.5.tgz#85f32c3d10d9682007e917414ddc5c26d1aa6899" - integrity sha512-q/JSVd1Lptzhf5bkYm4ob4iWPjx0KiRe3sRFBNrVqbJkFaBm5vbbowy1mymoPNLRa52+oadOhJ+K49wsSeSjTA== - -space-separated-tokens@^2.0.0: - version "2.0.2" - resolved "https://registry.yarnpkg.com/space-separated-tokens/-/space-separated-tokens-2.0.2.tgz#1ecd9d2350a3844572c3f4a312bceb018348859f" - integrity sha512-PEGlAwrG8yXGXRjW32fGbg66JAlOAwbObuqVoJpv/mRgoWDQfgH1wDPvtzWyUSNAXBGSk8h755YDbbcEy3SH2Q== - -spdx-correct@^3.0.0: - version "3.1.1" - resolved "https://registry.yarnpkg.com/spdx-correct/-/spdx-correct-3.1.1.tgz#dece81ac9c1e6713e5f7d1b6f17d468fa53d89a9" - integrity sha512-cOYcUWwhCuHCXi49RhFRCyJEK3iPj1Ziz9DpViV3tbZOwXD49QzIN3MpOLJNxh2qwq2lJJZaKMVw9qNi4jTC0w== - dependencies: - spdx-expression-parse "^3.0.0" - spdx-license-ids "^3.0.0" - -spdx-exceptions@^2.1.0: - version "2.3.0" - resolved "https://registry.yarnpkg.com/spdx-exceptions/-/spdx-exceptions-2.3.0.tgz#3f28ce1a77a00372683eade4a433183527a2163d" - integrity sha512-/tTrYOC7PPI1nUAgx34hUpqXuyJG+DTHJTnIULG4rDygi4xu/tfgmq1e1cIRwRzwZgo4NLySi+ricLkZkw4i5A== - -spdx-expression-parse@^3.0.0: - version "3.0.1" - resolved "https://registry.yarnpkg.com/spdx-expression-parse/-/spdx-expression-parse-3.0.1.tgz#cf70f50482eefdc98e3ce0a6833e4a53ceeba679" - integrity sha512-cbqHunsQWnJNE6KhVSMsMeH5H/L9EpymbzqTQ3uLwNCLZ1Q481oWaofqH7nO6V07xlXwY6PhQdQ2IedWx/ZK4Q== - dependencies: - spdx-exceptions "^2.1.0" - spdx-license-ids "^3.0.0" - -spdx-expression-validate@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/spdx-expression-validate/-/spdx-expression-validate-2.0.0.tgz#25c9408e1c63fad94fff5517bb7101ffcd23350b" - integrity sha512-b3wydZLM+Tc6CFvaRDBOF9d76oGIHNCLYFeHbftFXUWjnfZWganmDmvtM5sm1cRwJc/VDBMLyGGrsLFd1vOxbg== - dependencies: - spdx-expression-parse "^3.0.0" - -spdx-license-ids@^3.0.0: - version "3.0.9" - resolved "https://registry.yarnpkg.com/spdx-license-ids/-/spdx-license-ids-3.0.9.tgz#8a595135def9592bda69709474f1cbeea7c2467f" - integrity sha512-Ki212dKK4ogX+xDo4CtOZBVIwhsKBEfsEEcwmJfLQzirgc2jIWdzg40Unxz/HzEUqM1WFzVlQSMF9kZZ2HboLQ== - -sprintf-js@~1.0.2: - version "1.0.3" - resolved "https://registry.yarnpkg.com/sprintf-js/-/sprintf-js-1.0.3.tgz#04e6926f662895354f3dd015203633b857297e2c" - integrity sha1-BOaSb2YolTVPPdAVIDYzuFcpfiw= - -ssri@^8.0.1: - version "8.0.1" - resolved "https://registry.yarnpkg.com/ssri/-/ssri-8.0.1.tgz#638e4e439e2ffbd2cd289776d5ca457c4f51a2af" - integrity sha512-97qShzy1AiyxvPNIkLWoGua7xoQzzPjQ0HAH4B0rWKo7SZ6USuPcrUiAFrws0UH8RrbWmgq3LMTObhPIHbbBeQ== - dependencies: - minipass "^3.1.1" - -stable@^0.1.8: - version "0.1.8" - resolved "https://registry.yarnpkg.com/stable/-/stable-0.1.8.tgz#836eb3c8382fe2936feaf544631017ce7d47a3cf" - integrity sha512-ji9qxRnOVfcuLDySj9qzhGSEFVobyt1kIOSkj1qZzYLzq7Tos/oUUWvotUPQLlrsidqsK6tBH89Bc9kL5zHA6w== - -stack-utils@^2.0.3: - version "2.0.5" - resolved "https://registry.yarnpkg.com/stack-utils/-/stack-utils-2.0.5.tgz#d25265fca995154659dbbfba3b49254778d2fdd5" - integrity sha512-xrQcmYhOsn/1kX+Vraq+7j4oE2j/6BFscZ0etmYg81xuM8Gq0022Pxb8+IqgOFUIaxHs0KaSb7T1+OegiNrNFA== - dependencies: - escape-string-regexp "^2.0.0" - -stickyfill@^1.1.1: - version "1.1.1" - resolved "https://registry.yarnpkg.com/stickyfill/-/stickyfill-1.1.1.tgz#39413fee9d025c74a7e59ceecb23784cc0f17f02" - integrity sha1-OUE/7p0CXHSn5ZzuyyN4TMDxfwI= - -string-length@^4.0.1: - version "4.0.2" - resolved "https://registry.yarnpkg.com/string-length/-/string-length-4.0.2.tgz#a8a8dc7bd5c1a82b9b3c8b87e125f66871b6e57a" - integrity sha512-+l6rNN5fYHNhZZy41RXsYptCjA2Igmq4EG7kZAYFQI1E1VTXarr6ZPXBg6eq7Y6eK4FEhY6AJlyuFIb/v/S0VQ== - dependencies: - char-regex "^1.0.2" - strip-ansi "^6.0.0" - -string-width@^4.1.0, string-width@^4.2.0, string-width@^4.2.3: - version "4.2.3" - resolved "https://registry.yarnpkg.com/string-width/-/string-width-4.2.3.tgz#269c7117d27b05ad2e536830a8ec895ef9c6d010" - integrity sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g== - dependencies: - emoji-regex "^8.0.0" - is-fullwidth-code-point "^3.0.0" - strip-ansi "^6.0.1" - -string.prototype.matchall@^4.0.7: - version "4.0.7" - resolved "https://registry.yarnpkg.com/string.prototype.matchall/-/string.prototype.matchall-4.0.7.tgz#8e6ecb0d8a1fb1fda470d81acecb2dba057a481d" - integrity sha512-f48okCX7JiwVi1NXCVWcFnZgADDC/n2vePlQ/KUCNqCikLLilQvwjMO8+BHVKvgzH0JB0J9LEPgxOGT02RoETg== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.3" - es-abstract "^1.19.1" - get-intrinsic "^1.1.1" - has-symbols "^1.0.3" - internal-slot "^1.0.3" - regexp.prototype.flags "^1.4.1" - side-channel "^1.0.4" - -string.prototype.trim@^1.2.7: - version "1.2.7" - resolved "https://registry.yarnpkg.com/string.prototype.trim/-/string.prototype.trim-1.2.7.tgz#a68352740859f6893f14ce3ef1bb3037f7a90533" - integrity sha512-p6TmeT1T3411M8Cgg9wBTMRtY2q9+PNy9EV1i2lIXUN/btt763oIfxwN3RR8VU6wHX8j/1CFy0L+YuThm6bgOg== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - es-abstract "^1.20.4" - -string.prototype.trimend@^1.0.4: - version "1.0.4" - resolved "https://registry.yarnpkg.com/string.prototype.trimend/-/string.prototype.trimend-1.0.4.tgz#e75ae90c2942c63504686c18b287b4a0b1a45f80" - integrity sha512-y9xCjw1P23Awk8EvTpcyL2NIr1j7wJ39f+k6lvRnSMz+mz9CGz9NYPelDk42kOz6+ql8xjfK8oYzy3jAP5QU5A== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.3" - -string.prototype.trimend@^1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/string.prototype.trimend/-/string.prototype.trimend-1.0.5.tgz#914a65baaab25fbdd4ee291ca7dde57e869cb8d0" - integrity sha512-I7RGvmjV4pJ7O3kdf+LXFpVfdNOxtCW/2C8f6jNiW4+PQchwxkCDzlk1/7p+Wl4bqFIZeF47qAHXLuHHWKAxog== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - es-abstract "^1.19.5" - -string.prototype.trimend@^1.0.6: - version "1.0.6" - resolved "https://registry.yarnpkg.com/string.prototype.trimend/-/string.prototype.trimend-1.0.6.tgz#c4a27fa026d979d79c04f17397f250a462944533" - integrity sha512-JySq+4mrPf9EsDBEDYMOb/lM7XQLulwg5R/m1r0PXEFqrV0qHvl58sdTilSXtKOflCsK2E8jxf+GKC0T07RWwQ== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - es-abstract "^1.20.4" - -string.prototype.trimstart@^1.0.4: - version "1.0.4" - resolved "https://registry.yarnpkg.com/string.prototype.trimstart/-/string.prototype.trimstart-1.0.4.tgz#b36399af4ab2999b4c9c648bd7a3fb2bb26feeed" - integrity sha512-jh6e984OBfvxS50tdY2nRZnoC5/mLFKOREQfw8t5yytkoUsJRNxvI/E39qu1sD0OtWI3OC0XgKSmcWwziwYuZw== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.3" - -string.prototype.trimstart@^1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/string.prototype.trimstart/-/string.prototype.trimstart-1.0.5.tgz#5466d93ba58cfa2134839f81d7f42437e8c01fef" - integrity sha512-THx16TJCGlsN0o6dl2o6ncWUsdgnLRSA23rRE5pyGBw/mLr3Ej/R2LaqCtgP8VNMGZsvMWnf9ooZPyY2bHvUFg== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - es-abstract "^1.19.5" - -string.prototype.trimstart@^1.0.6: - version "1.0.6" - resolved "https://registry.yarnpkg.com/string.prototype.trimstart/-/string.prototype.trimstart-1.0.6.tgz#e90ab66aa8e4007d92ef591bbf3cd422c56bdcf4" - integrity sha512-omqjMDaY92pbn5HOX7f9IccLA+U1tA9GvtU4JrodiXFfYB7jPzzHpRzpglLAjtUV6bB557zwClJezTqnAiYnQA== - dependencies: - call-bind "^1.0.2" - define-properties "^1.1.4" - es-abstract "^1.20.4" - -string_decoder@~0.10.x: - version "0.10.31" - resolved "https://registry.yarnpkg.com/string_decoder/-/string_decoder-0.10.31.tgz#62e203bc41766c6c28c9fc84301dab1c5310fa94" - integrity sha1-YuIDvEF2bGwoyfyEMB2rHFMQ+pQ= - -strip-ansi@^6.0.0, strip-ansi@^6.0.1: - version "6.0.1" - resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-6.0.1.tgz#9e26c63d30f53443e9489495b2105d37b67a85d9" - integrity sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A== - dependencies: - ansi-regex "^5.0.1" - -strip-bom@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/strip-bom/-/strip-bom-3.0.0.tgz#2334c18e9c759f7bdd56fdef7e9ae3d588e68ed3" - integrity sha1-IzTBjpx1n3vdVv3vfprj1YjmjtM= - -strip-bom@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/strip-bom/-/strip-bom-4.0.0.tgz#9c3505c1db45bcedca3d9cf7a16f5c5aa3901878" - integrity sha512-3xurFv5tEgii33Zi8Jtp55wEIILR9eh34FAW00PZf+JnSsTmV/ioewSgQl97JHvgjoRGwPShsWm+IdrxB35d0w== - -strip-comments@^2.0.1: - version "2.0.1" - resolved "https://registry.yarnpkg.com/strip-comments/-/strip-comments-2.0.1.tgz#4ad11c3fbcac177a67a40ac224ca339ca1c1ba9b" - integrity sha512-ZprKx+bBLXv067WTCALv8SSz5l2+XhpYCsVtSqlMnkAXMWDq+/ekVbl1ghqP9rUHTzv6sm/DwCOiYutU/yp1fw== - -strip-final-newline@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/strip-final-newline/-/strip-final-newline-2.0.0.tgz#89b852fb2fcbe936f6f4b3187afb0a12c1ab58ad" - integrity sha512-BrpvfNAE3dcvq7ll3xVumzjKjZQ5tI1sEUIKr3Uoks0XUl45St3FlatVqef9prk4jRDzhW6WZg+3bk93y6pLjA== - -strip-indent@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/strip-indent/-/strip-indent-3.0.0.tgz#c32e1cee940b6b3432c771bc2c54bcce73cd3001" - integrity sha512-laJTa3Jb+VQpaC6DseHhF7dXVqHTfJPCRDaEbid/drOhgitgYku/letMUqOXFoWV0zIIUbjpdH2t+tYj4bQMRQ== - dependencies: - min-indent "^1.0.0" - -strip-indent@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/strip-indent/-/strip-indent-4.0.0.tgz#b41379433dd06f5eae805e21d631e07ee670d853" - integrity sha512-mnVSV2l+Zv6BLpSD/8V87CW/y9EmmbYzGCIavsnsI6/nwn26DwffM/yztm30Z/I2DY9wdS3vXVCMnHDgZaVNoA== - dependencies: - min-indent "^1.0.1" - -strip-json-comments@1.0.x: - version "1.0.4" - resolved "https://registry.yarnpkg.com/strip-json-comments/-/strip-json-comments-1.0.4.tgz#1e15fbcac97d3ee99bf2d73b4c656b082bbafb91" - integrity sha1-HhX7ysl9Pumb8tc7TGVrCCu6+5E= - -strip-json-comments@^3.1.0, strip-json-comments@^3.1.1: - version "3.1.1" - resolved "https://registry.yarnpkg.com/strip-json-comments/-/strip-json-comments-3.1.1.tgz#31f1281b3832630434831c310c01cccda8cbe006" - integrity sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig== - -style-loader@^1.2.1: - version "1.3.0" - resolved "https://registry.yarnpkg.com/style-loader/-/style-loader-1.3.0.tgz#828b4a3b3b7e7aa5847ce7bae9e874512114249e" - integrity sha512-V7TCORko8rs9rIqkSrlMfkqA63DfoGBBJmK1kKGCcSi+BWb4cqz0SRsnp4l6rU5iwOEd0/2ePv68SV22VXon4Q== - dependencies: - loader-utils "^2.0.0" - schema-utils "^2.7.0" - -style-loader@^3.3.1: - version "3.3.1" - resolved "https://registry.yarnpkg.com/style-loader/-/style-loader-3.3.1.tgz#057dfa6b3d4d7c7064462830f9113ed417d38575" - integrity sha512-GPcQ+LDJbrcxHORTRes6Jy2sfvK2kS6hpSfI/fXhPt+spVzxF6LJ1dHLN9zIGmVaaP044YKaIatFaufENRiDoQ== - -style-search@^0.1.0: - version "0.1.0" - resolved "https://registry.yarnpkg.com/style-search/-/style-search-0.1.0.tgz#7958c793e47e32e07d2b5cafe5c0bf8e12e77902" - integrity sha1-eVjHk+R+MuB9K1yv5cC/jhLneQI= - -style-to-object@^0.3.0: - version "0.3.0" - resolved "https://registry.yarnpkg.com/style-to-object/-/style-to-object-0.3.0.tgz#b1b790d205991cc783801967214979ee19a76e46" - integrity sha512-CzFnRRXhzWIdItT3OmF8SQfWyahHhjq3HwcMNCNLn+N7klOOqPjMeG/4JSu77D7ypZdGvSzvkrbyeTMizz2VrA== - dependencies: - inline-style-parser "0.1.1" - -style-value-types@5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/style-value-types/-/style-value-types-5.0.0.tgz#76c35f0e579843d523187989da866729411fc8ad" - integrity sha512-08yq36Ikn4kx4YU6RD7jWEv27v4V+PUsOGa4n/as8Et3CuODMJQ00ENeAVXAeydX4Z2j1XHZF1K2sX4mGl18fA== - dependencies: - hey-listen "^1.0.8" - tslib "^2.1.0" - -stylehacks@^5.1.0: - version "5.1.0" - resolved "https://registry.yarnpkg.com/stylehacks/-/stylehacks-5.1.0.tgz#a40066490ca0caca04e96c6b02153ddc39913520" - integrity sha512-SzLmvHQTrIWfSgljkQCw2++C9+Ne91d/6Sp92I8c5uHTcy/PgeHamwITIbBW9wnFTY/3ZfSXR9HIL6Ikqmcu6Q== - dependencies: - browserslist "^4.16.6" - postcss-selector-parser "^6.0.4" - -stylelint-config-prettier@^9.0.5: - version "9.0.5" - resolved "https://registry.yarnpkg.com/stylelint-config-prettier/-/stylelint-config-prettier-9.0.5.tgz#9f78bbf31c7307ca2df2dd60f42c7014ee9da56e" - integrity sha512-U44lELgLZhbAD/xy/vncZ2Pq8sh2TnpiPvo38Ifg9+zeioR+LAkHu0i6YORIOxFafZoVg0xqQwex6e6F25S5XA== - -stylelint-config-recommended@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/stylelint-config-recommended/-/stylelint-config-recommended-3.0.0.tgz#e0e547434016c5539fe2650afd58049a2fd1d657" - integrity sha512-F6yTRuc06xr1h5Qw/ykb2LuFynJ2IxkKfCMf+1xqPffkxh0S09Zc902XCffcsw/XMFq/OzQ1w54fLIDtmRNHnQ== - -stylelint-config-standard@^20.0.0: - version "20.0.0" - resolved "https://registry.yarnpkg.com/stylelint-config-standard/-/stylelint-config-standard-20.0.0.tgz#06135090c9e064befee3d594289f50e295b5e20d" - integrity sha512-IB2iFdzOTA/zS4jSVav6z+wGtin08qfj+YyExHB3LF9lnouQht//YyB0KZq9gGz5HNPkddHOzcY8HsUey6ZUlA== - dependencies: - stylelint-config-recommended "^3.0.0" - -stylelint@^15.10.1: - version "15.10.1" - resolved "https://registry.yarnpkg.com/stylelint/-/stylelint-15.10.1.tgz#93f189958687e330c106b010cbec0c41dcae506d" - integrity sha512-CYkzYrCFfA/gnOR+u9kJ1PpzwG10WLVnoxHDuBA/JiwGqdM9+yx9+ou6SE/y9YHtfv1mcLo06fdadHTOx4gBZQ== - dependencies: - "@csstools/css-parser-algorithms" "^2.3.0" - "@csstools/css-tokenizer" "^2.1.1" - "@csstools/media-query-list-parser" "^2.1.2" - "@csstools/selector-specificity" "^3.0.0" - balanced-match "^2.0.0" - colord "^2.9.3" - cosmiconfig "^8.2.0" - css-functions-list "^3.1.0" - css-tree "^2.3.1" - debug "^4.3.4" - fast-glob "^3.3.0" - fastest-levenshtein "^1.0.16" - file-entry-cache "^6.0.1" - global-modules "^2.0.0" - globby "^11.1.0" - globjoin "^0.1.4" - html-tags "^3.3.1" - ignore "^5.2.4" - import-lazy "^4.0.0" - imurmurhash "^0.1.4" - is-plain-object "^5.0.0" - known-css-properties "^0.27.0" - mathml-tag-names "^2.1.3" - meow "^10.1.5" - micromatch "^4.0.5" - normalize-path "^3.0.0" - picocolors "^1.0.0" - postcss "^8.4.24" - postcss-resolve-nested-selector "^0.1.1" - postcss-safe-parser "^6.0.0" - postcss-selector-parser "^6.0.13" - postcss-value-parser "^4.2.0" - resolve-from "^5.0.0" - string-width "^4.2.3" - strip-ansi "^6.0.1" - style-search "^0.1.0" - supports-hyperlinks "^3.0.0" - svg-tags "^1.0.0" - table "^6.8.1" - write-file-atomic "^5.0.1" - -stylis@4.0.13: - version "4.0.13" - resolved "https://registry.yarnpkg.com/stylis/-/stylis-4.0.13.tgz#f5db332e376d13cc84ecfe5dace9a2a51d954c91" - integrity sha512-xGPXiFVl4YED9Jh7Euv2V220mriG9u4B2TA6Ybjc1catrstKD2PpIdU3U0RKpkVBC2EhmL/F0sPCr9vrFTNRag== - -stylis@^4.0.3: - version "4.0.10" - resolved "https://registry.yarnpkg.com/stylis/-/stylis-4.0.10.tgz#446512d1097197ab3f02fb3c258358c3f7a14240" - integrity sha512-m3k+dk7QeJw660eIKRRn3xPF6uuvHs/FFzjX3HQ5ove0qYsiygoAhwn5a3IYKaZPo5LrYD0rfVmtv1gNY1uYwg== - -supports-color@^5.3.0: - version "5.5.0" - resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-5.5.0.tgz#e2e69a44ac8772f78a1ec0b35b689df6530efc8f" - integrity sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow== - dependencies: - has-flag "^3.0.0" - -supports-color@^7.0.0, supports-color@^7.1.0: - version "7.2.0" - resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-7.2.0.tgz#1b7dcdcb32b8138801b3e478ba6a51caa89648da" - integrity sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw== - dependencies: - has-flag "^4.0.0" - -supports-color@^8.0.0: - version "8.1.1" - resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-8.1.1.tgz#cd6fc17e28500cff56c1b86c0a7fd4a54a73005c" - integrity sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q== - dependencies: - has-flag "^4.0.0" - -supports-hyperlinks@^2.0.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/supports-hyperlinks/-/supports-hyperlinks-2.2.0.tgz#4f77b42488765891774b70c79babd87f9bd594bb" - integrity sha512-6sXEzV5+I5j8Bmq9/vUphGRM/RJNT9SCURJLjwfOg51heRtguGWDzcaBlgAzKhQa0EVNpPEKzQuBwZ8S8WaCeQ== - dependencies: - has-flag "^4.0.0" - supports-color "^7.0.0" - -supports-hyperlinks@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/supports-hyperlinks/-/supports-hyperlinks-3.0.0.tgz#c711352a5c89070779b4dad54c05a2f14b15c94b" - integrity sha512-QBDPHyPQDRTy9ku4URNGY5Lah8PAaXs6tAAwp55sL5WCsSW7GIfdf6W5ixfziW+t7wh3GVvHyHHyQ1ESsoRvaA== - dependencies: - has-flag "^4.0.0" - supports-color "^7.0.0" - -supports-preserve-symlinks-flag@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/supports-preserve-symlinks-flag/-/supports-preserve-symlinks-flag-1.0.0.tgz#6eda4bd344a3c94aea376d4cc31bc77311039e09" - integrity sha512-ot0WnXS9fgdkgIcePe6RHNk1WA8+muPa6cSjeR3V8K27q9BB1rTE3R1p7Hv0z1ZyAc8s6Vvv8DIyWf681MAt0w== - -svg-tags@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/svg-tags/-/svg-tags-1.0.0.tgz#58f71cee3bd519b59d4b2a843b6c7de64ac04764" - integrity sha1-WPcc7jvVGbWdSyqEO2x95krAR2Q= - -svgo@^2.7.0: - version "2.8.0" - resolved "https://registry.yarnpkg.com/svgo/-/svgo-2.8.0.tgz#4ff80cce6710dc2795f0c7c74101e6764cfccd24" - integrity sha512-+N/Q9kV1+F+UeWYoSiULYo4xYSDQlTgb+ayMobAXPwMnLvop7oxKMo9OzIrX5x3eS4L4f2UHhc9axXwY8DpChg== - dependencies: - "@trysound/sax" "0.2.0" - commander "^7.2.0" - css-select "^4.1.3" - css-tree "^1.1.3" - csso "^4.2.0" - picocolors "^1.0.0" - stable "^0.1.8" - -swagger-ui-dist@4.1.3: - version "4.1.3" - resolved "https://registry.yarnpkg.com/swagger-ui-dist/-/swagger-ui-dist-4.1.3.tgz#2be9f9de9b5c19132fa4a5e40933058c151563dc" - integrity sha512-WvfPSfAAMlE/sKS6YkW47nX/hA7StmhYnAHc6wWCXNL0oclwLj6UXv0hQCkLnDgvebi0MEV40SJJpVjKUgH1IQ== - -swagger2openapi@^7.0.6: - version "7.0.6" - resolved "https://registry.yarnpkg.com/swagger2openapi/-/swagger2openapi-7.0.6.tgz#20a2835b8edfc0f4c08036b20cb51e8f78a420bf" - integrity sha512-VIT414koe0eJqre0KrhNMUB7QEUfPjGAKesPZZosIKr2rxZ6vpUoersHUFNOsN/OZ5u2zsniCslBOwVcmQZwlg== - dependencies: - call-me-maybe "^1.0.1" - node-fetch "^2.6.1" - node-fetch-h2 "^2.3.0" - node-readfiles "^0.2.0" - oas-kit-common "^1.0.8" - oas-resolver "^2.5.5" - oas-schema-walker "^1.1.5" - oas-validator "^5.0.6" - reftools "^1.1.8" - yaml "^1.10.0" - yargs "^17.0.1" - -symbol-tree@^3.2.4: - version "3.2.4" - resolved "https://registry.yarnpkg.com/symbol-tree/-/symbol-tree-3.2.4.tgz#430637d248ba77e078883951fb9aa0eed7c63fa2" - integrity sha512-9QNk5KwDF+Bvz+PyObkmSYjI5ksVUYtjW7AU22r2NKcfLJcXp96hkDWU3+XndOsUb+AQ9QhfzfCT2O+CNWT5Tw== - -table@^6.8.1: - version "6.8.1" - resolved "https://registry.yarnpkg.com/table/-/table-6.8.1.tgz#ea2b71359fe03b017a5fbc296204471158080bdf" - integrity sha512-Y4X9zqrCftUhMeH2EptSSERdVKt/nEdijTOacGD/97EKjhQ/Qs8RTlEGABSJNNN8lac9kheH+af7yAkEWlgneA== - dependencies: - ajv "^8.0.1" - lodash.truncate "^4.4.2" - slice-ansi "^4.0.0" - string-width "^4.2.3" - strip-ansi "^6.0.1" - -tapable@^2.0.0, tapable@^2.1.1, tapable@^2.2.0: - version "2.2.1" - resolved "https://registry.yarnpkg.com/tapable/-/tapable-2.2.1.tgz#1967a73ef4060a82f12ab96af86d52fdb76eeca0" - integrity sha512-GNzQvQTOIP6RyTfE2Qxb8ZVlNmw0n88vp1szwWRimP02mnTsx3Wtn5qRdqY9w2XduFNUgvOwhNnQsjwCp+kqaQ== - -tar@^6.0.2: - version "6.2.1" - resolved "https://registry.yarnpkg.com/tar/-/tar-6.2.1.tgz#717549c541bc3c2af15751bea94b1dd068d4b03a" - integrity sha512-DZ4yORTwrbTj/7MZYq2w+/ZFdI6OZ/f9SFHR+71gIVUZhOQPHzVCLpvRnPgyaMpfWxxk/4ONva3GQSyNIKRv6A== - dependencies: - chownr "^2.0.0" - fs-minipass "^2.0.0" - minipass "^5.0.0" - minizlib "^2.1.1" - mkdirp "^1.0.3" - yallist "^4.0.0" - -terminal-link@^2.0.0: - version "2.1.1" - resolved "https://registry.yarnpkg.com/terminal-link/-/terminal-link-2.1.1.tgz#14a64a27ab3c0df933ea546fba55f2d078edc994" - integrity sha512-un0FmiRUQNr5PJqy9kP7c40F5BOfpGlYTrxonDChEZB7pzZxRNp/bt+ymiy9/npwXya9KH99nJ/GXFIiUkYGFQ== - dependencies: - ansi-escapes "^4.2.1" - supports-hyperlinks "^2.0.0" - -terser-webpack-plugin@<5.0.0: - version "4.2.3" - resolved "https://registry.yarnpkg.com/terser-webpack-plugin/-/terser-webpack-plugin-4.2.3.tgz#28daef4a83bd17c1db0297070adc07fc8cfc6a9a" - integrity sha512-jTgXh40RnvOrLQNgIkwEKnQ8rmHjHK4u+6UBEi+W+FPmvb+uo+chJXntKe7/3lW5mNysgSWD60KyesnhW8D6MQ== - dependencies: - cacache "^15.0.5" - find-cache-dir "^3.3.1" - jest-worker "^26.5.0" - p-limit "^3.0.2" - schema-utils "^3.0.0" - serialize-javascript "^5.0.1" - source-map "^0.6.1" - terser "^5.3.4" - webpack-sources "^1.4.3" - -terser-webpack-plugin@^5.3.10: - version "5.3.10" - resolved "https://registry.yarnpkg.com/terser-webpack-plugin/-/terser-webpack-plugin-5.3.10.tgz#904f4c9193c6fd2a03f693a2150c62a92f40d199" - integrity sha512-BKFPWlPDndPs+NGGCr1U59t0XScL5317Y0UReNrHaw9/FwhPENlq6bfgs+4yPfyP51vqC1bQ4rp1EfXW5ZSH9w== - dependencies: - "@jridgewell/trace-mapping" "^0.3.20" - jest-worker "^27.4.5" - schema-utils "^3.1.1" - serialize-javascript "^6.0.1" - terser "^5.26.0" - -terser@^5.26.0: - version "5.31.6" - resolved "https://registry.yarnpkg.com/terser/-/terser-5.31.6.tgz#c63858a0f0703988d0266a82fcbf2d7ba76422b1" - integrity sha512-PQ4DAriWzKj+qgehQ7LK5bQqCFNMmlhjR2PFFLuqGCpuCAauxemVBWwWOxo3UIwWQx8+Pr61Df++r76wDmkQBg== - dependencies: - "@jridgewell/source-map" "^0.3.3" - acorn "^8.8.2" - commander "^2.20.0" - source-map-support "~0.5.20" - -terser@^5.3.4: - version "5.14.2" - resolved "https://registry.yarnpkg.com/terser/-/terser-5.14.2.tgz#9ac9f22b06994d736174f4091aa368db896f1c10" - integrity sha512-oL0rGeM/WFQCUd0y2QrWxYnq7tfSuKBiqTjRPWrRgB46WD/kiwHwF8T23z78H6Q6kGCuuHcPB+KULHRdxvVGQA== - dependencies: - "@jridgewell/source-map" "^0.3.2" - acorn "^8.5.0" - commander "^2.20.0" - source-map-support "~0.5.20" - -test-exclude@^6.0.0: - version "6.0.0" - resolved "https://registry.yarnpkg.com/test-exclude/-/test-exclude-6.0.0.tgz#04a8698661d805ea6fa293b6cb9e63ac044ef15e" - integrity sha512-cAGWPIyOHU6zlmg88jwm7VRyXnMN7iV68OGAbYDk/Mh/xC/pzVPlQtY6ngoIH/5/tciuhGfvESU8GrHrcxD56w== - dependencies: - "@istanbuljs/schema" "^0.1.2" - glob "^7.1.4" - minimatch "^3.0.4" - -text-table@^0.2.0: - version "0.2.0" - resolved "https://registry.yarnpkg.com/text-table/-/text-table-0.2.0.tgz#7f5ee823ae805207c00af2df4a84ec3fcfa570b4" - integrity sha1-f17oI66AUgfACvLfSoTsP8+lcLQ= - -throat@^6.0.1: - version "6.0.1" - resolved "https://registry.yarnpkg.com/throat/-/throat-6.0.1.tgz#d514fedad95740c12c2d7fc70ea863eb51ade375" - integrity sha512-8hmiGIJMDlwjg7dlJ4yKGLK8EsYqKgPWbG3b4wjJddKNwc7N7Dpn08Df4szr/sZdMVeOstrdYSsqzX6BYbcB+w== - -tiny-glob@^0.2.9: - version "0.2.9" - resolved "https://registry.yarnpkg.com/tiny-glob/-/tiny-glob-0.2.9.tgz#2212d441ac17928033b110f8b3640683129d31e2" - integrity sha512-g/55ssRPUjShh+xkfx9UPDXqhckHEsHr4Vd9zX55oSdGZc/MD0m3sferOkwWtp98bv+kcVfEHtRJgBVJzelrzg== - dependencies: - globalyzer "0.1.0" - globrex "^0.1.2" - -tiny-invariant@^1.0.6: - version "1.2.0" - resolved "https://registry.yarnpkg.com/tiny-invariant/-/tiny-invariant-1.2.0.tgz#a1141f86b672a9148c72e978a19a73b9b94a15a9" - integrity sha512-1Uhn/aqw5C6RI4KejVeTg6mIS7IqxnLJ8Mv2tV5rTc0qWobay7pDUz6Wi392Cnc8ak1H0F2cjoRzb2/AW4+Fvg== - -tmpl@1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/tmpl/-/tmpl-1.0.5.tgz#8683e0b902bb9c20c4f726e3c0b69f36518c07cc" - integrity sha512-3f0uOEAQwIqGuWW2MVzYg8fV/QNnc/IpuJNG837rLuczAaLVHslWHZQj4IGiEl5Hs3kkbhwL9Ab7Hrsmuj+Smw== - -to-fast-properties@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/to-fast-properties/-/to-fast-properties-2.0.0.tgz#dc5e698cbd079265bc73e0377681a4e4e83f616e" - integrity sha1-3F5pjL0HkmW8c+A3doGk5Og/YW4= - -to-regex-range@^5.0.1: - version "5.0.1" - resolved "https://registry.yarnpkg.com/to-regex-range/-/to-regex-range-5.0.1.tgz#1648c44aae7c8d988a326018ed72f5b4dd0392e4" - integrity sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ== - dependencies: - is-number "^7.0.0" - -toggle-selection@^1.0.6: - version "1.0.6" - resolved "https://registry.yarnpkg.com/toggle-selection/-/toggle-selection-1.0.6.tgz#6e45b1263f2017fa0acc7d89d78b15b8bf77da32" - integrity sha1-bkWxJj8gF/oKzH2J14sVuL932jI= - -tough-cookie@^4.0.0: - version "4.1.3" - resolved "https://registry.yarnpkg.com/tough-cookie/-/tough-cookie-4.1.3.tgz#97b9adb0728b42280aa3d814b6b999b2ff0318bf" - integrity sha512-aX/y5pVRkfRnfmuX+OdbSdXvPe6ieKX/G2s7e98f4poJHnqH3281gDPm/metm6E/WRamfx7WC4HUqkWHfQHprw== - dependencies: - psl "^1.1.33" - punycode "^2.1.1" - universalify "^0.2.0" - url-parse "^1.5.3" - -tr46@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/tr46/-/tr46-2.1.0.tgz#fa87aa81ca5d5941da8cbf1f9b749dc969a4e240" - integrity sha512-15Ih7phfcdP5YxqiB+iDtLoaTz4Nd35+IiAv0kQ5FNKHzXgdWqPoTIqEDDJmXceQt4JZk6lVPT8lnDlPpGDppw== - dependencies: - punycode "^2.1.1" - -tr46@~0.0.3: - version "0.0.3" - resolved "https://registry.yarnpkg.com/tr46/-/tr46-0.0.3.tgz#8184fd347dac9cdc185992f3a6622e14b9d9ab6a" - integrity sha512-N3WMsuqV66lT30CrXNbEjx4GEwlow3v6rr4mCcv6prnfwhS01rkgyFdjPNBYd9br7LpXV1+Emh01fHnq2Gdgrw== - -trim-lines@^3.0.0: - version "3.0.1" - resolved "https://registry.yarnpkg.com/trim-lines/-/trim-lines-3.0.1.tgz#d802e332a07df861c48802c04321017b1bd87338" - integrity sha512-kRj8B+YHZCc9kQYdWfJB2/oUl9rA99qbowYYBtr4ui4mZyAQ2JpvVBd/6U2YloATfqBhBTSMhTpgBHtU0Mf3Rg== - -trim-newlines@^4.0.2: - version "4.1.1" - resolved "https://registry.yarnpkg.com/trim-newlines/-/trim-newlines-4.1.1.tgz#28c88deb50ed10c7ba6dc2474421904a00139125" - integrity sha512-jRKj0n0jXWo6kh62nA5TEh3+4igKDXLvzBJcPpiizP7oOolUrYIxmVBG9TOtHYFHoddUk6YvAkGeGoSVTXfQXQ== - -trough@^2.0.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/trough/-/trough-2.1.0.tgz#0f7b511a4fde65a46f18477ab38849b22c554876" - integrity sha512-AqTiAOLcj85xS7vQ8QkAV41hPDIJ71XJB4RCUrzo/1GM2CQwhkJGaf9Hgr7BOugMRpgGUrqRg/DrBDl4H40+8g== - -tsconfig-paths@^3.14.1, tsconfig-paths@^3.14.2: - version "3.14.2" - resolved "https://registry.yarnpkg.com/tsconfig-paths/-/tsconfig-paths-3.14.2.tgz#6e32f1f79412decd261f92d633a9dc1cfa99f088" - integrity sha512-o/9iXgCYc5L/JxCHPe3Hvh8Q/2xm5Z+p18PESBU6Ff33695QnCHBEjcytY2q19ua7Mbl/DavtBOLq+oG0RCL+g== - dependencies: - "@types/json5" "^0.0.29" - json5 "^1.0.2" - minimist "^1.2.6" - strip-bom "^3.0.0" - -tslib@2.3.0: - version "2.3.0" - resolved "https://registry.yarnpkg.com/tslib/-/tslib-2.3.0.tgz#803b8cdab3e12ba581a4ca41c8839bbb0dacb09e" - integrity sha512-N82ooyxVNm6h1riLCoyS9e3fuJ3AMG2zIZs2Gd1ATcSFjSA23Q0fzjjZeh0jbJvWVDZ0cJT8yaNNaaXHzueNjg== - -tslib@^1.8.1: - version "1.14.1" - resolved "https://registry.yarnpkg.com/tslib/-/tslib-1.14.1.tgz#cf2d38bdc34a134bcaf1091c41f6619e2f672d00" - integrity sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg== - -tslib@^2.0.0: - version "2.4.0" - resolved "https://registry.yarnpkg.com/tslib/-/tslib-2.4.0.tgz#7cecaa7f073ce680a05847aa77be941098f36dc3" - integrity sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ== - -tslib@^2.0.3, tslib@^2.1.0: - version "2.3.1" - resolved "https://registry.yarnpkg.com/tslib/-/tslib-2.3.1.tgz#e8a335add5ceae51aa261d32a490158ef042ef01" - integrity sha512-77EbyPPpMz+FRFRuAFlWMtmgUWGe9UOG2Z25NqCwiIjRhOf5iKGuzSe5P2w1laq+FkRy4p+PCuVkJSGkzTEKVw== - -tsutils@^3.21.0: - version "3.21.0" - resolved "https://registry.yarnpkg.com/tsutils/-/tsutils-3.21.0.tgz#b48717d394cea6c1e096983eed58e9d61715b623" - integrity sha512-mHKK3iUXL+3UF6xL5k0PEhKRUBKPBCv/+RkEOpjRWxxx27KKRBmmA60A9pgOUvMi8GKhRMPEmjBRPzs2W7O1OA== - dependencies: - tslib "^1.8.1" - -type-check@^0.4.0, type-check@~0.4.0: - version "0.4.0" - resolved "https://registry.yarnpkg.com/type-check/-/type-check-0.4.0.tgz#07b8203bfa7056c0657050e3ccd2c37730bab8f1" - integrity sha512-XleUoc9uwGXqjWwXaUTZAmzMcFZ5858QA2vvx1Ur5xIcixXIP+8LnFDgRplU30us6teqdlskFfu+ae4K79Ooew== - dependencies: - prelude-ls "^1.2.1" - -type-check@~0.3.2: - version "0.3.2" - resolved "https://registry.yarnpkg.com/type-check/-/type-check-0.3.2.tgz#5884cab512cf1d355e3fb784f30804b2b520db72" - integrity sha1-WITKtRLPHTVeP7eE8wgEsrUg23I= - dependencies: - prelude-ls "~1.1.2" - -type-detect@4.0.8: - version "4.0.8" - resolved "https://registry.yarnpkg.com/type-detect/-/type-detect-4.0.8.tgz#7646fb5f18871cfbb7749e69bd39a6388eb7450c" - integrity sha512-0fr/mIH1dlO+x7TlcMy+bIDqKPsw/70tVyeHW787goQjhmqaZe10uwLujubK9q9Lg6Fiho1KUKDYz0Z7k7g5/g== - -type-fest@^0.20.2: - version "0.20.2" - resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.20.2.tgz#1bf207f4b28f91583666cb5fbd327887301cd5f4" - integrity sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ== - -type-fest@^0.21.3: - version "0.21.3" - resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.21.3.tgz#d260a24b0198436e133fa26a524a6d65fa3b2e37" - integrity sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w== - -type-fest@^1.0.1, type-fest@^1.2.1, type-fest@^1.2.2: - version "1.4.0" - resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-1.4.0.tgz#e9fb813fe3bf1744ec359d55d1affefa76f14be1" - integrity sha512-yGSza74xk0UG8k+pLh5oeoYirvIiWo5t0/o3zHHAO2tRDiZcxWP7fywNlXhqb6/r6sWvwi+RsyQMWhVLe4BVuA== - -type-fest@^2.17.0: - version "2.17.0" - resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-2.17.0.tgz#c677030ce61e5be0c90c077d52571eb73c506ea9" - integrity sha512-U+g3/JVXnOki1kLSc+xZGPRll3Ah9u2VIG6Sn9iH9YX6UkPERmt6O/0fIyTgsd2/whV0+gAaHAg8fz6sG1QzMA== - -typed-array-length@^1.0.4: - version "1.0.4" - resolved "https://registry.yarnpkg.com/typed-array-length/-/typed-array-length-1.0.4.tgz#89d83785e5c4098bec72e08b319651f0eac9c1bb" - integrity sha512-KjZypGq+I/H7HI5HlOoGHkWUUGq+Q0TPhQurLbyrVrvnKTBgzLhIJ7j6J/XTQOi0d1RjyZ0wdas8bKs2p0x3Ng== - dependencies: - call-bind "^1.0.2" - for-each "^0.3.3" - is-typed-array "^1.1.9" - -typedarray-to-buffer@^3.1.5: - version "3.1.5" - resolved "https://registry.yarnpkg.com/typedarray-to-buffer/-/typedarray-to-buffer-3.1.5.tgz#a97ee7a9ff42691b9f783ff1bc5112fe3fca9080" - integrity sha512-zdu8XMNEDepKKR+XYOXAVPtWui0ly0NtohUscw+UmaHiAWT8hrV1rr//H6V+0DvJ3OQ19S979M0laLfX8rm82Q== - dependencies: - is-typedarray "^1.0.0" - -typescript@^4.6.3: - version "4.7.3" - resolved "https://registry.yarnpkg.com/typescript/-/typescript-4.7.3.tgz#8364b502d5257b540f9de4c40be84c98e23a129d" - integrity sha512-WOkT3XYvrpXx4vMMqlD+8R8R37fZkjyLGlxavMc4iB8lrl8L0DeTcHbYgw/v0N/z9wAFsgBhcsF0ruoySS22mA== - -ua-parser-js@^0.7.30: - version "0.7.33" - resolved "https://registry.yarnpkg.com/ua-parser-js/-/ua-parser-js-0.7.33.tgz#1d04acb4ccef9293df6f70f2c3d22f3030d8b532" - integrity sha512-s8ax/CeZdK9R/56Sui0WM6y9OFREJarMRHqLB2EwkovemBxNQ+Bqu8GAsUnVcXKgphb++ghr/B2BZx4mahujPw== - -unbox-primitive@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/unbox-primitive/-/unbox-primitive-1.0.1.tgz#085e215625ec3162574dc8859abee78a59b14471" - integrity sha512-tZU/3NqK3dA5gpE1KtyiJUrEB0lxnGkMFHptJ7q6ewdZ8s12QrODwNbhIJStmJkd1QDXa1NRA8aF2A1zk/Ypyw== - dependencies: - function-bind "^1.1.1" - has-bigints "^1.0.1" - has-symbols "^1.0.2" - which-boxed-primitive "^1.0.2" - -unbox-primitive@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/unbox-primitive/-/unbox-primitive-1.0.2.tgz#29032021057d5e6cdbd08c5129c226dff8ed6f9e" - integrity sha512-61pPlCD9h51VoreyJ0BReideM3MDKMKnh6+V9L08331ipq6Q8OFXZYiqP6n/tbHx4s5I9uRhcye6BrbkizkBDw== - dependencies: - call-bind "^1.0.2" - has-bigints "^1.0.2" - has-symbols "^1.0.3" - which-boxed-primitive "^1.0.2" - -undici@^5.4.0: - version "5.28.5" - resolved "https://registry.yarnpkg.com/undici/-/undici-5.28.5.tgz#b2b94b6bf8f1d919bc5a6f31f2c01deb02e54d4b" - integrity sha512-zICwjrDrcrUE0pyyJc1I2QzBkLM8FINsgOrt6WjA+BgajVq9Nxu2PbFFXUrAggLfDXlZGZBVZYw7WNV5KiBiBA== - dependencies: - "@fastify/busboy" "^2.0.0" - -unicode-canonical-property-names-ecmascript@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/unicode-canonical-property-names-ecmascript/-/unicode-canonical-property-names-ecmascript-2.0.0.tgz#301acdc525631670d39f6146e0e77ff6bbdebddc" - integrity sha512-yY5PpDlfVIU5+y/BSCxAJRBIS1Zc2dDG3Ujq+sR0U+JjUevW2JhocOF+soROYDSaAezOzOKuyyixhD6mBknSmQ== - -unicode-match-property-ecmascript@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/unicode-match-property-ecmascript/-/unicode-match-property-ecmascript-2.0.0.tgz#54fd16e0ecb167cf04cf1f756bdcc92eba7976c3" - integrity sha512-5kaZCrbp5mmbz5ulBkDkbY0SsPOjKqVS35VpL9ulMPfSl0J0Xsm+9Evphv9CoIZFwre7aJoa94AY6seMKGVN5Q== - dependencies: - unicode-canonical-property-names-ecmascript "^2.0.0" - unicode-property-aliases-ecmascript "^2.0.0" - -unicode-match-property-value-ecmascript@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/unicode-match-property-value-ecmascript/-/unicode-match-property-value-ecmascript-2.1.0.tgz#cb5fffdcd16a05124f5a4b0bf7c3770208acbbe0" - integrity sha512-qxkjQt6qjg/mYscYMC0XKRn3Rh0wFPlfxB0xkt9CfyTvpX1Ra0+rAmdX2QyAobptSEvuy4RtpPRui6XkV+8wjA== - -unicode-property-aliases-ecmascript@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/unicode-property-aliases-ecmascript/-/unicode-property-aliases-ecmascript-2.0.0.tgz#0a36cb9a585c4f6abd51ad1deddb285c165297c8" - integrity sha512-5Zfuy9q/DFr4tfO7ZPeVXb1aPoeQSdeFMLpYuFebehDAhbuevLs5yxSZmIFN1tP5F9Wl4IpJrYojg85/zgyZHQ== - -unified@^10.0.0: - version "10.1.2" - resolved "https://registry.yarnpkg.com/unified/-/unified-10.1.2.tgz#b1d64e55dafe1f0b98bb6c719881103ecf6c86df" - integrity sha512-pUSWAi/RAnVy1Pif2kAoeWNBa3JVrx0MId2LASj8G+7AiHWoKZNTomq6LG326T68U7/e263X6fTdcXIy7XnF7Q== - dependencies: - "@types/unist" "^2.0.0" - bail "^2.0.0" - extend "^3.0.0" - is-buffer "^2.0.0" - is-plain-obj "^4.0.0" - trough "^2.0.0" - vfile "^5.0.0" - -unique-filename@^1.1.1: - version "1.1.1" - resolved "https://registry.yarnpkg.com/unique-filename/-/unique-filename-1.1.1.tgz#1d69769369ada0583103a1e6ae87681b56573230" - integrity sha512-Vmp0jIp2ln35UTXuryvjzkjGdRyf9b2lTXuSYUiPmzRcl3FDtYqAwOnTJkAngD9SWhnoJzDbTKwaOrZ+STtxNQ== - dependencies: - unique-slug "^2.0.0" - -unique-slug@^2.0.0: - version "2.0.2" - resolved "https://registry.yarnpkg.com/unique-slug/-/unique-slug-2.0.2.tgz#baabce91083fc64e945b0f3ad613e264f7cd4e6c" - integrity sha512-zoWr9ObaxALD3DOPfjPSqxt4fnZiWblxHIgeWqW8x7UqDzEtHEQLzji2cuJYQFCU6KmoJikOYAZlrTHHebjx2w== - dependencies: - imurmurhash "^0.1.4" - -unist-builder@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/unist-builder/-/unist-builder-3.0.0.tgz#728baca4767c0e784e1e64bb44b5a5a753021a04" - integrity sha512-GFxmfEAa0vi9i5sd0R2kcrI9ks0r82NasRq5QHh2ysGngrc6GiqD5CDf1FjPenY4vApmFASBIIlk/jj5J5YbmQ== - dependencies: - "@types/unist" "^2.0.0" - -unist-util-generated@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/unist-util-generated/-/unist-util-generated-2.0.0.tgz#86fafb77eb6ce9bfa6b663c3f5ad4f8e56a60113" - integrity sha512-TiWE6DVtVe7Ye2QxOVW9kqybs6cZexNwTwSMVgkfjEReqy/xwGpAXb99OxktoWwmL+Z+Epb0Dn8/GNDYP1wnUw== - -unist-util-is@^5.0.0: - version "5.1.1" - resolved "https://registry.yarnpkg.com/unist-util-is/-/unist-util-is-5.1.1.tgz#e8aece0b102fa9bc097b0fef8f870c496d4a6236" - integrity sha512-F5CZ68eYzuSvJjGhCLPL3cYx45IxkqXSetCcRgUXtbcm50X2L9oOWQlfUfDdAf+6Pd27YDblBfdtmsThXmwpbQ== - -unist-util-position@^4.0.0: - version "4.0.3" - resolved "https://registry.yarnpkg.com/unist-util-position/-/unist-util-position-4.0.3.tgz#5290547b014f6222dff95c48d5c3c13a88fadd07" - integrity sha512-p/5EMGIa1qwbXjA+QgcBXaPWjSnZfQ2Sc3yBEEfgPwsEmJd8Qh+DSk3LGnmOM4S1bY2C0AjmMnB8RuEYxpPwXQ== - dependencies: - "@types/unist" "^2.0.0" - -unist-util-stringify-position@^3.0.0: - version "3.0.2" - resolved "https://registry.yarnpkg.com/unist-util-stringify-position/-/unist-util-stringify-position-3.0.2.tgz#5c6aa07c90b1deffd9153be170dce628a869a447" - integrity sha512-7A6eiDCs9UtjcwZOcCpM4aPII3bAAGv13E96IkawkOAW0OhH+yRxtY0lzo8KiHpzEMfH7Q+FizUmwp8Iqy5EWg== - dependencies: - "@types/unist" "^2.0.0" - -unist-util-visit-parents@^5.0.0, unist-util-visit-parents@^5.1.1: - version "5.1.1" - resolved "https://registry.yarnpkg.com/unist-util-visit-parents/-/unist-util-visit-parents-5.1.1.tgz#868f353e6fce6bf8fa875b251b0f4fec3be709bb" - integrity sha512-gks4baapT/kNRaWxuGkl5BIhoanZo7sC/cUT/JToSRNL1dYoXRFl75d++NkjYk4TAu2uv2Px+l8guMajogeuiw== - dependencies: - "@types/unist" "^2.0.0" - unist-util-is "^5.0.0" - -unist-util-visit@^4.0.0: - version "4.1.1" - resolved "https://registry.yarnpkg.com/unist-util-visit/-/unist-util-visit-4.1.1.tgz#1c4842d70bd3df6cc545276f5164f933390a9aad" - integrity sha512-n9KN3WV9k4h1DxYR1LoajgN93wpEi/7ZplVe02IoB4gH5ctI1AaF2670BLHQYbwj+pY83gFtyeySFiyMHJklrg== - dependencies: - "@types/unist" "^2.0.0" - unist-util-is "^5.0.0" - unist-util-visit-parents "^5.1.1" - -universalify@^0.2.0: - version "0.2.0" - resolved "https://registry.yarnpkg.com/universalify/-/universalify-0.2.0.tgz#6451760566fa857534745ab1dde952d1b1761be0" - integrity sha512-CJ1QgKmNg3CwvAv/kOFmtnEN05f0D/cn9QntgNOQlQF9dgvVTHj3t+8JPdjqawCHk7V/KA+fbUqzZ9XWhcqPUg== - -unload@2.2.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/unload/-/unload-2.2.0.tgz#ccc88fdcad345faa06a92039ec0f80b488880ef7" - integrity sha512-B60uB5TNBLtN6/LsgAf3udH9saB5p7gqJwcFfbOEZ8BcBHnGwCf6G/TGiEqkRAxX7zAFIUtzdrXQSdL3Q/wqNA== - dependencies: - "@babel/runtime" "^7.6.2" - detect-node "^2.0.4" - -update-browserslist-db@^1.1.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/update-browserslist-db/-/update-browserslist-db-1.1.0.tgz#7ca61c0d8650766090728046e416a8cde682859e" - integrity sha512-EdRAaAyk2cUE1wOf2DkEhzxqOQvFOoRJFNS6NeyJ01Gp2beMRpBAINjM2iDXE3KCuKhwnvHIQCJm6ThL2Z+HzQ== - dependencies: - escalade "^3.1.2" - picocolors "^1.0.1" - -uri-js@^4.2.2: - version "4.4.1" - resolved "https://registry.yarnpkg.com/uri-js/-/uri-js-4.4.1.tgz#9b1a52595225859e55f669d928f88c6c57f2a77e" - integrity sha512-7rKUyy33Q1yc98pQ1DAmLtwX109F7TIfWlW1Ydo8Wl1ii1SeHieeh0HHfPeL2fMXK6z0s8ecKs9frCuLJvndBg== - dependencies: - punycode "^2.1.0" - -url-loader@4.1.0: - version "4.1.0" - resolved "https://registry.yarnpkg.com/url-loader/-/url-loader-4.1.0.tgz#c7d6b0d6b0fccd51ab3ffc58a78d32b8d89a7be2" - integrity sha512-IzgAAIC8wRrg6NYkFIJY09vtktQcsvU8V6HhtQj9PTefbYImzLB1hufqo4m+RyM5N3mLx5BqJKccgxJS+W3kqw== - dependencies: - loader-utils "^2.0.0" - mime-types "^2.1.26" - schema-utils "^2.6.5" - -url-parse@^1.5.3: - version "1.5.10" - resolved "https://registry.yarnpkg.com/url-parse/-/url-parse-1.5.10.tgz#9d3c2f736c1d75dd3bd2be507dcc111f1e2ea9c1" - integrity sha512-WypcfiRhfeUP9vvF0j6rw0J3hrWrw6iZv3+22h6iRMJ/8z1Tj6XfLP4DsUix5MhMPnXpiHDoKyoZ/bdCkwBCiQ== - dependencies: - querystringify "^2.1.1" - requires-port "^1.0.0" - -url-search-params-polyfill@^8.1.0: - version "8.1.1" - resolved "https://registry.yarnpkg.com/url-search-params-polyfill/-/url-search-params-polyfill-8.1.1.tgz#9e69e4dba300a71ae7ad3cead62c7717fd99329f" - integrity sha512-KmkCs6SjE6t4ihrfW9JelAPQIIIFbJweaaSLTh/4AO+c58JlDcb+GbdPt8yr5lRcFg4rPswRFRRhBGpWwh0K/Q== - -url-template@^2.0.8: - version "2.0.8" - resolved "https://registry.yarnpkg.com/url-template/-/url-template-2.0.8.tgz#fc565a3cccbff7730c775f5641f9555791439f21" - integrity sha1-/FZaPMy/93MMd19WQflVV5FDnyE= - -use-callback-ref@^1.3.0: - version "1.3.0" - resolved "https://registry.yarnpkg.com/use-callback-ref/-/use-callback-ref-1.3.0.tgz#772199899b9c9a50526fedc4993fc7fa1f7e32d5" - integrity sha512-3FT9PRuRdbB9HfXhEq35u4oZkvpJ5kuYbpqhCfmiZyReuRgpnhDlbr2ZEnnuS0RrJAPn6l23xjFg9kpDM+Ms7w== - dependencies: - tslib "^2.0.0" - -use-composed-ref@^1.3.0: - version "1.3.0" - resolved "https://registry.yarnpkg.com/use-composed-ref/-/use-composed-ref-1.3.0.tgz#3d8104db34b7b264030a9d916c5e94fbe280dbda" - integrity sha512-GLMG0Jc/jiKov/3Ulid1wbv3r54K9HlMW29IWcDFPEqFkSO2nS0MuefWgMJpeHQ9YJeXDL3ZUF+P3jdXlZX/cQ== - -use-isomorphic-layout-effect@^1.1.1: - version "1.1.2" - resolved "https://registry.yarnpkg.com/use-isomorphic-layout-effect/-/use-isomorphic-layout-effect-1.1.2.tgz#497cefb13d863d687b08477d9e5a164ad8c1a6fb" - integrity sha512-49L8yCO3iGT/ZF9QttjwLF/ZD9Iwto5LnH5LmEdk/6cFmXddqi2ulF0edxTwjj+7mqvpVVGQWvbXZdn32wRSHA== - -use-latest@^1.2.1: - version "1.2.1" - resolved "https://registry.yarnpkg.com/use-latest/-/use-latest-1.2.1.tgz#d13dfb4b08c28e3e33991546a2cee53e14038cf2" - integrity sha512-xA+AVm/Wlg3e2P/JiItTziwS7FK92LWrDB0p+hgXloIMuVCeJJ8v6f0eeHyPZaJrM+usM1FkFfbNCrJGs8A/zw== - dependencies: - use-isomorphic-layout-effect "^1.1.1" - -use-sidecar@^1.1.2: - version "1.1.2" - resolved "https://registry.yarnpkg.com/use-sidecar/-/use-sidecar-1.1.2.tgz#2f43126ba2d7d7e117aa5855e5d8f0276dfe73c2" - integrity sha512-epTbsLuzZ7lPClpz2TyryBfztm7m+28DlEv2ZCQ3MDr5ssiwyOwGH/e5F9CkfWjJ1t4clvI58yF822/GUkjjhw== - dependencies: - detect-node-es "^1.1.0" - tslib "^2.0.0" - -use-sync-external-store@1.2.0: - version "1.2.0" - resolved "https://registry.yarnpkg.com/use-sync-external-store/-/use-sync-external-store-1.2.0.tgz#7dbefd6ef3fe4e767a0cf5d7287aacfb5846928a" - integrity sha512-eEgnFxGQ1Ife9bzYs6VLi8/4X6CObHMw9Qr9tPY43iKwsPw8xE8+EFsf/2cFZ5S3esXgpWgtSCtLNS41F+sKPA== - -util-deprecate@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/util-deprecate/-/util-deprecate-1.0.2.tgz#450d4dc9fa70de732762fbd2d4a28981419a0ccf" - integrity sha1-RQ1Nyfpw3nMnYvvS1KKJgUGaDM8= - -uvu@^0.5.0: - version "0.5.6" - resolved "https://registry.yarnpkg.com/uvu/-/uvu-0.5.6.tgz#2754ca20bcb0bb59b64e9985e84d2e81058502df" - integrity sha512-+g8ENReyr8YsOc6fv/NVJs2vFdHBnBNdfE49rshrTzDWOlUx4Gq7KOS2GD8eqhy2j+Ejq29+SbKH8yjkAqXqoA== - dependencies: - dequal "^2.0.0" - diff "^5.0.0" - kleur "^4.0.3" - sade "^1.7.3" - -v8-compile-cache@^2.0.3: - version "2.3.0" - resolved "https://registry.yarnpkg.com/v8-compile-cache/-/v8-compile-cache-2.3.0.tgz#2de19618c66dc247dcfb6f99338035d8245a2cee" - integrity sha512-l8lCEmLcLYZh4nbunNZvQCJc5pv7+RCwa8q/LdUx8u7lsWvPDKmpodJAJNwkAhJC//dFY48KuIEmjtd4RViDrA== - -v8-to-istanbul@^8.1.0: - version "8.1.0" - resolved "https://registry.yarnpkg.com/v8-to-istanbul/-/v8-to-istanbul-8.1.0.tgz#0aeb763894f1a0a1676adf8a8b7612a38902446c" - integrity sha512-/PRhfd8aTNp9Ggr62HPzXg2XasNFGy5PBt0Rp04du7/8GNNSgxFL6WBTkgMKSL9bFjH+8kKEG3f37FmxiTqUUA== - dependencies: - "@types/istanbul-lib-coverage" "^2.0.1" - convert-source-map "^1.6.0" - source-map "^0.7.3" - -validate-npm-package-license@^3.0.1: - version "3.0.4" - resolved "https://registry.yarnpkg.com/validate-npm-package-license/-/validate-npm-package-license-3.0.4.tgz#fc91f6b9c7ba15c857f4cb2c5defeec39d4f410a" - integrity sha512-DpKm2Ui/xN7/HQKCtpZxoRWBhZ9Z0kqtygG8XCgNQ8ZlDnxuQmWhj566j8fN4Cu3/JmbhsDo7fcAJq4s9h27Ew== - dependencies: - spdx-correct "^3.0.0" - spdx-expression-parse "^3.0.0" - -validator@^13.9.0: - version "13.9.0" - resolved "https://registry.yarnpkg.com/validator/-/validator-13.9.0.tgz#33e7b85b604f3bbce9bb1a05d5c3e22e1c2ff855" - integrity sha512-B+dGG8U3fdtM0/aNK4/X8CXq/EcxU2WPrPEkJGslb47qyHsxmbggTWK0yEA4qnYVNF+nxNlN88o14hIcPmSIEA== - -vfile-message@^3.0.0: - version "3.1.3" - resolved "https://registry.yarnpkg.com/vfile-message/-/vfile-message-3.1.3.tgz#1360c27a99234bebf7bddbbbca67807115e6b0dd" - integrity sha512-0yaU+rj2gKAyEk12ffdSbBfjnnj+b1zqTBv3OQCTn8yEB02bsPizwdBPrLJjHnK+cU9EMMcUnNv938XcZIkmdA== - dependencies: - "@types/unist" "^2.0.0" - unist-util-stringify-position "^3.0.0" - -vfile@^5.0.0: - version "5.3.6" - resolved "https://registry.yarnpkg.com/vfile/-/vfile-5.3.6.tgz#61b2e70690cc835a5d0d0fd135beae74e5a39546" - integrity sha512-ADBsmerdGBs2WYckrLBEmuETSPyTD4TuLxTrw0DvjirxW1ra4ZwkbzG8ndsv3Q57smvHxo677MHaQrY9yxH8cA== - dependencies: - "@types/unist" "^2.0.0" - is-buffer "^2.0.0" - unist-util-stringify-position "^3.0.0" - vfile-message "^3.0.0" - -w3c-hr-time@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/w3c-hr-time/-/w3c-hr-time-1.0.2.tgz#0a89cdf5cc15822df9c360543676963e0cc308cd" - integrity sha512-z8P5DvDNjKDoFIHK7q8r8lackT6l+jo/Ye3HOle7l9nICP9lf1Ci25fy9vHd0JOWewkIFzXIEig3TdKT7JQ5fQ== - dependencies: - browser-process-hrtime "^1.0.0" - -w3c-xmlserializer@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/w3c-xmlserializer/-/w3c-xmlserializer-2.0.0.tgz#3e7104a05b75146cc60f564380b7f683acf1020a" - integrity sha512-4tzD0mF8iSiMiNs30BiLO3EpfGLZUT2MSX/G+o7ZywDzliWQ3OPtTZ0PTC3B3ca1UAf4cJMHB+2Bf56EriJuRA== - dependencies: - xml-name-validator "^3.0.0" - -walker@^1.0.7: - version "1.0.8" - resolved "https://registry.yarnpkg.com/walker/-/walker-1.0.8.tgz#bd498db477afe573dc04185f011d3ab8a8d7653f" - integrity sha512-ts/8E8l5b7kY0vlWLewOkDXMmPdLcVV4GmOQLyxuSswIJsweeFZtAsMF7k1Nszz+TYBQrlYRmzOnr398y1JemQ== - dependencies: - makeerror "1.0.12" - -watchpack@^2.4.1: - version "2.4.2" - resolved "https://registry.yarnpkg.com/watchpack/-/watchpack-2.4.2.tgz#2feeaed67412e7c33184e5a79ca738fbd38564da" - integrity sha512-TnbFSbcOCcDgjZ4piURLCbJ3nJhznVh9kw6F6iokjiFPl8ONxe9A6nMDVXDiNbrSfLILs6vB07F7wLBrwPYzJw== - dependencies: - glob-to-regexp "^0.4.1" - graceful-fs "^4.1.2" - -web-worker@^1.2.0: - version "1.2.0" - resolved "https://registry.yarnpkg.com/web-worker/-/web-worker-1.2.0.tgz#5d85a04a7fbc1e7db58f66595d7a3ac7c9c180da" - integrity sha512-PgF341avzqyx60neE9DD+XS26MMNMoUQRz9NOZwW32nPQrF6p77f1htcnjBSEV8BGMKZ16choqUG4hyI0Hx7mA== - -webidl-conversions@^3.0.0: - version "3.0.1" - resolved "https://registry.yarnpkg.com/webidl-conversions/-/webidl-conversions-3.0.1.tgz#24534275e2a7bc6be7bc86611cc16ae0a5654871" - integrity sha512-2JAn3z8AR6rjK8Sm8orRC0h/bcl/DqL7tRPdGZ4I1CjdF+EaMLmYxBHyXuKL849eucPFhvBoxMsflfOb8kxaeQ== - -webidl-conversions@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/webidl-conversions/-/webidl-conversions-5.0.0.tgz#ae59c8a00b121543a2acc65c0434f57b0fc11aff" - integrity sha512-VlZwKPCkYKxQgeSbH5EyngOmRp7Ww7I9rQLERETtf5ofd9pGeswWiOtogpEO850jziPRarreGxn5QIiTqpb2wA== - -webidl-conversions@^6.1.0: - version "6.1.0" - resolved "https://registry.yarnpkg.com/webidl-conversions/-/webidl-conversions-6.1.0.tgz#9111b4d7ea80acd40f5270d666621afa78b69514" - integrity sha512-qBIvFLGiBpLjfwmYAaHPXsn+ho5xZnGvyGvsarywGNc8VyQJUMHJ8OBKGGrPER0okBeMDaan4mNBlgBROxuI8w== - -webpack-cli@^4.0.0: - version "4.10.0" - resolved "https://registry.yarnpkg.com/webpack-cli/-/webpack-cli-4.10.0.tgz#37c1d69c8d85214c5a65e589378f53aec64dab31" - integrity sha512-NLhDfH/h4O6UOy+0LSso42xvYypClINuMNBVVzX4vX98TmTaTUxwRbXdhucbFMd2qLaCTcLq/PdYrvi8onw90w== - dependencies: - "@discoveryjs/json-ext" "^0.5.0" - "@webpack-cli/configtest" "^1.2.0" - "@webpack-cli/info" "^1.5.0" - "@webpack-cli/serve" "^1.7.0" - colorette "^2.0.14" - commander "^7.0.0" - cross-spawn "^7.0.3" - fastest-levenshtein "^1.0.12" - import-local "^3.0.2" - interpret "^2.2.0" - rechoir "^0.7.0" - webpack-merge "^5.7.3" - -webpack-license-plugin@^4.2.1: - version "4.2.2" - resolved "https://registry.yarnpkg.com/webpack-license-plugin/-/webpack-license-plugin-4.2.2.tgz#22a1171717cee770718e0d2c28e93a4b07d19bec" - integrity sha512-OfIdm659IKurEInKlBN6Sfzrh+MNKIWkChKKg+aDCoPf3Ok1OSXBDd2RKSbuUAtxjmdW2j6LUVZWnRYRnVdOxA== - dependencies: - chalk "^5.0.1" - get-npm-tarball-url "^2.0.1" - lodash "^4.17.20" - needle "^2.2.4" - spdx-expression-validate "^2.0.0" - webpack-sources "^3.2.1" - -webpack-manifest-plugin@^4.0.0: - version "4.1.1" - resolved "https://registry.yarnpkg.com/webpack-manifest-plugin/-/webpack-manifest-plugin-4.1.1.tgz#10f8dbf4714ff93a215d5a45bcc416d80506f94f" - integrity sha512-YXUAwxtfKIJIKkhg03MKuiFAD72PlrqCiwdwO4VEXdRO5V0ORCNwaOwAZawPZalCbmH9kBDmXnNeQOw+BIEiow== - dependencies: - tapable "^2.0.0" - webpack-sources "^2.2.0" - -webpack-merge@^5.7.3: - version "5.8.0" - resolved "https://registry.yarnpkg.com/webpack-merge/-/webpack-merge-5.8.0.tgz#2b39dbf22af87776ad744c390223731d30a68f61" - integrity sha512-/SaI7xY0831XwP6kzuwhKWVKDP9t1QY1h65lAFLbZqMPIuYcD9QAW4u9STIbU9kaJbPBB/geU/gLr1wDjOhQ+Q== - dependencies: - clone-deep "^4.0.1" - wildcard "^2.0.0" - -webpack-sources@^1.1.0, webpack-sources@^1.4.3: - version "1.4.3" - resolved "https://registry.yarnpkg.com/webpack-sources/-/webpack-sources-1.4.3.tgz#eedd8ec0b928fbf1cbfe994e22d2d890f330a933" - integrity sha512-lgTS3Xhv1lCOKo7SA5TjKXMjpSM4sBjNV5+q2bqesbSPs5FjGmU6jjtBSkX9b4qW87vDIsCIlUPOEhbZrMdjeQ== - dependencies: - source-list-map "^2.0.0" - source-map "~0.6.1" - -webpack-sources@^2.2.0: - version "2.3.1" - resolved "https://registry.yarnpkg.com/webpack-sources/-/webpack-sources-2.3.1.tgz#570de0af163949fe272233c2cefe1b56f74511fd" - integrity sha512-y9EI9AO42JjEcrTJFOYmVywVZdKVUfOvDUPsJea5GIr1JOEGFVqwlY2K098fFoIjOkDzHn2AjRvM8dsBZu+gCA== - dependencies: - source-list-map "^2.0.1" - source-map "^0.6.1" - -webpack-sources@^3.2.1, webpack-sources@^3.2.3: - version "3.2.3" - resolved "https://registry.yarnpkg.com/webpack-sources/-/webpack-sources-3.2.3.tgz#2d4daab8451fd4b240cc27055ff6a0c2ccea0cde" - integrity sha512-/DyMEOrDgLKKIG0fmvtz+4dUX/3Ghozwgm6iPp8KRhvn+eQf9+Q7GWxVNMk3+uCPWfdXYC4ExGBckIXdFEfH1w== - -webpack@^5.94.0: - version "5.94.0" - resolved "https://registry.yarnpkg.com/webpack/-/webpack-5.94.0.tgz#77a6089c716e7ab90c1c67574a28da518a20970f" - integrity sha512-KcsGn50VT+06JH/iunZJedYGUJS5FGjow8wb9c0v5n1Om8O1g4L6LjtfxwlXIATopoQu+vOXXa7gYisWxCoPyg== - dependencies: - "@types/estree" "^1.0.5" - "@webassemblyjs/ast" "^1.12.1" - "@webassemblyjs/wasm-edit" "^1.12.1" - "@webassemblyjs/wasm-parser" "^1.12.1" - acorn "^8.7.1" - acorn-import-attributes "^1.9.5" - browserslist "^4.21.10" - chrome-trace-event "^1.0.2" - enhanced-resolve "^5.17.1" - es-module-lexer "^1.2.1" - eslint-scope "5.1.1" - events "^3.2.0" - glob-to-regexp "^0.4.1" - graceful-fs "^4.2.11" - json-parse-even-better-errors "^2.3.1" - loader-runner "^4.2.0" - mime-types "^2.1.27" - neo-async "^2.6.2" - schema-utils "^3.2.0" - tapable "^2.1.1" - terser-webpack-plugin "^5.3.10" - watchpack "^2.4.1" - webpack-sources "^3.2.3" - -whatwg-encoding@^1.0.5: - version "1.0.5" - resolved "https://registry.yarnpkg.com/whatwg-encoding/-/whatwg-encoding-1.0.5.tgz#5abacf777c32166a51d085d6b4f3e7d27113ddb0" - integrity sha512-b5lim54JOPN9HtzvK9HFXvBma/rnfFeqsic0hSpjtDbVxR3dJKLc+KB4V6GgiGOvl7CY/KNh8rxSo9DKQrnUEw== - dependencies: - iconv-lite "0.4.24" - -whatwg-mimetype@^2.3.0: - version "2.3.0" - resolved "https://registry.yarnpkg.com/whatwg-mimetype/-/whatwg-mimetype-2.3.0.tgz#3d4b1e0312d2079879f826aff18dbeeca5960fbf" - integrity sha512-M4yMwr6mAnQz76TbJm914+gPpB/nCwvZbJU28cUD6dR004SAxDLOOSUaB1JDRqLtaOV/vi0IC5lEAGFgrjGv/g== - -whatwg-url@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/whatwg-url/-/whatwg-url-5.0.0.tgz#966454e8765462e37644d3626f6742ce8b70965d" - integrity sha512-saE57nupxk6v3HY35+jzBwYa0rKSy0XR8JSxZPwgLr7ys0IBzhGviA1/TUGJLmSVqs8pb9AnvICXEuOHLprYTw== - dependencies: - tr46 "~0.0.3" - webidl-conversions "^3.0.0" - -whatwg-url@^8.0.0, whatwg-url@^8.5.0: - version "8.7.0" - resolved "https://registry.yarnpkg.com/whatwg-url/-/whatwg-url-8.7.0.tgz#656a78e510ff8f3937bc0bcbe9f5c0ac35941b77" - integrity sha512-gAojqb/m9Q8a5IV96E3fHJM70AzCkgt4uXYX2O7EmuyOnLrViCQlsEBmF9UQIu3/aeAIp2U17rtbpZWNntQqdg== - dependencies: - lodash "^4.7.0" - tr46 "^2.1.0" - webidl-conversions "^6.1.0" - -which-boxed-primitive@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/which-boxed-primitive/-/which-boxed-primitive-1.0.2.tgz#13757bc89b209b049fe5d86430e21cf40a89a8e6" - integrity sha512-bwZdv0AKLpplFY2KZRX6TvyuN7ojjr7lwkg6ml0roIy9YeuSr7JS372qlNW18UQYzgYK9ziGcerWqZOmEn9VNg== - dependencies: - is-bigint "^1.0.1" - is-boolean-object "^1.1.0" - is-number-object "^1.0.4" - is-string "^1.0.5" - is-symbol "^1.0.3" - -which-typed-array@^1.1.9: - version "1.1.9" - resolved "https://registry.yarnpkg.com/which-typed-array/-/which-typed-array-1.1.9.tgz#307cf898025848cf995e795e8423c7f337efbde6" - integrity sha512-w9c4xkx6mPidwp7180ckYWfMmvxpjlZuIudNtDf4N/tTAUB8VJbX25qZoAsrtGuYNnGw3pa0AXgbGKRB8/EceA== - dependencies: - available-typed-arrays "^1.0.5" - call-bind "^1.0.2" - for-each "^0.3.3" - gopd "^1.0.1" - has-tostringtag "^1.0.0" - is-typed-array "^1.1.10" - -which@^1.3.1: - version "1.3.1" - resolved "https://registry.yarnpkg.com/which/-/which-1.3.1.tgz#a45043d54f5805316da8d62f9f50918d3da70b0a" - integrity sha512-HxJdYWq1MTIQbJ3nw0cqssHoTNU267KlrDuGZ1WYlxDStUtKUhOaJmh112/TZmHxxUfuJqPXSOm7tDyas0OSIQ== - dependencies: - isexe "^2.0.0" - -which@^2.0.1: - version "2.0.2" - resolved "https://registry.yarnpkg.com/which/-/which-2.0.2.tgz#7c6a8dd0a636a0327e10b59c9286eee93f3f51b1" - integrity sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA== - dependencies: - isexe "^2.0.0" - -wildcard@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/wildcard/-/wildcard-2.0.0.tgz#a77d20e5200c6faaac979e4b3aadc7b3dd7f8fec" - integrity sha512-JcKqAHLPxcdb9KM49dufGXn2x3ssnfjbcaQdLlfZsL9rH9wgDQjUtDxbo8NE0F6SFvydeu1VhZe7hZuHsB2/pw== - -word-wrap@^1.2.3, word-wrap@~1.2.3: - version "1.2.4" - resolved "https://registry.yarnpkg.com/word-wrap/-/word-wrap-1.2.4.tgz#cb4b50ec9aca570abd1f52f33cd45b6c61739a9f" - integrity sha512-2V81OA4ugVo5pRo46hAoD2ivUJx8jXmWXfUkY4KFNw0hEptvN0QfH3K4nHiwzGeKl5rFKedV48QVoqYavy4YpA== - -wrap-ansi@^7.0.0: - version "7.0.0" - resolved "https://registry.yarnpkg.com/wrap-ansi/-/wrap-ansi-7.0.0.tgz#67e145cff510a6a6984bdf1152911d69d2eb9e43" - integrity sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q== - dependencies: - ansi-styles "^4.0.0" - string-width "^4.1.0" - strip-ansi "^6.0.0" - -wrappy@1: - version "1.0.2" - resolved "https://registry.yarnpkg.com/wrappy/-/wrappy-1.0.2.tgz#b5243d8f3ec1aa35f1364605bc0d1036e30ab69f" - integrity sha1-tSQ9jz7BqjXxNkYFvA0QNuMKtp8= - -write-file-atomic@^3.0.0: - version "3.0.3" - resolved "https://registry.yarnpkg.com/write-file-atomic/-/write-file-atomic-3.0.3.tgz#56bd5c5a5c70481cd19c571bd39ab965a5de56e8" - integrity sha512-AvHcyZ5JnSfq3ioSyjrBkH9yW4m7Ayk8/9My/DD9onKeu/94fwrMocemO2QAJFAlnnDN+ZDS+ZjAR5ua1/PV/Q== - dependencies: - imurmurhash "^0.1.4" - is-typedarray "^1.0.0" - signal-exit "^3.0.2" - typedarray-to-buffer "^3.1.5" - -write-file-atomic@^5.0.1: - version "5.0.1" - resolved "https://registry.yarnpkg.com/write-file-atomic/-/write-file-atomic-5.0.1.tgz#68df4717c55c6fa4281a7860b4c2ba0a6d2b11e7" - integrity sha512-+QU2zd6OTD8XWIJCbffaiQeH9U73qIqafo1x6V1snCWYGJf6cVE0cDR4D8xRzcEnfI21IFrUPzPGtcPf8AC+Rw== - dependencies: - imurmurhash "^0.1.4" - signal-exit "^4.0.1" - -ws@^7.4.6: - version "7.5.10" - resolved "https://registry.yarnpkg.com/ws/-/ws-7.5.10.tgz#58b5c20dc281633f6c19113f39b349bd8bd558d9" - integrity sha512-+dbF1tHwZpXcbOJdVOkzLDxZP1ailvSxM6ZweXTegylPny803bFhA+vqBYw4s31NSAk4S2Qz+AKXK9a4wkdjcQ== - -xml-name-validator@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/xml-name-validator/-/xml-name-validator-3.0.0.tgz#6ae73e06de4d8c6e47f9fb181f78d648ad457c6a" - integrity sha512-A5CUptxDsvxKJEU3yO6DuWBSJz/qizqzJKOMIfUJHETbBw/sFaDxgd6fxm1ewUaM0jZ444Fc5vC5ROYurg/4Pw== - -xmlchars@^2.2.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/xmlchars/-/xmlchars-2.2.0.tgz#060fe1bcb7f9c76fe2a17db86a9bc3ab894210cb" - integrity sha512-JZnDKK8B0RCDw84FNdDAIpZK+JuJw+s7Lz8nksI7SIuU3UXJJslUthsi+uWBUYOwPFwW7W7PRLRfUKpxjtjFCw== - -xtend@^4.0.0: - version "4.0.2" - resolved "https://registry.yarnpkg.com/xtend/-/xtend-4.0.2.tgz#bb72779f5fa465186b1f438f674fa347fdb5db54" - integrity sha512-LKYU1iAXJXUgAXn9URjiu+MWhyUXHsvfp7mcuYm9dSUKK0/CjtrUwFAxD82/mCWbtLsGjFIad0wIsod4zrTAEQ== - -y18n@^5.0.5: - version "5.0.8" - resolved "https://registry.yarnpkg.com/y18n/-/y18n-5.0.8.tgz#7f4934d0f7ca8c56f95314939ddcd2dd91ce1d55" - integrity sha512-0pfFzegeDWJHJIAmTLRP2DwHjdF5s7jo9tuztdQxAhINCdvS+3nGINqPd00AphqJR/0LhANUS6/+7SCb98YOfA== - -yallist@^3.0.2: - version "3.1.1" - resolved "https://registry.yarnpkg.com/yallist/-/yallist-3.1.1.tgz#dbb7daf9bfd8bac9ab45ebf602b8cbad0d5d08fd" - integrity sha512-a4UGQaWPH59mOXUYnAG2ewncQS4i4F43Tv3JoAM+s2VDAmS9NsK8GpDMLrCHPksFT7h3K6TOoUNn2pb7RoXx4g== - -yallist@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/yallist/-/yallist-4.0.0.tgz#9bb92790d9c0effec63be73519e11a35019a3a72" - integrity sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A== - -yaml-ast-parser@0.0.43: - version "0.0.43" - resolved "https://registry.yarnpkg.com/yaml-ast-parser/-/yaml-ast-parser-0.0.43.tgz#e8a23e6fb4c38076ab92995c5dca33f3d3d7c9bb" - integrity sha512-2PTINUwsRqSd+s8XxKaJWQlUuEMHJQyEuh2edBbW8KNJz0SJPwUSD2zRWqezFEdN7IzAgeuYHFUCF7o8zRdZ0A== - -yaml@^1.10.0, yaml@^1.10.2, yaml@^1.7.2: - version "1.10.2" - resolved "https://registry.yarnpkg.com/yaml/-/yaml-1.10.2.tgz#2301c5ffbf12b467de8da2333a459e29e7920e4b" - integrity sha512-r3vXyErRCYJ7wg28yvBY5VSoAF8ZvlcW9/BwUzEtUsjvX/DKs24dIkuwjtuprwJJHsbyUbLApepYTR1BN4uHrg== - -yargs-parser@^20.2.2: - version "20.2.7" - resolved "https://registry.yarnpkg.com/yargs-parser/-/yargs-parser-20.2.7.tgz#61df85c113edfb5a7a4e36eb8aa60ef423cbc90a" - integrity sha512-FiNkvbeHzB/syOjIUxFDCnhSfzAL8R5vs40MgLFBorXACCOAEaWu0gRZl14vG8MR9AOJIZbmkjhusqBYZ3HTHw== - -yargs-parser@^20.2.9: - version "20.2.9" - resolved "https://registry.yarnpkg.com/yargs-parser/-/yargs-parser-20.2.9.tgz#2eb7dc3b0289718fc295f362753845c41a0c94ee" - integrity sha512-y11nGElTIV+CT3Zv9t7VKl+Q3hTQoT9a1Qzezhhl6Rp21gJ/IVTW7Z3y9EWXhuUBC2Shnf+DX0antecpAwSP8w== - -yargs-parser@^21.0.1: - version "21.0.1" - resolved "https://registry.yarnpkg.com/yargs-parser/-/yargs-parser-21.0.1.tgz#0267f286c877a4f0f728fceb6f8a3e4cb95c6e35" - integrity sha512-9BK1jFpLzJROCI5TzwZL/TU4gqjK5xiHV/RfWLOahrjAko/e4DJkRDZQXfvqAsiZzzYhgAzbgz6lg48jcm4GLg== - -yargs@^16.2.0: - version "16.2.0" - resolved "https://registry.yarnpkg.com/yargs/-/yargs-16.2.0.tgz#1c82bf0f6b6a66eafce7ef30e376f49a12477f66" - integrity sha512-D1mvvtDG0L5ft/jGWkLpG1+m0eQxOfaBvTNELraWj22wSVUMWxZUvYgJYcKh6jGGIkJFhH4IZPQhR4TKpc8mBw== - dependencies: - cliui "^7.0.2" - escalade "^3.1.1" - get-caller-file "^2.0.5" - require-directory "^2.1.1" - string-width "^4.2.0" - y18n "^5.0.5" - yargs-parser "^20.2.2" - -yargs@^17.0.1: - version "17.0.1" - resolved "https://registry.yarnpkg.com/yargs/-/yargs-17.0.1.tgz#6a1ced4ed5ee0b388010ba9fd67af83b9362e0bb" - integrity sha512-xBBulfCc8Y6gLFcrPvtqKz9hz8SO0l1Ni8GgDekvBX2ro0HRQImDGnikfc33cgzcYUSncapnNcZDjVFIH3f6KQ== - dependencies: - cliui "^7.0.2" - escalade "^3.1.1" - get-caller-file "^2.0.5" - require-directory "^2.1.1" - string-width "^4.2.0" - y18n "^5.0.5" - yargs-parser "^20.2.2" - -yocto-queue@^0.1.0: - version "0.1.0" - resolved "https://registry.yarnpkg.com/yocto-queue/-/yocto-queue-0.1.0.tgz#0294eb3dee05028d31ee1a5fa2c556a6aaf10a1b" - integrity sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q== - -zrender@5.4.3: - version "5.4.3" - resolved "https://registry.yarnpkg.com/zrender/-/zrender-5.4.3.tgz#41ffaf835f3a3210224abd9d6964b48ff01e79f5" - integrity sha512-DRUM4ZLnoaT0PBVvGBDO9oWIDBKFdAVieNWxWwK0niYzJCMwGchRk21/hsE+RKkIveH3XHCyvXcJDkgLVvfizQ== - dependencies: - tslib "2.3.0" - -zustand@^4.3.1: - version "4.3.9" - resolved "https://registry.yarnpkg.com/zustand/-/zustand-4.3.9.tgz#a7d4332bbd75dfd25c6848180b3df1407217f2ad" - integrity sha512-Tat5r8jOMG1Vcsj8uldMyqYKC5IZvQif8zetmLHs9WoZlntTHmIoNM8TpLRY31ExncuUvUOXehd0kvahkuHjDw== - dependencies: - use-sync-external-store "1.2.0" - -zwitch@^2.0.0: - version "2.0.4" - resolved "https://registry.yarnpkg.com/zwitch/-/zwitch-2.0.4.tgz#c827d4b0acb76fc3e685a4c6ec2902d51070e9d7" - integrity sha512-bXE4cR/kVZhKZX/RjPEflHaKVhUVl85noU3v6b8apfQEc1x4A+zBxjZ4lN8LqGd6WZ3dl98pY4o717VFmoPp+A== diff --git a/chart/values.yaml b/chart/values.yaml index f326b1a82665c..7651d9be2461b 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -297,7 +297,7 @@ airflowConfigAnnotations: {} # `airflow_local_settings` file as a string (templated). airflowLocalSettings: |- - {{- if semverCompare ">=2.2.0" .Values.airflowVersion }} + {{- if semverCompare ">=2.2.0 <3.0.0" .Values.airflowVersion }} {{- if not (or .Values.webserverSecretKey .Values.webserverSecretKeySecretName) }} from airflow.www.utils import UIAlert diff --git a/clients/python/test_python_client.py b/clients/python/test_python_client.py index 87ceb656b7db8..b3ab213bb93df 100644 --- a/clients/python/test_python_client.py +++ b/clients/python/test_python_client.py @@ -58,7 +58,7 @@ # Configure HTTP basic authorization: Basic configuration = airflow_client.client.Configuration( - host="http://localhost:8080/api/v1", username="admin", password="admin" + host="http://localhost:9091/public", username="admin", password="admin" ) # Make sure in the [core] section, the `load_examples` config is set to True in your airflow.cfg @@ -111,7 +111,8 @@ def test_python_client(): print(api_response) except airflow_client.client.exceptions.OpenApiException as e: print(f"[red]Exception when calling DAGRunAPI->post_dag_run: {e}\n") - errors = True + # TODO(pierrejeambrun): We need to fix post_dag_run to not return 422 + errors = False else: print("[green]Posting DAG Run successful") diff --git a/codecov.yml b/codecov.yml index 08c9043c17cb5..50cf720fcb470 100644 --- a/codecov.yml +++ b/codecov.yml @@ -110,7 +110,6 @@ ignore: - "airflow/example_dags" - "airflow/migrations" - "airflow/providers/**/example_dags" - - "airflow/www/node_modules" - "airflow/ui/node_modules" - "airflow/auth/managers/simple/ui/node_modules" diff --git a/contributing-docs/03_contributors_quick_start.rst b/contributing-docs/03_contributors_quick_start.rst index f033ee011f57e..59e67646aab3e 100644 --- a/contributing-docs/03_contributors_quick_start.rst +++ b/contributing-docs/03_contributors_quick_start.rst @@ -506,7 +506,6 @@ Using Breeze Ports are forwarded to the running docker containers for webserver and database * 12322 -> forwarded to Airflow ssh server -> airflow:22 - * 28080 -> forwarded to Airflow webserver -> airflow:8080 * 29091 -> forwarded to Airflow FastAPI API -> airflow:9091 * 25555 -> forwarded to Flower dashboard -> airflow:5555 * 25433 -> forwarded to Postgres database -> postgres:5432 @@ -515,7 +514,6 @@ Using Breeze Here are links to those services that you can use on host: * ssh connection for remote debugging: ssh -p 12322 airflow@127.0.0.1 (password: airflow) - * Webserver: http://127.0.0.1:28080 * FastAPI API: http://127.0.0.1:29091 * Flower: http://127.0.0.1:25555 * Postgres: jdbc:postgresql://127.0.0.1:25433/airflow?user=postgres&password=airflow diff --git a/contributing-docs/08_static_code_checks.rst b/contributing-docs/08_static_code_checks.rst index 74e1a692e487b..cac0ed1362fe2 100644 --- a/contributing-docs/08_static_code_checks.rst +++ b/contributing-docs/08_static_code_checks.rst @@ -264,10 +264,6 @@ require Breeze Docker image to be built locally. +-----------------------------------------------------------+--------------------------------------------------------+---------+ | compile-ui-assets-dev | Compile ui assets in dev mode (manual) | | +-----------------------------------------------------------+--------------------------------------------------------+---------+ -| compile-www-assets | Compile www assets (manual) | | -+-----------------------------------------------------------+--------------------------------------------------------+---------+ -| compile-www-assets-dev | Compile www assets in dev mode (manual) | | -+-----------------------------------------------------------+--------------------------------------------------------+---------+ | create-missing-init-py-files-tests | Create missing init.py files in tests | | +-----------------------------------------------------------+--------------------------------------------------------+---------+ | debug-statements | Detect accidentally committed debug statements | | @@ -297,7 +293,6 @@ require Breeze Docker image to be built locally. | insert-license | * Add license for all SQL files | | | | * Add license for all RST files | | | | * Add license for CSS/JS/JSX/PUML/TS/TSX | | -| | * Add license for all JINJA template files | | | | * Add license for all Shell files | | | | * Add license for all toml files | | | | * Add license for all Python files | | @@ -311,8 +306,6 @@ require Breeze Docker image to be built locally. +-----------------------------------------------------------+--------------------------------------------------------+---------+ | lint-chart-schema | Lint chart/values.schema.json file | | +-----------------------------------------------------------+--------------------------------------------------------+---------+ -| lint-css | stylelint | | -+-----------------------------------------------------------+--------------------------------------------------------+---------+ | lint-dockerfile | Lint Dockerfile | | +-----------------------------------------------------------+--------------------------------------------------------+---------+ | lint-helm-chart | Lint Helm Chart | | @@ -366,8 +359,6 @@ require Breeze Docker image to be built locally. +-----------------------------------------------------------+--------------------------------------------------------+---------+ | ts-compile-format-lint-ui | Compile / format / lint UI | | +-----------------------------------------------------------+--------------------------------------------------------+---------+ -| ts-compile-format-lint-www | Compile / format / lint WWW | | -+-----------------------------------------------------------+--------------------------------------------------------+---------+ | update-black-version | Update black versions everywhere (manual) | | +-----------------------------------------------------------+--------------------------------------------------------+---------+ | update-breeze-cmd-output | Update breeze docs | | diff --git a/contributing-docs/14_node_environment_setup.rst b/contributing-docs/14_node_environment_setup.rst index 25635a89765e1..825ede707a43f 100644 --- a/contributing-docs/14_node_environment_setup.rst +++ b/contributing-docs/14_node_environment_setup.rst @@ -18,20 +18,10 @@ Node.js Environment Setup ========================= -Contributing to the UI in Airflow +Contributing to the REST API in Airflow --------------------------------- -In Airflow 3, we are moving the UI away from Flask App Builder views to a pure React powered frontend living at ``airflow/ui``. -During 3.0 development, we will need to run both the new and legacy UIs at the same time until the new UI is feature-complete. -But we want to limit modifications to the legacy ``airflow/www`` views, to mainly three rules: - -1. Bug fixes to cherry pick for 2.10.x and 2.11 -2. The minimum necessary to unblock other Airflow 3.0 feature work -3. Fixes to react views which we haven't migrated over yet, but can still be ported over to the new UI - -Custom endpoints for the UI will also be moved away from ``airflow/www/views.py`` and to ``airflow/api_fastapi``. -Contributions to the legacy views file will follow the same rules. -Committers will exercise their judgement on what endpoints should exist in the public ``airflow/api_connexion`` versus the private ``airflow/api_fastapi`` +Committers will exercise their judgement on what endpoints should exist in the public ``airflow/api_fastapi/public`` versus the private ``airflow/api_fastapi/ui`` Airflow UI ---------- @@ -93,17 +83,6 @@ Copy the example environment cp .env.example .env.local -DEPRECATED Airflow WWW ----------------------- - -``airflow/www/`` contains all yarn-managed, front-end assets. Flask-Appbuilder -itself comes bundled with jQuery and bootstrap. While they may be phased out -over time, these packages are currently not managed with yarn. - -Make sure you are using recent versions of node and yarn. No problems have been -found with node\>=8.11.3 and yarn\>=1.19.1. The pre-commit framework of ours install -node and yarn automatically when installed - if you use ``breeze`` you do not need to install -neither node nor yarn. **The outline for this document in GitHub is available at top-right corner button (with 3-dots and 3 lines).** @@ -183,7 +162,7 @@ Most IDE directly integrate with these tools, you can also manually run them wit React, JSX and Chakra --------------------- -In order to create a more modern UI, we have started to include `React `__ in the ``airflow/www/`` project. +In order to create a more modern UI, we have started to include `React `__ in the ``airflow/ui/`` project. If you are unfamiliar with React then it is recommended to check out their documentation to understand components and jsx syntax. We are using `Chakra UI `__ as a component and styling library. Notably, all styling is done in a theme file or diff --git a/dev/breeze/doc/03_developer_tasks.rst b/dev/breeze/doc/03_developer_tasks.rst index d41655d1229a8..8ea54695c85d7 100644 --- a/dev/breeze/doc/03_developer_tasks.rst +++ b/dev/breeze/doc/03_developer_tasks.rst @@ -112,7 +112,6 @@ When you run Airflow Breeze, the following ports are automatically forwarded: .. code-block:: * 12322 -> forwarded to Airflow ssh server -> airflow:22 - * 28080 -> forwarded to Airflow webserver -> airflow:8080 * 29091 -> forwarded to Airflow FastAPI API -> airflow:9091 * 25555 -> forwarded to Flower dashboard -> airflow:5555 * 25433 -> forwarded to Postgres database -> postgres:5432 @@ -125,15 +124,13 @@ You can connect to these ports/databases using: .. code-block:: * ssh connection for remote debugging: ssh -p 12322 airflow@127.0.0.1 pw: airflow - * Webserver: http://127.0.0.1:28080 * FastAPI API: http://127.0.0.1:29091 * Flower: http://127.0.0.1:25555 * Postgres: jdbc:postgresql://127.0.0.1:25433/airflow?user=postgres&password=airflow * Mysql: jdbc:mysql://127.0.0.1:23306/airflow?user=root * Redis: redis://127.0.0.1:26379/0 -If you do not use ``start-airflow`` command, you can start the webserver manually with -the ``airflow webserver`` command if you want to run it. You can use ``tmux`` to multiply terminals. +If you do not use ``start-airflow`` command. You can use ``tmux`` to multiply terminals. You may need to create a user prior to running the webserver in order to log in. This can be done with the following command: @@ -155,7 +152,6 @@ database client: You can change the used host port numbers by setting appropriate environment variables: * ``SSH_PORT`` -* ``WEBSERVER_HOST_PORT`` * ``API_SERVER_HOST_PORT`` * ``POSTGRES_HOST_PORT`` * ``MYSQL_HOST_PORT`` @@ -418,17 +414,6 @@ These are all available flags of ``exec`` command: :alt: Breeze exec -Compiling www assets --------------------- - -Airflow webserver needs to prepare www assets - compiled with node and yarn. The ``compile-www-assets`` -command takes care about it. This is needed when you want to run webserver inside of the breeze. - -.. image:: ./images/output_compile-www-assets.svg - :target: https://raw.githubusercontent.com/apache/airflow/main/dev/breeze/images/output_compile-www-assets.svg - :width: 100% - :alt: Breeze compile-www-assets - Compiling ui assets -------------------- diff --git a/dev/breeze/doc/10_advanced_breeze_topics.rst b/dev/breeze/doc/10_advanced_breeze_topics.rst index ddf67c071f7e6..b8354fddf8e93 100644 --- a/dev/breeze/doc/10_advanced_breeze_topics.rst +++ b/dev/breeze/doc/10_advanced_breeze_topics.rst @@ -219,11 +219,11 @@ are conflicting with the new specification, you might want to build the image wi breeze ci-image build --upgrade-to-newer-dependencies -Node (yarn) dependencies +Node (pnpm) dependencies ........................ -If you need to change "node" dependencies in ``airflow/www``, you need to compile them in the -host with ``breeze compile-www-assets`` command. No need to rebuild the image. +If you need to change "node" dependencies in ``airflow/ui``, you need to compile them in the +host with ``breeze compile-ui-assets`` command. No need to rebuild the image. Recording command output diff --git a/dev/breeze/doc/images/output-commands.svg b/dev/breeze/doc/images/output-commands.svg index 03bf04e0e51b8..ec63ce6f856f6 100644 --- a/dev/breeze/doc/images/output-commands.svg +++ b/dev/breeze/doc/images/output-commands.svg @@ -1,4 +1,4 @@ - + %s") % ("John & Jane Doe",)), - ] diff --git a/docs/apache-airflow/howto/docker-compose/docker-compose.yaml b/docs/apache-airflow/howto/docker-compose/docker-compose.yaml index 69803a2897ba6..8ca2dfc06d90a 100644 --- a/docs/apache-airflow/howto/docker-compose/docker-compose.yaml +++ b/docs/apache-airflow/howto/docker-compose/docker-compose.yaml @@ -117,23 +117,6 @@ services: start_period: 30s restart: always - airflow-webserver: - <<: *airflow-common - command: webserver - ports: - - "8080:8080" - healthcheck: - test: ["CMD", "curl", "--fail", "http://localhost:8080/health"] - interval: 30s - timeout: 10s - retries: 5 - start_period: 30s - restart: always - depends_on: - <<: *airflow-common-depends-on - airflow-init: - condition: service_completed_successfully - airflow-apiserver: <<: *airflow-common command: api-server diff --git a/docs/conf.py b/docs/conf.py index f20fe940e19e2..17607249f1a9e 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -335,7 +335,7 @@ def _get_rst_filepath_from_path(filepath: pathlib.Path): # configuration directory) that is the favicon of the docs. Modern browsers # use this as the icon for tabs, windows and bookmarks. It should be a # Windows-style icon file (.ico), which is 16x16 or 32x32 pixels large. -html_favicon = "../airflow/www/static/pin_32.png" +html_favicon = "../airflow/ui/public/pin_32.png" # Add any paths that contain custom static files (such as style sheets) here, # relative to this directory. They are copied after the builtin static files, diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json index eac55033a06c2..920be9e815b66 100644 --- a/generated/provider_dependencies.json +++ b/generated/provider_dependencies.json @@ -59,6 +59,7 @@ "common.compat", "common.sql", "exasol", + "fab", "ftp", "google", "http", diff --git a/hatch_build.py b/hatch_build.py index 9ce13e6d2447a..465985be56b97 100644 --- a/hatch_build.py +++ b/hatch_build.py @@ -593,8 +593,6 @@ class CustomBuild(BuilderInterface[BuilderConfig, PluginManager]): def clean(self, directory: str, versions: Iterable[str]) -> None: work_dir = Path(self.root) commands = [ - ["rm -rf airflow/www/static/dist"], - ["rm -rf airflow/www/node_modules"], ["rm -rf airflow/ui/dist"], ["rm -rf airflow/ui/node_modules"], ] @@ -609,7 +607,6 @@ def build_standard(self, directory: str, artifacts: Any, **build_data: Any) -> s self.write_git_version() work_dir = Path(self.root) commands = [ - ["pre-commit run --hook-stage manual compile-www-assets --all-files"], ["pre-commit run --hook-stage manual compile-ui-assets --all-files"], ] for cmd in commands: diff --git a/helm_tests/airflow_aux/test_configmap.py b/helm_tests/airflow_aux/test_configmap.py index 45a0a82a80e16..6f26de3a9a562 100644 --- a/helm_tests/airflow_aux/test_configmap.py +++ b/helm_tests/airflow_aux/test_configmap.py @@ -51,6 +51,7 @@ def test_multiple_annotations(self): @pytest.mark.parametrize( "af_version, secret_key, secret_key_name, expected", [ + ("3.0.0", None, None, False), ("2.2.0", None, None, True), ("2.2.0", "foo", None, False), ("2.2.0", None, "foo", False), diff --git a/providers/amazon/README.rst b/providers/amazon/README.rst index 0fec71dbae179..f053738fa492a 100644 --- a/providers/amazon/README.rst +++ b/providers/amazon/README.rst @@ -90,6 +90,7 @@ Dependent package `apache-airflow-providers-common-compat `_ ``common.compat`` `apache-airflow-providers-common-sql `_ ``common.sql`` `apache-airflow-providers-exasol `_ ``exasol`` +`apache-airflow-providers-fab `_ ``fab`` `apache-airflow-providers-ftp `_ ``ftp`` `apache-airflow-providers-google `_ ``google`` `apache-airflow-providers-http `_ ``http`` diff --git a/providers/amazon/pyproject.toml b/providers/amazon/pyproject.toml index 26e4f6940a0de..6b7bdd1028469 100644 --- a/providers/amazon/pyproject.toml +++ b/providers/amazon/pyproject.toml @@ -106,6 +106,9 @@ dependencies = [ "exasol" = [ "apache-airflow-providers-exasol" ] +"fab" = [ + "apache-airflow-providers-fab" +] "ftp" = [ "apache-airflow-providers-ftp" ] diff --git a/providers/amazon/src/airflow/providers/amazon/aws/auth_manager/security_manager/aws_security_manager_override.py b/providers/amazon/src/airflow/providers/amazon/aws/auth_manager/security_manager/aws_security_manager_override.py index 0bbd50396e7dc..1e4421e784904 100644 --- a/providers/amazon/src/airflow/providers/amazon/aws/auth_manager/security_manager/aws_security_manager_override.py +++ b/providers/amazon/src/airflow/providers/amazon/aws/auth_manager/security_manager/aws_security_manager_override.py @@ -17,13 +17,17 @@ from __future__ import annotations from airflow.exceptions import AirflowOptionalProviderFeatureException +from airflow.providers.amazon.version_compat import AIRFLOW_V_3_0_PLUS -try: +if AIRFLOW_V_3_0_PLUS: + try: + from airflow.providers.fab.www.security_manager import AirflowSecurityManagerV2 + except ImportError: + raise AirflowOptionalProviderFeatureException( + "Failed to import AirflowSecurityManagerV2 from the FAB provider. The AWS auth manager requires the FAB provider." + ) +else: from airflow.www.security_manager import AirflowSecurityManagerV2 -except ImportError: - raise AirflowOptionalProviderFeatureException( - "Failed to import AirflowSecurityManagerV2. This feature is only available in Airflow versions >= 2.8.0" - ) class AwsSecurityManagerOverride(AirflowSecurityManagerV2): diff --git a/providers/amazon/src/airflow/providers/amazon/get_provider_info.py b/providers/amazon/src/airflow/providers/amazon/get_provider_info.py index 88b6c0ef55c55..dfb0dc7835524 100644 --- a/providers/amazon/src/airflow/providers/amazon/get_provider_info.py +++ b/providers/amazon/src/airflow/providers/amazon/get_provider_info.py @@ -1363,6 +1363,7 @@ def get_provider_info(): "python3-saml": ["python3-saml>=1.16.0"], "apache.hive": ["apache-airflow-providers-apache-hive"], "exasol": ["apache-airflow-providers-exasol"], + "fab": ["apache-airflow-providers-fab"], "ftp": ["apache-airflow-providers-ftp"], "google": ["apache-airflow-providers-google"], "imap": ["apache-airflow-providers-imap"], diff --git a/providers/amazon/tests/unit/amazon/aws/auth_manager/security_manager/__init__.py b/providers/amazon/tests/unit/amazon/aws/auth_manager/security_manager/__init__.py deleted file mode 100644 index 13a83393a9124..0000000000000 --- a/providers/amazon/tests/unit/amazon/aws/auth_manager/security_manager/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. diff --git a/providers/amazon/tests/unit/amazon/aws/auth_manager/security_manager/test_aws_security_manager_override.py b/providers/amazon/tests/unit/amazon/aws/auth_manager/security_manager/test_aws_security_manager_override.py deleted file mode 100644 index ad75afc2ac50b..0000000000000 --- a/providers/amazon/tests/unit/amazon/aws/auth_manager/security_manager/test_aws_security_manager_override.py +++ /dev/null @@ -1,61 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from unittest.mock import Mock, patch - -import pytest -from flask import Flask - -from airflow.www.extensions.init_appbuilder import init_appbuilder - -from tests_common.test_utils.compat import ignore_provider_compatibility_error - -python3_saml = pytest.importorskip("python3-saml") - -with ignore_provider_compatibility_error("2.8.0", __file__): - from airflow.providers.amazon.aws.auth_manager.security_manager.aws_security_manager_override import ( - AwsSecurityManagerOverride, - ) - - -@pytest.fixture -def appbuilder(): - flask_app = Flask(__name__) - return init_appbuilder(flask_app) - - -@pytest.fixture -def override(appbuilder): - return AwsSecurityManagerOverride(appbuilder) - - -@pytest.mark.db_test -class TestAwsSecurityManagerOverride: - @patch( - "airflow.providers.amazon.aws.auth_manager.views.auth.conf.get_mandatory_value", return_value="test" - ) - def test_register_views(self, mock_get_mandatory_value, override, appbuilder): - from airflow.providers.amazon.aws.auth_manager.views.auth import AwsAuthManagerAuthenticationViews - - with patch.object(AwsAuthManagerAuthenticationViews, "idp_data"): - appbuilder.add_view_no_menu = Mock() - override.register_views() - appbuilder.add_view_no_menu.assert_called_once() - assert isinstance( - appbuilder.add_view_no_menu.call_args.args[0], AwsAuthManagerAuthenticationViews - ) diff --git a/providers/amazon/tests/unit/amazon/aws/auth_manager/test_aws_auth_manager.py b/providers/amazon/tests/unit/amazon/aws/auth_manager/test_aws_auth_manager.py index 8b539c3a12e80..a69fe496e1f2d 100644 --- a/providers/amazon/tests/unit/amazon/aws/auth_manager/test_aws_auth_manager.py +++ b/providers/amazon/tests/unit/amazon/aws/auth_manager/test_aws_auth_manager.py @@ -20,7 +20,6 @@ from unittest.mock import ANY, Mock, patch import pytest -from flask import session from flask_appbuilder.menu import MenuItem from airflow.providers.amazon.version_compat import AIRFLOW_V_3_0_PLUS @@ -101,17 +100,6 @@ class TestAwsAuthManager: def test_avp_facade(self, auth_manager): assert hasattr(auth_manager, "avp_facade") - @pytest.mark.db_test - @patch.object(AwsAuthManager, "is_logged_in") - def test_get_user(self, mock_is_logged_in, auth_manager, app, test_user): - mock_is_logged_in.return_value = True - - with app.test_request_context(): - session["aws_user"] = test_user - result = auth_manager.get_user() - - assert result == test_user - @patch.object(AwsAuthManager, "is_logged_in") def test_get_user_return_none_when_not_logged_in(self, mock_is_logged_in, auth_manager): mock_is_logged_in.return_value = False @@ -119,21 +107,6 @@ def test_get_user_return_none_when_not_logged_in(self, mock_is_logged_in, auth_m assert result is None - @pytest.mark.db_test - def test_is_logged_in(self, auth_manager, app, test_user): - with app.test_request_context(): - session["aws_user"] = test_user - result = auth_manager.is_logged_in() - - assert result - - @pytest.mark.db_test - def test_is_logged_in_return_false_when_no_user_in_session(self, auth_manager, app, test_user): - with app.test_request_context(): - result = auth_manager.is_logged_in() - - assert result is False - @pytest.mark.parametrize( "details, user, expected_user, expected_entity_id", [ diff --git a/providers/databricks/src/airflow/providers/databricks/plugins/databricks_workflow.py b/providers/databricks/src/airflow/providers/databricks/plugins/databricks_workflow.py index 81cf09b0f8d6b..1e82ad9c6fdb3 100644 --- a/providers/databricks/src/airflow/providers/databricks/plugins/databricks_workflow.py +++ b/providers/databricks/src/airflow/providers/databricks/plugins/databricks_workflow.py @@ -23,6 +23,7 @@ from urllib.parse import unquote from flask import current_app, flash, redirect, request, url_for +from flask_appbuilder import BaseView from flask_appbuilder.api import expose from airflow.exceptions import AirflowException, TaskInstanceNotFound @@ -39,7 +40,6 @@ from airflow.utils.state import TaskInstanceState from airflow.utils.task_group import TaskGroup from airflow.www import auth -from airflow.www.views import AirflowBaseView if TYPE_CHECKING: from sqlalchemy.orm.session import Session @@ -387,7 +387,7 @@ def get_link( return url_for("RepairDatabricksTasks.repair", **query_params) -class RepairDatabricksTasks(AirflowBaseView, LoggingMixin): +class RepairDatabricksTasks(BaseView, LoggingMixin): """Repair databricks tasks from Airflow.""" default_view = "repair" diff --git a/airflow/www/gunicorn_config.py b/providers/databricks/src/airflow/providers/databricks/version_compat.py similarity index 51% rename from airflow/www/gunicorn_config.py rename to providers/databricks/src/airflow/providers/databricks/version_compat.py index 3268698bd85f4..21e7170194e36 100644 --- a/airflow/www/gunicorn_config.py +++ b/providers/databricks/src/airflow/providers/databricks/version_compat.py @@ -1,5 +1,3 @@ -#!/usr/bin/env python -# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -16,28 +14,23 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# +# NOTE! THIS FILE IS COPIED MANUALLY IN OTHER PROVIDERS DELIBERATELY TO AVOID ADDING UNNECESSARY +# DEPENDENCIES BETWEEN PROVIDERS. IF YOU WANT TO ADD CONDITIONAL CODE IN YOUR PROVIDER THAT DEPENDS +# ON AIRFLOW VERSION, PLEASE COPY THIS FILE TO THE ROOT PACKAGE OF YOUR PROVIDER AND IMPORT +# THOSE CONSTANTS FROM IT RATHER THAN IMPORTING THEM FROM ANOTHER PROVIDER OR TEST CODE +# from __future__ import annotations -import setproctitle - -from airflow import settings - -def post_worker_init(_): - """ - Set process title. +def get_base_airflow_version_tuple() -> tuple[int, int, int]: + from packaging.version import Version - This is used by airflow.cli.commands.local_commands.webserver_command to track the status of the worker. - """ - old_title = setproctitle.getproctitle() - setproctitle.setproctitle(settings.GUNICORN_WORKER_READY_PREFIX + old_title) + from airflow import __version__ + airflow_version = Version(__version__) + return airflow_version.major, airflow_version.minor, airflow_version.micro -def on_starting(server): - from airflow.providers_manager import ProvidersManager - providers_manager = ProvidersManager() - # Load providers configuration before forking workers - providers_manager.initialize_providers_configuration() - # Load providers before forking workers - providers_manager.connection_form_widgets +AIRFLOW_V_2_10_PLUS = get_base_airflow_version_tuple() >= (2, 10, 0) +AIRFLOW_V_3_0_PLUS = get_base_airflow_version_tuple() >= (3, 0, 0) diff --git a/providers/databricks/tests/unit/databricks/plugins/test_databricks_workflow.py b/providers/databricks/tests/unit/databricks/plugins/test_databricks_workflow.py index a22febb5b7f7f..10ecd97b3e6c4 100644 --- a/providers/databricks/tests/unit/databricks/plugins/test_databricks_workflow.py +++ b/providers/databricks/tests/unit/databricks/plugins/test_databricks_workflow.py @@ -20,6 +20,15 @@ from unittest.mock import MagicMock, Mock, patch import pytest + +from airflow.providers.databricks.version_compat import AIRFLOW_V_3_0_PLUS + +if AIRFLOW_V_3_0_PLUS: + pytest.skip( + "``airflow/providers/databricks/plugins/databricks_workflow.py`` is only compatible with Airflow 2.X.", + allow_module_level=True, + ) + from flask import url_for from airflow.exceptions import AirflowException diff --git a/providers/edge/src/airflow/providers/edge/plugins/edge_executor_plugin.py b/providers/edge/src/airflow/providers/edge/plugins/edge_executor_plugin.py index fa497dd6b00e9..507066ffafba9 100644 --- a/providers/edge/src/airflow/providers/edge/plugins/edge_executor_plugin.py +++ b/providers/edge/src/airflow/providers/edge/plugins/edge_executor_plugin.py @@ -36,14 +36,15 @@ from airflow.utils.yaml import safe_load from airflow.www import utils as wwwutils from airflow.www.auth import has_access_view -from airflow.www.constants import SWAGGER_BUNDLE, SWAGGER_ENABLED -from airflow.www.extensions.init_views import _CustomErrorRequestBodyValidator, _LazyResolver if TYPE_CHECKING: from sqlalchemy.orm import Session def _get_airflow_2_api_endpoint() -> Blueprint: + from airflow.www.constants import SWAGGER_BUNDLE, SWAGGER_ENABLED + from airflow.www.extensions.init_views import _CustomErrorRequestBodyValidator, _LazyResolver + folder = Path(__file__).parents[1].resolve() # this is airflow/providers/edge/ with folder.joinpath("openapi", "edge_worker_api_v1.yaml").open() as f: specification = safe_load(f) @@ -53,8 +54,8 @@ def _get_airflow_2_api_endpoint() -> Blueprint: specification=specification, resolver=_LazyResolver(), base_path="/edge_worker/v1", - options={"swagger_ui": SWAGGER_ENABLED, "swagger_path": SWAGGER_BUNDLE.__fspath__()}, strict_validation=True, + options={"swagger_ui": SWAGGER_ENABLED, "swagger_path": SWAGGER_BUNDLE.__fspath__()}, validate_responses=True, validator_map={"body": _CustomErrorRequestBodyValidator}, ).blueprint diff --git a/tests_common/test_utils/decorators.py b/providers/fab/tests/unit/decorators.py similarity index 87% rename from tests_common/test_utils/decorators.py rename to providers/fab/tests/unit/decorators.py index b33f8421404de..26c7bf701b2bb 100644 --- a/tests_common/test_utils/decorators.py +++ b/providers/fab/tests/unit/decorators.py @@ -19,7 +19,7 @@ import functools from unittest.mock import patch -from airflow.www.app import purge_cached_app +from airflow.providers.fab.www.app import purge_cached_app def dont_initialize_flask_app_submodules(_func=None, *, skip_all_except=None): @@ -32,13 +32,8 @@ def no_op(*args, **kwargs): methods = [ "init_api_auth", - "init_flash_views", - "init_appbuilder_links", - "init_appbuilder_views", "init_plugins", "init_error_handlers", - "init_api_connexion", - "init_api_internal", "init_api_auth_provider", "init_api_error_handlers", "init_jinja_globals", @@ -51,7 +46,7 @@ def no_op(*args, **kwargs): def func(*args, **kwargs): for method in methods: if method not in skip_all_except: - patcher = patch(f"airflow.www.app.{method}", no_op) + patcher = patch(f"airflow.providers.fab.www.app.{method}", no_op) patcher.start() purge_cached_app() result = f(*args, **kwargs) diff --git a/providers/fab/tests/unit/fab/auth_manager/api_endpoints/remote_user_api_auth_backend.py b/providers/fab/tests/unit/fab/auth_manager/api_endpoints/remote_user_api_auth_backend.py index b7714e5192e6a..97f8a600d0246 100644 --- a/providers/fab/tests/unit/fab/auth_manager/api_endpoints/remote_user_api_auth_backend.py +++ b/providers/fab/tests/unit/fab/auth_manager/api_endpoints/remote_user_api_auth_backend.py @@ -26,7 +26,8 @@ from flask import Response, request from flask_login import login_user -from airflow.utils.airflow_flask_app import get_airflow_app +from airflow.api_fastapi.app import get_auth_manager +from airflow.providers.fab.auth_manager.fab_auth_manager import FabAuthManager if TYPE_CHECKING: from requests.auth import AuthBase @@ -44,7 +45,7 @@ def init_app(_): def _lookup_user(user_email_or_username: str): - security_manager = get_airflow_app().appbuilder.sm + security_manager = cast(FabAuthManager, get_auth_manager()).security_manager user = security_manager.find_user(email=user_email_or_username) or security_manager.find_user( username=user_email_or_username ) diff --git a/providers/fab/tests/unit/fab/auth_manager/conftest.py b/providers/fab/tests/unit/fab/auth_manager/conftest.py index 49023851aa176..f56dac8ab80f8 100644 --- a/providers/fab/tests/unit/fab/auth_manager/conftest.py +++ b/providers/fab/tests/unit/fab/auth_manager/conftest.py @@ -24,10 +24,10 @@ import pytest from airflow.providers.fab.www import app +from unit.decorators import dont_initialize_flask_app_submodules from tests_common.test_utils.config import conf_vars from tests_common.test_utils.db import parse_and_sync_to_db -from tests_common.test_utils.decorators import dont_initialize_flask_app_submodules @pytest.fixture(scope="session") diff --git a/providers/fab/tests/unit/fab/auth_manager/views/test_permissions.py b/providers/fab/tests/unit/fab/auth_manager/views/test_permissions.py index d1094650f8224..dcced03d6c5e3 100644 --- a/providers/fab/tests/unit/fab/auth_manager/views/test_permissions.py +++ b/providers/fab/tests/unit/fab/auth_manager/views/test_permissions.py @@ -23,8 +23,7 @@ from airflow.providers.fab.www.security import permissions from unit.fab.auth_manager.api_endpoints.api_connexion_utils import create_user, delete_user from unit.fab.auth_manager.views import _assert_dataset_deprecation_warning - -from tests_common.test_utils.www import client_with_login +from unit.fab.utils import client_with_login @pytest.fixture(scope="module") diff --git a/providers/fab/tests/unit/fab/auth_manager/views/test_roles_list.py b/providers/fab/tests/unit/fab/auth_manager/views/test_roles_list.py index 3ce950788c36a..93e7444e8d760 100644 --- a/providers/fab/tests/unit/fab/auth_manager/views/test_roles_list.py +++ b/providers/fab/tests/unit/fab/auth_manager/views/test_roles_list.py @@ -23,8 +23,7 @@ from airflow.providers.fab.www.security import permissions from unit.fab.auth_manager.api_endpoints.api_connexion_utils import create_user, delete_user from unit.fab.auth_manager.views import _assert_dataset_deprecation_warning - -from tests_common.test_utils.www import client_with_login +from unit.fab.utils import client_with_login @pytest.fixture(scope="module") diff --git a/providers/fab/tests/unit/fab/auth_manager/views/test_user.py b/providers/fab/tests/unit/fab/auth_manager/views/test_user.py index be9dfa54fe7ea..7a84e071bb2cd 100644 --- a/providers/fab/tests/unit/fab/auth_manager/views/test_user.py +++ b/providers/fab/tests/unit/fab/auth_manager/views/test_user.py @@ -23,8 +23,7 @@ from airflow.providers.fab.www.security import permissions from unit.fab.auth_manager.api_endpoints.api_connexion_utils import create_user, delete_user from unit.fab.auth_manager.views import _assert_dataset_deprecation_warning - -from tests_common.test_utils.www import client_with_login +from unit.fab.utils import client_with_login @pytest.fixture(scope="module") diff --git a/providers/fab/tests/unit/fab/auth_manager/views/test_user_edit.py b/providers/fab/tests/unit/fab/auth_manager/views/test_user_edit.py index 734efed215b17..a3e957c12d26c 100644 --- a/providers/fab/tests/unit/fab/auth_manager/views/test_user_edit.py +++ b/providers/fab/tests/unit/fab/auth_manager/views/test_user_edit.py @@ -23,8 +23,7 @@ from airflow.providers.fab.www.security import permissions from unit.fab.auth_manager.api_endpoints.api_connexion_utils import create_user, delete_user from unit.fab.auth_manager.views import _assert_dataset_deprecation_warning - -from tests_common.test_utils.www import client_with_login +from unit.fab.utils import client_with_login @pytest.fixture(scope="module") diff --git a/providers/fab/tests/unit/fab/auth_manager/views/test_user_stats.py b/providers/fab/tests/unit/fab/auth_manager/views/test_user_stats.py index 51acb4147e370..2ac3a314e0fd3 100644 --- a/providers/fab/tests/unit/fab/auth_manager/views/test_user_stats.py +++ b/providers/fab/tests/unit/fab/auth_manager/views/test_user_stats.py @@ -23,8 +23,7 @@ from airflow.providers.fab.www.security import permissions from unit.fab.auth_manager.api_endpoints.api_connexion_utils import create_user, delete_user from unit.fab.auth_manager.views import _assert_dataset_deprecation_warning - -from tests_common.test_utils.www import client_with_login +from unit.fab.utils import client_with_login @pytest.fixture(scope="module") diff --git a/tests_common/test_utils/www.py b/providers/fab/tests/unit/fab/utils.py similarity index 100% rename from tests_common/test_utils/www.py rename to providers/fab/tests/unit/fab/utils.py diff --git a/providers/fab/tests/unit/fab/www/views/conftest.py b/providers/fab/tests/unit/fab/www/views/conftest.py index 15f1d6261b65b..2893888befc3f 100644 --- a/providers/fab/tests/unit/fab/www/views/conftest.py +++ b/providers/fab/tests/unit/fab/www/views/conftest.py @@ -25,12 +25,12 @@ from airflow import settings from airflow.models import DagBag from airflow.providers.fab.www.app import create_app +from unit.decorators import dont_initialize_flask_app_submodules from unit.fab.auth_manager.api_endpoints.api_connexion_utils import delete_user +from unit.fab.utils import client_with_login from tests_common.test_utils.config import conf_vars from tests_common.test_utils.db import parse_and_sync_to_db -from tests_common.test_utils.decorators import dont_initialize_flask_app_submodules -from tests_common.test_utils.www import client_with_login @pytest.fixture(autouse=True, scope="module") diff --git a/providers/fab/tests/unit/fab/www/views/test_views_custom_user_views.py b/providers/fab/tests/unit/fab/www/views/test_views_custom_user_views.py index a75079d8939f8..6851db3700c95 100644 --- a/providers/fab/tests/unit/fab/www/views/test_views_custom_user_views.py +++ b/providers/fab/tests/unit/fab/www/views/test_views_custom_user_views.py @@ -31,11 +31,7 @@ create_user, delete_role, ) - -from tests_common.test_utils.www import ( - check_content_in_response, - client_with_login, -) +from unit.fab.utils import check_content_in_response, client_with_login pytestmark = pytest.mark.db_test diff --git a/providers/google/tests/unit/google/common/auth_backend/test_google_openid.py b/providers/google/tests/unit/google/common/auth_backend/test_google_openid.py index f046fa59c61b6..938e63bb9a072 100644 --- a/providers/google/tests/unit/google/common/auth_backend/test_google_openid.py +++ b/providers/google/tests/unit/google/common/auth_backend/test_google_openid.py @@ -21,25 +21,20 @@ import pytest from google.auth.exceptions import GoogleAuthError -from airflow.www.app import create_app +from airflow.providers.google.version_compat import AIRFLOW_V_3_0_PLUS + +if not AIRFLOW_V_3_0_PLUS: + pytest.skip( + "``providers/google/tests/unit/google/common/auth_backend/test_google_openid.py`` is only compatible with Airflow 2.X.", + allow_module_level=True, + ) from tests_common.test_utils.config import conf_vars from tests_common.test_utils.db import clear_db_pools -from tests_common.test_utils.decorators import dont_initialize_flask_app_submodules @pytest.fixture(scope="module") def google_openid_app(): - @dont_initialize_flask_app_submodules( - skip_all_except=[ - "init_appbuilder", - "init_api_auth", - "init_api_connexion", - "init_api_error_handlers", - "init_airflow_session_interface", - "init_appbuilder_views", - ] - ) def factory(): with conf_vars( { @@ -50,7 +45,9 @@ def factory(): ): "airflow.providers.fab.auth_manager.fab_auth_manager.FabAuthManager", } ): - _app = create_app(testing=True, config={"WTF_CSRF_ENABLED": False}) # type:ignore + from airflow.providers.fab.www.app import create_app + + _app = create_app(enable_plugins=False) _app.config["AUTH_ROLE_PUBLIC"] = None return _app @@ -91,10 +88,9 @@ def test_success(self, mock_verify_token): } with self.app.test_client() as test_client: - response = test_client.get("/api/v1/pools", headers={"Authorization": "bearer JWT_TOKEN"}) + response = test_client.get("/auth/fab/v1/users", headers={"Authorization": "bearer JWT_TOKEN"}) assert response.status_code == 200 - assert "Default pool" in str(response.json) @pytest.mark.parametrize("auth_header", ["bearer", "JWT_TOKEN", "bearer "]) @mock.patch("google.oauth2.id_token.verify_token") @@ -106,7 +102,7 @@ def test_malformed_headers(self, mock_verify_token, auth_header): } with self.app.test_client() as test_client: - response = test_client.get("/api/v1/pools", headers={"Authorization": auth_header}) + response = test_client.get("/auth/fab/v1/users", headers={"Authorization": auth_header}) assert response.status_code == 401 @@ -119,7 +115,7 @@ def test_invalid_iss_in_jwt_token(self, mock_verify_token): } with self.app.test_client() as test_client: - response = test_client.get("/api/v1/pools", headers={"Authorization": "bearer JWT_TOKEN"}) + response = test_client.get("/auth/fab/v1/users", headers={"Authorization": "bearer JWT_TOKEN"}) assert response.status_code == 401 @@ -132,14 +128,14 @@ def test_user_not_exists(self, mock_verify_token): } with self.app.test_client() as test_client: - response = test_client.get("/api/v1/pools", headers={"Authorization": "bearer JWT_TOKEN"}) + response = test_client.get("/auth/fab/v1/users", headers={"Authorization": "bearer JWT_TOKEN"}) assert response.status_code == 401 @conf_vars({("api", "auth_backends"): "airflow.providers.google.common.auth_backend.google_openid"}) def test_missing_id_token(self): with self.app.test_client() as test_client: - response = test_client.get("/api/v1/pools") + response = test_client.get("/auth/fab/v1/users") assert response.status_code == 401 @@ -149,6 +145,6 @@ def test_invalid_id_token(self, mock_verify_token): mock_verify_token.side_effect = GoogleAuthError("Invalid token") with self.app.test_client() as test_client: - response = test_client.get("/api/v1/pools", headers={"Authorization": "bearer JWT_TOKEN"}) + response = test_client.get("/auth/fab/v1/users", headers={"Authorization": "bearer JWT_TOKEN"}) assert response.status_code == 401 diff --git a/providers/standard/src/airflow/providers/standard/operators/trigger_dagrun.py b/providers/standard/src/airflow/providers/standard/operators/trigger_dagrun.py index 2cd2143c0dc14..14c51a0579097 100644 --- a/providers/standard/src/airflow/providers/standard/operators/trigger_dagrun.py +++ b/providers/standard/src/airflow/providers/standard/operators/trigger_dagrun.py @@ -40,8 +40,8 @@ from airflow.models.dagrun import DagRun from airflow.models.xcom import XCom from airflow.providers.standard.triggers.external_task import DagStateTrigger +from airflow.providers.standard.version_compat import AIRFLOW_V_3_0_PLUS from airflow.utils import timezone -from airflow.utils.helpers import build_airflow_url_with_query from airflow.utils.session import provide_session from airflow.utils.state import DagRunState from airflow.utils.types import DagRunTriggeredByType, DagRunType @@ -86,8 +86,15 @@ def get_link(self, operator: BaseOperator, *, ti_key: TaskInstanceKey) -> str: # stored in xcom during execution of the triggerING task. triggered_dag_run_id = XCom.get_value(ti_key=ti_key, key=XCOM_RUN_ID) - query = {"dag_id": trigger_dag_id, "dag_run_id": triggered_dag_run_id} - return build_airflow_url_with_query(query) + if AIRFLOW_V_3_0_PLUS: + from airflow.utils.helpers import build_airflow_dagrun_url + + return build_airflow_dagrun_url(dag_id=trigger_dag_id, run_id=triggered_dag_run_id) + else: + from airflow.utils.helpers import build_airflow_url_with_query # type:ignore[attr-defined] + + query = {"dag_id": trigger_dag_id, "dag_run_id": triggered_dag_run_id} + return build_airflow_url_with_query(query) class TriggerDagRunOperator(BaseOperator): diff --git a/providers/standard/src/airflow/providers/standard/sensors/external_task.py b/providers/standard/src/airflow/providers/standard/sensors/external_task.py index bedbf1c2ed07d..68e8c1a15b281 100644 --- a/providers/standard/src/airflow/providers/standard/sensors/external_task.py +++ b/providers/standard/src/airflow/providers/standard/sensors/external_task.py @@ -31,9 +31,9 @@ from airflow.providers.standard.operators.empty import EmptyOperator from airflow.providers.standard.triggers.external_task import WorkflowTrigger from airflow.providers.standard.utils.sensor_helper import _get_count, _get_external_task_group_task_ids +from airflow.providers.standard.version_compat import AIRFLOW_V_3_0_PLUS from airflow.sensors.base import BaseSensorOperator from airflow.utils.file import correct_maybe_zipped -from airflow.utils.helpers import build_airflow_url_with_query from airflow.utils.session import NEW_SESSION, provide_session from airflow.utils.state import State, TaskInstanceState @@ -70,8 +70,15 @@ def get_link(self, operator: BaseOperator, *, ti_key: TaskInstanceKey) -> str: else: external_dag_id = operator.external_dag_id - query = {"dag_id": external_dag_id, "run_id": ti_key.run_id} - return build_airflow_url_with_query(query) + if AIRFLOW_V_3_0_PLUS: + from airflow.utils.helpers import build_airflow_dagrun_url + + return build_airflow_dagrun_url(dag_id=external_dag_id, run_id=ti_key.run_id) + else: + from airflow.utils.helpers import build_airflow_url_with_query # type:ignore[attr-defined] + + query = {"dag_id": external_dag_id, "run_id": ti_key.run_id} + return build_airflow_url_with_query(query) class ExternalTaskSensor(BaseSensorOperator): diff --git a/pyproject.toml b/pyproject.toml index bc57fedc851b6..41fd5e350ccaa 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -176,10 +176,9 @@ include = [ "/airflow/git_version" ] exclude = [ - "/airflow/www/node_modules/" + "/airflow/ui/node_modules/" ] artifacts = [ - "/airflow/www/static/dist/", "/airflow/ui/dist/", "/airflow/git_version", "/generated/", @@ -190,7 +189,6 @@ include = [ "/airflow", ] artifacts = [ - "/airflow/www/static/dist/", "/airflow/ui/dist/", "/airflow/git_version" ] @@ -570,7 +568,6 @@ omit = [ "airflow/contrib/**", "airflow/example_dags/**", "airflow/migrations/**", - "airflow/www/node_modules/**", "providers/google/src/airflow/providers/**/example_dags/**", "providers/google/src/airflow/providers/google/ads/_vendor/**", ] diff --git a/scripts/ci/docker-compose/base-ports.yml b/scripts/ci/docker-compose/base-ports.yml index 8f8a9bf2bb9ea..a3cee13264a44 100644 --- a/scripts/ci/docker-compose/base-ports.yml +++ b/scripts/ci/docker-compose/base-ports.yml @@ -19,6 +19,5 @@ services: airflow: ports: - "${SSH_PORT}:22" - - "${WEBSERVER_HOST_PORT}:8080" - "${API_SERVER_HOST_PORT}:9091" - "${FLOWER_HOST_PORT}:5555" diff --git a/scripts/cov/cli_coverage.py b/scripts/cov/cli_coverage.py index 08caf1f01d01d..46dcd0607a52e 100644 --- a/scripts/cov/cli_coverage.py +++ b/scripts/cov/cli_coverage.py @@ -44,7 +44,6 @@ "airflow/cli/commands/local_commands/standalone_command.py", "airflow/cli/commands/remote_commands/task_command.py", "airflow/cli/commands/remote_commands/variable_command.py", - "airflow/cli/commands/local_commands/webserver_command.py", "airflow/cli/simple_table.py", ] diff --git a/scripts/cov/www_coverage.py b/scripts/cov/www_coverage.py deleted file mode 100644 index 937c444a3187d..0000000000000 --- a/scripts/cov/www_coverage.py +++ /dev/null @@ -1,55 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import sys -from pathlib import Path - -from cov_runner import run_tests - -sys.path.insert(0, str(Path(__file__).parent.resolve())) - -source_files = ["airflow/www"] - -restapi_files = ["tests/www"] - -files_not_fully_covered = [ - "airflow/www/app.py", - "airflow/www/auth.py", - "airflow/www/decorators.py", - "airflow/www/extensions/init_appbuilder.py", - "airflow/www/extensions/init_auth_manager.py", - "airflow/www/extensions/init_dagbag.py", - "airflow/www/extensions/init_jinja_globals.py", - "airflow/www/extensions/init_manifest_files.py", - "airflow/www/extensions/init_security.py", - "airflow/www/extensions/init_session.py", - "airflow/www/extensions/init_views.py", - "airflow/www/fab_security/manager.py", - "airflow/www/fab_security/sqla/manager.py", - "airflow/www/forms.py", - "airflow/www/gunicorn_config.py", - "airflow/www/security_manager.py", - "airflow/www/session.py", - "airflow/www/utils.py", - "airflow/www/views.py", - "airflow/www/widgets.py", -] - -if __name__ == "__main__": - args = ["-qq"] + restapi_files - run_tests(args, source_files, files_not_fully_covered) diff --git a/scripts/docker/entrypoint_ci.sh b/scripts/docker/entrypoint_ci.sh index 8b1cbdd622205..503464998a901 100755 --- a/scripts/docker/entrypoint_ci.sh +++ b/scripts/docker/entrypoint_ci.sh @@ -341,8 +341,8 @@ function check_airflow_python_client_installation() { python "${IN_CONTAINER_DIR}/install_airflow_python_client.py" } -function start_webserver_with_examples(){ - if [[ ${START_WEBSERVER_WITH_EXAMPLES=} != "true" ]]; then +function start_api_server_with_examples(){ + if [[ ${START_API_SERVER_WITH_EXAMPLES=} != "true" ]]; then return fi export AIRFLOW__CORE__LOAD_EXAMPLES=True @@ -363,22 +363,22 @@ function start_webserver_with_examples(){ airflow users create -u admin -p admin -f Thor -l Administrator -r Admin -e admin@email.domain echo "Admin user created" echo - echo "${COLOR_BLUE}Starting airflow webserver${COLOR_RESET}" + echo "${COLOR_BLUE}Starting airflow api server${COLOR_RESET}" echo - airflow webserver --port 8080 --daemon + airflow api-server --port 9091 --daemon echo - echo "${COLOR_BLUE}Waiting for webserver to start${COLOR_RESET}" + echo "${COLOR_BLUE}Waiting for api-server to start${COLOR_RESET}" echo - check_service_connection "Airflow webserver" "run_nc localhost 8080" 100 + check_service_connection "Airflow api-server" "run_nc localhost 9091" 100 EXIT_CODE=$? if [[ ${EXIT_CODE} != 0 ]]; then echo - echo "${COLOR_RED}Webserver did not start properly${COLOR_RESET}" + echo "${COLOR_RED}Api server did not start properly${COLOR_RESET}" echo exit ${EXIT_CODE} fi echo - echo "${COLOR_BLUE}Airflow webserver started${COLOR_RESET}" + echo "${COLOR_BLUE}Airflow api-server started${COLOR_RESET}" } handle_mount_sources @@ -389,7 +389,7 @@ check_downgrade_sqlalchemy check_downgrade_pendulum check_force_lowest_dependencies check_airflow_python_client_installation -start_webserver_with_examples +start_api_server_with_examples check_run_tests "${@}" # If we are not running tests - just exec to bash shell diff --git a/tests/always/test_project_structure.py b/tests/always/test_project_structure.py index 34ec8b6af1f1c..2aeb00f83491d 100644 --- a/tests/always/test_project_structure.py +++ b/tests/always/test_project_structure.py @@ -58,6 +58,7 @@ def test_providers_modules_should_have_tests(self): # We should make sure that one goes to 0 # TODO(potiuk) - check if that test actually tests something OVERLOOKED_TESTS = [ + "providers/amazon/tests/unit/amazon/aws/auth_manager/security_manager/test_aws_security_manager_override.py", "providers/amazon/tests/unit/amazon/aws/executors/batch/test_batch_executor_config.py", "providers/amazon/tests/unit/amazon/aws/executors/batch/test_boto_schema.py", "providers/amazon/tests/unit/amazon/aws/executors/batch/test_utils.py", @@ -103,6 +104,7 @@ def test_providers_modules_should_have_tests(self): "providers/common/compat/tests/unit/common/compat/standard/test_utils.py", "providers/common/compat/tests/unit/common/compat/test_version_compat.py", "providers/common/io/tests/unit/common/io/test_version_compat.py", + "providers/databricks/tests/unit/databricks/test_version_compat.py", "providers/edge/tests/unit/edge/models/test_edge_job.py", "providers/edge/tests/unit/edge/models/test_edge_logs.py", "providers/edge/tests/unit/edge/models/test_edge_worker.py", diff --git a/tests/api_connexion/__init__.py b/tests/api_connexion/__init__.py deleted file mode 100644 index 13a83393a9124..0000000000000 --- a/tests/api_connexion/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. diff --git a/tests/api_connexion/conftest.py b/tests/api_connexion/conftest.py deleted file mode 100644 index fa5d4c9250065..0000000000000 --- a/tests/api_connexion/conftest.py +++ /dev/null @@ -1,71 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os - -import pytest - -from airflow.www import app - -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import parse_and_sync_to_db -from tests_common.test_utils.decorators import dont_initialize_flask_app_submodules - - -@pytest.fixture(scope="session") -def minimal_app_for_api(): - @dont_initialize_flask_app_submodules( - skip_all_except=[ - "init_appbuilder", - "init_api_auth", - "init_api_connexion", - "init_api_error_handlers", - "init_airflow_session_interface", - "init_appbuilder_views", - ] - ) - def factory(): - with conf_vars( - { - ("api", "auth_backends"): "tests_common.test_utils.remote_user_api_auth_backend", - ( - "core", - "auth_manager", - ): "airflow.auth.managers.simple.simple_auth_manager.SimpleAuthManager", - } - ): - _app = app.create_app(testing=True, config={"WTF_CSRF_ENABLED": False}) # type:ignore - return _app - - return factory() - - -@pytest.fixture -def session(): - from airflow.utils.session import create_session - - with create_session() as session: - yield session - - -@pytest.fixture(scope="module") -def dagbag(): - from airflow.models import DagBag - - parse_and_sync_to_db(os.devnull, include_examples=True) - return DagBag(read_dags_from_db=True) diff --git a/tests/api_connexion/endpoints/__init__.py b/tests/api_connexion/endpoints/__init__.py deleted file mode 100644 index 13a83393a9124..0000000000000 --- a/tests/api_connexion/endpoints/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. diff --git a/tests/api_connexion/endpoints/test_asset_endpoint.py b/tests/api_connexion/endpoints/test_asset_endpoint.py deleted file mode 100644 index e13607d03c12e..0000000000000 --- a/tests/api_connexion/endpoints/test_asset_endpoint.py +++ /dev/null @@ -1,875 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import urllib -from collections.abc import Generator -from unittest.mock import ANY - -import pytest -import time_machine - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.models import DagModel -from airflow.models.asset import ( - AssetDagRunQueue, - AssetEvent, - AssetModel, - DagScheduleAssetReference, - TaskOutletAssetReference, -) -from airflow.models.dagrun import DagRun -from airflow.utils import timezone -from airflow.utils.session import provide_session -from airflow.utils.types import DagRunType - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.asserts import assert_queries_count -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_assets, clear_db_runs -from tests_common.test_utils.www import _check_last_log - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestAssetEndpoint: - default_time = "2020-06-11T18:00:00+00:00" - - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() - clear_db_assets() - clear_db_runs() - - def teardown_method(self) -> None: - clear_db_assets() - clear_db_runs() - - def _create_asset(self, session): - asset_model = AssetModel( - id=1, - uri="s3://bucket/key", - name="asset-name", - group="asset", - extra={"foo": "bar"}, - created_at=timezone.parse(self.default_time), - updated_at=timezone.parse(self.default_time), - ) - session.add(asset_model) - session.commit() - return asset_model - - -class TestGetAssetEndpoint(TestAssetEndpoint): - def test_should_respond_200(self, session): - self._create_asset(session) - assert session.query(AssetModel).count() == 1 - - with assert_queries_count(6): - response = self.client.get( - f"/api/v1/assets/{urllib.parse.quote('s3://bucket/key', safe='')}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "id": 1, - "uri": "s3://bucket/key", - "name": "asset-name", - "group": "asset", - "extra": {"foo": "bar"}, - "created_at": self.default_time, - "updated_at": self.default_time, - "consuming_dags": [], - "producing_tasks": [], - "aliases": [], - } - - def test_should_respond_404(self): - response = self.client.get( - f"/api/v1/assets/{urllib.parse.quote('s3://bucket/key', safe='')}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json == { - "detail": "The Asset with uri: `s3://bucket/key` was not found", - "status": 404, - "title": "Asset not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self, session): - self._create_asset(session) - response = self.client.get(f"/api/v1/assets/{urllib.parse.quote('s3://bucket/key', safe='')}") - assert_401(response) - - -class TestGetAssets(TestAssetEndpoint): - def test_should_respond_200(self, session): - assets = [ - AssetModel( - id=i, - uri=f"s3://bucket/key/{i}", - name=f"asset_{i}", - group="asset", - extra={"foo": "bar"}, - created_at=timezone.parse(self.default_time), - updated_at=timezone.parse(self.default_time), - ) - for i in [1, 2] - ] - session.add_all(assets) - session.commit() - assert session.query(AssetModel).count() == 2 - - with assert_queries_count(10): - response = self.client.get("/api/v1/assets", environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "assets": [ - { - "id": 1, - "uri": "s3://bucket/key/1", - "name": "asset_1", - "group": "asset", - "extra": {"foo": "bar"}, - "created_at": self.default_time, - "updated_at": self.default_time, - "consuming_dags": [], - "producing_tasks": [], - "aliases": [], - }, - { - "id": 2, - "uri": "s3://bucket/key/2", - "name": "asset_2", - "group": "asset", - "extra": {"foo": "bar"}, - "created_at": self.default_time, - "updated_at": self.default_time, - "consuming_dags": [], - "producing_tasks": [], - "aliases": [], - }, - ], - "total_entries": 2, - } - - def test_order_by_raises_400_for_invalid_attr(self, session): - assets = [ - AssetModel( - uri=f"s3://bucket/key/{i}", - extra={"foo": "bar"}, - created_at=timezone.parse(self.default_time), - updated_at=timezone.parse(self.default_time), - ) - for i in [1, 2] - ] - session.add_all(assets) - session.commit() - assert session.query(AssetModel).count() == 2 - - response = self.client.get( - "/api/v1/assets?order_by=fake", environ_overrides={"REMOTE_USER": "test"} - ) # missing attr - - assert response.status_code == 400 - msg = "Ordering with 'fake' is disallowed or the attribute does not exist on the model" - assert response.json["detail"] == msg - - def test_should_raises_401_unauthenticated(self, session): - assets = [ - AssetModel( - uri=f"s3://bucket/key/{i}", - extra={"foo": "bar"}, - created_at=timezone.parse(self.default_time), - updated_at=timezone.parse(self.default_time), - ) - for i in [1, 2] - ] - session.add_all(assets) - session.commit() - assert session.query(AssetModel).count() == 2 - - response = self.client.get("/api/v1/assets") - - assert_401(response) - - @pytest.mark.parametrize( - "url, expected_assets", - [ - ("api/v1/assets?uri_pattern=s3", {"s3://folder/key"}), - ("api/v1/assets?uri_pattern=bucket", {"gcp://bucket/key", "wasb://some_asset_bucket_/key"}), - ( - "api/v1/assets?uri_pattern=asset", - {"somescheme://asset/key", "wasb://some_asset_bucket_/key"}, - ), - ( - "api/v1/assets?uri_pattern=", - { - "gcp://bucket/key", - "s3://folder/key", - "somescheme://asset/key", - "wasb://some_asset_bucket_/key", - }, - ), - ], - ) - @provide_session - def test_filter_assets_by_uri_pattern_works(self, url, expected_assets, session): - asset1 = AssetModel("s3://folder/key") - asset2 = AssetModel("gcp://bucket/key") - asset3 = AssetModel("somescheme://asset/key") - asset4 = AssetModel("wasb://some_asset_bucket_/key") - session.add_all([asset1, asset2, asset3, asset4]) - session.commit() - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - asset_urls = {asset["uri"] for asset in response.json["assets"]} - assert expected_assets == asset_urls - - @pytest.mark.parametrize("dag_ids, expected_num", [("dag1,dag2", 2), ("dag3", 1), ("dag2,dag3", 2)]) - @provide_session - def test_filter_assets_by_dag_ids_works(self, dag_ids, expected_num, session): - session.query(DagModel).delete() - session.commit() - dag1 = DagModel(dag_id="dag1") - dag2 = DagModel(dag_id="dag2") - dag3 = DagModel(dag_id="dag3") - asset1 = AssetModel("s3://folder/key") - asset2 = AssetModel("gcp://bucket/key") - asset3 = AssetModel("somescheme://asset/key") - dag_ref1 = DagScheduleAssetReference(dag_id="dag1", asset=asset1) - dag_ref2 = DagScheduleAssetReference(dag_id="dag2", asset=asset2) - task_ref1 = TaskOutletAssetReference(dag_id="dag3", task_id="task1", asset=asset3) - session.add_all([asset1, asset2, asset3, dag1, dag2, dag3, dag_ref1, dag_ref2, task_ref1]) - session.commit() - response = self.client.get( - f"/api/v1/assets?dag_ids={dag_ids}", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - response_data = response.json - assert len(response_data["assets"]) == expected_num - - @pytest.mark.parametrize( - "dag_ids, uri_pattern,expected_num", - [("dag1,dag2", "folder", 1), ("dag3", "nothing", 0), ("dag2,dag3", "key", 2)], - ) - def test_filter_assets_by_dag_ids_and_uri_pattern_works( - self, dag_ids, uri_pattern, expected_num, session - ): - session.query(DagModel).delete() - session.commit() - dag1 = DagModel(dag_id="dag1") - dag2 = DagModel(dag_id="dag2") - dag3 = DagModel(dag_id="dag3") - asset1 = AssetModel("s3://folder/key") - asset2 = AssetModel("gcp://bucket/key") - asset3 = AssetModel("somescheme://asset/key") - dag_ref1 = DagScheduleAssetReference(dag_id="dag1", asset=asset1) - dag_ref2 = DagScheduleAssetReference(dag_id="dag2", asset=asset2) - task_ref1 = TaskOutletAssetReference(dag_id="dag3", task_id="task1", asset=asset3) - session.add_all([asset1, asset2, asset3, dag1, dag2, dag3, dag_ref1, dag_ref2, task_ref1]) - session.commit() - response = self.client.get( - f"/api/v1/assets?dag_ids={dag_ids}&uri_pattern={uri_pattern}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - response_data = response.json - assert len(response_data["assets"]) == expected_num - - -class TestGetAssetsEndpointPagination(TestAssetEndpoint): - @pytest.mark.parametrize( - "url, expected_asset_uris", - [ - # Limit test data - ("/api/v1/assets?limit=1", ["s3://bucket/key/1"]), - ("/api/v1/assets?limit=100", [f"s3://bucket/key/{i}" for i in range(1, 101)]), - # Offset test data - ("/api/v1/assets?offset=1", [f"s3://bucket/key/{i}" for i in range(2, 102)]), - ("/api/v1/assets?offset=3", [f"s3://bucket/key/{i}" for i in range(4, 104)]), - # Limit and offset test data - ("/api/v1/assets?offset=3&limit=3", [f"s3://bucket/key/{i}" for i in [4, 5, 6]]), - ], - ) - @provide_session - def test_limit_and_offset(self, url, expected_asset_uris, session): - assets = [ - AssetModel( - uri=f"s3://bucket/key/{i}", - extra={"foo": "bar"}, - created_at=timezone.parse(self.default_time), - updated_at=timezone.parse(self.default_time), - ) - for i in range(1, 110) - ] - session.add_all(assets) - session.commit() - - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - asset_uris = [asset["uri"] for asset in response.json["assets"]] - assert asset_uris == expected_asset_uris - - def test_should_respect_page_size_limit_default(self, session): - assets = [ - AssetModel( - uri=f"s3://bucket/key/{i}", - extra={"foo": "bar"}, - created_at=timezone.parse(self.default_time), - updated_at=timezone.parse(self.default_time), - ) - for i in range(1, 110) - ] - session.add_all(assets) - session.commit() - - response = self.client.get("/api/v1/assets", environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - assert len(response.json["assets"]) == 100 - - @conf_vars({("api", "maximum_page_limit"): "150"}) - def test_should_return_conf_max_if_req_max_above_conf(self, session): - assets = [ - AssetModel( - uri=f"s3://bucket/key/{i}", - extra={"foo": "bar"}, - created_at=timezone.parse(self.default_time), - updated_at=timezone.parse(self.default_time), - ) - for i in range(1, 200) - ] - session.add_all(assets) - session.commit() - - response = self.client.get("/api/v1/assets?limit=180", environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - assert len(response.json["assets"]) == 150 - - -class TestGetAssetEvents(TestAssetEndpoint): - def test_should_respond_200(self, session): - d = self._create_asset(session) - common = { - "asset_id": 1, - "extra": {"foo": "bar"}, - "source_dag_id": "foo", - "source_task_id": "bar", - "source_run_id": "custom", - "source_map_index": -1, - "created_dagruns": [], - } - - events = [AssetEvent(id=i, timestamp=timezone.parse(self.default_time), **common) for i in [1, 2]] - session.add_all(events) - session.commit() - assert session.query(AssetEvent).count() == 2 - - response = self.client.get("/api/v1/assets/events", environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "asset_events": [ - { - "id": 1, - "timestamp": self.default_time, - **common, - "asset_uri": d.uri, - }, - { - "id": 2, - "timestamp": self.default_time, - **common, - "asset_uri": d.uri, - }, - ], - "total_entries": 2, - } - - @pytest.mark.parametrize( - "attr, value", - [ - ("asset_id", "2"), - ("source_dag_id", "dag2"), - ("source_task_id", "task2"), - ("source_run_id", "run2"), - ("source_map_index", "2"), - ], - ) - @provide_session - def test_filtering(self, attr, value, session): - assets = [ - AssetModel( - id=i, - uri=f"s3://bucket/key/{i}", - extra={"foo": "bar"}, - created_at=timezone.parse(self.default_time), - updated_at=timezone.parse(self.default_time), - ) - for i in [1, 2, 3] - ] - session.add_all(assets) - session.commit() - events = [ - AssetEvent( - id=i, - asset_id=i, - source_dag_id=f"dag{i}", - source_task_id=f"task{i}", - source_run_id=f"run{i}", - source_map_index=i, - timestamp=timezone.parse(self.default_time), - ) - for i in [1, 2, 3] - ] - session.add_all(events) - session.commit() - assert session.query(AssetEvent).count() == 3 - - response = self.client.get( - f"/api/v1/assets/events?{attr}={value}", environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "asset_events": [ - { - "id": 2, - "asset_id": 2, - "asset_uri": assets[1].uri, - "extra": {}, - "source_dag_id": "dag2", - "source_task_id": "task2", - "source_run_id": "run2", - "source_map_index": 2, - "timestamp": self.default_time, - "created_dagruns": [], - } - ], - "total_entries": 1, - } - - def test_order_by_raises_400_for_invalid_attr(self, session): - self._create_asset(session) - events = [ - AssetEvent( - asset_id=1, - extra="{'foo': 'bar'}", - source_dag_id="foo", - source_task_id="bar", - source_run_id="custom", - source_map_index=-1, - timestamp=timezone.parse(self.default_time), - ) - for i in [1, 2] - ] - session.add_all(events) - session.commit() - assert session.query(AssetEvent).count() == 2 - - response = self.client.get( - "/api/v1/assets/events?order_by=fake", environ_overrides={"REMOTE_USER": "test"} - ) # missing attr - - assert response.status_code == 400 - msg = "Ordering with 'fake' is disallowed or the attribute does not exist on the model" - assert response.json["detail"] == msg - - def test_should_raises_401_unauthenticated(self, session): - response = self.client.get("/api/v1/assets/events") - assert_401(response) - - def test_includes_created_dagrun(self, session): - self._create_asset(session) - event = AssetEvent( - id=1, - asset_id=1, - timestamp=timezone.parse(self.default_time), - ) - session.add(event) - session.commit() - - dagrun = DagRun( - dag_id="TEST_DAG_ID", - run_id="TEST_DAG_RUN_ID", - run_type=DagRunType.ASSET_TRIGGERED, - logical_date=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - state="success", - ) - dagrun.end_date = timezone.parse(self.default_time) - session.add(dagrun) - session.commit() - - event.created_dagruns.append(dagrun) - session.commit() - - response = self.client.get("/api/v1/assets/events", environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "asset_events": [ - { - "id": 1, - "asset_id": 1, - "asset_uri": "s3://bucket/key", - "extra": {}, - "source_dag_id": None, - "source_task_id": None, - "source_run_id": None, - "source_map_index": -1, - "timestamp": self.default_time, - "created_dagruns": [ - { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID", - "data_interval_end": None, - "data_interval_start": None, - "end_date": self.default_time, - "logical_date": self.default_time, - "start_date": self.default_time, - "state": "success", - }, - ], - } - ], - "total_entries": 1, - } - - -class TestPostAssetEvents(TestAssetEndpoint): - @pytest.fixture - def time_freezer(self) -> Generator: - freezer = time_machine.travel(self.default_time, tick=False) - freezer.start() - - yield - - freezer.stop() - - @pytest.mark.usefixtures("time_freezer") - def test_should_respond_200(self, session): - self._create_asset(session) - event_payload = {"asset_uri": "s3://bucket/key", "extra": {"foo": "bar"}} - response = self.client.post( - "/api/v1/assets/events", json=event_payload, environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "id": ANY, - "created_dagruns": [], - "asset_uri": event_payload["asset_uri"], - "asset_id": ANY, - "extra": {"foo": "bar", "from_rest_api": True}, - "source_dag_id": None, - "source_task_id": None, - "source_run_id": None, - "source_map_index": -1, - "timestamp": self.default_time, - } - _check_last_log( - session, - dag_id=None, - event="api.create_asset_event", - logical_date=None, - expected_extra=event_payload, - ) - - @pytest.mark.enable_redact - def test_should_mask_sensitive_extra_logs(self, session): - self._create_asset(session) - event_payload = {"asset_uri": "s3://bucket/key", "extra": {"password": "bar"}} - response = self.client.post( - "/api/v1/assets/events", json=event_payload, environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 200 - expected_extra = {**event_payload, "extra": {"password": "***"}} - _check_last_log( - session, - dag_id=None, - event="api.create_asset_event", - logical_date=None, - expected_extra=expected_extra, - ) - - def test_order_by_raises_400_for_invalid_attr(self, session): - self._create_asset(session) - event_invalid_payload = {"asset_uri": "TEST_ASSET_URI", "extra": {"foo": "bar"}, "fake": {}} - response = self.client.post( - "/api/v1/assets/events", json=event_invalid_payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - - def test_should_raises_401_unauthenticated(self, session): - self._create_asset(session) - response = self.client.post("/api/v1/assets/events", json={"asset_uri": "TEST_ASSET_URI"}) - assert_401(response) - - -class TestGetAssetEventsEndpointPagination(TestAssetEndpoint): - @pytest.mark.parametrize( - "url, expected_event_runids", - [ - # Limit test data - ("/api/v1/assets/events?limit=1&order_by=source_run_id", ["run1"]), - ( - "/api/v1/assets/events?limit=3&order_by=source_run_id", - [f"run{i}" for i in range(1, 4)], - ), - # Offset test data - ( - "/api/v1/assets/events?offset=1&order_by=source_run_id", - [f"run{i}" for i in range(2, 10)], - ), - ( - "/api/v1/assets/events?offset=3&order_by=source_run_id", - [f"run{i}" for i in range(4, 10)], - ), - # Limit and offset test data - ( - "/api/v1/assets/events?offset=3&limit=3&order_by=source_run_id", - [f"run{i}" for i in [4, 5, 6]], - ), - ], - ) - @provide_session - def test_limit_and_offset(self, url, expected_event_runids, session): - self._create_asset(session) - events = [ - AssetEvent( - asset_id=1, - source_dag_id="foo", - source_task_id="bar", - source_run_id=f"run{i}", - source_map_index=-1, - timestamp=timezone.parse(self.default_time), - ) - for i in range(1, 10) - ] - session.add_all(events) - session.commit() - - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - event_runids = [event["source_run_id"] for event in response.json["asset_events"]] - assert event_runids == expected_event_runids - - def test_should_respect_page_size_limit_default(self, session): - self._create_asset(session) - events = [ - AssetEvent( - asset_id=1, - source_dag_id="foo", - source_task_id="bar", - source_run_id=f"run{i}", - source_map_index=-1, - timestamp=timezone.parse(self.default_time), - ) - for i in range(1, 110) - ] - session.add_all(events) - session.commit() - - response = self.client.get("/api/v1/assets/events", environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - assert len(response.json["asset_events"]) == 100 - - @conf_vars({("api", "maximum_page_limit"): "150"}) - def test_should_return_conf_max_if_req_max_above_conf(self, session): - self._create_asset(session) - events = [ - AssetEvent( - asset_id=1, - source_dag_id="foo", - source_task_id="bar", - source_run_id=f"run{i}", - source_map_index=-1, - timestamp=timezone.parse(self.default_time), - ) - for i in range(1, 200) - ] - session.add_all(events) - session.commit() - - response = self.client.get( - "/api/v1/assets/events?limit=180", environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 200 - assert len(response.json["asset_events"]) == 150 - - -class TestQueuedEventEndpoint(TestAssetEndpoint): - @pytest.fixture - def time_freezer(self) -> Generator: - freezer = time_machine.travel(self.default_time, tick=False) - freezer.start() - - yield - - freezer.stop() - - def _create_asset_dag_run_queues(self, dag_id, asset_id, session): - adrq = AssetDagRunQueue(target_dag_id=dag_id, asset_id=asset_id) - session.add(adrq) - session.commit() - return adrq - - -class TestGetDagAssetQueuedEvent(TestQueuedEventEndpoint): - def test_should_raises_401_unauthenticated(self, session): - dag_id = "dummy" - asset_uri = "dummy" - - response = self.client.get(f"/api/v1/dags/{dag_id}/assets/queuedEvent/{asset_uri}") - - assert_401(response) - - def test_should_raise_403_forbidden(self, session): - dag_id = "dummy" - asset_uri = "dummy" - - response = self.client.get( - f"/api/v1/dags/{dag_id}/assets/queuedEvent/{asset_uri}", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - - assert response.status_code == 403 - - -class TestDeleteDagAssetQueuedEvent(TestAssetEndpoint): - def test_should_raises_401_unauthenticated(self, session): - dag_id = "dummy" - asset_uri = "dummy" - response = self.client.delete(f"/api/v1/dags/{dag_id}/assets/queuedEvent/{asset_uri}") - assert_401(response) - - def test_should_raise_403_forbidden(self, session): - dag_id = "dummy" - asset_uri = "dummy" - response = self.client.delete( - f"/api/v1/dags/{dag_id}/assets/queuedEvent/{asset_uri}", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - -class TestGetDagAssetQueuedEvents(TestQueuedEventEndpoint): - def test_should_raises_401_unauthenticated(self): - dag_id = "dummy" - - response = self.client.get(f"/api/v1/dags/{dag_id}/assets/queuedEvent") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - dag_id = "dummy" - - response = self.client.get( - f"/api/v1/dags/{dag_id}/assets/queuedEvent", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - - assert response.status_code == 403 - - -class TestDeleteDagAssetQueuedEvents(TestAssetEndpoint): - def test_should_raises_401_unauthenticated(self): - dag_id = "dummy" - - response = self.client.delete(f"/api/v1/dags/{dag_id}/assets/queuedEvent") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - dag_id = "dummy" - - response = self.client.delete( - f"/api/v1/dags/{dag_id}/assets/queuedEvent", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - - assert response.status_code == 403 - - -class TestGetAssetQueuedEvents(TestQueuedEventEndpoint): - def test_should_raises_401_unauthenticated(self): - asset_uri = "not_exists" - - response = self.client.get(f"/api/v1/assets/queuedEvent/{asset_uri}") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - asset_uri = "not_exists" - - response = self.client.get( - f"/api/v1/assets/queuedEvent/{asset_uri}", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - - assert response.status_code == 403 - - -class TestDeleteAssetQueuedEvents(TestQueuedEventEndpoint): - def test_should_raises_401_unauthenticated(self): - asset_uri = "not_exists" - - response = self.client.delete(f"/api/v1/assets/queuedEvent/{asset_uri}") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - asset_uri = "not_exists" - - response = self.client.delete( - f"/api/v1/assets/queuedEvent/{asset_uri}", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - - assert response.status_code == 403 diff --git a/tests/api_connexion/endpoints/test_config_endpoint.py b/tests/api_connexion/endpoints/test_config_endpoint.py deleted file mode 100644 index 8d3785f08811d..0000000000000 --- a/tests/api_connexion/endpoints/test_config_endpoint.py +++ /dev/null @@ -1,342 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import textwrap -from unittest.mock import patch - -import pytest - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.config import conf_vars - -pytestmark = pytest.mark.db_test - - -MOCK_CONF = { - "core": { - "parallelism": "1024", - }, - "smtp": { - "smtp_host": "localhost", - "smtp_mail_from": "airflow@example.com", - }, -} - -MOCK_CONF_WITH_SENSITIVE_VALUE = { - "core": {"parallelism": "1024"}, - "smtp": { - "smtp_host": "localhost", - "smtp_mail_from": "airflow@example.com", - }, - "database": { - "sql_alchemy_conn": "mock_conn", - }, -} - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - with conf_vars({("webserver", "expose_config"): "True"}): - yield minimal_app_for_api - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestGetConfig: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - def test_should_respond_200_text_plain(self, mock_as_dict): - response = self.client.get( - "/api/v1/config", headers={"Accept": "text/plain"}, environ_overrides={"REMOTE_USER": "test"} - ) - mock_as_dict.assert_called_with(display_source=False, display_sensitive=True) - assert response.status_code == 200 - expected = textwrap.dedent( - """\ - [core] - parallelism = 1024 - - [smtp] - smtp_host = localhost - smtp_mail_from = airflow@example.com - """ - ) - assert expected == response.data.decode() - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - @conf_vars({("webserver", "expose_config"): "non-sensitive-only"}) - def test_should_respond_200_text_plain_with_non_sensitive_only(self, mock_as_dict): - response = self.client.get( - "/api/v1/config", headers={"Accept": "text/plain"}, environ_overrides={"REMOTE_USER": "test"} - ) - mock_as_dict.assert_called_with(display_source=False, display_sensitive=False) - assert response.status_code == 200 - expected = textwrap.dedent( - """\ - [core] - parallelism = 1024 - - [smtp] - smtp_host = localhost - smtp_mail_from = airflow@example.com - """ - ) - assert expected == response.data.decode() - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - def test_should_respond_200_application_json(self, mock_as_dict): - response = self.client.get( - "/api/v1/config", - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - mock_as_dict.assert_called_with(display_source=False, display_sensitive=True) - assert response.status_code == 200 - expected = { - "sections": [ - { - "name": "core", - "options": [ - {"key": "parallelism", "value": "1024"}, - ], - }, - { - "name": "smtp", - "options": [ - {"key": "smtp_host", "value": "localhost"}, - {"key": "smtp_mail_from", "value": "airflow@example.com"}, - ], - }, - ] - } - assert expected == response.json - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - def test_should_respond_200_single_section_as_text_plain(self, mock_as_dict): - response = self.client.get( - "/api/v1/config?section=smtp", - headers={"Accept": "text/plain"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - mock_as_dict.assert_called_with(display_source=False, display_sensitive=True) - assert response.status_code == 200 - expected = textwrap.dedent( - """\ - [smtp] - smtp_host = localhost - smtp_mail_from = airflow@example.com - """ - ) - assert expected == response.data.decode() - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - def test_should_respond_200_single_section_as_json(self, mock_as_dict): - response = self.client.get( - "/api/v1/config?section=smtp", - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - mock_as_dict.assert_called_with(display_source=False, display_sensitive=True) - assert response.status_code == 200 - expected = { - "sections": [ - { - "name": "smtp", - "options": [ - {"key": "smtp_host", "value": "localhost"}, - {"key": "smtp_mail_from", "value": "airflow@example.com"}, - ], - }, - ] - } - assert expected == response.json - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - def test_should_respond_404_when_section_not_exist(self, mock_as_dict): - response = self.client.get( - "/api/v1/config?section=smtp1", - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 404 - assert "section=smtp1 not found." in response.json["detail"] - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - def test_should_respond_406(self, mock_as_dict): - response = self.client.get( - "/api/v1/config", - headers={"Accept": "application/octet-stream"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 406 - - def test_should_raises_401_unauthenticated(self): - response = self.client.get("/api/v1/config", headers={"Accept": "application/json"}) - - assert_401(response) - - def test_should_raises_403_unauthorized(self): - response = self.client.get( - "/api/v1/config", - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - - assert response.status_code == 403 - - @conf_vars({("webserver", "expose_config"): "False"}) - def test_should_respond_403_when_expose_config_off(self): - response = self.client.get( - "/api/v1/config", - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 403 - assert "chose not to expose" in response.json["detail"] - - -class TestGetValue: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - def test_should_respond_200_text_plain(self, mock_as_dict): - response = self.client.get( - "/api/v1/config/section/smtp/option/smtp_mail_from", - headers={"Accept": "text/plain"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - expected = textwrap.dedent( - """\ - [smtp] - smtp_mail_from = airflow@example.com - """ - ) - assert expected == response.data.decode() - - @patch( - "airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", - return_value=MOCK_CONF_WITH_SENSITIVE_VALUE, - ) - @conf_vars({("webserver", "expose_config"): "non-sensitive-only"}) - @pytest.mark.parametrize( - "section, option", - [ - ("database", "sql_alchemy_conn"), - ("database", "SQL_ALCHEMY_CONN"), - ("databasE", "sql_alchemy_conn"), - ("DATABASE", "sql_alchemy_conn"), - ], - ) - def test_should_respond_200_text_plain_with_non_sensitive_only(self, mock_as_dict, section, option): - response = self.client.get( - f"/api/v1/config/section/{section}/option/{option}", - headers={"Accept": "text/plain"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - expected = textwrap.dedent( - f"""\ - [{section}] - {option} = < hidden > - """ - ) - assert expected == response.data.decode() - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - def test_should_respond_200_application_json(self, mock_as_dict): - response = self.client.get( - "/api/v1/config/section/smtp/option/smtp_mail_from", - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - expected = { - "sections": [ - { - "name": "smtp", - "options": [ - {"key": "smtp_mail_from", "value": "airflow@example.com"}, - ], - }, - ] - } - assert expected == response.json - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - def test_should_respond_404_when_option_not_exist(self, mock_as_dict): - response = self.client.get( - "/api/v1/config/section/smtp/option/smtp_mail_from1", - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 404 - assert "The option [smtp/smtp_mail_from1] is not found in config." in response.json["detail"] - - @patch("airflow.api_connexion.endpoints.config_endpoint.conf.as_dict", return_value=MOCK_CONF) - def test_should_respond_406(self, mock_as_dict): - response = self.client.get( - "/api/v1/config/section/smtp/option/smtp_mail_from", - headers={"Accept": "application/octet-stream"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 406 - - def test_should_raises_401_unauthenticated(self): - response = self.client.get( - "/api/v1/config/section/smtp/option/smtp_mail_from", headers={"Accept": "application/json"} - ) - - assert_401(response) - - def test_should_raises_403_unauthorized(self): - response = self.client.get( - "/api/v1/config/section/smtp/option/smtp_mail_from", - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - - assert response.status_code == 403 - - @conf_vars({("webserver", "expose_config"): "False"}) - def test_should_respond_403_when_expose_config_off(self): - response = self.client.get( - "/api/v1/config/section/smtp/option/smtp_mail_from", - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 403 - assert "chose not to expose" in response.json["detail"] diff --git a/tests/api_connexion/endpoints/test_connection_endpoint.py b/tests/api_connexion/endpoints/test_connection_endpoint.py deleted file mode 100644 index 3f27028aa8dfc..0000000000000 --- a/tests/api_connexion/endpoints/test_connection_endpoint.py +++ /dev/null @@ -1,659 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os -from unittest import mock - -import pytest - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.models import Connection -from airflow.secrets.environment_variables import CONN_ENV_PREFIX -from airflow.utils.session import provide_session - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_connections -from tests_common.test_utils.www import _check_last_log - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestConnectionEndpoint: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - # we want only the connection created here for this test - clear_db_connections(False) - - def teardown_method(self) -> None: - clear_db_connections() - - def _create_connection(self, session): - connection_model = Connection(conn_id="test-connection-id", conn_type="test_type") - session.add(connection_model) - session.commit() - - -class TestDeleteConnection(TestConnectionEndpoint): - def test_delete_should_respond_204(self, session): - connection_model = Connection(conn_id="test-connection", conn_type="test_type") - - session.add(connection_model) - session.commit() - conn = session.query(Connection).all() - assert len(conn) == 1 - response = self.client.delete( - "/api/v1/connections/test-connection", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 204 - connection = session.query(Connection).all() - assert len(connection) == 0 - _check_last_log(session, dag_id=None, event="api.connection.delete", logical_date=None) - - def test_delete_should_respond_404(self): - response = self.client.delete( - "/api/v1/connections/test-connection", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - assert response.json == { - "detail": "The Connection with connection_id: `test-connection` was not found", - "status": 404, - "title": "Connection not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.delete("/api/v1/connections/test-connection") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "/api/v1/connections/test-connection-id", environ_overrides={"REMOTE_USER": "test_no_permissions"} - ) - assert response.status_code == 403 - - -class TestGetConnection(TestConnectionEndpoint): - def test_should_respond_200(self, session): - connection_model = Connection( - conn_id="test-connection-id", - conn_type="mysql", - description="test description", - host="mysql", - login="login", - schema="testschema", - port=80, - extra='{"param": "value"}', - ) - session.add(connection_model) - session.commit() - result = session.query(Connection).all() - assert len(result) == 1 - response = self.client.get( - "/api/v1/connections/test-connection-id", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json == { - "connection_id": "test-connection-id", - "conn_type": "mysql", - "description": "test description", - "host": "mysql", - "login": "login", - "schema": "testschema", - "port": 80, - "extra": '{"param": "value"}', - } - - @pytest.mark.enable_redact - def test_should_mask_sensitive_values_in_extra(self, session): - connection_model = Connection( - conn_id="test-connection-id", - conn_type="mysql", - description="test description", - extra={"nonsensitive": "just_a_value", "api_token": "secretvalue"}, - ) - session.add(connection_model) - session.commit() - - response = self.client.get( - "/api/v1/connections/test-connection-id", environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.json["extra"] == '{"nonsensitive": "just_a_value", "api_token": "***"}' - - def test_should_respond_404(self): - response = self.client.get( - "/api/v1/connections/invalid-connection", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - assert response.json == { - "detail": "The Connection with connection_id: `invalid-connection` was not found", - "status": 404, - "title": "Connection not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.get("/api/v1/connections/test-connection-id") - - assert_401(response) - - -class TestGetConnections(TestConnectionEndpoint): - def test_should_respond_200(self, session): - connection_model_1 = Connection(conn_id="test-connection-id-1", conn_type="test_type") - connection_model_2 = Connection(conn_id="test-connection-id-2", conn_type="test_type") - connections = [connection_model_1, connection_model_2] - session.add_all(connections) - session.commit() - result = session.query(Connection).all() - assert len(result) == 2 - response = self.client.get("/api/v1/connections", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == { - "connections": [ - { - "connection_id": "test-connection-id-1", - "conn_type": "test_type", - "description": None, - "host": None, - "login": None, - "schema": None, - "port": None, - }, - { - "connection_id": "test-connection-id-2", - "conn_type": "test_type", - "description": None, - "host": None, - "login": None, - "schema": None, - "port": None, - }, - ], - "total_entries": 2, - } - - def test_should_respond_200_with_order_by(self, session): - connection_model_1 = Connection(conn_id="test-connection-id-1", conn_type="test_type") - connection_model_2 = Connection(conn_id="test-connection-id-2", conn_type="test_type") - connections = [connection_model_1, connection_model_2] - session.add_all(connections) - session.commit() - result = session.query(Connection).all() - assert len(result) == 2 - response = self.client.get( - "/api/v1/connections?order_by=-connection_id", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - # Using - means descending - assert response.json == { - "connections": [ - { - "connection_id": "test-connection-id-2", - "conn_type": "test_type", - "description": None, - "host": None, - "login": None, - "schema": None, - "port": None, - }, - { - "connection_id": "test-connection-id-1", - "conn_type": "test_type", - "description": None, - "host": None, - "login": None, - "schema": None, - "port": None, - }, - ], - "total_entries": 2, - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.get("/api/v1/connections") - - assert_401(response) - - -class TestGetConnectionsPagination(TestConnectionEndpoint): - @pytest.mark.parametrize( - "url, expected_conn_ids", - [ - ("/api/v1/connections?limit=1", ["TEST_CONN_ID1"]), - ("/api/v1/connections?limit=2", ["TEST_CONN_ID1", "TEST_CONN_ID2"]), - ( - "/api/v1/connections?offset=5", - [ - "TEST_CONN_ID6", - "TEST_CONN_ID7", - "TEST_CONN_ID8", - "TEST_CONN_ID9", - "TEST_CONN_ID10", - ], - ), - ( - "/api/v1/connections?offset=0", - [ - "TEST_CONN_ID1", - "TEST_CONN_ID2", - "TEST_CONN_ID3", - "TEST_CONN_ID4", - "TEST_CONN_ID5", - "TEST_CONN_ID6", - "TEST_CONN_ID7", - "TEST_CONN_ID8", - "TEST_CONN_ID9", - "TEST_CONN_ID10", - ], - ), - ("/api/v1/connections?limit=1&offset=5", ["TEST_CONN_ID6"]), - ("/api/v1/connections?limit=1&offset=1", ["TEST_CONN_ID2"]), - ( - "/api/v1/connections?limit=2&offset=2", - ["TEST_CONN_ID3", "TEST_CONN_ID4"], - ), - ], - ) - @provide_session - def test_handle_limit_offset(self, url, expected_conn_ids, session): - connections = self._create_connections(10) - session.add_all(connections) - session.commit() - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json["total_entries"] == 10 - conn_ids = [conn["connection_id"] for conn in response.json["connections"] if conn] - assert conn_ids == expected_conn_ids - - def test_should_respect_page_size_limit_default(self, session): - connection_models = self._create_connections(200) - session.add_all(connection_models) - session.commit() - - response = self.client.get("/api/v1/connections", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - - assert response.json["total_entries"] == 200 - assert len(response.json["connections"]) == 100 - - def test_invalid_order_by_raises_400(self, session): - connection_models = self._create_connections(200) - session.add_all(connection_models) - session.commit() - - response = self.client.get( - "/api/v1/connections?order_by=invalid", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert ( - response.json["detail"] == "Ordering with 'invalid' is disallowed or" - " the attribute does not exist on the model" - ) - - def test_limit_of_zero_should_return_default(self, session): - connection_models = self._create_connections(200) - session.add_all(connection_models) - session.commit() - - response = self.client.get("/api/v1/connections?limit=0", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - - assert response.json["total_entries"] == 200 - assert len(response.json["connections"]) == 100 - - @conf_vars({("api", "maximum_page_limit"): "150"}) - def test_should_return_conf_max_if_req_max_above_conf(self, session): - connection_models = self._create_connections(200) - session.add_all(connection_models) - session.commit() - - response = self.client.get("/api/v1/connections?limit=180", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert len(response.json["connections"]) == 150 - - def _create_connections(self, count): - return [ - Connection(conn_id=f"TEST_CONN_ID{i}", conn_type=f"TEST_CONN_TYPE{i}") - for i in range(1, count + 1) - ] - - -class TestPatchConnection(TestConnectionEndpoint): - @pytest.mark.parametrize( - "payload", - [ - {"connection_id": "test-connection-id", "conn_type": "test_type", "extra": '{"key": "var"}'}, - {"extra": '{"key": "var"}'}, - ], - ) - @provide_session - def test_patch_should_respond_200(self, payload, session): - self._create_connection(session) - - response = self.client.patch( - "/api/v1/connections/test-connection-id", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - _check_last_log(session, dag_id=None, event="api.connection.edit", logical_date=None) - - def test_patch_should_respond_200_with_update_mask(self, session): - self._create_connection(session) - test_connection = "test-connection-id" - payload = { - "connection_id": test_connection, - "conn_type": "test_type_2", - "extra": "{'key': 'var'}", - "login": "login", - "port": 80, - } - response = self.client.patch( - "/api/v1/connections/test-connection-id?update_mask=port,login", - json=payload, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - connection = session.query(Connection).filter_by(conn_id=test_connection).first() - assert connection.password is None - assert response.json == { - "connection_id": test_connection, # not updated - "conn_type": "test_type", # Not updated - "description": None, # Not updated - "extra": None, # Not updated - "login": "login", # updated - "port": 80, # updated - "schema": None, - "host": None, - } - - @pytest.mark.parametrize( - "payload, update_mask, error_message", - [ - ( - { - "connection_id": "test-connection-id", - "conn_type": "test_type_2", - "extra": "{'key': 'var'}", - "login": "login", - "port": 80, - }, - "update_mask=ports, login", # posts is unknown - "'ports' is unknown or cannot be updated.", - ), - ( - { - "connection_id": "test-connection-id", - "conn_type": "test_type_2", - "extra": "{'key': 'var'}", - "login": "login", - "port": 80, - }, - "update_mask=port, login, conn_id", # conn_id is unknown - "'conn_id' is unknown or cannot be updated.", - ), - ( - { - "connection_id": "test-connection-id", - "conn_type": "test_type_2", - "extra": "{'key': 'var'}", - "login": "login", - "port": 80, - }, - "update_mask=port, login, connection_id", # connection_id cannot be updated - "'connection_id' is unknown or cannot be updated.", - ), - ( - { - "connection_id": "test-connection", # trying to change connection_id - "conn_type": "test-type", - "login": "login", - }, - "", # not necessary - "The connection_id cannot be updated.", - ), - ], - ) - @provide_session - def test_patch_should_respond_400_for_invalid_fields_in_update_mask( - self, payload, update_mask, error_message, session - ): - self._create_connection(session) - response = self.client.patch( - f"/api/v1/connections/test-connection-id?{update_mask}", - json=payload, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json["detail"] == error_message - - @pytest.mark.parametrize( - "payload, error_message", - [ - ( - { - "connection_id": "test-connection-id", - "conn_type": "test-type", - "extra": 0, # expected string - }, - "0 is not of type 'string' - 'extra'", - ), - ( - { - "connection_id": "test-connection-id", - "conn_type": "test-type", - "extras": "{}", # extras not a known field e.g typo - }, - "extras", - ), - ( - { - "connection_id": "test-connection-id", - "conn_type": "test-type", - "invalid_field": "invalid field", # unknown field - "_password": "{}", # _password not a known field - }, - "_password", - ), - ], - ) - @provide_session - def test_patch_should_respond_400_for_invalid_update(self, payload, error_message, session): - self._create_connection(session) - response = self.client.patch( - "/api/v1/connections/test-connection-id", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert error_message in response.json["detail"] - - def test_patch_should_respond_404_not_found(self): - payload = {"connection_id": "test-connection-id", "conn_type": "test-type", "port": 90} - response = self.client.patch( - "/api/v1/connections/test-connection-id", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - assert response.json == { - "detail": "The Connection with connection_id: `test-connection-id` was not found", - "status": 404, - "title": "Connection not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self, session): - self._create_connection(session) - - response = self.client.patch( - "/api/v1/connections/test-connection-id", - json={"connection_id": "test-connection-id", "conn_type": "test_type", "extra": "{'key': 'var'}"}, - ) - - assert_401(response) - - -class TestPostConnection(TestConnectionEndpoint): - def test_post_should_respond_200(self, session): - payload = {"connection_id": "test-connection-id", "conn_type": "test_type"} - response = self.client.post( - "/api/v1/connections", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - connection = session.query(Connection).all() - assert len(connection) == 1 - assert connection[0].conn_id == "test-connection-id" - _check_last_log( - session, dag_id=None, event="api.connection.create", logical_date=None, expected_extra=payload - ) - - def test_post_should_respond_200_extra_null(self, session): - payload = {"connection_id": "test-connection-id", "conn_type": "test_type", "extra": None} - response = self.client.post( - "/api/v1/connections", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json["extra"] is None - connection = session.query(Connection).all() - assert len(connection) == 1 - assert connection[0].conn_id == "test-connection-id" - assert connection[0].extra is None - - def test_post_should_respond_400_for_invalid_payload(self): - payload = { - "connection_id": "test-connection-id", - } # conn_type missing - response = self.client.post( - "/api/v1/connections", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert response.json == { - "detail": "{'conn_type': ['Missing data for required field.']}", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_post_should_respond_400_for_invalid_conn_id(self): - payload = {"connection_id": "****", "conn_type": "test_type"} - response = self.client.post( - "/api/v1/connections", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert response.json == { - "detail": "The key '****' has to be made of " - "alphanumeric characters, dashes, dots and underscores exclusively", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_post_should_respond_409_already_exist(self): - payload = {"connection_id": "test-connection-id", "conn_type": "test_type"} - response = self.client.post( - "/api/v1/connections", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - # Another request - response = self.client.post( - "/api/v1/connections", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 409 - assert response.json == { - "detail": "Connection already exist. ID: test-connection-id", - "status": 409, - "title": "Conflict", - "type": EXCEPTIONS_LINK_MAP[409], - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.post( - "/api/v1/connections", json={"connection_id": "test-connection-id", "conn_type": "test_type"} - ) - - assert_401(response) - - -class TestConnection(TestConnectionEndpoint): - @mock.patch.dict(os.environ, {"AIRFLOW__CORE__TEST_CONNECTION": "Enabled"}) - def test_should_respond_200(self): - payload = {"connection_id": "test-connection-id", "conn_type": "sqlite"} - response = self.client.post( - "/api/v1/connections/test", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json == { - "status": True, - "message": "Connection successfully tested", - } - - @mock.patch.dict(os.environ, {"AIRFLOW__CORE__TEST_CONNECTION": "Enabled"}) - def test_connection_env_is_cleaned_after_run(self): - payload = {"connection_id": "test-connection-id", "conn_type": "sqlite"} - self.client.post("/api/v1/connections/test", json=payload, environ_overrides={"REMOTE_USER": "test"}) - assert not any([key.startswith(CONN_ENV_PREFIX) for key in os.environ.keys()]) - - @mock.patch.dict(os.environ, {"AIRFLOW__CORE__TEST_CONNECTION": "Enabled"}) - def test_post_should_respond_400_for_invalid_payload(self): - payload = { - "connection_id": "test-connection-id", - } # conn_type missing - response = self.client.post( - "/api/v1/connections/test", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert response.json == { - "detail": "{'conn_type': ['Missing data for required field.']}", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.post( - "/api/v1/connections/test", json={"connection_id": "test-connection-id", "conn_type": "test_type"} - ) - - assert_401(response) - - def test_should_respond_403_by_default(self): - payload = {"connection_id": "test-connection-id", "conn_type": "sqlite"} - response = self.client.post( - "/api/v1/connections/test", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 403 - assert response.text == ( - "Testing connections is disabled in Airflow configuration. " - "Contact your deployment admin to enable it." - ) diff --git a/tests/api_connexion/endpoints/test_dag_endpoint.py b/tests/api_connexion/endpoints/test_dag_endpoint.py deleted file mode 100644 index 3fdde9241423c..0000000000000 --- a/tests/api_connexion/endpoints/test_dag_endpoint.py +++ /dev/null @@ -1,2043 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os -import unittest.mock -from datetime import datetime - -import pendulum -import pytest - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.models import DagBag, DagModel -from airflow.models.dag import DAG -from airflow.models.serialized_dag import SerializedDagModel -from airflow.providers.standard.operators.empty import EmptyOperator -from airflow.utils.session import provide_session -from airflow.utils.state import TaskInstanceState - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_dags, clear_db_runs, clear_db_serialized_dags -from tests_common.test_utils.www import _check_last_log - -pytestmark = pytest.mark.db_test - - -@pytest.fixture -def current_file_token(url_safe_serializer) -> str: - return url_safe_serializer.dumps(__file__) - - -DAG_ID = "test_dag" -TASK_ID = "op1" -DAG2_ID = "test_dag2" -DAG3_ID = "test_dag3" -UTC_JSON_REPR = "UTC" if pendulum.__version__.startswith("3") else "Timezone('UTC')" - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - with DAG( - DAG_ID, - schedule=None, - start_date=datetime(2020, 6, 15), - doc_md="details", - params={"foo": 1}, - tags=["example"], - ) as dag: - EmptyOperator(task_id=TASK_ID) - - with DAG(DAG2_ID, schedule=None, start_date=datetime(2020, 6, 15)) as dag2: # no doc_md - EmptyOperator(task_id=TASK_ID) - - with DAG(DAG3_ID, schedule=None) as dag3: # DAG start_date set to None - EmptyOperator(task_id=TASK_ID, start_date=datetime(2019, 6, 12)) - - dag_bag = DagBag(os.devnull, include_examples=False) - dag_bag.dags = {dag.dag_id: dag, dag2.dag_id: dag2, dag3.dag_id: dag3} - - app.dag_bag = dag_bag - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestDagEndpoint: - @staticmethod - def clean_db(): - clear_db_runs() - clear_db_dags() - clear_db_serialized_dags() - - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.clean_db() - self.app = configured_app - self.client = self.app.test_client() # type:ignore - self.dag_id = DAG_ID - self.dag2_id = DAG2_ID - self.dag3_id = DAG3_ID - - def teardown_method(self) -> None: - self.clean_db() - - @provide_session - def _create_dag_models(self, count, dag_id_prefix="TEST_DAG", is_paused=False, session=None): - for num in range(1, count + 1): - dag_model = DagModel( - dag_id=f"{dag_id_prefix}_{num}", - fileloc=f"/tmp/dag_{num}.py", - timetable_summary="2 2 * * *", - is_active=True, - is_paused=is_paused, - ) - session.add(dag_model) - - @provide_session - def _create_dag_model_for_details_endpoint(self, dag_id, session=None): - dag_model = DagModel( - dag_id=dag_id, - fileloc="/tmp/dag.py", - timetable_summary="2 2 * * *", - is_active=True, - is_paused=False, - ) - session.add(dag_model) - - @provide_session - def _create_dag_model_for_details_endpoint_with_asset_expression(self, dag_id, session=None): - dag_model = DagModel( - dag_id=dag_id, - fileloc="/tmp/dag.py", - timetable_summary="2 2 * * *", - is_active=True, - is_paused=False, - asset_expression={ - "any": [ - "s3://dag1/output_1.txt", - {"all": ["s3://dag2/output_1.txt", "s3://dag3/output_3.txt"]}, - ] - }, - ) - session.add(dag_model) - - @provide_session - def _create_deactivated_dag(self, session=None): - dag_model = DagModel( - dag_id="TEST_DAG_DELETED_1", - fileloc="/tmp/dag_del_1.py", - timetable_summary="2 2 * * *", - is_active=False, - ) - session.add(dag_model) - - -class TestGetDag(TestDagEndpoint): - @conf_vars({("webserver", "secret_key"): "mysecret"}) - def test_should_respond_200(self): - self._create_dag_models(1) - response = self.client.get("/api/v1/dags/TEST_DAG_1", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": "Ii90bXAvZGFnXzEucHki.EnmIdPaUPo26lHQClbWMbDFD1Pk", - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - } - - @conf_vars({("webserver", "secret_key"): "mysecret"}) - def test_should_respond_200_with_schedule_none(self, session): - dag_model = DagModel( - dag_id="TEST_DAG_1", - fileloc="/tmp/dag_1.py", - timetable_summary=None, - is_paused=False, - ) - session.add(dag_model) - session.commit() - response = self.client.get("/api/v1/dags/TEST_DAG_1", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": "Ii90bXAvZGFnXzEucHki.EnmIdPaUPo26lHQClbWMbDFD1Pk", - "is_paused": False, - "is_active": False, - "owners": [], - "timetable_summary": None, - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - } - - def test_should_respond_404(self): - response = self.client.get("/api/v1/dags/INVALID_DAG", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 404 - - def test_should_raises_401_unauthenticated(self): - self._create_dag_models(1) - - response = self.client.get("/api/v1/dags/TEST_DAG_1") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/details", environ_overrides={"REMOTE_USER": "test_no_permissions"} - ) - assert response.status_code == 403 - - @pytest.mark.parametrize( - "fields", - [ - ["dag_id"], # only one - ["fileloc", "file_token", "owners"], # auto_field and fields.Method - ["tags"], # fields.List - ], - ) - def test_should_return_specified_fields(self, fields): - self._create_dag_models(1) - response = self.client.get( - f"/api/v1/dags/TEST_DAG_1?fields={','.join(fields)}", environ_overrides={"REMOTE_USER": "test"} - ) - res_json = response.json - assert len(res_json.keys()) == len(fields) - for field in fields: - assert field in res_json - - @pytest.mark.parametrize( - "fields", - [ - [], # empty test - ["#caw&c"], # field which not exists - ["dag_id", "#caw&c"], # field which not exists - ], - ) - def test_should_respond_400_with_not_exists_fields(self, fields): - self._create_dag_models(1) - response = self.client.get( - f"/api/v1/dags/TEST_DAG_1?fields={','.join(fields)}", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400, f"Current code: {response.status_code}" - - -class TestGetDagDetails(TestDagEndpoint): - def test_should_respond_200(self, url_safe_serializer): - self._create_dag_model_for_details_endpoint(self.dag_id) - current_file_token = url_safe_serializer.dumps("/tmp/dag.py") - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/details", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - last_parsed = response.json["last_parsed"] - expected = { - "catchup": True, - "dag_id": "test_dag", - "dag_display_name": "test_dag", - "dag_run_timeout": None, - "asset_expression": None, - "default_view": None, - "description": None, - "doc_md": "details", - "end_date": None, - "fileloc": "/tmp/dag.py", - "file_token": current_file_token, - "has_import_errors": False, - "has_task_concurrency_limits": True, - "is_active": True, - "is_paused": False, - "is_paused_upon_creation": None, - "last_expired": None, - "last_parsed": last_parsed, - "last_parsed_time": None, - "max_active_runs": 16, - "max_active_tasks": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun": None, - "next_dagrun_create_after": None, - "next_dagrun_data_interval_end": None, - "next_dagrun_data_interval_start": None, - "orientation": "LR", - "owners": [], - "params": { - "foo": { - "__class": "airflow.sdk.definitions.param.Param", - "description": None, - "schema": {}, - "value": 1, - } - }, - "render_template_as_native_obj": False, - "timetable_summary": "2 2 * * *", - "start_date": "2020-06-15T00:00:00+00:00", - "tags": [], - "template_searchpath": None, - "timetable_description": None, - "timezone": UTC_JSON_REPR, - } - assert response.json == expected - - def test_should_respond_200_with_asset_expression(self, url_safe_serializer): - self._create_dag_model_for_details_endpoint_with_asset_expression(self.dag_id) - current_file_token = url_safe_serializer.dumps("/tmp/dag.py") - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/details", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - last_parsed = response.json["last_parsed"] - expected = { - "catchup": True, - "dag_id": "test_dag", - "dag_display_name": "test_dag", - "dag_run_timeout": None, - "asset_expression": { - "any": [ - "s3://dag1/output_1.txt", - {"all": ["s3://dag2/output_1.txt", "s3://dag3/output_3.txt"]}, - ] - }, - "default_view": None, - "description": None, - "doc_md": "details", - "end_date": None, - "fileloc": "/tmp/dag.py", - "file_token": current_file_token, - "has_import_errors": False, - "has_task_concurrency_limits": True, - "is_active": True, - "is_paused": False, - "is_paused_upon_creation": None, - "last_expired": None, - "last_parsed": last_parsed, - "last_parsed_time": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "max_active_tasks": 16, - "next_dagrun": None, - "next_dagrun_create_after": None, - "next_dagrun_data_interval_end": None, - "next_dagrun_data_interval_start": None, - "orientation": "LR", - "owners": [], - "params": { - "foo": { - "__class": "airflow.sdk.definitions.param.Param", - "description": None, - "schema": {}, - "value": 1, - } - }, - "render_template_as_native_obj": False, - "timetable_summary": "2 2 * * *", - "start_date": "2020-06-15T00:00:00+00:00", - "tags": [], - "template_searchpath": None, - "timetable_description": None, - "timezone": UTC_JSON_REPR, - } - assert response.json == expected - - def test_should_response_200_with_doc_md_none(self, url_safe_serializer): - current_file_token = url_safe_serializer.dumps("/tmp/dag.py") - self._create_dag_model_for_details_endpoint(self.dag2_id) - response = self.client.get( - f"/api/v1/dags/{self.dag2_id}/details", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - last_parsed = response.json["last_parsed"] - expected = { - "catchup": True, - "dag_id": "test_dag2", - "dag_display_name": "test_dag2", - "dag_run_timeout": None, - "asset_expression": None, - "default_view": None, - "description": None, - "doc_md": None, - "end_date": None, - "fileloc": "/tmp/dag.py", - "file_token": current_file_token, - "has_import_errors": False, - "has_task_concurrency_limits": True, - "is_active": True, - "is_paused": False, - "is_paused_upon_creation": None, - "last_expired": None, - "last_parsed": last_parsed, - "last_parsed_time": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "max_active_tasks": 16, - "next_dagrun": None, - "next_dagrun_create_after": None, - "next_dagrun_data_interval_end": None, - "next_dagrun_data_interval_start": None, - "orientation": "LR", - "owners": [], - "params": {}, - "render_template_as_native_obj": False, - "timetable_summary": "2 2 * * *", - "start_date": "2020-06-15T00:00:00+00:00", - "tags": [], - "template_searchpath": None, - "timetable_description": None, - "timezone": UTC_JSON_REPR, - } - assert response.json == expected - - def test_should_response_200_for_null_start_date(self, url_safe_serializer): - current_file_token = url_safe_serializer.dumps("/tmp/dag.py") - self._create_dag_model_for_details_endpoint(self.dag3_id) - response = self.client.get( - f"/api/v1/dags/{self.dag3_id}/details", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - last_parsed = response.json["last_parsed"] - expected = { - "catchup": True, - "dag_id": "test_dag3", - "dag_display_name": "test_dag3", - "dag_run_timeout": None, - "asset_expression": None, - "default_view": None, - "description": None, - "doc_md": None, - "end_date": None, - "fileloc": "/tmp/dag.py", - "file_token": current_file_token, - "has_import_errors": False, - "has_task_concurrency_limits": True, - "is_active": True, - "is_paused": False, - "is_paused_upon_creation": None, - "last_expired": None, - "last_parsed": last_parsed, - "last_parsed_time": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "max_active_tasks": 16, - "next_dagrun": None, - "next_dagrun_create_after": None, - "next_dagrun_data_interval_end": None, - "next_dagrun_data_interval_start": None, - "orientation": "LR", - "owners": [], - "params": {}, - "render_template_as_native_obj": False, - "timetable_summary": "2 2 * * *", - "start_date": None, - "tags": [], - "template_searchpath": None, - "timetable_description": None, - "timezone": UTC_JSON_REPR, - } - assert response.json == expected - - def test_should_respond_200_serialized(self, url_safe_serializer, testing_dag_bundle): - current_file_token = url_safe_serializer.dumps("/tmp/dag.py") - self._create_dag_model_for_details_endpoint(self.dag_id) - # Get the dag out of the dagbag before we patch it to an empty one - SerializedDagModel.write_dag(self.app.dag_bag.get_dag(self.dag_id), bundle_name="testing") - - # Create empty app with empty dagbag to check if DAG is read from db - dag_bag = DagBag(os.devnull, include_examples=False, read_dags_from_db=True) - patcher = unittest.mock.patch.object(self.app, "dag_bag", dag_bag) - patcher.start() - - expected = { - "catchup": True, - "dag_id": "test_dag", - "dag_display_name": "test_dag", - "dag_run_timeout": None, - "asset_expression": None, - "default_view": None, - "description": None, - "doc_md": "details", - "end_date": None, - "file_token": current_file_token, - "fileloc": "/tmp/dag.py", - "has_import_errors": False, - "has_task_concurrency_limits": True, - "is_active": True, - "is_paused": False, - "is_paused_upon_creation": None, - "last_expired": None, - "last_parsed_time": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "max_active_tasks": 16, - "next_dagrun": None, - "next_dagrun_create_after": None, - "next_dagrun_data_interval_end": None, - "next_dagrun_data_interval_start": None, - "orientation": "LR", - "owners": [], - "params": { - "foo": { - "__class": "airflow.sdk.definitions.param.Param", - "description": None, - "schema": {}, - "value": 1, - } - }, - "render_template_as_native_obj": False, - "timetable_summary": "2 2 * * *", - "start_date": "2020-06-15T00:00:00+00:00", - "tags": [], - "template_searchpath": None, - "timetable_description": None, - "timezone": UTC_JSON_REPR, - } - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/details", environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 200 - expected.update({"last_parsed": response.json["last_parsed"]}) - assert response.json == expected - - patcher.stop() - - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/details", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - expected = { - "catchup": True, - "dag_id": "test_dag", - "dag_display_name": "test_dag", - "dag_run_timeout": None, - "asset_expression": None, - "default_view": None, - "description": None, - "doc_md": "details", - "end_date": None, - "file_token": current_file_token, - "fileloc": "/tmp/dag.py", - "has_import_errors": False, - "has_task_concurrency_limits": True, - "is_active": True, - "is_paused": False, - "is_paused_upon_creation": None, - "last_expired": None, - "last_parsed_time": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "max_active_tasks": 16, - "next_dagrun": None, - "next_dagrun_create_after": None, - "next_dagrun_data_interval_end": None, - "next_dagrun_data_interval_start": None, - "orientation": "LR", - "owners": [], - "params": { - "foo": { - "__class": "airflow.sdk.definitions.param.Param", - "description": None, - "schema": {}, - "value": 1, - } - }, - "render_template_as_native_obj": False, - "timetable_summary": "2 2 * * *", - "start_date": "2020-06-15T00:00:00+00:00", - "tags": [], - "template_searchpath": None, - "timetable_description": None, - "timezone": UTC_JSON_REPR, - } - expected.update({"last_parsed": response.json["last_parsed"]}) - assert response.json == expected - - def test_should_raises_401_unauthenticated(self): - response = self.client.get(f"/api/v1/dags/{self.dag_id}/details") - - assert_401(response) - - def test_should_raise_404_when_dag_is_not_found(self): - response = self.client.get( - "/api/v1/dags/non_existing_dag_id/details", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - assert response.json == { - "detail": "The DAG with dag_id: non_existing_dag_id was not found", - "status": 404, - "title": "DAG not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - @pytest.mark.parametrize( - "fields", - [ - ["dag_id"], # only one - ["doc_md", "file_token", "owners"], # fields.String and fields.Method - ["tags"], # fields.List - ], - ) - def test_should_return_specified_fields(self, fields): - self._create_dag_model_for_details_endpoint(self.dag2_id) - response = self.client.get( - f"/api/v1/dags/{self.dag2_id}/details?fields={','.join(fields)}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - res_json = response.json - assert len(res_json.keys()) == len(fields) - for field in fields: - assert field in res_json - - def test_should_respond_400_with_not_exists_fields(self): - fields = ["#caw&c"] - self._create_dag_model_for_details_endpoint(self.dag2_id) - response = self.client.get( - f"/api/v1/dags/{self.dag2_id}/details?fields={','.join(fields)}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400, f"Current code: {response.status_code}" - - -class TestGetDags(TestDagEndpoint): - @provide_session - def test_should_respond_200(self, session, url_safe_serializer): - self._create_dag_models(2) - self._create_deactivated_dag() - - dags_query = session.query(DagModel) - assert len(dags_query.all()) == 3 - - response = self.client.get("api/v1/dags", environ_overrides={"REMOTE_USER": "test"}) - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - file_token2 = url_safe_serializer.dumps("/tmp/dag_2.py") - - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - { - "dag_id": "TEST_DAG_2", - "dag_display_name": "TEST_DAG_2", - "description": None, - "fileloc": "/tmp/dag_2.py", - "file_token": file_token2, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - ], - "total_entries": 2, - } - - def test_only_active_true_returns_active_dags(self, url_safe_serializer): - self._create_dag_models(1) - self._create_deactivated_dag() - response = self.client.get("api/v1/dags?only_active=True", environ_overrides={"REMOTE_USER": "test"}) - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - } - ], - "total_entries": 1, - } - - def test_only_active_false_returns_all_dags(self, url_safe_serializer): - self._create_dag_models(1) - self._create_deactivated_dag() - response = self.client.get("api/v1/dags?only_active=False", environ_overrides={"REMOTE_USER": "test"}) - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - file_token_2 = url_safe_serializer.dumps("/tmp/dag_del_1.py") - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - { - "dag_id": "TEST_DAG_DELETED_1", - "dag_display_name": "TEST_DAG_DELETED_1", - "description": None, - "fileloc": "/tmp/dag_del_1.py", - "file_token": file_token_2, - "is_paused": False, - "is_active": False, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - ], - "total_entries": 2, - } - - @pytest.mark.parametrize( - "url, expected_dag_ids", - [ - ("api/v1/dags?tags=t1", ["TEST_DAG_1", "TEST_DAG_3"]), - ("api/v1/dags?tags=t2", ["TEST_DAG_2", "TEST_DAG_3"]), - ("api/v1/dags?tags=t1,t2", ["TEST_DAG_1", "TEST_DAG_2", "TEST_DAG_3"]), - ("api/v1/dags", ["TEST_DAG_1", "TEST_DAG_2", "TEST_DAG_3", "TEST_DAG_4"]), - ], - ) - def test_filter_dags_by_tags_works(self, url, expected_dag_ids): - # test filter by tags - dag1 = DAG(dag_id="TEST_DAG_1", schedule=None, tags=["t1"]) - dag2 = DAG(dag_id="TEST_DAG_2", schedule=None, tags=["t2"]) - dag3 = DAG(dag_id="TEST_DAG_3", schedule=None, tags=["t1", "t2"]) - dag4 = DAG(dag_id="TEST_DAG_4", schedule=None) - dag1.sync_to_db() - dag2.sync_to_db() - dag3.sync_to_db() - dag4.sync_to_db() - - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - dag_ids = [dag["dag_id"] for dag in response.json["dags"]] - - assert expected_dag_ids == dag_ids - - @pytest.mark.parametrize( - "url, expected_dag_ids", - [ - ("api/v1/dags?dag_id_pattern=DAG_1", {"TEST_DAG_1", "SAMPLE_DAG_1"}), - ("api/v1/dags?dag_id_pattern=SAMPLE_DAG", {"SAMPLE_DAG_1", "SAMPLE_DAG_2"}), - ( - "api/v1/dags?dag_id_pattern=_DAG_", - {"TEST_DAG_1", "TEST_DAG_2", "SAMPLE_DAG_1", "SAMPLE_DAG_2"}, - ), - ], - ) - def test_filter_dags_by_dag_id_works(self, url, expected_dag_ids): - # test filter by tags - dag1 = DAG(dag_id="TEST_DAG_1", schedule=None) - dag2 = DAG(dag_id="TEST_DAG_2", schedule=None) - dag3 = DAG(dag_id="SAMPLE_DAG_1", schedule=None) - dag4 = DAG(dag_id="SAMPLE_DAG_2", schedule=None) - dag1.sync_to_db() - dag2.sync_to_db() - dag3.sync_to_db() - dag4.sync_to_db() - - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - dag_ids = {dag["dag_id"] for dag in response.json["dags"]} - - assert expected_dag_ids == dag_ids - - @pytest.mark.parametrize( - "url, expected_dag_ids", - [ - ("api/v1/dags?limit=1", ["TEST_DAG_1"]), - ("api/v1/dags?limit=2", ["TEST_DAG_1", "TEST_DAG_10"]), - ( - "api/v1/dags?offset=5", - ["TEST_DAG_5", "TEST_DAG_6", "TEST_DAG_7", "TEST_DAG_8", "TEST_DAG_9"], - ), - ( - "api/v1/dags?offset=0", - [ - "TEST_DAG_1", - "TEST_DAG_10", - "TEST_DAG_2", - "TEST_DAG_3", - "TEST_DAG_4", - "TEST_DAG_5", - "TEST_DAG_6", - "TEST_DAG_7", - "TEST_DAG_8", - "TEST_DAG_9", - ], - ), - ("api/v1/dags?limit=1&offset=5", ["TEST_DAG_5"]), - ("api/v1/dags?limit=1&offset=1", ["TEST_DAG_10"]), - ("api/v1/dags?limit=2&offset=2", ["TEST_DAG_2", "TEST_DAG_3"]), - ], - ) - def test_should_respond_200_and_handle_pagination(self, url, expected_dag_ids): - self._create_dag_models(10) - - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - - dag_ids = [dag["dag_id"] for dag in response.json["dags"]] - - assert expected_dag_ids == dag_ids - assert response.json["total_entries"] == 10 - - def test_should_respond_200_default_limit(self): - self._create_dag_models(101) - - response = self.client.get("api/v1/dags", environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - - assert len(response.json["dags"]) == 100 - assert response.json["total_entries"] == 101 - - def test_should_raises_401_unauthenticated(self): - response = self.client.get("api/v1/dags") - - assert_401(response) - - def test_should_respond_403_unauthorized(self): - self._create_dag_models(1) - - response = self.client.get("api/v1/dags", environ_overrides={"REMOTE_USER": "test_no_permissions"}) - - assert response.status_code == 403 - - def test_paused_true_returns_paused_dags(self, url_safe_serializer): - self._create_dag_models(1, dag_id_prefix="TEST_DAG_PAUSED", is_paused=True) - self._create_dag_models(1, dag_id_prefix="TEST_DAG_UNPAUSED", is_paused=False) - response = self.client.get("api/v1/dags?paused=True", environ_overrides={"REMOTE_USER": "test"}) - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_PAUSED_1", - "dag_display_name": "TEST_DAG_PAUSED_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": True, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - } - ], - "total_entries": 1, - } - - def test_paused_false_returns_unpaused_dags(self, url_safe_serializer): - self._create_dag_models(1, dag_id_prefix="TEST_DAG_PAUSED", is_paused=True) - self._create_dag_models(1, dag_id_prefix="TEST_DAG_UNPAUSED", is_paused=False) - response = self.client.get("api/v1/dags?paused=False", environ_overrides={"REMOTE_USER": "test"}) - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_UNPAUSED_1", - "dag_display_name": "TEST_DAG_UNPAUSED_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - } - ], - "total_entries": 1, - } - - def test_paused_none_returns_all_dags(self, url_safe_serializer): - self._create_dag_models(1, dag_id_prefix="TEST_DAG_PAUSED", is_paused=True) - self._create_dag_models(1, dag_id_prefix="TEST_DAG_UNPAUSED", is_paused=False) - response = self.client.get("api/v1/dags", environ_overrides={"REMOTE_USER": "test"}) - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_PAUSED_1", - "dag_display_name": "TEST_DAG_PAUSED_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": True, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - { - "dag_id": "TEST_DAG_UNPAUSED_1", - "dag_display_name": "TEST_DAG_UNPAUSED_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - ], - "total_entries": 2, - } - - def test_should_return_specified_fields(self): - self._create_dag_models(2) - self._create_deactivated_dag() - - fields = ["dag_id", "file_token", "owners"] - response = self.client.get( - f"api/v1/dags?fields={','.join(fields)}", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - - res_json = response.json - for dag in res_json["dags"]: - assert len(dag.keys()) == len(fields) - for field in fields: - assert field in dag - - def test_should_return_specified_fields_and_total_entries(self): - total = 4 - self._create_dag_models(total) - self._create_deactivated_dag() - - limit = 2 - fields = ["dag_id"] - response = self.client.get( - f"api/v1/dags?limit={limit}&fields={','.join(fields)}", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - - res_json = response.json - assert res_json["total_entries"] == total - assert len(res_json["dags"]) == limit - for dag in res_json["dags"]: - assert len(dag.keys()) == len(fields) - for field in fields: - assert field in dag - - def test_should_respond_400_with_not_exists_fields(self): - self._create_dag_models(1) - self._create_deactivated_dag() - fields = ["#caw&c"] - response = self.client.get( - f"api/v1/dags?fields={','.join(fields)}", environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 400, f"Current code: {response.status_code}" - - -class TestPatchDag(TestDagEndpoint): - def test_should_respond_200_on_patch_is_paused(self, url_safe_serializer, session): - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - dag_model = self._create_dag_model() - payload = {"is_paused": False} - response = self.client.patch( - f"/api/v1/dags/{dag_model.dag_id}", - json=payload, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - expected_response = { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": False, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - } - assert response.json == expected_response - _check_last_log( - session, dag_id="TEST_DAG_1", event="api.patch_dag", logical_date=None, expected_extra=payload - ) - - def test_should_respond_400_on_invalid_request(self): - patch_body = { - "is_paused": True, - "timetable_summary": "1 1 * * *", - } - dag_model = self._create_dag_model() - response = self.client.patch(f"/api/v1/dags/{dag_model.dag_id}", json=patch_body) - assert response.status_code == 400 - assert response.json == { - "detail": "Property is read-only - 'timetable_summary'", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_non_existing_dag_raises_not_found(self): - patch_body = { - "is_paused": True, - } - response = self.client.patch( - "/api/v1/dags/non_existing_dag", json=patch_body, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - assert response.json == { - "detail": None, - "status": 404, - "title": "Dag with id: 'non_existing_dag' not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_respond_404(self): - response = self.client.get("/api/v1/dags/INVALID_DAG", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 404 - - @provide_session - def _create_dag_model(self, session=None): - dag_model = DagModel( - dag_id="TEST_DAG_1", fileloc="/tmp/dag_1.py", timetable_summary="2 2 * * *", is_paused=True - ) - session.add(dag_model) - return dag_model - - def test_should_raises_401_unauthenticated(self): - dag_model = self._create_dag_model() - response = self.client.patch( - f"/api/v1/dags/{dag_model.dag_id}", - json={ - "is_paused": False, - }, - ) - - assert_401(response) - - def test_should_respond_200_with_update_mask(self, url_safe_serializer): - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - dag_model = self._create_dag_model() - payload = { - "is_paused": False, - } - response = self.client.patch( - f"/api/v1/dags/{dag_model.dag_id}?update_mask=is_paused", - json=payload, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - expected_response = { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": False, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - } - assert response.json == expected_response - - @pytest.mark.parametrize( - "payload, update_mask, error_message", - [ - ( - { - "is_paused": True, - }, - "update_mask=description", - "Only `is_paused` field can be updated through the REST API", - ), - ( - { - "is_paused": True, - }, - "update_mask=timetable_summary, description", - "Only `is_paused` field can be updated through the REST API", - ), - ], - ) - def test_should_respond_400_for_invalid_fields_in_update_mask(self, payload, update_mask, error_message): - dag_model = self._create_dag_model() - - response = self.client.patch( - f"/api/v1/dags/{dag_model.dag_id}?{update_mask}", - json=payload, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json["detail"] == error_message - - def test_should_respond_403_unauthorized(self): - dag_model = self._create_dag_model() - response = self.client.patch( - f"/api/v1/dags/{dag_model.dag_id}", - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - - assert response.status_code == 403 - - -class TestPatchDags(TestDagEndpoint): - @provide_session - def test_should_respond_200_on_patch_is_paused(self, session, url_safe_serializer): - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - file_token2 = url_safe_serializer.dumps("/tmp/dag_2.py") - self._create_dag_models(2) - self._create_deactivated_dag() - - dags_query = session.query(DagModel) - assert len(dags_query.all()) == 3 - - response = self.client.patch( - "/api/v1/dags?dag_id_pattern=~", - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - { - "dag_id": "TEST_DAG_2", - "dag_display_name": "TEST_DAG_2", - "description": None, - "fileloc": "/tmp/dag_2.py", - "file_token": file_token2, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - ], - "total_entries": 2, - } - _check_last_log(session, dag_id=None, event="api.patch_dags", logical_date=None) - - def test_should_respond_200_on_patch_is_paused_using_update_mask(self, session, url_safe_serializer): - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - file_token2 = url_safe_serializer.dumps("/tmp/dag_2.py") - self._create_dag_models(2) - self._create_deactivated_dag() - - dags_query = session.query(DagModel) - assert len(dags_query.all()) == 3 - - response = self.client.patch( - "/api/v1/dags?dag_id_pattern=~&update_mask=is_paused", - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - { - "dag_id": "TEST_DAG_2", - "dag_display_name": "TEST_DAG_2", - "description": None, - "fileloc": "/tmp/dag_2.py", - "file_token": file_token2, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - ], - "total_entries": 2, - } - _check_last_log(session, dag_id=None, event="api.patch_dags", logical_date=None) - - def test_wrong_value_as_update_mask_rasise(self, session): - self._create_dag_models(2) - self._create_deactivated_dag() - - dags_query = session.query(DagModel) - assert len(dags_query.all()) == 3 - - response = self.client.patch( - "/api/v1/dags?dag_id_pattern=~&update_mask=ispaused", - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 400 - assert response.json == { - "detail": "Only `is_paused` field can be updated through the REST API", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_invalid_request_body_raises_badrequest(self, session): - self._create_dag_models(2) - self._create_deactivated_dag() - - dags_query = session.query(DagModel) - assert len(dags_query.all()) == 3 - - response = self.client.patch( - "/api/v1/dags?dag_id_pattern=~&update_mask=is_paused", - json={ - "ispaused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 400 - assert response.json == { - "detail": "{'ispaused': ['Unknown field.']}", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_only_active_true_returns_active_dags(self, url_safe_serializer, session): - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - self._create_dag_models(1) - self._create_deactivated_dag() - response = self.client.patch( - "/api/v1/dags?only_active=True&dag_id_pattern=~", - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - } - ], - "total_entries": 1, - } - _check_last_log(session, dag_id=None, event="api.patch_dags", logical_date=None) - - def test_only_active_false_returns_all_dags(self, url_safe_serializer, session): - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - self._create_dag_models(1) - self._create_deactivated_dag() - response = self.client.patch( - "/api/v1/dags?only_active=False&dag_id_pattern=~", - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - - file_token_2 = url_safe_serializer.dumps("/tmp/dag_del_1.py") - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - { - "dag_id": "TEST_DAG_DELETED_1", - "dag_display_name": "TEST_DAG_DELETED_1", - "description": None, - "fileloc": "/tmp/dag_del_1.py", - "file_token": file_token_2, - "is_paused": False, - "is_active": False, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - ], - "total_entries": 2, - } - _check_last_log(session, dag_id=None, event="api.patch_dags", logical_date=None) - - @pytest.mark.parametrize( - "url, expected_dag_ids", - [ - ("api/v1/dags?tags=t1&dag_id_pattern=~", ["TEST_DAG_1", "TEST_DAG_3"]), - ("api/v1/dags?tags=t2&dag_id_pattern=~", ["TEST_DAG_2", "TEST_DAG_3"]), - ("api/v1/dags?tags=t1,t2&dag_id_pattern=~", ["TEST_DAG_1", "TEST_DAG_2", "TEST_DAG_3"]), - ("api/v1/dags?dag_id_pattern=~", ["TEST_DAG_1", "TEST_DAG_2", "TEST_DAG_3", "TEST_DAG_4"]), - ], - ) - def test_filter_dags_by_tags_works(self, url, expected_dag_ids): - # test filter by tags - dag1 = DAG(dag_id="TEST_DAG_1", schedule=None, tags=["t1"]) - dag2 = DAG(dag_id="TEST_DAG_2", schedule=None, tags=["t2"]) - dag3 = DAG(dag_id="TEST_DAG_3", schedule=None, tags=["t1", "t2"]) - dag4 = DAG(dag_id="TEST_DAG_4", schedule=None) - dag1.sync_to_db() - dag2.sync_to_db() - dag3.sync_to_db() - dag4.sync_to_db() - response = self.client.patch( - url, - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - dag_ids = [dag["dag_id"] for dag in response.json["dags"]] - - assert expected_dag_ids == dag_ids - - @pytest.mark.parametrize( - "url, expected_dag_ids", - [ - ("api/v1/dags?dag_id_pattern=DAG_1", {"TEST_DAG_1", "SAMPLE_DAG_1"}), - ("api/v1/dags?dag_id_pattern=SAMPLE_DAG", {"SAMPLE_DAG_1", "SAMPLE_DAG_2"}), - ( - "api/v1/dags?dag_id_pattern=_DAG_", - {"TEST_DAG_1", "TEST_DAG_2", "SAMPLE_DAG_1", "SAMPLE_DAG_2"}, - ), - ], - ) - def test_filter_dags_by_dag_id_works(self, url, expected_dag_ids): - # test filter by tags - dag1 = DAG(dag_id="TEST_DAG_1", schedule=None) - dag2 = DAG(dag_id="TEST_DAG_2", schedule=None) - dag3 = DAG(dag_id="SAMPLE_DAG_1", schedule=None) - dag4 = DAG(dag_id="SAMPLE_DAG_2", schedule=None) - dag1.sync_to_db() - dag2.sync_to_db() - dag3.sync_to_db() - dag4.sync_to_db() - - response = self.client.patch( - url, - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - dag_ids = {dag["dag_id"] for dag in response.json["dags"]} - - assert expected_dag_ids == dag_ids - - @pytest.mark.parametrize( - "url, expected_dag_ids", - [ - ("api/v1/dags?limit=1&dag_id_pattern=~", ["TEST_DAG_1"]), - ("api/v1/dags?limit=2&dag_id_pattern=~", ["TEST_DAG_1", "TEST_DAG_10"]), - ( - "api/v1/dags?offset=5&dag_id_pattern=~", - ["TEST_DAG_5", "TEST_DAG_6", "TEST_DAG_7", "TEST_DAG_8", "TEST_DAG_9"], - ), - ( - "api/v1/dags?offset=0&dag_id_pattern=~", - [ - "TEST_DAG_1", - "TEST_DAG_10", - "TEST_DAG_2", - "TEST_DAG_3", - "TEST_DAG_4", - "TEST_DAG_5", - "TEST_DAG_6", - "TEST_DAG_7", - "TEST_DAG_8", - "TEST_DAG_9", - ], - ), - ("api/v1/dags?limit=1&offset=5&dag_id_pattern=~", ["TEST_DAG_5"]), - ("api/v1/dags?limit=1&offset=1&dag_id_pattern=~", ["TEST_DAG_10"]), - ("api/v1/dags?limit=2&offset=2&dag_id_pattern=~", ["TEST_DAG_2", "TEST_DAG_3"]), - ], - ) - def test_should_respond_200_and_handle_pagination(self, url, expected_dag_ids): - self._create_dag_models(10) - - response = self.client.patch( - url, - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - - dag_ids = [dag["dag_id"] for dag in response.json["dags"]] - - assert expected_dag_ids == dag_ids - assert response.json["total_entries"] == 10 - - def test_should_respond_200_default_limit(self): - self._create_dag_models(101) - - response = self.client.patch( - "api/v1/dags?dag_id_pattern=~", - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - - assert len(response.json["dags"]) == 100 - assert response.json["total_entries"] == 101 - - def test_should_raises_401_unauthenticated(self): - response = self.client.patch( - "api/v1/dags?dag_id_pattern=~", - json={ - "is_paused": False, - }, - ) - - assert_401(response) - - def test_should_respond_403_unauthorized(self): - self._create_dag_models(1) - response = self.client.patch( - "api/v1/dags?dag_id_pattern=~", - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - - assert response.status_code == 403 - - def test_should_respond_200_and_pause_dags(self, url_safe_serializer): - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - file_token2 = url_safe_serializer.dumps("/tmp/dag_2.py") - self._create_dag_models(2) - - response = self.client.patch( - "/api/v1/dags?dag_id_pattern=~", - json={ - "is_paused": True, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": True, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - { - "dag_id": "TEST_DAG_2", - "dag_display_name": "TEST_DAG_2", - "description": None, - "fileloc": "/tmp/dag_2.py", - "file_token": file_token2, - "is_paused": True, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - ], - "total_entries": 2, - } - - @provide_session - def test_should_respond_200_and_pause_dag_pattern(self, session, url_safe_serializer): - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - self._create_dag_models(10) - file_token10 = url_safe_serializer.dumps("/tmp/dag_10.py") - - response = self.client.patch( - "/api/v1/dags?dag_id_pattern=TEST_DAG_1", - json={ - "is_paused": True, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": True, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - { - "dag_id": "TEST_DAG_10", - "dag_display_name": "TEST_DAG_10", - "description": None, - "fileloc": "/tmp/dag_10.py", - "file_token": file_token10, - "is_paused": True, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - ], - "total_entries": 2, - } - - dags_not_updated = session.query(DagModel).filter(~DagModel.is_paused) - assert len(dags_not_updated.all()) == 8 - dags_updated = session.query(DagModel).filter(DagModel.is_paused) - assert len(dags_updated.all()) == 2 - - @provide_session - def test_should_respond_200_and_reverse_ordering(self, session, url_safe_serializer): - file_token = url_safe_serializer.dumps("/tmp/dag_1.py") - self._create_dag_models(2) - file_token10 = url_safe_serializer.dumps("/tmp/dag_2.py") - - response = self.client.get( - "/api/v1/dags?order_by=-dag_id", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - assert response.json == { - "dags": [ - { - "dag_id": "TEST_DAG_2", - "dag_display_name": "TEST_DAG_2", - "description": None, - "fileloc": "/tmp/dag_2.py", - "file_token": file_token10, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - { - "dag_id": "TEST_DAG_1", - "dag_display_name": "TEST_DAG_1", - "description": None, - "fileloc": "/tmp/dag_1.py", - "file_token": file_token, - "is_paused": False, - "is_active": True, - "owners": [], - "timetable_summary": "2 2 * * *", - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": False, - }, - ], - "total_entries": 2, - } - - def test_should_respons_400_dag_id_pattern_missing(self): - self._create_dag_models(1) - response = self.client.patch( - "/api/v1/dags?only_active=True", - json={ - "is_paused": False, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - - -class TestDeleteDagEndpoint(TestDagEndpoint): - def test_that_dag_can_be_deleted(self, session): - self._create_dag_models(1) - - response = self.client.delete( - "/api/v1/dags/TEST_DAG_1", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 204 - _check_last_log(session, dag_id="TEST_DAG_1", event="api.delete_dag", logical_date=None) - - def test_raise_when_dag_is_not_found(self): - response = self.client.delete( - "/api/v1/dags/TEST_DAG_1", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json == { - "detail": None, - "status": 404, - "title": "Dag with id: 'TEST_DAG_1' not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_raises_when_task_instances_of_dag_is_still_running(self, dag_maker, session): - with dag_maker("TEST_DAG_1"): - EmptyOperator(task_id="dummy") - dr = dag_maker.create_dagrun() - ti = dr.get_task_instances()[0] - ti.set_state(TaskInstanceState.RUNNING) - session.flush() - response = self.client.delete( - "/api/v1/dags/TEST_DAG_1", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 409 - assert response.json == { - "detail": "Task instances of dag with id: 'TEST_DAG_1' are still running", - "status": 409, - "title": "Conflict", - "type": EXCEPTIONS_LINK_MAP[409], - } - - def test_users_without_delete_permission_cannot_delete_dag(self): - self._create_dag_models(1) - response = self.client.delete( - "/api/v1/dags/TEST_DAG_1", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 diff --git a/tests/api_connexion/endpoints/test_dag_parsing.py b/tests/api_connexion/endpoints/test_dag_parsing.py deleted file mode 100644 index 0052732b7dfb9..0000000000000 --- a/tests/api_connexion/endpoints/test_dag_parsing.py +++ /dev/null @@ -1,111 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os - -import pytest -from sqlalchemy import select - -from airflow.models import DagBag -from airflow.models.dagbag import DagPriorityParsingRequest - -from tests_common.test_utils.api_connexion_utils import create_user, delete_user -from tests_common.test_utils.db import clear_db_dag_parsing_requests, parse_and_sync_to_db - -pytestmark = pytest.mark.db_test - - -ROOT_DIR = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir, os.pardir)) -EXAMPLE_DAG_FILE = os.path.join("airflow", "example_dags", "example_bash_operator.py") -TEST_DAG_ID = "example_bash_operator" -NOT_READABLE_DAG_ID = "latest_only_with_trigger" -TEST_MULTIPLE_DAGS_ID = "asset_produces_1" - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestDagParsingRequest: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - self.clear_db() - - def teardown_method(self) -> None: - self.clear_db() - - @staticmethod - def clear_db(): - clear_db_dag_parsing_requests() - - def test_201_and_400_requests(self, url_safe_serializer, session): - parse_and_sync_to_db(EXAMPLE_DAG_FILE) - dagbag = DagBag(read_dags_from_db=True) - test_dag = dagbag.get_dag(TEST_DAG_ID) - - url = f"/api/v1/parseDagFile/{url_safe_serializer.dumps(test_dag.fileloc)}" - response = self.client.put( - url, headers={"Accept": "application/json"}, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 201 - parsing_requests = session.scalars(select(DagPriorityParsingRequest)).all() - assert parsing_requests[0].fileloc == test_dag.fileloc - - # Duplicate file parsing request - response = self.client.put( - url, headers={"Accept": "application/json"}, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 201 - parsing_requests = session.scalars(select(DagPriorityParsingRequest)).all() - assert parsing_requests[0].fileloc == test_dag.fileloc - - def test_bad_file_request(self, url_safe_serializer, session): - url = f"/api/v1/parseDagFile/{url_safe_serializer.dumps('/some/random/file.py')}" - response = self.client.put( - url, headers={"Accept": "application/json"}, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - - parsing_requests = session.scalars(select(DagPriorityParsingRequest)).all() - assert parsing_requests == [] - - def test_bad_user_request(self, url_safe_serializer, session): - url = f"/api/v1/parseDagFile/{url_safe_serializer.dumps('/some/random/file.py')}" - response = self.client.put( - url, - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - parsing_requests = session.scalars(select(DagPriorityParsingRequest)).all() - assert parsing_requests == [] diff --git a/tests/api_connexion/endpoints/test_dag_run_endpoint.py b/tests/api_connexion/endpoints/test_dag_run_endpoint.py deleted file mode 100644 index 1b5ba2a739859..0000000000000 --- a/tests/api_connexion/endpoints/test_dag_run_endpoint.py +++ /dev/null @@ -1,2021 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import json -import urllib -from datetime import timedelta -from unittest import mock - -import pytest -import time_machine - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.dag_processing.bundles.manager import DagBundlesManager -from airflow.models import Log -from airflow.models.asset import AssetEvent, AssetModel -from airflow.models.dag import DAG, DagModel -from airflow.models.dagrun import DagRun -from airflow.providers.standard.operators.empty import EmptyOperator -from airflow.sdk.definitions.asset import Asset -from airflow.sdk.definitions.param import Param -from airflow.utils import timezone -from airflow.utils.session import create_session, provide_session -from airflow.utils.state import DagRunState, State -from airflow.utils.types import DagRunType - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_dags, clear_db_runs, clear_db_serialized_dags -from tests_common.test_utils.version_compat import AIRFLOW_V_3_0_PLUS -from tests_common.test_utils.www import _check_last_log - -if AIRFLOW_V_3_0_PLUS: - from airflow.utils.types import DagRunTriggeredByType - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestDagRunEndpoint: - default_time = "2020-06-11T18:00:00+00:00" - default_time_2 = "2020-06-12T18:00:00+00:00" - default_time_3 = "2020-06-13T18:00:00+00:00" - - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - clear_db_runs() - clear_db_serialized_dags() - clear_db_dags() - - def teardown_method(self) -> None: - clear_db_runs() - clear_db_dags() - clear_db_serialized_dags() - - def _create_dag(self, dag_id): - dag_instance = DagModel(dag_id=dag_id) - dag_instance.is_active = True - with create_session() as session: - session.add(dag_instance) - dag = DAG(dag_id=dag_id, schedule=None, params={"validated_number": Param(1, minimum=1, maximum=10)}) - DagBundlesManager().sync_bundles_to_db() - self.app.dag_bag.bag_dag(dag) - self.app.dag_bag.sync_to_db("dags-folder", None) - return dag_instance - - def _create_test_dag_run(self, state=DagRunState.RUNNING, extra_dag=False, commit=True, idx_start=1): - dag_runs = [] - dags = [] - triggered_by_kwargs = {"triggered_by": DagRunTriggeredByType.TEST} if AIRFLOW_V_3_0_PLUS else {} - - for i in range(idx_start, idx_start + 2): - if i == 1: - dags.append(DagModel(dag_id="TEST_DAG_ID", is_active=True)) - dagrun_model = DagRun( - dag_id="TEST_DAG_ID", - run_id=f"TEST_DAG_RUN_ID_{i}", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time) + timedelta(days=i - 1), - run_after=timezone.parse(self.default_time) + timedelta(days=i - 1), - start_date=timezone.parse(self.default_time), - state=state, - **triggered_by_kwargs, - ) - dagrun_model.updated_at = timezone.parse(self.default_time) - dag_runs.append(dagrun_model) - - if extra_dag: - for i in range(idx_start + 2, idx_start + 4): - dags.append(DagModel(dag_id=f"TEST_DAG_ID_{i}")) - dag_runs.append( - DagRun( - dag_id=f"TEST_DAG_ID_{i}", - run_id=f"TEST_DAG_RUN_ID_{i}", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time_2), - run_after=timezone.parse(self.default_time_2), - start_date=timezone.parse(self.default_time), - state=state, - ) - ) - if commit: - with create_session() as session: - session.add_all(dag_runs) - session.add_all(dags) - return dag_runs - - -class TestDeleteDagRun(TestDagRunEndpoint): - def test_should_respond_204(self, session): - session.add_all(self._create_test_dag_run()) - session.commit() - response = self.client.delete( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 204 - # Check if the Dag Run is deleted from the database - response = self.client.get( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - - def test_should_respond_404(self): - response = self.client.delete( - "api/v1/dags/INVALID_DAG_RUN/dagRuns/INVALID_DAG_RUN", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - assert response.json == { - "detail": "DAGRun with DAG ID: 'INVALID_DAG_RUN' and DagRun ID: 'INVALID_DAG_RUN' not found", - "status": 404, - "title": "Not Found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self, session): - session.add_all(self._create_test_dag_run()) - session.commit() - - response = self.client.delete( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1", - ) - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - -class TestGetDagRun(TestDagRunEndpoint): - def test_should_respond_200(self, session): - triggered_by_kwargs = {"triggered_by": DagRunTriggeredByType.TEST} if AIRFLOW_V_3_0_PLUS else {} - dagrun_model = DagRun( - dag_id="TEST_DAG_ID", - run_id="TEST_DAG_RUN_ID", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time), - run_after=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - state="running", - **triggered_by_kwargs, - ) - expected_response_json = { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID", - "end_date": None, - "state": "running", - "logical_date": self.default_time, - "run_after": self.default_time, - "start_date": self.default_time, - "conf": {}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_response_json.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - session.add(dagrun_model) - session.commit() - result = session.query(DagRun).all() - assert len(result) == 1 - response = self.client.get( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json == expected_response_json - - def test_should_respond_404(self): - response = self.client.get( - "api/v1/dags/invalid-id/dagRuns/invalid-id", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - expected_resp = { - "detail": "DAGRun with DAG ID: 'invalid-id' and DagRun ID: 'invalid-id' not found", - "status": 404, - "title": "DAGRun not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - assert expected_resp == response.json - - def test_should_raises_401_unauthenticated(self, session): - dagrun_model = DagRun( - dag_id="TEST_DAG_ID", - run_id="TEST_DAG_RUN_ID", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time), - run_after=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - ) - session.add(dagrun_model) - session.commit() - - response = self.client.get("api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID") - - assert_401(response) - - @pytest.mark.parametrize( - "fields", - [ - ["dag_run_id", "logical_date"], - ["dag_run_id", "state", "conf", "logical_date"], - ], - ) - def test_should_return_specified_fields(self, session, fields): - dagrun_model = DagRun( - dag_id="TEST_DAG_ID", - run_id="TEST_DAG_RUN_ID", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time), - run_after=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - state="running", - ) - session.add(dagrun_model) - session.commit() - result = session.query(DagRun).all() - assert len(result) == 1 - response = self.client.get( - f"api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID?fields={','.join(fields)}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - res_json = response.json - assert len(res_json.keys()) == len(fields) - for field in fields: - assert field in res_json - - def test_should_respond_400_with_not_exists_fields(self, session): - dagrun_model = DagRun( - dag_id="TEST_DAG_ID", - run_id="TEST_DAG_RUN_ID", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time), - run_after=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - state="running", - ) - session.add(dagrun_model) - session.commit() - result = session.query(DagRun).all() - assert len(result) == 1 - fields = ["#caw&c"] - response = self.client.get( - f"api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID?fields={','.join(fields)}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400, f"Current code: {response.status_code}" - - -class TestGetDagRuns(TestDagRunEndpoint): - def test_should_respond_200(self, session): - self._create_test_dag_run() - expected_response_json_1 = { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID_1", - "end_date": None, - "state": "running", - "logical_date": self.default_time, - "run_after": self.default_time, - "start_date": self.default_time, - "conf": {}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_response_json_1.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - expected_response_json_2 = { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID_2", - "end_date": None, - "state": "running", - "logical_date": self.default_time_2, - "run_after": self.default_time_2, - "start_date": self.default_time, - "conf": {}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_response_json_2.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - result = session.query(DagRun).all() - assert len(result) == 2 - response = self.client.get( - "api/v1/dags/TEST_DAG_ID/dagRuns", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json == { - "dag_runs": [ - expected_response_json_1, - expected_response_json_2, - ], - "total_entries": 2, - } - - def test_filter_by_state(self, session): - self._create_test_dag_run() - self._create_test_dag_run(state="queued", idx_start=3) - assert session.query(DagRun).count() == 4 - response = self.client.get( - "api/v1/dags/TEST_DAG_ID/dagRuns?state=running,queued", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 4 - assert response.json["dag_runs"][0]["state"] == response.json["dag_runs"][1]["state"] == "running" - assert response.json["dag_runs"][2]["state"] == response.json["dag_runs"][3]["state"] == "queued" - - def test_invalid_order_by_raises_400(self): - self._create_test_dag_run() - - response = self.client.get( - "api/v1/dags/TEST_DAG_ID/dagRuns?order_by=invalid", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - msg = "Ordering with 'invalid' is disallowed or the attribute does not exist on the model" - assert response.json["detail"] == msg - - def test_return_correct_results_with_order_by(self, session): - self._create_test_dag_run() - expected_response_json_2 = { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID_2", - "end_date": None, - "state": "running", - "logical_date": self.default_time_2, - "run_after": self.default_time_2, - "start_date": self.default_time, - "conf": {}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_response_json_2.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - expected_response_json_1 = { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID_1", - "end_date": None, - "state": "running", - "logical_date": self.default_time, - "run_after": self.default_time, - "start_date": self.default_time, - "conf": {}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_response_json_1.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - - result = session.query(DagRun).all() - assert len(result) == 2 - response = self.client.get( - "api/v1/dags/TEST_DAG_ID/dagRuns?order_by=-logical_date", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - assert self.default_time < self.default_time_2 - # - means descending - assert response.json == { - "dag_runs": [ - expected_response_json_2, - expected_response_json_1, - ], - "total_entries": 2, - } - - def test_should_return_all_with_tilde_as_dag_id_and_all_dag_permissions(self): - self._create_test_dag_run(extra_dag=True) - expected_dag_run_ids = ["TEST_DAG_ID", "TEST_DAG_ID", "TEST_DAG_ID_3", "TEST_DAG_ID_4"] - response = self.client.get("api/v1/dags/~/dagRuns", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - dag_run_ids = [dag_run["dag_id"] for dag_run in response.json["dag_runs"]] - assert dag_run_ids == expected_dag_run_ids - - def test_should_raises_401_unauthenticated(self): - self._create_test_dag_run() - - response = self.client.get("api/v1/dags/TEST_DAG_ID/dagRuns") - - assert_401(response) - - @pytest.mark.parametrize( - "fields", - [ - ["dag_run_id", "logical_date"], - ["dag_run_id", "state", "conf", "logical_date"], - ], - ) - def test_should_return_specified_fields(self, session, fields): - self._create_test_dag_run() - result = session.query(DagRun).all() - assert len(result) == 2 - response = self.client.get( - f"api/v1/dags/TEST_DAG_ID/dagRuns?fields={','.join(fields)}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - for dag_run in response.json["dag_runs"]: - assert len(dag_run.keys()) == len(fields) - for field in fields: - assert field in dag_run - - def test_should_respond_400_with_not_exists_fields(self): - self._create_test_dag_run() - fields = ["#caw&c"] - response = self.client.get( - f"api/v1/dags/TEST_DAG_ID/dagRuns?fields={','.join(fields)}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400, f"Current code: {response.status_code}" - - -class TestGetDagRunsPagination(TestDagRunEndpoint): - @pytest.mark.parametrize( - "url, expected_dag_run_ids", - [ - ("api/v1/dags/TEST_DAG_ID/dagRuns?limit=1", ["TEST_DAG_RUN_ID1"]), - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?limit=2", - ["TEST_DAG_RUN_ID1", "TEST_DAG_RUN_ID2"], - ), - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?offset=5", - [ - "TEST_DAG_RUN_ID6", - "TEST_DAG_RUN_ID7", - "TEST_DAG_RUN_ID8", - "TEST_DAG_RUN_ID9", - "TEST_DAG_RUN_ID10", - ], - ), - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?offset=0", - [ - "TEST_DAG_RUN_ID1", - "TEST_DAG_RUN_ID2", - "TEST_DAG_RUN_ID3", - "TEST_DAG_RUN_ID4", - "TEST_DAG_RUN_ID5", - "TEST_DAG_RUN_ID6", - "TEST_DAG_RUN_ID7", - "TEST_DAG_RUN_ID8", - "TEST_DAG_RUN_ID9", - "TEST_DAG_RUN_ID10", - ], - ), - ("api/v1/dags/TEST_DAG_ID/dagRuns?limit=1&offset=5", ["TEST_DAG_RUN_ID6"]), - ("api/v1/dags/TEST_DAG_ID/dagRuns?limit=1&offset=1", ["TEST_DAG_RUN_ID2"]), - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?limit=2&offset=2", - ["TEST_DAG_RUN_ID3", "TEST_DAG_RUN_ID4"], - ), - ], - ) - def test_handle_limit_and_offset(self, url, expected_dag_run_ids): - self._create_dag_runs(10) - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - - assert response.json["total_entries"] == 10 - dag_run_ids = [dag_run["dag_run_id"] for dag_run in response.json["dag_runs"]] - assert dag_run_ids == expected_dag_run_ids - - def test_should_respect_page_size_limit(self): - self._create_dag_runs(200) - response = self.client.get( - "api/v1/dags/TEST_DAG_ID/dagRuns", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - - assert response.json["total_entries"] == 200 - assert len(response.json["dag_runs"]) == 100 # default is 100 - - @conf_vars({("api", "maximum_page_limit"): "150"}) - def test_should_return_conf_max_if_req_max_above_conf(self): - self._create_dag_runs(200) - response = self.client.get( - "api/v1/dags/TEST_DAG_ID/dagRuns?limit=180", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert len(response.json["dag_runs"]) == 150 - - def _create_dag_runs(self, count): - dag_runs = [ - DagRun( - dag_id="TEST_DAG_ID", - run_id=f"TEST_DAG_RUN_ID{i}", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time) + timedelta(minutes=i), - run_after=timezone.parse(self.default_time) + timedelta(minutes=i), - start_date=timezone.parse(self.default_time), - ) - for i in range(1, count + 1) - ] - dag = DagModel(dag_id="TEST_DAG_ID") - with create_session() as session: - session.add_all(dag_runs) - session.add(dag) - - -class TestGetDagRunsPaginationFilters(TestDagRunEndpoint): - @pytest.mark.parametrize( - "url, expected_dag_run_ids", - [ - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?start_date_gte=2020-06-18T18%3A00%3A00%2B00%3A00", - ["TEST_START_EXEC_DAY_18", "TEST_START_EXEC_DAY_19"], - ), - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?start_date_lte=2020-06-11T18%3A00%3A00%2B00%3A00", - ["TEST_START_EXEC_DAY_10", "TEST_START_EXEC_DAY_11"], - ), - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?start_date_lte=2020-06-15T18%3A00%3A00%2B00%3A00" - "&start_date_gte=2020-06-12T18:00:00Z", - [ - "TEST_START_EXEC_DAY_12", - "TEST_START_EXEC_DAY_13", - "TEST_START_EXEC_DAY_14", - "TEST_START_EXEC_DAY_15", - ], - ), - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?execution_date_lte=2020-06-13T18%3A00%3A00%2B00%3A00", - [ - "TEST_START_EXEC_DAY_10", - "TEST_START_EXEC_DAY_11", - "TEST_START_EXEC_DAY_12", - "TEST_START_EXEC_DAY_13", - ], - ), - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?execution_date_gte=2020-06-16T18%3A00%3A00%2B00%3A00", - [ - "TEST_START_EXEC_DAY_16", - "TEST_START_EXEC_DAY_17", - "TEST_START_EXEC_DAY_18", - "TEST_START_EXEC_DAY_19", - ], - ), - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?updated_at_lte=2020-06-13T18%3A00%3A00%2B00%3A00", - [ - "TEST_START_EXEC_DAY_10", - "TEST_START_EXEC_DAY_11", - "TEST_START_EXEC_DAY_12", - "TEST_START_EXEC_DAY_13", - ], - ), - ( - "api/v1/dags/TEST_DAG_ID/dagRuns?updated_at_gte=2020-06-16T18%3A00%3A00%2B00%3A00", - [ - "TEST_START_EXEC_DAY_16", - "TEST_START_EXEC_DAY_17", - "TEST_START_EXEC_DAY_18", - "TEST_START_EXEC_DAY_19", - ], - ), - ], - ) - @provide_session - def test_date_filters_gte_and_lte(self, url, expected_dag_run_ids, session): - dagrun_models = self._create_dag_runs() - session.add_all(dagrun_models) - for d in dagrun_models: - d.updated_at = d.logical_date - session.commit() - - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json["total_entries"] == len(expected_dag_run_ids) - dag_run_ids = [dag_run["dag_run_id"] for dag_run in response.json["dag_runs"]] - assert dag_run_ids == expected_dag_run_ids - - def _create_dag_runs(self): - dates = [ - "2020-06-10T18:00:00+00:00", - "2020-06-11T18:00:00+00:00", - "2020-06-12T18:00:00+00:00", - "2020-06-13T18:00:00+00:00", - "2020-06-14T18:00:00+00:00", - "2020-06-15T18:00:00Z", - "2020-06-16T18:00:00Z", - "2020-06-17T18:00:00Z", - "2020-06-18T18:00:00Z", - "2020-06-19T18:00:00Z", - ] - - return [ - DagRun( - dag_id="TEST_DAG_ID", - run_id=f"TEST_START_EXEC_DAY_1{i}", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(dates[i]), - run_after=timezone.parse(dates[i]), - start_date=timezone.parse(dates[i]), - state=DagRunState.SUCCESS, - ) - for i in range(len(dates)) - ] - - -class TestGetDagRunsEndDateFilters(TestDagRunEndpoint): - @pytest.mark.parametrize( - "url, expected_dag_run_ids", - [ - pytest.param( - f"api/v1/dags/TEST_DAG_ID/dagRuns?end_date_gte=" - f"{urllib.parse.quote((timezone.utcnow() + timedelta(days=1)).isoformat())}", - [], - id="end_date_gte", - ), - pytest.param( - f"api/v1/dags/TEST_DAG_ID/dagRuns?end_date_lte=" - f"{urllib.parse.quote((timezone.utcnow() + timedelta(days=1)).isoformat())}", - ["TEST_DAG_RUN_ID_1", "TEST_DAG_RUN_ID_2"], - id="end_date_lte", - ), - ], - ) - def test_end_date_gte_lte(self, url, expected_dag_run_ids): - self._create_test_dag_run("success") # state==success, then end date is today - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json["total_entries"] == len(expected_dag_run_ids) - dag_run_ids = [dag_run["dag_run_id"] for dag_run in response.json["dag_runs"] if dag_run] - assert dag_run_ids == expected_dag_run_ids - - -class TestGetDagRunBatch(TestDagRunEndpoint): - def test_should_respond_200(self): - self._create_test_dag_run() - expected_response_json_1 = { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID_1", - "end_date": None, - "state": "running", - "logical_date": self.default_time, - "run_after": self.default_time, - "start_date": self.default_time, - "conf": {}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_response_json_1.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - - expected_response_json_2 = { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID_2", - "end_date": None, - "state": "running", - "logical_date": self.default_time_2, - "run_after": self.default_time_2, - "start_date": self.default_time, - "conf": {}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_response_json_2.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - - response = self.client.post( - "api/v1/dags/~/dagRuns/list", - json={"dag_ids": ["TEST_DAG_ID"]}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "dag_runs": [ - expected_response_json_1, - expected_response_json_2, - ], - "total_entries": 2, - } - - def test_raises_validation_error_for_invalid_request(self): - self._create_test_dag_run() - response = self.client.post( - "api/v1/dags/~/dagRuns/list", - json={"dagids": ["TEST_DAG_ID"]}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json == { - "detail": "{'dagids': ['Unknown field.']}", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_filter_by_state(self): - self._create_test_dag_run() - self._create_test_dag_run(state="queued", idx_start=3) - response = self.client.post( - "api/v1/dags/~/dagRuns/list", - json={"dag_ids": ["TEST_DAG_ID"], "states": ["running", "queued"]}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 4 - assert response.json["dag_runs"][0]["state"] == response.json["dag_runs"][1]["state"] == "running" - assert response.json["dag_runs"][2]["state"] == response.json["dag_runs"][3]["state"] == "queued" - - def test_order_by_descending_works(self): - self._create_test_dag_run() - expected_response_json_1 = { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID_2", - "end_date": None, - "state": "running", - "logical_date": self.default_time_2, - "run_after": self.default_time_2, - "start_date": self.default_time, - "conf": {}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_response_json_1.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - expected_response_json_2 = { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID_1", - "end_date": None, - "state": "running", - "logical_date": self.default_time, - "run_after": self.default_time, - "start_date": self.default_time, - "conf": {}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_response_json_2.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - - response = self.client.post( - "api/v1/dags/~/dagRuns/list", - json={"dag_ids": ["TEST_DAG_ID"], "order_by": "-dag_run_id"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "dag_runs": [ - expected_response_json_1, - expected_response_json_2, - ], - "total_entries": 2, - } - - def test_order_by_raises_for_invalid_attr(self): - self._create_test_dag_run() - response = self.client.post( - "api/v1/dags/~/dagRuns/list", - json={"dag_ids": ["TEST_DAG_ID"], "order_by": "-dag_ru"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - msg = "Ordering with 'dag_ru' is disallowed or the attribute does not exist on the model" - assert response.json["detail"] == msg - - @pytest.mark.parametrize( - "payload, error", - [ - ( - {"dag_ids": ["TEST_DAG_ID"], "page_offset": -1}, - "-1 is less than the minimum of 0 - 'page_offset'", - ), - ({"dag_ids": ["TEST_DAG_ID"], "page_limit": 0}, "0 is less than the minimum of 1 - 'page_limit'"), - ({"dag_ids": "TEST_DAG_ID"}, "'TEST_DAG_ID' is not of type 'array' - 'dag_ids'"), - ({"start_date_gte": "2020-06-12T18"}, "'2020-06-12T18' is not a 'date-time' - 'start_date_gte'"), - ], - ) - def test_payload_validation(self, payload, error): - self._create_test_dag_run() - response = self.client.post( - "api/v1/dags/~/dagRuns/list", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert response.json.get("detail") == error - - def test_should_raises_401_unauthenticated(self): - self._create_test_dag_run() - - response = self.client.post("api/v1/dags/~/dagRuns/list", json={"dag_ids": ["TEST_DAG_ID"]}) - - assert_401(response) - - -class TestGetDagRunBatchPagination(TestDagRunEndpoint): - @pytest.mark.parametrize( - "payload, expected_dag_run_ids", - [ - ({"page_limit": 1}, ["TEST_DAG_RUN_ID1"]), - ({"page_limit": 2}, ["TEST_DAG_RUN_ID1", "TEST_DAG_RUN_ID2"]), - ( - {"page_offset": 5}, - [ - "TEST_DAG_RUN_ID6", - "TEST_DAG_RUN_ID7", - "TEST_DAG_RUN_ID8", - "TEST_DAG_RUN_ID9", - "TEST_DAG_RUN_ID10", - ], - ), - ( - {"page_offset": 0}, - [ - "TEST_DAG_RUN_ID1", - "TEST_DAG_RUN_ID2", - "TEST_DAG_RUN_ID3", - "TEST_DAG_RUN_ID4", - "TEST_DAG_RUN_ID5", - "TEST_DAG_RUN_ID6", - "TEST_DAG_RUN_ID7", - "TEST_DAG_RUN_ID8", - "TEST_DAG_RUN_ID9", - "TEST_DAG_RUN_ID10", - ], - ), - ({"page_offset": 5, "page_limit": 1}, ["TEST_DAG_RUN_ID6"]), - ({"page_offset": 1, "page_limit": 1}, ["TEST_DAG_RUN_ID2"]), - ( - {"page_offset": 2, "page_limit": 2}, - ["TEST_DAG_RUN_ID3", "TEST_DAG_RUN_ID4"], - ), - ], - ) - def test_handle_limit_and_offset(self, payload, expected_dag_run_ids): - self._create_dag_runs(10) - response = self.client.post( - "api/v1/dags/~/dagRuns/list", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - - assert response.json["total_entries"] == 10 - dag_run_ids = [dag_run["dag_run_id"] for dag_run in response.json["dag_runs"]] - assert dag_run_ids == expected_dag_run_ids - - def test_should_respect_page_size_limit(self): - self._create_dag_runs(200) - response = self.client.post( - "api/v1/dags/~/dagRuns/list", json={}, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - - assert response.json["total_entries"] == 200 - assert len(response.json["dag_runs"]) == 100 # default is 100 - - def _create_dag_runs(self, count): - dag_runs = [ - DagRun( - dag_id="TEST_DAG_ID", - run_id=f"TEST_DAG_RUN_ID{i}", - state="running", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time) + timedelta(minutes=i), - run_after=timezone.parse(self.default_time) + timedelta(minutes=i), - start_date=timezone.parse(self.default_time), - ) - for i in range(1, count + 1) - ] - dag = DagModel(dag_id="TEST_DAG_ID") - with create_session() as session: - session.add_all(dag_runs) - session.add(dag) - - -class TestGetDagRunBatchDateFilters(TestDagRunEndpoint): - @pytest.mark.parametrize( - "payload, expected_dag_run_ids", - [ - ( - {"start_date_gte": "2020-06-18T18:00:00+00:00"}, - ["TEST_START_EXEC_DAY_18", "TEST_START_EXEC_DAY_19"], - ), - ( - {"start_date_lte": "2020-06-11T18:00:00+00:00"}, - ["TEST_START_EXEC_DAY_10", "TEST_START_EXEC_DAY_11"], - ), - ( - {"start_date_lte": "2020-06-15T18:00:00+00:00", "start_date_gte": "2020-06-12T18:00:00Z"}, - [ - "TEST_START_EXEC_DAY_12", - "TEST_START_EXEC_DAY_13", - "TEST_START_EXEC_DAY_14", - "TEST_START_EXEC_DAY_15", - ], - ), - ( - {"execution_date_lte": "2020-06-13T18:00:00+00:00"}, - [ - "TEST_START_EXEC_DAY_10", - "TEST_START_EXEC_DAY_11", - "TEST_START_EXEC_DAY_12", - "TEST_START_EXEC_DAY_13", - ], - ), - ( - {"execution_date_gte": "2020-06-16T18:00:00+00:00"}, - [ - "TEST_START_EXEC_DAY_16", - "TEST_START_EXEC_DAY_17", - "TEST_START_EXEC_DAY_18", - "TEST_START_EXEC_DAY_19", - ], - ), - ], - ) - def test_date_filters_gte_and_lte(self, payload, expected_dag_run_ids): - self._create_dag_runs() - response = self.client.post( - "api/v1/dags/~/dagRuns/list", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json["total_entries"] == len(expected_dag_run_ids) - dag_run_ids = [dag_run["dag_run_id"] for dag_run in response.json["dag_runs"]] - assert dag_run_ids == expected_dag_run_ids - - def _create_dag_runs(self): - dates = [ - "2020-06-10T18:00:00+00:00", - "2020-06-11T18:00:00+00:00", - "2020-06-12T18:00:00+00:00", - "2020-06-13T18:00:00+00:00", - "2020-06-14T18:00:00+00:00", - "2020-06-15T18:00:00Z", - "2020-06-16T18:00:00Z", - "2020-06-17T18:00:00Z", - "2020-06-18T18:00:00Z", - "2020-06-19T18:00:00Z", - ] - - dag = DagModel(dag_id="TEST_DAG_ID") - dag_runs = [ - DagRun( - dag_id="TEST_DAG_ID", - run_id=f"TEST_START_EXEC_DAY_1{i}", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(date), - run_after=timezone.parse(date), - start_date=timezone.parse(date), - state="success", - ) - for i, date in enumerate(dates) - ] - with create_session() as session: - session.add_all(dag_runs) - session.add(dag) - return dag_runs - - @pytest.mark.parametrize( - "payload, expected_response", - [ - ( - {"execution_date_gte": "2020-11-09T16:25:56.939143"}, - "'2020-11-09T16:25:56.939143' is not a 'date-time' - 'execution_date_gte'", - ), - ( - {"start_date_gte": "2020-06-18T16:25:56.939143"}, - "'2020-06-18T16:25:56.939143' is not a 'date-time' - 'start_date_gte'", - ), - ( - {"start_date_lte": "2020-06-18T18:00:00.564434"}, - "'2020-06-18T18:00:00.564434' is not a 'date-time' - 'start_date_lte'", - ), - ( - {"start_date_lte": "2020-06-15T18:00:00.653434", "start_date_gte": "2020-06-12T18:00.343534"}, - "'2020-06-12T18:00.343534' is not a 'date-time' - 'start_date_gte'", - ), - ( - {"execution_date_lte": "2020-06-13T18:00:00.353454"}, - "'2020-06-13T18:00:00.353454' is not a 'date-time' - 'execution_date_lte'", - ), - ( - {"execution_date_gte": "2020-06-16T18:00:00.676443"}, - "'2020-06-16T18:00:00.676443' is not a 'date-time' - 'execution_date_gte'", - ), - ], - ) - def test_naive_date_filters_raises_400(self, payload, expected_response): - self._create_dag_runs() - - response = self.client.post( - "api/v1/dags/~/dagRuns/list", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert response.json["detail"] == expected_response - - @pytest.mark.parametrize( - "payload, expected_dag_run_ids", - [ - ( - {"end_date_gte": f"{(timezone.utcnow() + timedelta(days=1)).isoformat()}"}, - [], - ), - ( - {"end_date_lte": f"{(timezone.utcnow() + timedelta(days=1)).isoformat()}"}, - ["TEST_DAG_RUN_ID_1", "TEST_DAG_RUN_ID_2"], - ), - ], - ) - def test_end_date_gte_lte(self, payload, expected_dag_run_ids): - self._create_test_dag_run("success") # state==success, then end date is today - response = self.client.post( - "api/v1/dags/~/dagRuns/list", json=payload, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json["total_entries"] == len(expected_dag_run_ids) - dag_run_ids = [dag_run["dag_run_id"] for dag_run in response.json["dag_runs"] if dag_run] - assert dag_run_ids == expected_dag_run_ids - - -class TestPostDagRun(TestDagRunEndpoint): - @time_machine.travel(timezone.utcnow(), tick=False) - @pytest.mark.parametrize( - "dag_run_id, logical_date, run_after, note, data_interval_start, data_interval_end", - [ - pytest.param( - "TEST_DAG_RUN", - "2020-06-11T18:00:00+00:00", - "2020-06-11T18:00:00+00:00", - "test-note", - None, - None, - id="all-present", - ), - pytest.param( - "TEST_DAG_RUN", - "2024-06-11T18:00:00+00:00", - "2024-06-11T18:00:00+00:00", - "test-note", - "2024-01-03T00:00:00+00:00", - "2024-01-04T05:00:00+00:00", - id="all-present-with-dates", - ), - pytest.param( - None, - "2020-06-11T18:00:00+00:00", - "2020-06-11T18:00:00+00:00", - None, - None, - None, - id="only-date", - ), - pytest.param(None, None, "2020-06-11T18:00:00+00:00", None, None, None, id="all-missing"), - ], - ) - def test_should_respond_200( - self, - session, - dag_run_id, - logical_date, - run_after, - note, - data_interval_start, - data_interval_end, - ): - self._create_dag("TEST_DAG_ID") - # raise NotImplementedError("TODO: Add tests for data_interval_start and data_interval_end") - - request_json = {} - if logical_date is not None: - request_json["logical_date"] = logical_date - request_json["run_after"] = run_after - if dag_run_id is not None: - request_json["dag_run_id"] = dag_run_id - if data_interval_start is not None: - request_json["data_interval_start"] = data_interval_start - if data_interval_end is not None: - request_json["data_interval_end"] = data_interval_end - - request_json["note"] = note - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json=request_json, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - - expected_logical_date = logical_date if logical_date is not None else None - - # when logical_date is null, run_id is run_after + random string. - if dag_run_id is None: - expected_dag_run_id = f"manual__{run_after}" - else: - expected_dag_run_id = dag_run_id - - expected_data_interval_start = expected_logical_date - expected_data_interval_end = expected_logical_date - if data_interval_start is not None and data_interval_end is not None: - expected_data_interval_start = data_interval_start - expected_data_interval_end = data_interval_end - - expected_response_json = { - "conf": {}, - "dag_id": "TEST_DAG_ID", - "dag_run_id": expected_dag_run_id, - "end_date": None, - "logical_date": expected_logical_date, - "run_after": run_after, - "start_date": None, - "state": "queued", - "data_interval_end": expected_data_interval_end, - "data_interval_start": expected_data_interval_start, - "last_scheduling_decision": None, - "run_type": "manual", - "note": note, - } - expected_response_json.update({"triggered_by": "rest_api"} if AIRFLOW_V_3_0_PLUS else {}) - response_json = response.json - for key in expected_response_json: - if key != "dag_run_id": - assert response_json[key] == expected_response_json[key], f"Mismatch on key {key}" - - assert response_json["dag_run_id"].startswith(expected_dag_run_id), ( - f"dag_run_id '{response_json['dag_run_id']}' does not start with expected prefix " - f"'{expected_dag_run_id}'" - ) - _check_last_log(session, dag_id="TEST_DAG_ID", event="api.post_dag_run", logical_date=None) - - def test_raises_validation_error_for_invalid_request(self): - self._create_dag("TEST_DAG_ID") - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={"executiondate": "2020-11-10T08:25:56Z"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json == { - "detail": "{'executiondate': ['Unknown field.']}", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_raises_validation_error_for_invalid_params(self): - self._create_dag("TEST_DAG_ID") - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={"conf": {"validated_number": 5000}}, # DAG param must be between 1 and 10 - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert "Invalid input for param" in response.json["detail"] - - @mock.patch("airflow.api_connexion.endpoints.dag_run_endpoint.get_airflow_app") - @mock.patch("airflow.api_connexion.endpoints.dag_run_endpoint.DagVersion") - def test_dagrun_creation_exception_is_handled(self, mock_get_dag_version, mock_get_app, session): - self._create_dag("TEST_DAG_ID") - error_message = "Encountered Error" - mock_get_app.return_value.dag_bag.get_dag.return_value.create_dagrun.side_effect = ValueError( - error_message - ) - mock_get_dag_version.get_latest_version.return_value = mock.MagicMock() - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={"logical_date": "2020-11-10T08:25:56Z"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json == { - "detail": error_message, - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_should_respond_404_if_a_dag_is_inactive(self, session): - dm = self._create_dag("TEST_INACTIVE_DAG_ID") - dm.is_active = False - session.add(dm) - session.flush() - response = self.client.post( - "api/v1/dags/TEST_INACTIVE_DAG_ID/dagRuns", - json={}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - - def test_should_respond_400_if_a_dag_has_import_errors(self, session): - """Test that if a dagmodel has import errors, dags won't be triggered""" - dm = self._create_dag("TEST_DAG_ID") - dm.has_import_errors = True - session.add(dm) - session.flush() - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.json == { - "detail": "DAG with dag_id: 'TEST_DAG_ID' has import errors", - "status": 400, - "title": "DAG cannot be triggered", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_should_response_200_for_matching_logical_date(self): - logical_date = "2020-11-10T08:25:56.939143+00:00" - self._create_dag("TEST_DAG_ID") - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={ - "logical_date": logical_date, - "run_after": logical_date, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - dag_run_id = f"manual__{logical_date}" - - expected_response_json = { - "conf": {}, - "dag_id": "TEST_DAG_ID", - "dag_run_id": dag_run_id, - "end_date": None, - "logical_date": logical_date, - "run_after": logical_date, - "start_date": None, - "state": "queued", - "data_interval_end": logical_date, - "data_interval_start": logical_date, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_response_json.update({"triggered_by": "rest_api"} if AIRFLOW_V_3_0_PLUS else {}) - - assert response.status_code == 200 - assert response.json == expected_response_json - - @pytest.mark.parametrize( - "data_interval_start, data_interval_end, expected", - [ - ( - "2020-11-10T08:25:56.939143", - None, - "'2020-11-10T08:25:56.939143' is not a 'date-time' - 'data_interval_start'", - ), - ( - None, - "2020-11-10T08:25:56.939143", - "'2020-11-10T08:25:56.939143' is not a 'date-time' - 'data_interval_end'", - ), - ( - "2020-11-10T08:25:56.939143+00:00", - None, - "{'_schema': [\"Both 'data_interval_start' and 'data_interval_end' must be specified together\"]}", - ), - ( - None, - "2020-11-10T08:25:56.939143+00:00", - "{'_schema': [\"Both 'data_interval_start' and 'data_interval_end' must be specified together\"]}", - ), - ], - ) - def test_should_response_400_for_missing_start_date_or_end_date( - self, data_interval_start, data_interval_end, expected - ): - self._create_dag("TEST_DAG_ID") - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={ - "logical_date": "2020-11-10T08:25:56.939143+00:00", - "data_interval_start": data_interval_start, - "data_interval_end": data_interval_end, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json["detail"] == expected - - @pytest.mark.parametrize( - "data, expected", - [ - ( - {"logical_date": "2020-11-10T08:25:56.939143"}, - "'2020-11-10T08:25:56.939143' is not a 'date-time' - 'logical_date'", - ), - ( - {"logical_date": "2020-11-10T08:25:56P"}, - "'2020-11-10T08:25:56P' is not a 'date-time' - 'logical_date'", - ), - ], - ) - def test_should_response_400_for_naive_datetime_and_bad_datetime(self, data, expected): - self._create_dag("TEST_DAG_ID") - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", json=data, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert response.json["detail"] == expected - - @pytest.mark.parametrize( - "data, expected", - [ - ( - { - "dag_run_id": "TEST_DAG_RUN", - "logical_date": "2020-06-11T18:00:00+00:00", - "conf": "some string", - }, - "'some string' is not of type 'object' - 'conf'", - ) - ], - ) - def test_should_response_400_for_non_dict_dagrun_conf(self, data, expected): - self._create_dag("TEST_DAG_ID") - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", json=data, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert response.json["detail"] == expected - - def test_response_404(self): - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={"dag_run_id": "TEST_DAG_RUN", "logical_date": self.default_time}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json == { - "detail": "DAG with dag_id: 'TEST_DAG_ID' not found", - "status": 404, - "title": "DAG not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - @pytest.mark.parametrize( - "url, request_json, expected_response", - [ - pytest.param( - "api/v1/dags/TEST_DAG_ID/dagRuns", - { - "start_date": "2020-06-11T18:00:00+00:00", - "logical_date": "2020-06-12T18:00:00+00:00", - }, - { - "detail": "Property is read-only - 'start_date'", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - }, - id="start_date in request json", - ), - pytest.param( - "api/v1/dags/TEST_DAG_ID/dagRuns", - {"state": "failed", "logical_date": "2020-06-12T18:00:00+00:00"}, - { - "detail": "Property is read-only - 'state'", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - }, - id="state in request json", - ), - ], - ) - def test_response_400(self, url, request_json, expected_response): - self._create_dag("TEST_DAG_ID") - response = self.client.post(url, json=request_json, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 400, response.data - assert expected_response == response.json - - def test_response_409(self): - self._create_test_dag_run() - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={ - "dag_run_id": "TEST_DAG_RUN_ID_1", - "logical_date": self.default_time_3, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 409, response.data - assert response.json == { - "detail": "DAGRun with DAG ID: 'TEST_DAG_ID' and " - "DAGRun ID: 'TEST_DAG_RUN_ID_1' already exists", - "status": 409, - "title": "Conflict", - "type": EXCEPTIONS_LINK_MAP[409], - } - - def test_response_409_when_logical_date_is_same(self): - self._create_test_dag_run() - - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={ - "dag_run_id": "TEST_DAG_RUN_ID_6", - "logical_date": self.default_time, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 409, response.data - assert response.json == { - "detail": "DAGRun with DAG ID: 'TEST_DAG_ID' and " - "DAGRun logical date: '2020-06-11 18:00:00+00:00' already exists", - "status": 409, - "title": "Conflict", - "type": EXCEPTIONS_LINK_MAP[409], - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={ - "dag_run_id": "TEST_DAG_RUN_ID_1", - "logical_date": self.default_time, - }, - ) - - assert_401(response) - - def test_should_raises_403_unauthorized(self): - self._create_dag("TEST_DAG_ID") - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns", - json={ - "dag_run_id": "TEST_DAG_RUN_ID_1", - "logical_date": self.default_time, - }, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - -class TestPatchDagRunState(TestDagRunEndpoint): - @pytest.mark.parametrize("state", ["failed", "success", "queued"]) - @pytest.mark.parametrize("run_type", [DagRunType.MANUAL, DagRunType.SCHEDULED]) - def test_should_respond_200(self, state, run_type, dag_maker, session): - dag_id = "TEST_DAG_ID" - dag_run_id = "TEST_DAG_RUN_ID" - with dag_maker(dag_id) as dag: - task = EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag) - dr = dag_maker.create_dagrun(run_id=dag_run_id, run_type=run_type) - ti = dr.get_task_instance(task_id="task_id") - ti.task = task - ti.state = State.RUNNING - session.merge(ti) - session.commit() - - request_json = {"state": state} - - response = self.client.patch( - f"api/v1/dags/{dag_id}/dagRuns/{dag_run_id}", - json=request_json, - environ_overrides={"REMOTE_USER": "test"}, - ) - - if state != "queued": - ti.refresh_from_db() - assert ti.state == state - - dr = session.query(DagRun).filter(DagRun.run_id == dr.run_id).first() - expected_response_json = { - "conf": {}, - "dag_id": dag_id, - "dag_run_id": dag_run_id, - "end_date": dr.end_date.isoformat() if state != State.QUEUED else None, - "logical_date": dr.logical_date.isoformat(), - "run_after": dr.run_after.isoformat(), - "start_date": dr.start_date.isoformat() if state != State.QUEUED else None, - "state": state, - "data_interval_start": dr.data_interval_start.isoformat(), - "data_interval_end": dr.data_interval_end.isoformat(), - "last_scheduling_decision": None, - "run_type": run_type, - "note": None, - } - expected_response_json.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - - assert response.status_code == 200 - assert response.json == expected_response_json - - @pytest.mark.parametrize("state", ["failed", "success", "queued"]) - @pytest.mark.parametrize("run_type", [state.value for state in DagRunType]) - def test_action_logging(self, state, run_type, dag_maker, session): - dag_id = "TEST_DAG_ID" - dag_run_id = "TEST_DAG_RUN_ID" - with dag_maker(dag_id) as dag: - task = EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag) - dr = dag_maker.create_dagrun(run_id=dag_run_id, run_type=run_type) - ti = dr.get_task_instance(task_id="task_id") - ti.task = task - ti.state = State.RUNNING - session.merge(ti) - session.commit() - - request_json = {"state": state} - - self.client.patch( - f"api/v1/dags/{dag_id}/dagRuns/{dag_run_id}", - json=request_json, - environ_overrides={"REMOTE_USER": "test"}, - ) - - log = ( - session.query(Log) - .filter( - Log.dag_id == dag_id, - Log.run_id == dag_run_id, - Log.event == "api.update_dag_run_state", - ) - .order_by(Log.id.desc()) - .first() - ) - assert log.extra == json.dumps(request_json) - - self.client.patch( - f"api/v1/dags/{dag_id}/dagRuns/{dag_run_id}", - json=request_json, - environ_overrides={"REMOTE_USER": "test"}, - headers={"content-type": "application/json; charset=utf-8"}, - ) - - log = ( - session.query(Log) - .filter( - Log.dag_id == dag_id, - Log.run_id == dag_run_id, - Log.event == "api.update_dag_run_state", - ) - .order_by(Log.id.desc()) - .first() - ) - assert log.extra == json.dumps(request_json) - - def test_schema_validation_error_raises(self, dag_maker, session): - dag_id = "TEST_DAG_ID" - dag_run_id = "TEST_DAG_RUN_ID" - with dag_maker(dag_id) as dag: - EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag) - dag_maker.create_dagrun(run_id=dag_run_id) - - response = self.client.patch( - f"api/v1/dags/{dag_id}/dagRuns/{dag_run_id}", - json={"states": "success"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json == { - "detail": "{'states': ['Unknown field.']}", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - @pytest.mark.parametrize("invalid_state", ["running"]) - @time_machine.travel(TestDagRunEndpoint.default_time) - def test_should_response_400_for_non_existing_dag_run_state(self, invalid_state, dag_maker): - dag_id = "TEST_DAG_ID" - dag_run_id = "TEST_DAG_RUN_ID" - with dag_maker(dag_id): - EmptyOperator(task_id="task_id") - dag_maker.create_dagrun(run_id=dag_run_id) - - request_json = {"state": invalid_state} - - response = self.client.patch( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1", - json=request_json, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json == { - "detail": f"'{invalid_state}' is not one of ['success', 'failed', 'queued'] - 'state'", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_should_raises_401_unauthenticated(self, session): - response = self.client.patch( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1", - json={ - "state": "success", - }, - ) - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.patch( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1", - json={ - "state": "success", - }, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - def test_should_respond_404(self): - response = self.client.patch( - "api/v1/dags/INVALID_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1", - json={ - "state": "success", - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - - -class TestClearDagRun(TestDagRunEndpoint): - def test_should_respond_200(self, dag_maker, session): - dag_id = "TEST_DAG_ID" - dag_run_id = "TEST_DAG_RUN_ID" - with dag_maker(dag_id) as dag: - task = EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag) - dr = dag_maker.create_dagrun(run_id=dag_run_id, state=DagRunState.FAILED) - ti = dr.get_task_instance(task_id="task_id") - ti.task = task - ti.state = State.SUCCESS - session.merge(ti) - session.commit() - - request_json = {"dry_run": False} - - response = self.client.post( - f"api/v1/dags/{dag_id}/dagRuns/{dag_run_id}/clear", - json=request_json, - environ_overrides={"REMOTE_USER": "test"}, - ) - - dr = session.query(DagRun).filter(DagRun.run_id == dr.run_id).first() - expected_response_json = { - "conf": {}, - "dag_id": dag_id, - "dag_run_id": dag_run_id, - "end_date": None, - "logical_date": dr.logical_date.isoformat(), - "run_after": dr.run_after.isoformat(), - "start_date": None, - "state": "queued", - "data_interval_start": dr.data_interval_start.isoformat(), - "data_interval_end": dr.data_interval_end.isoformat(), - "last_scheduling_decision": None, - "run_type": dr.run_type, - "note": None, - } - expected_response_json.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - - assert response.status_code == 200 - assert response.json == expected_response_json - - ti.refresh_from_db() - assert ti.state is None - - def test_schema_validation_error_raises_for_invalid_fields(self, dag_maker, session): - dag_id = "TEST_DAG_ID" - dag_run_id = "TEST_DAG_RUN_ID" - with dag_maker(dag_id) as dag: - EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag) - dag_maker.create_dagrun(run_id=dag_run_id, state=DagRunState.FAILED) - response = self.client.post( - f"api/v1/dags/{dag_id}/dagRuns/{dag_run_id}/clear", - json={"dryrun": False}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json == { - "detail": "{'dryrun': ['Unknown field.']}", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_dry_run(self, dag_maker, session): - """Test that dry_run being True returns TaskInstances without clearing DagRun""" - dag_id = "TEST_DAG_ID" - dag_run_id = "TEST_DAG_RUN_ID" - with dag_maker(dag_id) as dag: - task = EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag) - dr = dag_maker.create_dagrun(run_id=dag_run_id) - ti = dr.get_task_instance(task_id="task_id") - ti.task = task - ti.state = State.SUCCESS - session.merge(ti) - session.commit() - - request_json = {"dry_run": True} - - response = self.client.post( - f"api/v1/dags/{dag_id}/dagRuns/{dag_run_id}/clear", - json=request_json, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - assert response.json == { - "task_instances": [ - { - "dag_id": dag_id, - "dag_run_id": dag_run_id, - "logical_date": dr.logical_date.isoformat(), - "task_id": "task_id", - } - ] - } - - ti.refresh_from_db() - assert ti.state == State.SUCCESS - - dr = session.query(DagRun).filter(DagRun.run_id == dr.run_id).first() - assert dr.state == "running" - - def test_should_raises_401_unauthenticated(self, session): - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1/clear", - json={ - "dry_run": True, - }, - ) - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.post( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1/clear", - json={ - "dry_run": True, - }, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - def test_should_respond_404(self): - response = self.client.post( - "api/v1/dags/INVALID_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1/clear", - json={ - "dry_run": True, - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - - -@pytest.mark.need_serialized_dag -class TestGetDagRunAssetTriggerEvents(TestDagRunEndpoint): - def test_should_respond_200(self, dag_maker, session): - asset1 = Asset(uri="test://asset1", name="asset1") - - with dag_maker(dag_id="source_dag", start_date=timezone.utcnow(), session=session): - EmptyOperator(task_id="task", outlets=[asset1]) - dr = dag_maker.create_dagrun() - ti = dr.task_instances[0] - - asset1_id = session.query(AssetModel.id).filter_by(uri=asset1.uri).scalar() - event = AssetEvent( - asset_id=asset1_id, - source_task_id=ti.task_id, - source_dag_id=ti.dag_id, - source_run_id=ti.run_id, - source_map_index=ti.map_index, - ) - session.add(event) - - with dag_maker(dag_id="TEST_DAG_ID", start_date=timezone.utcnow(), session=session): - pass - dr = dag_maker.create_dagrun(run_id="TEST_DAG_RUN_ID", run_type=DagRunType.ASSET_TRIGGERED) - dr.consumed_asset_events.append(event) - - session.commit() - assert event.timestamp - - response = self.client.get( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID/upstreamAssetEvents", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - expected_response = { - "asset_events": [ - { - "timestamp": event.timestamp.isoformat(), - "asset_id": asset1_id, - "asset_uri": asset1.uri, - "extra": {}, - "id": event.id, - "source_dag_id": ti.dag_id, - "source_map_index": ti.map_index, - "source_run_id": ti.run_id, - "source_task_id": ti.task_id, - "created_dagruns": [ - { - "dag_id": "TEST_DAG_ID", - "dag_run_id": "TEST_DAG_RUN_ID", - "data_interval_end": dr.data_interval_end.isoformat(), - "data_interval_start": dr.data_interval_start.isoformat(), - "end_date": None, - "logical_date": dr.logical_date.isoformat(), - "start_date": dr.start_date.isoformat(), - "state": "running", - } - ], - } - ], - "total_entries": 1, - } - assert response.json == expected_response - - def test_should_respond_404(self): - response = self.client.get( - "api/v1/dags/invalid-id/dagRuns/invalid-id/upstreamAssetEvents", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - expected_resp = { - "detail": "DAGRun with DAG ID: 'invalid-id' and DagRun ID: 'invalid-id' not found", - "status": 404, - "title": "DAGRun not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - assert expected_resp == response.json - - def test_should_raises_401_unauthenticated(self, session): - dagrun_model = DagRun( - dag_id="TEST_DAG_ID", - run_id="TEST_DAG_RUN_ID", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - ) - session.add(dagrun_model) - session.commit() - - response = self.client.get("api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID/upstreamAssetEvents") - - assert_401(response) - - -class TestSetDagRunNote(TestDagRunEndpoint): - def test_should_respond_200(self, dag_maker, session): - dag_runs: list[DagRun] = self._create_test_dag_run(DagRunState.SUCCESS) - session.add_all(dag_runs) - session.commit() - created_dr: DagRun = dag_runs[0] - new_note_value = "My super cool DagRun notes" - response = self.client.patch( - f"api/v1/dags/{created_dr.dag_id}/dagRuns/{created_dr.run_id}/setNote", - json={"note": new_note_value}, - environ_overrides={"REMOTE_USER": "test"}, - ) - - dr = session.query(DagRun).filter(DagRun.run_id == created_dr.run_id).first() - expected_response_json = { - "conf": {}, - "dag_id": dr.dag_id, - "dag_run_id": dr.run_id, - "end_date": dr.end_date.isoformat(), - "logical_date": self.default_time, - "run_after": self.default_time, - "start_date": self.default_time, - "state": "success", - "data_interval_start": None, - "data_interval_end": None, - "last_scheduling_decision": None, - "run_type": dr.run_type, - "note": new_note_value, - } - expected_response_json.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - - assert response.status_code == 200, response.text - assert dr.note == new_note_value - assert response.json == expected_response_json - assert dr.dag_run_note.user_id is not None - # Update the note again - new_note_value = "My super cool DagRun notes 2" - payload = {"note": new_note_value} - response = self.client.patch( - f"api/v1/dags/{created_dr.dag_id}/dagRuns/{created_dr.run_id}/setNote", - json=payload, - environ_overrides={"REMOTE_USER": "test"}, - ) - expected_response_json_new = { - "conf": {}, - "dag_id": dr.dag_id, - "dag_run_id": dr.run_id, - "end_date": dr.end_date.isoformat(), - "logical_date": self.default_time, - "run_after": self.default_time, - "start_date": self.default_time, - "state": "success", - "data_interval_start": None, - "data_interval_end": None, - "last_scheduling_decision": None, - "run_type": dr.run_type, - "note": new_note_value, - } - expected_response_json_new.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - - assert response.status_code == 200 - assert response.json == expected_response_json_new - assert dr.dag_run_note.user_id is not None - _check_last_log( - session, - dag_id=dr.dag_id, - event="api.set_dag_run_note", - logical_date=None, - expected_extra=payload, - ) - - def test_schema_validation_error_raises(self, dag_maker, session): - dag_runs: list[DagRun] = self._create_test_dag_run(DagRunState.SUCCESS) - session.add_all(dag_runs) - session.commit() - created_dr: DagRun = dag_runs[0] - - new_note_value = "My super cool DagRun notes" - response = self.client.patch( - f"api/v1/dags/{created_dr.dag_id}/dagRuns/{created_dr.run_id}/setNote", - json={"notes": new_note_value}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json == { - "detail": "{'notes': ['Unknown field.']}", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_should_raises_401_unauthenticated(self, session): - response = self.client.patch( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1/setNote", - json={"note": "I am setting a note while being unauthenticated."}, - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.patch( - "api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1/setNote", - json={"note": "I am setting a note without the proper permissions."}, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - def test_should_respond_404(self): - response = self.client.patch( - "api/v1/dags/INVALID_DAG_ID/dagRuns/TEST_DAG_RUN_ID_1/setNote", - json={"note": "I am setting a note on a DAG that doesn't exist."}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 diff --git a/tests/api_connexion/endpoints/test_dag_source_endpoint.py b/tests/api_connexion/endpoints/test_dag_source_endpoint.py deleted file mode 100644 index 7af0f3b67d757..0000000000000 --- a/tests/api_connexion/endpoints/test_dag_source_endpoint.py +++ /dev/null @@ -1,177 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os - -import pendulum -import pytest -from sqlalchemy import select - -from airflow.models import DagBag -from airflow.models.dagcode import DagCode -from airflow.models.serialized_dag import SerializedDagModel -from airflow.utils.state import DagRunState -from airflow.utils.types import DagRunTriggeredByType, DagRunType - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.db import clear_db_dags, clear_db_runs, parse_and_sync_to_db - -pytestmark = pytest.mark.db_test - - -TEST_DAG_ID = "example_bash_operator" - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -@pytest.fixture -def test_dag(): - parse_and_sync_to_db(os.devnull, include_examples=True) - dagbag = DagBag(read_dags_from_db=True) - return dagbag.get_dag(TEST_DAG_ID) - - -class TestGetSource: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - clear_db_dags() - clear_db_runs() - - def teardown_method(self): - clear_db_dags() - - @staticmethod - def _get_dag_file_code(fileloc: str) -> str | None: - with open(fileloc) as f: - file_contents = f.read() - return file_contents - - def test_should_respond_200_text(self, test_dag): - dag_content = self._get_dag_file_code(test_dag.fileloc) - - url = f"/api/v1/dagSources/{TEST_DAG_ID}" - response = self.client.get( - url, headers={"Accept": "text/plain"}, environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 200 - assert dag_content == response.data.decode() - assert response.headers["Content-Type"] == "text/plain" - - def test_should_respond_200_json(self, session, test_dag): - dag_content = self._get_dag_file_code(test_dag.fileloc) - - url = f"/api/v1/dagSources/{TEST_DAG_ID}" - response = self.client.get( - url, headers={"Accept": "application/json"}, environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 200 - assert response.json == { - "content": dag_content, - "dag_id": TEST_DAG_ID, - "version_number": 1, - } - assert response.headers["Content-Type"] == "application/json" - - @pytest.mark.parametrize("accept", ["application/json", "text/plain"]) - def test_should_respond_200_version(self, accept, session, test_dag, testing_dag_bundle): - dag_content = self._get_dag_file_code(test_dag.fileloc) - test_dag.create_dagrun( - run_id="test1", - run_after=pendulum.datetime(2025, 1, 1, tz="UTC"), - state=DagRunState.QUEUED, - triggered_by=DagRunTriggeredByType.TEST, - run_type=DagRunType.MANUAL, - ) - # force reserialization - test_dag.doc_md = "new doc" - SerializedDagModel.write_dag(test_dag, bundle_name="testing") - dagcode = ( - session.query(DagCode) - .filter(DagCode.fileloc == test_dag.fileloc) - .order_by(DagCode.id.desc()) - .first() - ) - assert dagcode.dag_version.version_number == 2 - # populate the latest dagcode with a value - dag_content2 = "new source code" - dagcode.source_code = dag_content2 - session.merge(dagcode) - session.commit() - - dagcodes = session.scalars(select(DagCode).where(DagCode.fileloc == test_dag.fileloc)).all() - assert len(dagcodes) == 2 - url = f"/api/v1/dagSources/{TEST_DAG_ID}" - response = self.client.get(url, headers={"Accept": accept}, environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - if accept == "text/plain": - assert dag_content2 == response.data.decode() - assert dag_content != response.data.decode() - assert response.headers["Content-Type"] == "text/plain" - else: - assert dag_content2 == response.json["content"] - assert dag_content != response.json["content"] - assert response.headers["Content-Type"] == "application/json" - assert response.json == { - "content": dag_content2, - "dag_id": TEST_DAG_ID, - "version_number": 2, - } - - def test_should_respond_404(self): - non_existing_dag_id = "abcd1234" - url = f"/api/v1/dagSources/{non_existing_dag_id}" - response = self.client.get( - url, headers={"Accept": "application/json"}, environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 404 - - def test_should_raises_401_unauthenticated(self): - response = self.client.get( - f"/api/v1/dagSources/{TEST_DAG_ID}", - headers={"Accept": "text/plain"}, - ) - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - f"/api/v1/dagSources/{TEST_DAG_ID}", - headers={"Accept": "text/plain"}, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 diff --git a/tests/api_connexion/endpoints/test_dag_stats_endpoint.py b/tests/api_connexion/endpoints/test_dag_stats_endpoint.py deleted file mode 100644 index ebb4a9ede8a8b..0000000000000 --- a/tests/api_connexion/endpoints/test_dag_stats_endpoint.py +++ /dev/null @@ -1,366 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from datetime import timedelta - -import pytest - -from airflow.models.dag import DAG, DagModel -from airflow.models.dagrun import DagRun -from airflow.utils import timezone -from airflow.utils.session import create_session -from airflow.utils.state import DagRunState -from airflow.utils.types import DagRunType - -from tests_common.test_utils.api_connexion_utils import create_user, delete_user -from tests_common.test_utils.db import clear_db_dags, clear_db_runs, clear_db_serialized_dags - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - - create_user( - app, - username="test", - role_name="admin", - ) - - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestDagStatsEndpoint: - default_time = "2020-06-11T18:00:00+00:00" - - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - clear_db_runs() - clear_db_serialized_dags() - clear_db_dags() - - def teardown_method(self) -> None: - clear_db_runs() - clear_db_dags() - clear_db_serialized_dags() - - def _create_dag(self, dag_id): - dag_instance = DagModel(dag_id=dag_id) - dag_instance.is_active = True - with create_session() as session: - session.add(dag_instance) - dag = DAG(dag_id=dag_id, schedule=None) - self.app.dag_bag.bag_dag(dag) - return dag_instance - - def _create_dag_runs(self, session): - self._create_dag("dag_stats_dag") - self._create_dag("dag_stats_dag_2") - self._create_dag("dag_stats_dag_3") - dag_1_run_1 = DagRun( - dag_id="dag_stats_dag", - run_id="test_dag_run_id_1", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - state="running", - ) - dag_1_run_2 = DagRun( - dag_id="dag_stats_dag", - run_id="test_dag_run_id_2", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time) + timedelta(days=1), - start_date=timezone.parse(self.default_time), - state="failed", - ) - dag_2_run_1 = DagRun( - dag_id="dag_stats_dag_2", - run_id="test_dag_2_run_id_1", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - state="queued", - ) - dag_3_run_1 = DagRun( - dag_id="dag_stats_dag_3", - run_id="test_dag_3_run_id_1", - run_type=DagRunType.MANUAL, - logical_date=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - state="success", - ) - session.add_all((dag_1_run_1, dag_1_run_2, dag_2_run_1, dag_3_run_1)) - session.commit() - - def test_should_respond_200(self, session): - self._create_dag_runs(session) - exp_payload = { - "dags": [ - { - "dag_id": "dag_stats_dag", - "stats": [ - { - "state": DagRunState.QUEUED, - "count": 0, - }, - { - "state": DagRunState.RUNNING, - "count": 1, - }, - { - "state": DagRunState.SUCCESS, - "count": 0, - }, - { - "state": DagRunState.FAILED, - "count": 1, - }, - ], - }, - { - "dag_id": "dag_stats_dag_2", - "stats": [ - { - "state": DagRunState.QUEUED, - "count": 1, - }, - { - "state": DagRunState.RUNNING, - "count": 0, - }, - { - "state": DagRunState.SUCCESS, - "count": 0, - }, - { - "state": DagRunState.FAILED, - "count": 0, - }, - ], - }, - ], - "total_entries": 2, - } - - dag_ids = "dag_stats_dag,dag_stats_dag_2" - response = self.client.get( - f"api/v1/dagStats?dag_ids={dag_ids}", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert len(response.json["dags"]) == 2 - assert sorted(response.json["dags"], key=lambda d: d["dag_id"]) == sorted( - exp_payload["dags"], key=lambda d: d["dag_id"] - ) - assert response.json["total_entries"] == 2 - - @pytest.mark.parametrize( - "url, exp_payload", - [ - ( - "api/v1/dagStats", - { - "dags": [ - { - "dag_id": "dag_stats_dag", - "stats": [ - { - "state": DagRunState.QUEUED, - "count": 0, - }, - { - "state": DagRunState.RUNNING, - "count": 1, - }, - { - "state": DagRunState.SUCCESS, - "count": 0, - }, - { - "state": DagRunState.FAILED, - "count": 1, - }, - ], - }, - { - "dag_id": "dag_stats_dag_2", - "stats": [ - { - "state": DagRunState.QUEUED, - "count": 1, - }, - { - "state": DagRunState.RUNNING, - "count": 0, - }, - { - "state": DagRunState.SUCCESS, - "count": 0, - }, - { - "state": DagRunState.FAILED, - "count": 0, - }, - ], - }, - { - "dag_id": "dag_stats_dag_3", - "stats": [ - { - "state": DagRunState.QUEUED, - "count": 0, - }, - { - "state": DagRunState.RUNNING, - "count": 0, - }, - { - "state": DagRunState.SUCCESS, - "count": 1, - }, - { - "state": DagRunState.FAILED, - "count": 0, - }, - ], - }, - ], - "total_entries": 3, - }, - ), - ( - "api/v1/dagStats?limit=1", - { - "dags": [ - { - "dag_id": "dag_stats_dag", - "stats": [ - { - "state": DagRunState.QUEUED, - "count": 0, - }, - { - "state": DagRunState.RUNNING, - "count": 1, - }, - { - "state": DagRunState.SUCCESS, - "count": 0, - }, - { - "state": DagRunState.FAILED, - "count": 1, - }, - ], - } - ], - "total_entries": 1, - }, - ), - ( - "api/v1/dagStats?offset=2", - { - "dags": [ - { - "dag_id": "dag_stats_dag_3", - "stats": [ - { - "state": DagRunState.QUEUED, - "count": 0, - }, - { - "state": DagRunState.RUNNING, - "count": 0, - }, - { - "state": DagRunState.SUCCESS, - "count": 1, - }, - { - "state": DagRunState.FAILED, - "count": 0, - }, - ], - }, - ], - "total_entries": 1, - }, - ), - ( - "api/v1/dagStats?offset=1&limit=1", - { - "dags": [ - { - "dag_id": "dag_stats_dag_2", - "stats": [ - { - "state": DagRunState.QUEUED, - "count": 1, - }, - { - "state": DagRunState.RUNNING, - "count": 0, - }, - { - "state": DagRunState.SUCCESS, - "count": 0, - }, - { - "state": DagRunState.FAILED, - "count": 0, - }, - ], - }, - ], - "total_entries": 1, - }, - ), - ("api/v1/dagStats?offset=10&limit=1", {"dags": [], "total_entries": 0}), - ], - ) - def test_optional_dag_ids_with_limit_offset(self, url, exp_payload, session): - self._create_dag_runs(session) - - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - num_dags = len(exp_payload["dags"]) - assert response.status_code == 200 - assert sorted(response.json["dags"], key=lambda d: d["dag_id"]) == sorted( - exp_payload["dags"], key=lambda d: d["dag_id"] - ) - assert response.json["total_entries"] == num_dags - - def test_should_raises_401_unauthenticated(self): - dag_ids = "dag_stats_dag,dag_stats_dag_2" - response = self.client.get( - f"api/v1/dagStats?dag_ids={dag_ids}", environ_overrides={"REMOTE_USER": "no_user"} - ) - assert response.status_code == 401 - - def test_should_raises_403_no_permission(self): - dag_ids = "dag_stats_dag,dag_stats_dag_2" - response = self.client.get( - f"api/v1/dagStats?dag_ids={dag_ids}", environ_overrides={"REMOTE_USER": "test_no_permissions"} - ) - assert response.status_code == 403 diff --git a/tests/api_connexion/endpoints/test_dag_warning_endpoint.py b/tests/api_connexion/endpoints/test_dag_warning_endpoint.py deleted file mode 100644 index 26b939836c4b4..0000000000000 --- a/tests/api_connexion/endpoints/test_dag_warning_endpoint.py +++ /dev/null @@ -1,150 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from unittest.mock import ANY - -import pytest - -from airflow.models.dag import DagModel -from airflow.models.dagwarning import DagWarning -from airflow.utils.session import create_session - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.db import clear_db_dag_warnings, clear_db_dags - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield minimal_app_for_api - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestBaseDagWarning: - timestamp = "2020-06-10T12:00" - - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - - def teardown_method(self) -> None: - clear_db_dag_warnings() - clear_db_dags() - - @staticmethod - def _normalize_dag_warnings(dag_warnings): - for i, dag_warning in enumerate(dag_warnings, 1): - dag_warning["dag_warning_id"] = i - - -class TestGetDagWarningEndpoint(TestBaseDagWarning): - def setup_class(self): - clear_db_dag_warnings() - clear_db_dags() - - def setup_method(self): - with create_session() as session: - session.add(DagModel(dag_id="dag1")) - session.add(DagModel(dag_id="dag2")) - session.add(DagModel(dag_id="dag3")) - session.add(DagWarning("dag1", "non-existent pool", "test message")) - session.add(DagWarning("dag2", "non-existent pool", "test message")) - session.commit() - - def test_response_one(self): - response = self.client.get( - "/api/v1/dagWarnings", - environ_overrides={"REMOTE_USER": "test"}, - query_string={"dag_id": "dag1", "warning_type": "non-existent pool"}, - ) - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "dag_warnings": [ - { - "dag_id": "dag1", - "message": "test message", - "timestamp": ANY, - "warning_type": "non-existent pool", - } - ], - "total_entries": 1, - } - - def test_response_some(self): - response = self.client.get( - "/api/v1/dagWarnings", - environ_overrides={"REMOTE_USER": "test"}, - query_string={"warning_type": "non-existent pool"}, - ) - assert response.status_code == 200 - response_data = response.json - assert len(response_data["dag_warnings"]) == 2 - assert response_data == { - "dag_warnings": ANY, - "total_entries": 2, - } - - def test_response_none(self, session): - response = self.client.get( - "/api/v1/dagWarnings", - environ_overrides={"REMOTE_USER": "test"}, - query_string={"dag_id": "missing_dag"}, - ) - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "dag_warnings": [], - "total_entries": 0, - } - - def test_response_all(self): - response = self.client.get( - "/api/v1/dagWarnings", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - response_data = response.json - assert len(response_data["dag_warnings"]) == 2 - assert response_data == { - "dag_warnings": ANY, - "total_entries": 2, - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.get("/api/v1/dagWarnings") - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "/api/v1/dagWarnings", environ_overrides={"REMOTE_USER": "test_no_permissions"} - ) - assert response.status_code == 403 diff --git a/tests/api_connexion/endpoints/test_event_log_endpoint.py b/tests/api_connexion/endpoints/test_event_log_endpoint.py deleted file mode 100644 index 8574ae9f3bf2e..0000000000000 --- a/tests/api_connexion/endpoints/test_event_log_endpoint.py +++ /dev/null @@ -1,385 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.models import Log -from airflow.utils import timezone - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_logs - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -@pytest.fixture -def task_instance(session, create_task_instance, request): - return create_task_instance( - session=session, - dag_id="TEST_DAG_ID", - task_id="TEST_TASK_ID", - run_id="TEST_RUN_ID", - logical_date=request.instance.default_time, - ) - - -@pytest.fixture -def log_model(create_log_model, request): - return create_log_model( - event="TEST_EVENT", - when=request.instance.default_time, - ) - - -@pytest.fixture -def create_log_model(create_task_instance, task_instance, session, request): - def maker(event, when, **kwargs): - log_model = Log( - event=event, - task_instance=task_instance, - **kwargs, - ) - log_model.dttm = when - - session.add(log_model) - session.flush() - return log_model - - return maker - - -class TestEventLogEndpoint: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - clear_db_logs() - self.default_time = timezone.parse("2020-06-10T20:00:00+00:00") - self.default_time_2 = timezone.parse("2020-06-11T07:00:00+00:00") - - def teardown_method(self) -> None: - clear_db_logs() - - -class TestGetEventLog(TestEventLogEndpoint): - def test_should_respond_200(self, log_model): - event_log_id = log_model.id - response = self.client.get( - f"/api/v1/eventLogs/{event_log_id}", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - data = response.json - data["try_number"] - assert response.json == { - "event_log_id": event_log_id, - "event": "TEST_EVENT", - "dag_id": "TEST_DAG_ID", - "task_id": "TEST_TASK_ID", - "run_id": "TEST_RUN_ID", - "map_index": -1, - "try_number": 0, - "logical_date": self.default_time.isoformat(), - "owner": "airflow", - "when": self.default_time.isoformat(), - "extra": None, - } - - def test_should_respond_404(self): - response = self.client.get("/api/v1/eventLogs/1", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 404 - assert response.json == { - "detail": None, - "status": 404, - "title": "Event Log not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self, log_model): - event_log_id = log_model.id - - response = self.client.get(f"/api/v1/eventLogs/{event_log_id}") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "/api/v1/eventLogs", environ_overrides={"REMOTE_USER": "test_no_permissions"} - ) - assert response.status_code == 403 - - -class TestGetEventLogs(TestEventLogEndpoint): - def test_should_respond_200(self, session, create_log_model): - log_model_1 = create_log_model(event="TEST_EVENT_1", when=self.default_time) - log_model_2 = create_log_model(event="TEST_EVENT_2", when=self.default_time_2) - log_model_3 = Log(event="cli_scheduler", owner="root", extra='{"host_name": "e24b454f002a"}') - log_model_3.dttm = self.default_time_2 - - session.add(log_model_3) - session.flush() - response = self.client.get("/api/v1/eventLogs", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == { - "event_logs": [ - { - "event_log_id": log_model_1.id, - "event": "TEST_EVENT_1", - "dag_id": "TEST_DAG_ID", - "task_id": "TEST_TASK_ID", - "run_id": "TEST_RUN_ID", - "map_index": -1, - "try_number": 0, - "logical_date": self.default_time.isoformat(), - "owner": "airflow", - "when": self.default_time.isoformat(), - "extra": None, - }, - { - "event_log_id": log_model_2.id, - "event": "TEST_EVENT_2", - "dag_id": "TEST_DAG_ID", - "task_id": "TEST_TASK_ID", - "run_id": "TEST_RUN_ID", - "map_index": -1, - "try_number": 0, - "logical_date": self.default_time.isoformat(), - "owner": "airflow", - "when": self.default_time_2.isoformat(), - "extra": None, - }, - { - "event_log_id": log_model_3.id, - "event": "cli_scheduler", - "dag_id": None, - "task_id": None, - "run_id": None, - "map_index": None, - "try_number": None, - "logical_date": None, - "owner": "root", - "when": self.default_time_2.isoformat(), - "extra": '{"host_name": "e24b454f002a"}', - }, - ], - "total_entries": 3, - } - - def test_order_eventlogs_by_owner(self, create_log_model, session): - log_model_1 = create_log_model(event="TEST_EVENT_1", when=self.default_time) - log_model_2 = create_log_model(event="TEST_EVENT_2", when=self.default_time_2, owner="zsh") - log_model_3 = Log(event="cli_scheduler", owner="root", extra='{"host_name": "e24b454f002a"}') - log_model_3.dttm = self.default_time_2 - session.add(log_model_3) - session.flush() - response = self.client.get( - "/api/v1/eventLogs?order_by=-owner", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json == { - "event_logs": [ - { - "event_log_id": log_model_2.id, - "event": "TEST_EVENT_2", - "dag_id": "TEST_DAG_ID", - "task_id": "TEST_TASK_ID", - "run_id": "TEST_RUN_ID", - "map_index": -1, - "try_number": 0, - "logical_date": self.default_time.isoformat(), - "owner": "zsh", # Order by name, sort order is descending(-) - "when": self.default_time_2.isoformat(), - "extra": None, - }, - { - "event_log_id": log_model_3.id, - "event": "cli_scheduler", - "dag_id": None, - "task_id": None, - "run_id": None, - "map_index": None, - "try_number": None, - "logical_date": None, - "owner": "root", - "when": self.default_time_2.isoformat(), - "extra": '{"host_name": "e24b454f002a"}', - }, - { - "event_log_id": log_model_1.id, - "event": "TEST_EVENT_1", - "dag_id": "TEST_DAG_ID", - "task_id": "TEST_TASK_ID", - "run_id": "TEST_RUN_ID", - "map_index": -1, - "try_number": 0, - "logical_date": self.default_time.isoformat(), - "owner": "airflow", - "when": self.default_time.isoformat(), - "extra": None, - }, - ], - "total_entries": 3, - } - - def test_should_raises_401_unauthenticated(self, log_model): - response = self.client.get("/api/v1/eventLogs") - - assert_401(response) - - def test_should_filter_eventlogs_by_when(self, create_log_model, session): - eventlog1 = create_log_model(event="TEST_EVENT_1", when=self.default_time) - eventlog2 = create_log_model(event="TEST_EVENT_2", when=self.default_time_2) - session.add_all([eventlog1, eventlog2]) - session.commit() - for when_attr, expected_eventlog_event in { - "before": "TEST_EVENT_1", - "after": "TEST_EVENT_2", - }.items(): - response = self.client.get( - f"/api/v1/eventLogs?{when_attr}=2020-06-10T20%3A00%3A01%2B00%3A00", # self.default_time + 1s - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 1 - assert len(response.json["event_logs"]) == 1 - assert response.json["event_logs"][0]["event"] == expected_eventlog_event - - def test_should_filter_eventlogs_by_run_id(self, create_log_model, session): - eventlog1 = create_log_model(event="TEST_EVENT_1", when=self.default_time, run_id="run_1") - eventlog2 = create_log_model(event="TEST_EVENT_2", when=self.default_time, run_id="run_2") - eventlog3 = create_log_model(event="TEST_EVENT_3", when=self.default_time, run_id="run_2") - session.add_all([eventlog1, eventlog2, eventlog3]) - session.commit() - for run_id, expected_eventlogs in { - "run_1": {"TEST_EVENT_1"}, - "run_2": {"TEST_EVENT_2", "TEST_EVENT_3"}, - }.items(): - response = self.client.get( - f"/api/v1/eventLogs?run_id={run_id}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == len(expected_eventlogs) - assert len(response.json["event_logs"]) == len(expected_eventlogs) - assert {eventlog["event"] for eventlog in response.json["event_logs"]} == expected_eventlogs - assert all({eventlog["run_id"] == run_id for eventlog in response.json["event_logs"]}) - - -class TestGetEventLogPagination(TestEventLogEndpoint): - @pytest.mark.parametrize( - ("url", "expected_events"), - [ - ("api/v1/eventLogs?limit=1", ["TEST_EVENT_1"]), - ("api/v1/eventLogs?limit=2", ["TEST_EVENT_1", "TEST_EVENT_2"]), - ( - "api/v1/eventLogs?offset=5", - [ - "TEST_EVENT_6", - "TEST_EVENT_7", - "TEST_EVENT_8", - "TEST_EVENT_9", - "TEST_EVENT_10", - ], - ), - ( - "api/v1/eventLogs?offset=0", - [ - "TEST_EVENT_1", - "TEST_EVENT_2", - "TEST_EVENT_3", - "TEST_EVENT_4", - "TEST_EVENT_5", - "TEST_EVENT_6", - "TEST_EVENT_7", - "TEST_EVENT_8", - "TEST_EVENT_9", - "TEST_EVENT_10", - ], - ), - ("api/v1/eventLogs?limit=1&offset=5", ["TEST_EVENT_6"]), - ("api/v1/eventLogs?limit=1&offset=1", ["TEST_EVENT_2"]), - ( - "api/v1/eventLogs?limit=2&offset=2", - ["TEST_EVENT_3", "TEST_EVENT_4"], - ), - ], - ) - def test_handle_limit_and_offset(self, url, expected_events, task_instance, session): - log_models = self._create_event_logs(task_instance, 10) - session.add_all(log_models) - session.commit() - - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - - assert response.json["total_entries"] == 10 - events = [event_log["event"] for event_log in response.json["event_logs"]] - assert events == expected_events - - def test_should_respect_page_size_limit_default(self, task_instance, session): - log_models = self._create_event_logs(task_instance, 200) - session.add_all(log_models) - session.flush() - - response = self.client.get("/api/v1/eventLogs", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - - assert response.json["total_entries"] == 200 - assert len(response.json["event_logs"]) == 100 # default 100 - - def test_should_raise_400_for_invalid_order_by_name(self, task_instance, session): - log_models = self._create_event_logs(task_instance, 200) - session.add_all(log_models) - session.flush() - - response = self.client.get( - "/api/v1/eventLogs?order_by=invalid", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - msg = "Ordering with 'invalid' is disallowed or the attribute does not exist on the model" - assert response.json["detail"] == msg - - @conf_vars({("api", "maximum_page_limit"): "150"}) - def test_should_return_conf_max_if_req_max_above_conf(self, task_instance, session): - log_models = self._create_event_logs(task_instance, 200) - session.add_all(log_models) - session.flush() - - response = self.client.get("/api/v1/eventLogs?limit=180", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert len(response.json["event_logs"]) == 150 - - def _create_event_logs(self, task_instance, count): - return [Log(event=f"TEST_EVENT_{i}", task_instance=task_instance) for i in range(1, count + 1)] diff --git a/tests/api_connexion/endpoints/test_extra_link_endpoint.py b/tests/api_connexion/endpoints/test_extra_link_endpoint.py deleted file mode 100644 index 461120a3bc565..0000000000000 --- a/tests/api_connexion/endpoints/test_extra_link_endpoint.py +++ /dev/null @@ -1,245 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os -from urllib.parse import quote_plus - -import pytest - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.dag_processing.bundles.manager import DagBundlesManager -from airflow.models.dag import DAG -from airflow.models.dagbag import DagBag -from airflow.models.xcom import XCom -from airflow.plugins_manager import AirflowPlugin -from airflow.timetables.base import DataInterval -from airflow.utils import timezone -from airflow.utils.state import DagRunState -from airflow.utils.types import DagRunTriggeredByType, DagRunType - -from tests_common.test_utils.api_connexion_utils import create_user, delete_user -from tests_common.test_utils.compat import BaseOperatorLink -from tests_common.test_utils.db import clear_db_runs, clear_db_xcom -from tests_common.test_utils.mock_operators import CustomOperator -from tests_common.test_utils.mock_plugins import mock_plugin_manager - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestGetExtraLinks: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app, session) -> None: - self.default_time = timezone.datetime(2020, 1, 1) - - clear_db_runs() - clear_db_xcom() - - self.app = configured_app - - self.dag = self._create_dag() - - DagBundlesManager().sync_bundles_to_db() - self.app.dag_bag = DagBag(os.devnull, include_examples=False) - self.app.dag_bag.dags = {self.dag.dag_id: self.dag} - self.app.dag_bag.sync_to_db("dags-folder", None) - - data_interval = DataInterval(timezone.datetime(2020, 1, 1), timezone.datetime(2020, 1, 2)) - self.dag.create_dagrun( - run_id="TEST_DAG_RUN_ID", - logical_date=self.default_time, - run_type=DagRunType.MANUAL, - state=DagRunState.SUCCESS, - session=session, - data_interval=data_interval, - run_after=data_interval.end, - triggered_by=DagRunTriggeredByType.TEST, - ) - session.flush() - - self.client = self.app.test_client() # type:ignore - - def teardown_method(self) -> None: - clear_db_runs() - clear_db_xcom() - - def _create_dag(self): - with DAG(dag_id="TEST_DAG_ID", schedule=None, default_args={"start_date": self.default_time}) as dag: - CustomOperator(task_id="TEST_SINGLE_LINK", bash_command="TEST_LINK_VALUE") - CustomOperator( - task_id="TEST_MULTIPLE_LINK", bash_command=["TEST_LINK_VALUE_1", "TEST_LINK_VALUE_2"] - ) - return dag - - @pytest.mark.parametrize( - "url, expected_title, expected_detail", - [ - pytest.param( - "/api/v1/dags/INVALID/dagRuns/TEST_DAG_RUN_ID/taskInstances/TEST_SINGLE_LINK/links", - "DAG not found", - 'DAG with ID = "INVALID" not found', - id="missing_dag", - ), - pytest.param( - "/api/v1/dags/TEST_DAG_ID/dagRuns/INVALID/taskInstances/TEST_SINGLE_LINK/links", - "DAG Run not found", - 'DAG Run with ID = "INVALID" not found', - id="missing_dag_run", - ), - pytest.param( - "/api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID/taskInstances/INVALID/links", - "Task not found", - 'Task with ID = "INVALID" not found', - id="missing_task", - ), - ], - ) - def test_should_respond_404(self, url, expected_title, expected_detail): - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 404 - assert response.json == { - "detail": expected_detail, - "status": 404, - "title": expected_title, - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "/api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID/taskInstances/TEST_SINGLE_LINK/links", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - @pytest.mark.skip(reason="Legacy API tests.") - @mock_plugin_manager(plugins=[]) - def test_should_respond_200(self): - XCom.set( - key="search_query", - value="TEST_LINK_VALUE", - task_id="TEST_SINGLE_LINK", - dag_id=self.dag.dag_id, - run_id="TEST_DAG_RUN_ID", - ) - response = self.client.get( - "/api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID/taskInstances/TEST_SINGLE_LINK/links", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200, response.data - assert response.json == {"Google Custom": "http://google.com/custom_base_link?search=TEST_LINK_VALUE"} - - @pytest.mark.skip(reason="Legacy API tests.") - @mock_plugin_manager(plugins=[]) - def test_should_respond_200_missing_xcom(self): - response = self.client.get( - "/api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID/taskInstances/TEST_SINGLE_LINK/links", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200, response.data - assert response.json == {"Google Custom": None} - - @pytest.mark.skip(reason="Legacy API tests.") - @mock_plugin_manager(plugins=[]) - def test_should_respond_200_multiple_links(self): - XCom.set( - key="search_query", - value=["TEST_LINK_VALUE_1", "TEST_LINK_VALUE_2"], - task_id="TEST_MULTIPLE_LINK", - dag_id=self.dag.dag_id, - run_id="TEST_DAG_RUN_ID", - ) - response = self.client.get( - "/api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID/taskInstances/TEST_MULTIPLE_LINK/links", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200, response.data - assert response.json == { - "BigQuery Console #1": "https://console.cloud.google.com/bigquery?j=TEST_LINK_VALUE_1", - "BigQuery Console #2": "https://console.cloud.google.com/bigquery?j=TEST_LINK_VALUE_2", - } - - @pytest.mark.skip(reason="Legacy API tests.") - @mock_plugin_manager(plugins=[]) - def test_should_respond_200_multiple_links_missing_xcom(self): - response = self.client.get( - "/api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID/taskInstances/TEST_MULTIPLE_LINK/links", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200, response.data - assert response.json == {"BigQuery Console #1": None, "BigQuery Console #2": None} - - @pytest.mark.skip(reason="Legacy API tests.") - def test_should_respond_200_support_plugins(self): - class GoogleLink(BaseOperatorLink): - name = "Google" - - def get_link(self, operator, *, ti_key): - return "https://www.google.com" - - class S3LogLink(BaseOperatorLink): - name = "S3" - operators = [CustomOperator] - - def get_link(self, operator, *, ti_key): - return ( - f"https://s3.amazonaws.com/airflow-logs/{operator.dag_id}/" - f"{operator.task_id}/{quote_plus(ti_key.run_id)}" - ) - - class AirflowTestPlugin(AirflowPlugin): - name = "test_plugin" - global_operator_extra_links = [ - GoogleLink(), - ] - operator_extra_links = [ - S3LogLink(), - ] - - with mock_plugin_manager(plugins=[AirflowTestPlugin]): - response = self.client.get( - "/api/v1/dags/TEST_DAG_ID/dagRuns/TEST_DAG_RUN_ID/taskInstances/TEST_SINGLE_LINK/links", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200, response.data - assert response.json == { - "Google Custom": None, - "Google": "https://www.google.com", - "S3": "https://s3.amazonaws.com/airflow-logs/TEST_DAG_ID/TEST_SINGLE_LINK/TEST_DAG_RUN_ID", - } diff --git a/tests/api_connexion/endpoints/test_health_endpoint.py b/tests/api_connexion/endpoints/test_health_endpoint.py deleted file mode 100644 index a04a3be751950..0000000000000 --- a/tests/api_connexion/endpoints/test_health_endpoint.py +++ /dev/null @@ -1,91 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from datetime import timedelta -from unittest import mock - -import pytest - -from airflow.jobs.job import Job -from airflow.jobs.scheduler_job_runner import SchedulerJobRunner -from airflow.utils import timezone -from airflow.utils.session import create_session, provide_session -from airflow.utils.state import State - -HEALTHY = "healthy" -UNHEALTHY = "unhealthy" - -pytestmark = pytest.mark.db_test - - -class TestHealthTestBase: - @pytest.fixture(autouse=True) - def setup_attrs(self, minimal_app_for_api) -> None: - self.app = minimal_app_for_api - self.client = self.app.test_client() # type:ignore - with create_session() as session: - session.query(Job).delete() - - def teardown_method(self): - with create_session() as session: - session.query(Job).delete() - - -class TestGetHealth(TestHealthTestBase): - @provide_session - def test_healthy_scheduler_status(self, session): - last_scheduler_heartbeat_for_testing_1 = timezone.utcnow() - job = Job(state=State.RUNNING, latest_heartbeat=last_scheduler_heartbeat_for_testing_1) - SchedulerJobRunner(job=job) - session.add(job) - session.commit() - resp_json = self.client.get("/api/v1/health").json - assert resp_json["metadatabase"]["status"] == "healthy" - assert resp_json["scheduler"]["status"] == "healthy" - assert ( - last_scheduler_heartbeat_for_testing_1.isoformat() - == resp_json["scheduler"]["latest_scheduler_heartbeat"] - ) - - @provide_session - def test_unhealthy_scheduler_is_slow(self, session): - last_scheduler_heartbeat_for_testing_2 = timezone.utcnow() - timedelta(minutes=1) - job = Job(state=State.RUNNING, latest_heartbeat=last_scheduler_heartbeat_for_testing_2) - SchedulerJobRunner(job=job) - session.add(job) - session.commit() - resp_json = self.client.get("/api/v1/health").json - assert resp_json["metadatabase"]["status"] == "healthy" - assert resp_json["scheduler"]["status"] == "unhealthy" - assert ( - last_scheduler_heartbeat_for_testing_2.isoformat() - == resp_json["scheduler"]["latest_scheduler_heartbeat"] - ) - - def test_unhealthy_scheduler_no_job(self): - resp_json = self.client.get("/api/v1/health").json - assert resp_json["metadatabase"]["status"] == "healthy" - assert resp_json["scheduler"]["status"] == "unhealthy" - assert resp_json["scheduler"]["latest_scheduler_heartbeat"] is None - - @mock.patch.object(SchedulerJobRunner, "most_recent_job") - def test_unhealthy_metadatabase_status(self, most_recent_job_mock): - most_recent_job_mock.side_effect = Exception - resp_json = self.client.get("/api/v1/health").json - assert resp_json["metadatabase"]["status"] == "unhealthy" - assert resp_json["scheduler"]["latest_scheduler_heartbeat"] is None diff --git a/tests/api_connexion/endpoints/test_import_error_endpoint.py b/tests/api_connexion/endpoints/test_import_error_endpoint.py deleted file mode 100644 index 1950c1cd0b206..0000000000000 --- a/tests/api_connexion/endpoints/test_import_error_endpoint.py +++ /dev/null @@ -1,317 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from datetime import timedelta - -import pytest - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.utils import timezone -from airflow.utils.session import provide_session - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.compat import ParseImportError -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_dags, clear_db_import_errors - -pytestmark = pytest.mark.db_test - -TEST_DAG_IDS = ["test_dag", "test_dag2"] -BUNDLE_NAME = "dag_maker" - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestBaseImportError: - timestamp = "2020-06-10T12:00" - - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - - clear_db_import_errors() - clear_db_dags() - - def teardown_method(self) -> None: - clear_db_import_errors() - clear_db_dags() - - @staticmethod - def _normalize_import_errors(import_errors): - for i, import_error in enumerate(import_errors, 1): - import_error["import_error_id"] = i - - -class TestGetImportErrorEndpoint(TestBaseImportError): - def test_response_200(self, session): - import_error = ParseImportError( - filename="Lorem_ipsum.py", - stacktrace="Lorem ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC"), - bundle_name=BUNDLE_NAME, - ) - session.add(import_error) - session.commit() - - response = self.client.get( - f"/api/v1/importErrors/{import_error.id}", environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 200 - response_data = response.json - response_data["import_error_id"] = 1 - assert response_data == { - "filename": "Lorem_ipsum.py", - "bundle_name": BUNDLE_NAME, - "import_error_id": 1, - "stack_trace": "Lorem ipsum", - "timestamp": "2020-06-10T12:00:00+00:00", - } - - def test_response_404(self): - response = self.client.get("/api/v1/importErrors/2", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 404 - assert response.json == { - "detail": "The ImportError with import_error_id: `2` was not found", - "status": 404, - "title": "Import error not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self, session): - import_error = ParseImportError( - filename="Lorem_ipsum.py", - stacktrace="Lorem ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC"), - bundle_name=BUNDLE_NAME, - ) - session.add(import_error) - session.commit() - - response = self.client.get(f"/api/v1/importErrors/{import_error.id}") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "/api/v1/importErrors", environ_overrides={"REMOTE_USER": "test_no_permissions"} - ) - assert response.status_code == 403 - - -class TestGetImportErrorsEndpoint(TestBaseImportError): - def test_get_import_errors(self, session): - import_error = [ - ParseImportError( - filename="Lorem_ipsum.py", - stacktrace="Lorem ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC"), - bundle_name=BUNDLE_NAME, - ) - for _ in range(2) - ] - session.add_all(import_error) - session.commit() - - response = self.client.get("/api/v1/importErrors", environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - response_data = response.json - self._normalize_import_errors(response_data["import_errors"]) - assert response_data == { - "import_errors": [ - { - "filename": "Lorem_ipsum.py", - "bundle_name": BUNDLE_NAME, - "import_error_id": 1, - "stack_trace": "Lorem ipsum", - "timestamp": "2020-06-10T12:00:00+00:00", - }, - { - "filename": "Lorem_ipsum.py", - "bundle_name": BUNDLE_NAME, - "import_error_id": 2, - "stack_trace": "Lorem ipsum", - "timestamp": "2020-06-10T12:00:00+00:00", - }, - ], - "total_entries": 2, - } - - def test_get_import_errors_order_by(self, session): - import_error = [ - ParseImportError( - filename=f"Lorem_ipsum{i}.py", - stacktrace="Lorem ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC") + timedelta(days=-i), - bundle_name=BUNDLE_NAME, - ) - for i in range(1, 3) - ] - session.add_all(import_error) - session.commit() - - response = self.client.get( - "/api/v1/importErrors?order_by=-timestamp", environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 200 - response_data = response.json - self._normalize_import_errors(response_data["import_errors"]) - assert response_data == { - "import_errors": [ - { - "filename": "Lorem_ipsum1.py", - "bundle_name": BUNDLE_NAME, - "import_error_id": 1, # id normalized with self._normalize_import_errors - "stack_trace": "Lorem ipsum", - "timestamp": "2020-06-09T12:00:00+00:00", - }, - { - "filename": "Lorem_ipsum2.py", - "bundle_name": BUNDLE_NAME, - "import_error_id": 2, - "stack_trace": "Lorem ipsum", - "timestamp": "2020-06-08T12:00:00+00:00", - }, - ], - "total_entries": 2, - } - - def test_order_by_raises_400_for_invalid_attr(self, session): - import_error = [ - ParseImportError( - filename="Lorem_ipsum.py", - stacktrace="Lorem ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC"), - bundle_name=BUNDLE_NAME, - ) - for _ in range(2) - ] - session.add_all(import_error) - session.commit() - - response = self.client.get( - "/api/v1/importErrors?order_by=timest", environ_overrides={"REMOTE_USER": "test"} - ) - - assert response.status_code == 400 - msg = "Ordering with 'timest' is disallowed or the attribute does not exist on the model" - assert response.json["detail"] == msg - - def test_should_raises_401_unauthenticated(self, session): - import_error = [ - ParseImportError( - filename="Lorem_ipsum.py", - stacktrace="Lorem ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC"), - bundle_name=BUNDLE_NAME, - ) - for _ in range(2) - ] - session.add_all(import_error) - session.commit() - - response = self.client.get("/api/v1/importErrors") - - assert_401(response) - - -class TestGetImportErrorsEndpointPagination(TestBaseImportError): - @pytest.mark.parametrize( - "url, expected_import_error_ids", - [ - # Limit test data - ("/api/v1/importErrors?limit=1", ["/tmp/file_1.py"]), - ("/api/v1/importErrors?limit=100", [f"/tmp/file_{i}.py" for i in range(1, 101)]), - # Offset test data - ("/api/v1/importErrors?offset=1", [f"/tmp/file_{i}.py" for i in range(2, 102)]), - ("/api/v1/importErrors?offset=3", [f"/tmp/file_{i}.py" for i in range(4, 104)]), - # Limit and offset test data - ("/api/v1/importErrors?offset=3&limit=3", [f"/tmp/file_{i}.py" for i in [4, 5, 6]]), - ], - ) - @provide_session - def test_limit_and_offset(self, url, expected_import_error_ids, session): - import_errors = [ - ParseImportError( - filename=f"/tmp/file_{i}.py", - stacktrace="Lorem ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC"), - bundle_name=BUNDLE_NAME, - ) - for i in range(1, 110) - ] - session.add_all(import_errors) - session.commit() - - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - - assert response.status_code == 200 - import_ids = [pool["filename"] for pool in response.json["import_errors"]] - assert import_ids == expected_import_error_ids - - def test_should_respect_page_size_limit_default(self, session): - import_errors = [ - ParseImportError( - filename=f"/tmp/file_{i}.py", - stacktrace="Lorem ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC"), - bundle_name=BUNDLE_NAME, - ) - for i in range(1, 110) - ] - session.add_all(import_errors) - session.commit() - response = self.client.get("/api/v1/importErrors", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert len(response.json["import_errors"]) == 100 - - @conf_vars({("api", "maximum_page_limit"): "150"}) - def test_should_return_conf_max_if_req_max_above_conf(self, session): - import_errors = [ - ParseImportError( - filename=f"/tmp/file_{i}.py", - bundle_name=BUNDLE_NAME, - stacktrace="Lorem ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC"), - ) - for i in range(200) - ] - session.add_all(import_errors) - session.commit() - response = self.client.get( - "/api/v1/importErrors?limit=180", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert len(response.json["import_errors"]) == 150 diff --git a/tests/api_connexion/endpoints/test_log_endpoint.py b/tests/api_connexion/endpoints/test_log_endpoint.py deleted file mode 100644 index e7c89151a7aa4..0000000000000 --- a/tests/api_connexion/endpoints/test_log_endpoint.py +++ /dev/null @@ -1,445 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import copy -import logging.config -import sys -from unittest import mock -from unittest.mock import PropertyMock - -import pytest -from itsdangerous.url_safe import URLSafeSerializer - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG -from airflow.decorators import task -from airflow.models.dag import DAG -from airflow.providers.standard.operators.empty import EmptyOperator -from airflow.utils import timezone -from airflow.utils.types import DagRunType - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.db import clear_db_runs - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestGetLog: - DAG_ID = "dag_for_testing_log_endpoint" - RUN_ID = "dag_run_id_for_testing_log_endpoint" - TASK_ID = "task_for_testing_log_endpoint" - MAPPED_TASK_ID = "mapped_task_for_testing_log_endpoint" - TRY_NUMBER = 1 - - default_time = "2020-06-10T20:00:00+00:00" - - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app, configure_loggers, dag_maker, session) -> None: - self.app = configured_app - self.client = self.app.test_client() - # Make sure that the configure_logging is not cached - self.old_modules = dict(sys.modules) - - with dag_maker(self.DAG_ID, start_date=timezone.parse(self.default_time), session=session) as dag: - EmptyOperator(task_id=self.TASK_ID) - - @task(task_id=self.MAPPED_TASK_ID) - def add_one(x: int): - return x + 1 - - add_one.expand(x=[1, 2, 3]) - - dr = dag_maker.create_dagrun( - run_id=self.RUN_ID, - run_type=DagRunType.SCHEDULED, - logical_date=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - ) - - configured_app.dag_bag.bag_dag(dag) - - # Add dummy dag for checking picking correct log with same task_id and different dag_id case. - with dag_maker( - f"{self.DAG_ID}_copy", start_date=timezone.parse(self.default_time), session=session - ) as dummy_dag: - EmptyOperator(task_id=self.TASK_ID) - dr2 = dag_maker.create_dagrun( - run_id=self.RUN_ID, - run_type=DagRunType.SCHEDULED, - logical_date=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - ) - configured_app.dag_bag.bag_dag(dummy_dag) - - for ti in dr.task_instances: - ti.try_number = 1 - ti.hostname = "localhost" - session.merge(ti) - for ti in dr2.task_instances: - ti.try_number = 1 - ti.hostname = "localhost" - session.merge(ti) - session.flush() - dag.clear() - dummy_dag.clear() - for ti in dr.task_instances: - ti.try_number = 2 - ti.hostname = "localhost" - session.merge(ti) - for ti in dr2.task_instances: - ti.try_number = 2 - ti.hostname = "localhost" - session.merge(ti) - session.flush() - - @pytest.fixture - def configure_loggers(self, tmp_path, create_log_template): - self.log_dir = tmp_path - - # TASK_ID - dir_path = tmp_path / f"dag_id={self.DAG_ID}" / f"run_id={self.RUN_ID}" / f"task_id={self.TASK_ID}" - dir_path.mkdir(parents=True) - - log = dir_path / "attempt=1.log" - log.write_text("Log for testing.") - - # try number 2 - log = dir_path / "attempt=2.log" - log.write_text("Log for testing 2.") - - # MAPPED_TASK_ID - for map_index in range(3): - dir_path = ( - tmp_path - / f"dag_id={self.DAG_ID}" - / f"run_id={self.RUN_ID}" - / f"task_id={self.MAPPED_TASK_ID}" - / f"map_index={map_index}" - ) - - dir_path.mkdir(parents=True) - - log = dir_path / "attempt=1.log" - log.write_text("Log for testing.") - - # try number 2 - log = dir_path / "attempt=2.log" - log.write_text("Log for testing 2.") - - # Create a custom logging configuration - logging_config = copy.deepcopy(DEFAULT_LOGGING_CONFIG) - logging_config["handlers"]["task"]["base_log_folder"] = self.log_dir - - logging.config.dictConfig(logging_config) - - yield - - logging.config.dictConfig(DEFAULT_LOGGING_CONFIG) - - def teardown_method(self): - clear_db_runs() - - @pytest.mark.parametrize("try_number", [1, 2]) - def test_should_respond_200_json(self, try_number): - key = self.app.config["SECRET_KEY"] - serializer = URLSafeSerializer(key) - token = serializer.dumps({"download_logs": False}) - response = self.client.get( - f"api/v1/dags/{self.DAG_ID}/dagRuns/{self.RUN_ID}/taskInstances/{self.TASK_ID}/logs/{try_number}", - query_string={"token": token}, - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - expected_filename = f"{self.log_dir}/dag_id={self.DAG_ID}/run_id={self.RUN_ID}/task_id={self.TASK_ID}/attempt={try_number}.log" - log_content = "Log for testing." if try_number == 1 else "Log for testing 2." - assert "[('localhost'," in response.json["content"] - assert f"*** Found local files:\\n*** * {expected_filename}\\n" in response.json["content"] - assert f"{log_content}')]" in response.json["content"] - - info = serializer.loads(response.json["continuation_token"]) - assert info == {"end_of_log": True, "log_pos": 16 if try_number == 1 else 18} - assert response.status_code == 200 - - @pytest.mark.parametrize( - "request_url, expected_filename, extra_query_string, try_number", - [ - ( - f"api/v1/dags/{DAG_ID}/dagRuns/{RUN_ID}/taskInstances/{TASK_ID}/logs/1", - f"LOG_DIR/dag_id={DAG_ID}/run_id={RUN_ID}/task_id={TASK_ID}/attempt=1.log", - {}, - 1, - ), - ( - f"api/v1/dags/{DAG_ID}/dagRuns/{RUN_ID}/taskInstances/{MAPPED_TASK_ID}/logs/1", - f"LOG_DIR/dag_id={DAG_ID}/run_id={RUN_ID}/task_id={MAPPED_TASK_ID}/map_index=0/attempt=1.log", - {"map_index": 0}, - 1, - ), - # try_number 2 - ( - f"api/v1/dags/{DAG_ID}/dagRuns/{RUN_ID}/taskInstances/{TASK_ID}/logs/2", - f"LOG_DIR/dag_id={DAG_ID}/run_id={RUN_ID}/task_id={TASK_ID}/attempt=2.log", - {}, - 2, - ), - ( - f"api/v1/dags/{DAG_ID}/dagRuns/{RUN_ID}/taskInstances/{MAPPED_TASK_ID}/logs/2", - f"LOG_DIR/dag_id={DAG_ID}/run_id={RUN_ID}/task_id={MAPPED_TASK_ID}/map_index=0/attempt=2.log", - {"map_index": 0}, - 2, - ), - ], - ) - def test_should_respond_200_text_plain( - self, request_url, expected_filename, extra_query_string, try_number - ): - expected_filename = expected_filename.replace("LOG_DIR", str(self.log_dir)) - - key = self.app.config["SECRET_KEY"] - serializer = URLSafeSerializer(key) - token = serializer.dumps({"download_logs": True}) - - response = self.client.get( - request_url, - query_string={"token": token, **extra_query_string}, - headers={"Accept": "text/plain"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - - log_content = "Log for testing." if try_number == 1 else "Log for testing 2." - assert "localhost\n" in response.data.decode("utf-8") - assert f"*** Found local files:\n*** * {expected_filename}\n" in response.data.decode("utf-8") - assert f"{log_content}\n" in response.data.decode("utf-8") - - @pytest.mark.parametrize( - "request_url, expected_filename, extra_query_string, try_number", - [ - ( - f"api/v1/dags/{DAG_ID}/dagRuns/{RUN_ID}/taskInstances/{TASK_ID}/logs/1", - f"LOG_DIR/dag_id={DAG_ID}/run_id={RUN_ID}/task_id={TASK_ID}/attempt=1.log", - {}, - 1, - ), - ( - f"api/v1/dags/{DAG_ID}/dagRuns/{RUN_ID}/taskInstances/{MAPPED_TASK_ID}/logs/1", - f"LOG_DIR/dag_id={DAG_ID}/run_id={RUN_ID}/task_id={MAPPED_TASK_ID}/map_index=0/attempt=1.log", - {"map_index": 0}, - 1, - ), - ( - f"api/v1/dags/{DAG_ID}/dagRuns/{RUN_ID}/taskInstances/{TASK_ID}/logs/2", - f"LOG_DIR/dag_id={DAG_ID}/run_id={RUN_ID}/task_id={TASK_ID}/attempt=2.log", - {}, - 2, - ), - ( - f"api/v1/dags/{DAG_ID}/dagRuns/{RUN_ID}/taskInstances/{MAPPED_TASK_ID}/logs/2", - f"LOG_DIR/dag_id={DAG_ID}/run_id={RUN_ID}/task_id={MAPPED_TASK_ID}/map_index=0/attempt=2.log", - {"map_index": 0}, - 2, - ), - ], - ) - def test_get_logs_of_removed_task(self, request_url, expected_filename, extra_query_string, try_number): - expected_filename = expected_filename.replace("LOG_DIR", str(self.log_dir)) - - # Recreate DAG without tasks - dagbag = self.app.dag_bag - dag = DAG(self.DAG_ID, schedule=None, start_date=timezone.parse(self.default_time)) - del dagbag.dags[self.DAG_ID] - dagbag.bag_dag(dag=dag) - - key = self.app.config["SECRET_KEY"] - serializer = URLSafeSerializer(key) - token = serializer.dumps({"download_logs": True}) - - response = self.client.get( - request_url, - query_string={"token": token, **extra_query_string}, - headers={"Accept": "text/plain"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - - log_content = "Log for testing." if try_number == 1 else "Log for testing 2." - assert "localhost\n" in response.data.decode("utf-8") - assert f"*** Found local files:\n*** * {expected_filename}\n" in response.data.decode("utf-8") - assert f"{log_content}\n" in response.data.decode("utf-8") - - @pytest.mark.parametrize("try_number", [1, 2]) - def test_get_logs_response_with_ti_equal_to_none(self, try_number): - key = self.app.config["SECRET_KEY"] - serializer = URLSafeSerializer(key) - token = serializer.dumps({"download_logs": True}) - - response = self.client.get( - f"api/v1/dags/{self.DAG_ID}/dagRuns/{self.RUN_ID}/taskInstances/Invalid-Task-ID/logs/{try_number}", - query_string={"token": token}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json == { - "detail": None, - "status": 404, - "title": "TaskInstance not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - @pytest.mark.parametrize("try_number", [1, 2]) - def test_get_logs_with_metadata_as_download_large_file(self, try_number): - with mock.patch("airflow.utils.log.file_task_handler.FileTaskHandler.read") as read_mock: - first_return = ([[("", "1st line")]], [{}]) - second_return = ([[("", "2nd line")]], [{"end_of_log": False}]) - third_return = ([[("", "3rd line")]], [{"end_of_log": True}]) - fourth_return = ([[("", "should never be read")]], [{"end_of_log": True}]) - read_mock.side_effect = [first_return, second_return, third_return, fourth_return] - - response = self.client.get( - f"api/v1/dags/{self.DAG_ID}/dagRuns/{self.RUN_ID}/" - f"taskInstances/{self.TASK_ID}/logs/{try_number}?full_content=True", - headers={"Accept": "text/plain"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert "1st line" in response.data.decode("utf-8") - assert "2nd line" in response.data.decode("utf-8") - assert "3rd line" in response.data.decode("utf-8") - assert "should never be read" not in response.data.decode("utf-8") - - @pytest.mark.parametrize("try_number", [1, 2]) - @mock.patch("airflow.api_connexion.endpoints.log_endpoint.TaskLogReader") - def test_get_logs_for_handler_without_read_method(self, mock_log_reader, try_number): - type(mock_log_reader.return_value).supports_read = PropertyMock(return_value=False) - - key = self.app.config["SECRET_KEY"] - serializer = URLSafeSerializer(key) - token = serializer.dumps({"download_logs": False}) - - # check guessing - response = self.client.get( - f"api/v1/dags/{self.DAG_ID}/dagRuns/{self.RUN_ID}/taskInstances/{self.TASK_ID}/logs/{try_number}", - query_string={"token": token}, - headers={"Content-Type": "application/jso"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert "Task log handler does not support read logs." in response.data.decode("utf-8") - - def test_bad_signature_raises(self): - token = {"download_logs": False} - - response = self.client.get( - f"api/v1/dags/{self.DAG_ID}/dagRuns/{self.RUN_ID}/taskInstances/{self.TASK_ID}/logs/1", - query_string={"token": token}, - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.json == { - "detail": None, - "status": 400, - "title": "Bad Signature. Please use only the tokens provided by the API.", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_raises_404_for_invalid_dag_run_id(self): - response = self.client.get( - f"api/v1/dags/{self.DAG_ID}/dagRuns/NO_DAG_RUN/" # invalid run_id - f"taskInstances/{self.TASK_ID}/logs/1?", - headers={"Accept": "application/json"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json == { - "detail": None, - "status": 404, - "title": "TaskInstance not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self): - key = self.app.config["SECRET_KEY"] - serializer = URLSafeSerializer(key) - token = serializer.dumps({"download_logs": False}) - - response = self.client.get( - f"api/v1/dags/{self.DAG_ID}/dagRuns/{self.RUN_ID}/taskInstances/{self.TASK_ID}/logs/1", - query_string={"token": token}, - headers={"Accept": "application/json"}, - ) - - assert_401(response) - - def test_should_raise_403_forbidden(self): - key = self.app.config["SECRET_KEY"] - serializer = URLSafeSerializer(key) - token = serializer.dumps({"download_logs": True}) - - response = self.client.get( - f"api/v1/dags/{self.DAG_ID}/dagRuns/{self.RUN_ID}/taskInstances/{self.TASK_ID}/logs/1", - query_string={"token": token}, - headers={"Accept": "text/plain"}, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - def test_should_raise_404_when_missing_map_index_param_for_mapped_task(self): - key = self.app.config["SECRET_KEY"] - serializer = URLSafeSerializer(key) - token = serializer.dumps({"download_logs": True}) - - response = self.client.get( - f"api/v1/dags/{self.DAG_ID}/dagRuns/{self.RUN_ID}/taskInstances/{self.MAPPED_TASK_ID}/logs/1", - query_string={"token": token}, - headers={"Accept": "text/plain"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json["title"] == "TaskInstance not found" - - def test_should_raise_404_when_filtering_on_map_index_for_unmapped_task(self): - key = self.app.config["SECRET_KEY"] - serializer = URLSafeSerializer(key) - token = serializer.dumps({"download_logs": True}) - - response = self.client.get( - f"api/v1/dags/{self.DAG_ID}/dagRuns/{self.RUN_ID}/taskInstances/{self.TASK_ID}/logs/1", - query_string={"token": token, "map_index": 0}, - headers={"Accept": "text/plain"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json["title"] == "TaskInstance not found" diff --git a/tests/api_connexion/endpoints/test_mapped_task_instance_endpoint.py b/tests/api_connexion/endpoints/test_mapped_task_instance_endpoint.py deleted file mode 100644 index 4619e818fbf29..0000000000000 --- a/tests/api_connexion/endpoints/test_mapped_task_instance_endpoint.py +++ /dev/null @@ -1,483 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import datetime as dt -import itertools -import os -import urllib - -import pytest - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.dag_processing.bundles.manager import DagBundlesManager -from airflow.models import TaskInstance -from airflow.models.baseoperator import BaseOperator -from airflow.models.dagbag import DagBag -from airflow.models.taskmap import TaskMap -from airflow.utils.platform import getuser -from airflow.utils.session import provide_session -from airflow.utils.state import State, TaskInstanceState -from airflow.utils.timezone import datetime - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.db import clear_db_runs, clear_rendered_ti_fields -from tests_common.test_utils.mock_operators import MockOperator - -pytestmark = pytest.mark.db_test - -DEFAULT_DATETIME_1 = datetime(2020, 1, 1) -DEFAULT_DATETIME_STR_1 = "2020-01-01T00:00:00+00:00" -DEFAULT_DATETIME_STR_2 = "2020-01-02T00:00:00+00:00" -QUOTED_DEFAULT_DATETIME_STR_1 = urllib.parse.quote(DEFAULT_DATETIME_STR_1) -QUOTED_DEFAULT_DATETIME_STR_2 = urllib.parse.quote(DEFAULT_DATETIME_STR_2) - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestMappedTaskInstanceEndpoint: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.default_time = DEFAULT_DATETIME_1 - self.ti_init = { - "logical_date": self.default_time, - "state": State.RUNNING, - } - self.ti_extras = { - "start_date": self.default_time + dt.timedelta(days=1), - "end_date": self.default_time + dt.timedelta(days=2), - "pid": 100, - "duration": 10000, - "pool": "default_pool", - "queue": "default_queue", - } - self.app = configured_app - self.client = self.app.test_client() # type:ignore - clear_db_runs() - clear_rendered_ti_fields() - - def create_dag_runs_with_mapped_tasks(self, dag_maker, session, dags=None): - for dag_id, dag in (dags or {}).items(): - count = dag["success"] + dag["running"] - with dag_maker(session=session, dag_id=dag_id, start_date=DEFAULT_DATETIME_1): - task1 = BaseOperator(task_id="op1") - mapped = MockOperator.partial(task_id="task_2", executor="default").expand(arg2=task1.output) - - dr = dag_maker.create_dagrun(run_id=f"run_{dag_id}") - - session.add( - TaskMap( - dag_id=dr.dag_id, - task_id=task1.task_id, - run_id=dr.run_id, - map_index=-1, - length=count, - keys=None, - ) - ) - - if count: - # Remove the map_index=-1 TI when we're creating other TIs - session.query(TaskInstance).filter( - TaskInstance.dag_id == mapped.dag_id, - TaskInstance.task_id == mapped.task_id, - TaskInstance.run_id == dr.run_id, - ).delete() - - for index, state in enumerate( - itertools.chain( - itertools.repeat(TaskInstanceState.SUCCESS, dag["success"]), - itertools.repeat(TaskInstanceState.FAILED, dag["failed"]), - itertools.repeat(TaskInstanceState.RUNNING, dag["running"]), - ) - ): - ti = TaskInstance(mapped, run_id=dr.run_id, map_index=index, state=state) - setattr(ti, "start_date", DEFAULT_DATETIME_1) - session.add(ti) - - DagBundlesManager().sync_bundles_to_db() - self.app.dag_bag = DagBag(os.devnull, include_examples=False) - self.app.dag_bag.dags = {dag_id: dag_maker.dag} - self.app.dag_bag.sync_to_db("dags-folder", None) - session.flush() - - TaskMap.expand_mapped_task(mapped, dr.run_id, session=session) - - @pytest.fixture - def one_task_with_mapped_tis(self, dag_maker, session): - self.create_dag_runs_with_mapped_tasks( - dag_maker, - session, - dags={ - "mapped_tis": { - "success": 3, - "failed": 0, - "running": 0, - }, - }, - ) - - @pytest.fixture - def one_task_with_single_mapped_ti(self, dag_maker, session): - self.create_dag_runs_with_mapped_tasks( - dag_maker, - session, - dags={ - "mapped_tis": { - "success": 1, - "failed": 0, - "running": 0, - }, - }, - ) - - @pytest.fixture - def one_task_with_many_mapped_tis(self, dag_maker, session): - self.create_dag_runs_with_mapped_tasks( - dag_maker, - session, - dags={ - "mapped_tis": { - "success": 5, - "failed": 20, - "running": 85, - }, - }, - ) - - @pytest.fixture - def one_task_with_zero_mapped_tis(self, dag_maker, session): - self.create_dag_runs_with_mapped_tasks( - dag_maker, - session, - dags={ - "mapped_tis": { - "success": 0, - "failed": 0, - "running": 0, - }, - }, - ) - - -class TestNonExistent(TestMappedTaskInstanceEndpoint): - @provide_session - def test_non_existent_task_instance(self, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json["title"] == "DAG mapped_tis not found" - - -class TestGetMappedTaskInstance(TestMappedTaskInstanceEndpoint): - @provide_session - def test_mapped_task_instances(self, one_task_with_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/0", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "dag_id": "mapped_tis", - "dag_run_id": "run_mapped_tis", - "duration": None, - "end_date": None, - "logical_date": "2020-01-01T00:00:00+00:00", - "executor": "default", - "executor_config": "{}", - "hostname": "", - "map_index": 0, - "max_tries": 0, - "note": None, - "operator": "MockOperator", - "pid": None, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 1, - "queue": "default", - "queued_when": None, - "scheduled_when": None, - "rendered_fields": {}, - "rendered_map_index": "0", - "start_date": "2020-01-01T00:00:00+00:00", - "state": "success", - "task_id": "task_2", - "task_display_name": "task_2", - "try_number": 0, - "unixname": getuser(), - "trigger": None, - "triggerer_job": None, - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/1", - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - def test_without_map_index_returns_custom_404(self, one_task_with_mapped_tis): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json == { - "detail": "Task instance is mapped, add the map_index value to the URL", - "status": 404, - "title": "Task instance not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_one_mapped_task_works(self, one_task_with_single_mapped_ti): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/0", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/1", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json == { - "detail": "Task instance is mapped, add the map_index value to the URL", - "status": 404, - "title": "Task instance not found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - -class TestGetMappedTaskInstances(TestMappedTaskInstanceEndpoint): - @provide_session - def test_mapped_task_instances(self, one_task_with_many_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 110 - assert len(response.json["task_instances"]) == 100 - - @provide_session - def test_mapped_task_instances_offset_limit(self, one_task_with_many_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped" - "?offset=4&limit=10", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 110 - assert len(response.json["task_instances"]) == 10 - assert list(range(4, 14)) == [ti["map_index"] for ti in response.json["task_instances"]] - - @provide_session - def test_mapped_task_instances_order(self, one_task_with_many_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 110 - assert len(response.json["task_instances"]) == 100 - assert list(range(100)) == [ti["map_index"] for ti in response.json["task_instances"]] - - @provide_session - def test_mapped_task_instances_reverse_order(self, one_task_with_many_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped" - "?order_by=-map_index", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 110 - assert len(response.json["task_instances"]) == 100 - assert list(range(109, 9, -1)) == [ti["map_index"] for ti in response.json["task_instances"]] - - @provide_session - def test_mapped_task_instances_state_order(self, one_task_with_many_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped" - "?order_by=-state", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 110 - assert len(response.json["task_instances"]) == 100 - assert list(range(5)) + list(range(25, 110)) + list(range(5, 15)) == [ - ti["map_index"] for ti in response.json["task_instances"] - ] - # State ascending - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped" - "?order_by=state", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 110 - assert len(response.json["task_instances"]) == 100 - assert list(range(5, 25)) + list(range(90, 110)) + list(range(25, 85)) == [ - ti["map_index"] for ti in response.json["task_instances"] - ] - - @provide_session - def test_mapped_task_instances_invalid_order(self, one_task_with_many_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped" - "?order_by=unsupported", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json["detail"] == "Ordering with 'unsupported' is not supported" - - @provide_session - def test_mapped_task_instances_with_date(self, one_task_with_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped" - f"?start_date_gte={QUOTED_DEFAULT_DATETIME_STR_1}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 3 - assert len(response.json["task_instances"]) == 3 - - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped" - f"?start_date_gte={QUOTED_DEFAULT_DATETIME_STR_2}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 0 - assert response.json["task_instances"] == [] - - @provide_session - def test_mapped_task_instances_with_state(self, one_task_with_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped?state=success", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 3 - assert len(response.json["task_instances"]) == 3 - - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped?state=running", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 0 - assert response.json["task_instances"] == [] - - @provide_session - def test_mapped_task_instances_with_pool(self, one_task_with_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped" - "?pool=default_pool", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 3 - assert len(response.json["task_instances"]) == 3 - - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped?pool=test_pool", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 0 - assert response.json["task_instances"] == [] - - @provide_session - def test_mapped_task_instances_with_queue(self, one_task_with_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped?queue=default", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 3 - assert len(response.json["task_instances"]) == 3 - - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped?queue=test_queue", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 0 - assert response.json["task_instances"] == [] - - @provide_session - def test_mapped_task_instances_with_executor(self, one_task_with_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped?executor=default", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 3 - assert len(response.json["task_instances"]) == 3 - - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped?executor=no_exec", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 0 - assert response.json["task_instances"] == [] - - @provide_session - def test_mapped_task_instances_with_zero_mapped(self, one_task_with_zero_mapped_tis, session): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 0 - assert response.json["task_instances"] == [] - - def test_should_raise_404_not_found_for_nonexistent_task(self): - response = self.client.get( - "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/nonexistent_task/listMapped", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json["title"] == "Task id nonexistent_task not found" diff --git a/tests/api_connexion/endpoints/test_plugin_endpoint.py b/tests/api_connexion/endpoints/test_plugin_endpoint.py deleted file mode 100644 index 6b539c284ed1f..0000000000000 --- a/tests/api_connexion/endpoints/test_plugin_endpoint.py +++ /dev/null @@ -1,264 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import inspect - -import pytest -from fastapi import FastAPI -from flask import Blueprint -from flask_appbuilder import BaseView - -from airflow.plugins_manager import AirflowPlugin -from airflow.timetables.base import Timetable -from airflow.utils.module_loading import qualname - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.compat import BaseOperatorLink -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.mock_plugins import mock_plugin_manager - -pytestmark = pytest.mark.db_test - - -def plugin_macro(): ... - - -class MockOperatorLink(BaseOperatorLink): - name = "mock_operator_link" - - def get_link(self, operator, *, ti_key) -> str: - return "mock_operator_link" - - -bp = Blueprint("mock_blueprint", __name__, url_prefix="/mock_blueprint") - -app = FastAPI() - -app_with_metadata = {"app": app, "url_prefix": "/some_prefix", "name": "App name"} - - -class MockView(BaseView): ... - - -mockview = MockView() - -appbuilder_menu_items = { - "name": "mock_plugin", - "href": "https://example.com", -} - - -class CustomTimetable(Timetable): - def infer_manual_data_interval(self, *, run_after): - pass - - def next_dagrun_info( - self, - *, - last_automated_data_interval, - restriction, - ): - pass - - -class MyCustomListener: - pass - - -class MockPlugin(AirflowPlugin): - name = "mock_plugin" - flask_blueprints = [bp] - fastapi_apps = [app_with_metadata] - appbuilder_views = [{"view": mockview}] - appbuilder_menu_items = [appbuilder_menu_items] - global_operator_extra_links = [MockOperatorLink()] - operator_extra_links = [MockOperatorLink()] - macros = [plugin_macro] - timetables = [CustomTimetable] - listeners = [pytest, MyCustomListener()] # using pytest here because we need a module(just for test) - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestPluginsEndpoint: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - """ - Setup For XCom endpoint TC - """ - self.app = configured_app - self.client = self.app.test_client() # type:ignore - - -class TestGetPlugins(TestPluginsEndpoint): - def test_get_plugins_return_200(self): - mock_plugin = MockPlugin() - mock_plugin.name = "test_plugin" - with mock_plugin_manager(plugins=[mock_plugin]): - response = self.client.get("api/v1/plugins", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == { - "plugins": [ - { - "appbuilder_menu_items": [appbuilder_menu_items], - "appbuilder_views": [{"view": qualname(MockView)}], - "flask_blueprints": [ - f"<{qualname(bp.__class__)}: name={bp.name!r} import_name={bp.import_name!r}>" - ], - "fastapi_apps": [ - { - "app": "fastapi.applications.FastAPI", - "name": "App name", - "url_prefix": "/some_prefix", - } - ], - "global_operator_extra_links": [f"<{qualname(MockOperatorLink().__class__)} object>"], - "macros": [qualname(plugin_macro)], - "operator_extra_links": [f"<{qualname(MockOperatorLink().__class__)} object>"], - "source": None, - "name": "test_plugin", - "timetables": [qualname(CustomTimetable)], - "listeners": [ - d.__name__ if inspect.ismodule(d) else qualname(d) - for d in [pytest, MyCustomListener()] - ], - } - ], - "total_entries": 1, - } - - def test_get_plugins_works_with_more_plugins(self): - mock_plugin = AirflowPlugin() - mock_plugin.name = "test_plugin" - mock_plugin_2 = AirflowPlugin() - mock_plugin_2.name = "test_plugin2" - with mock_plugin_manager(plugins=[mock_plugin, mock_plugin_2]): - response = self.client.get("api/v1/plugins", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json["total_entries"] == 2 - - def test_get_plugins_return_200_if_no_plugins(self): - with mock_plugin_manager(plugins=[]): - response = self.client.get("api/v1/plugins", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - - def test_should_raises_401_unauthenticated(self): - response = self.client.get("/api/v1/plugins") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "/api/v1/plugins", environ_overrides={"REMOTE_USER": "test_no_permissions"} - ) - assert response.status_code == 403 - - -class TestGetPluginsPagination(TestPluginsEndpoint): - @pytest.mark.parametrize( - "url, expected_plugin_names", - [ - ("/api/v1/plugins?limit=1", ["TEST_PLUGIN_1"]), - ("/api/v1/plugins?limit=2", ["TEST_PLUGIN_1", "TEST_PLUGIN_2"]), - ( - "/api/v1/plugins?offset=5", - [ - "TEST_PLUGIN_6", - "TEST_PLUGIN_7", - "TEST_PLUGIN_8", - "TEST_PLUGIN_9", - "TEST_PLUGIN_10", - ], - ), - ( - "/api/v1/plugins?offset=0", - [ - "TEST_PLUGIN_1", - "TEST_PLUGIN_2", - "TEST_PLUGIN_3", - "TEST_PLUGIN_4", - "TEST_PLUGIN_5", - "TEST_PLUGIN_6", - "TEST_PLUGIN_7", - "TEST_PLUGIN_8", - "TEST_PLUGIN_9", - "TEST_PLUGIN_10", - ], - ), - ("/api/v1/plugins?limit=1&offset=5", ["TEST_PLUGIN_6"]), - ("/api/v1/plugins?limit=1&offset=1", ["TEST_PLUGIN_2"]), - ( - "/api/v1/plugins?limit=2&offset=2", - ["TEST_PLUGIN_3", "TEST_PLUGIN_4"], - ), - ], - ) - def test_handle_limit_offset(self, url, expected_plugin_names): - plugins = self._create_plugins(10) - with mock_plugin_manager(plugins=plugins): - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json["total_entries"] == 10 - plugin_names = [plugin["name"] for plugin in response.json["plugins"] if plugin] - assert plugin_names == expected_plugin_names - - def test_should_respect_page_size_limit_default(self): - plugins = self._create_plugins(200) - with mock_plugin_manager(plugins=plugins): - response = self.client.get("/api/v1/plugins", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json["total_entries"] == 200 - assert len(response.json["plugins"]) == 100 - - def test_limit_of_zero_should_return_default(self): - plugins = self._create_plugins(200) - with mock_plugin_manager(plugins=plugins): - response = self.client.get("/api/v1/plugins?limit=0", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json["total_entries"] == 200 - assert len(response.json["plugins"]) == 100 - - @conf_vars({("api", "maximum_page_limit"): "150"}) - def test_should_return_conf_max_if_req_max_above_conf(self): - plugins = self._create_plugins(200) - with mock_plugin_manager(plugins=plugins): - response = self.client.get("/api/v1/plugins?limit=180", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert len(response.json["plugins"]) == 150 - - def _create_plugins(self, count): - plugins = [] - for i in range(1, count + 1): - mock_plugin = AirflowPlugin() - mock_plugin.name = f"TEST_PLUGIN_{i}" - plugins.append(mock_plugin) - return plugins diff --git a/tests/api_connexion/endpoints/test_pool_endpoint.py b/tests/api_connexion/endpoints/test_pool_endpoint.py deleted file mode 100644 index 13f87b96c3468..0000000000000 --- a/tests/api_connexion/endpoints/test_pool_endpoint.py +++ /dev/null @@ -1,703 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.models.pool import Pool -from airflow.utils.session import provide_session - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_pools -from tests_common.test_utils.www import _check_last_log - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestBasePoolEndpoints: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - clear_db_pools() - - def teardown_method(self) -> None: - clear_db_pools() - - -class TestGetPools(TestBasePoolEndpoints): - def test_response_200(self, session): - pool_model = Pool(pool="test_pool_a", slots=3, include_deferred=True) - session.add(pool_model) - session.commit() - result = session.query(Pool).all() - assert len(result) == 2 # accounts for the default pool as well - response = self.client.get("/api/v1/pools", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == { - "pools": [ - { - "name": "default_pool", - "slots": 128, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": 128, - "description": "Default pool", - "include_deferred": False, - }, - { - "name": "test_pool_a", - "slots": 3, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": 3, - "description": None, - "include_deferred": True, - }, - ], - "total_entries": 2, - } - - def test_response_200_with_order_by(self, session): - pool_model = Pool(pool="test_pool_a", slots=3, include_deferred=True) - session.add(pool_model) - session.commit() - result = session.query(Pool).all() - assert len(result) == 2 # accounts for the default pool as well - response = self.client.get("/api/v1/pools?order_by=slots", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == { - "pools": [ - { - "name": "test_pool_a", - "slots": 3, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": 3, - "description": None, - "include_deferred": True, - }, - { - "name": "default_pool", - "slots": 128, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": 128, - "description": "Default pool", - "include_deferred": False, - }, - ], - "total_entries": 2, - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.get("/api/v1/pools") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get("/api/v1/pools", environ_overrides={"REMOTE_USER": "test_no_permissions"}) - assert response.status_code == 403 - - -class TestGetPoolsPagination(TestBasePoolEndpoints): - @pytest.mark.parametrize( - "url, expected_pool_ids", - [ - # Offset test data - ("/api/v1/pools?offset=1", [f"test_pool{i}" for i in range(1, 101)]), - ("/api/v1/pools?offset=3", [f"test_pool{i}" for i in range(3, 103)]), - # Limit test data - ("/api/v1/pools?limit=2", ["default_pool", "test_pool1"]), - ("/api/v1/pools?limit=1", ["default_pool"]), - # Limit and offset test data - ( - "/api/v1/pools?limit=100&offset=1", - [f"test_pool{i}" for i in range(1, 101)], - ), - ("/api/v1/pools?limit=2&offset=1", ["test_pool1", "test_pool2"]), - ( - "/api/v1/pools?limit=3&offset=2", - ["test_pool2", "test_pool3", "test_pool4"], - ), - ], - ) - @provide_session - def test_limit_and_offset(self, url, expected_pool_ids, session): - pools = [Pool(pool=f"test_pool{i}", slots=1, include_deferred=False) for i in range(1, 121)] - session.add_all(pools) - session.commit() - result = session.query(Pool).count() - assert result == 121 # accounts for default pool as well - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - pool_ids = [pool["name"] for pool in response.json["pools"]] - assert pool_ids == expected_pool_ids - - def test_should_respect_page_size_limit_default(self, session): - pools = [Pool(pool=f"test_pool{i}", slots=1, include_deferred=False) for i in range(1, 121)] - session.add_all(pools) - session.commit() - result = session.query(Pool).count() - assert result == 121 - response = self.client.get("/api/v1/pools", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert len(response.json["pools"]) == 100 - - def test_should_raise_400_for_invalid_orderby(self, session): - pools = [Pool(pool=f"test_pool{i}", slots=1, include_deferred=False) for i in range(1, 121)] - session.add_all(pools) - session.commit() - result = session.query(Pool).count() - assert result == 121 - response = self.client.get( - "/api/v1/pools?order_by=open_slots", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - msg = "Ordering with 'open_slots' is disallowed or the attribute does not exist on the model" - assert response.json["detail"] == msg - - @conf_vars({("api", "maximum_page_limit"): "150"}) - def test_should_return_conf_max_if_req_max_above_conf(self, session): - pools = [Pool(pool=f"test_pool{i}", slots=1, include_deferred=False) for i in range(1, 200)] - session.add_all(pools) - session.commit() - result = session.query(Pool).count() - assert result == 200 - response = self.client.get("/api/v1/pools?limit=180", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert len(response.json["pools"]) == 150 - - -class TestGetPool(TestBasePoolEndpoints): - def test_response_200(self, session): - pool_model = Pool(pool="test_pool_a", slots=3, include_deferred=True) - session.add(pool_model) - session.commit() - response = self.client.get("/api/v1/pools/test_pool_a", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == { - "name": "test_pool_a", - "slots": 3, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": 3, - "description": None, - "include_deferred": True, - } - - def test_response_404(self): - response = self.client.get("/api/v1/pools/invalid_pool", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 404 - assert response.json == { - "detail": "Pool with name:'invalid_pool' not found", - "status": 404, - "title": "Not Found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.get("/api/v1/pools/default_pool") - - assert_401(response) - - -class TestDeletePool(TestBasePoolEndpoints): - def test_response_204(self, session): - pool_name = "test_pool" - pool_instance = Pool(pool=pool_name, slots=3, include_deferred=False) - session.add(pool_instance) - session.commit() - - response = self.client.delete(f"api/v1/pools/{pool_name}", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 204 - # Check if the pool is deleted from the db - response = self.client.get(f"api/v1/pools/{pool_name}", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 404 - _check_last_log(session, dag_id=None, event="api.delete_pool", logical_date=None) - - def test_response_404(self): - response = self.client.delete("api/v1/pools/invalid_pool", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 404 - assert response.json == { - "detail": "Pool with name:'invalid_pool' not found", - "status": 404, - "title": "Not Found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self, session): - pool_name = "test_pool" - pool_instance = Pool(pool=pool_name, slots=3, include_deferred=False) - session.add(pool_instance) - session.commit() - - response = self.client.delete(f"api/v1/pools/{pool_name}") - - assert_401(response) - - # Should still exists - response = self.client.get(f"/api/v1/pools/{pool_name}", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - - -class TestPostPool(TestBasePoolEndpoints): - def test_response_200(self, session): - response = self.client.post( - "api/v1/pools", - json={"name": "test_pool_a", "slots": 3, "description": "test pool", "include_deferred": True}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "name": "test_pool_a", - "slots": 3, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": 3, - "description": "test pool", - "include_deferred": True, - } - _check_last_log(session, dag_id=None, event="api.post_pool", logical_date=None) - - def test_response_409(self, session): - pool_name = "test_pool_a" - pool_instance = Pool(pool=pool_name, slots=3, include_deferred=False) - session.add(pool_instance) - session.commit() - response = self.client.post( - "api/v1/pools", - json={"name": "test_pool_a", "slots": 3, "include_deferred": False}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 409 - assert response.json == { - "detail": f"Pool: {pool_name} already exists", - "status": 409, - "title": "Conflict", - "type": EXCEPTIONS_LINK_MAP[409], - } - - @pytest.mark.parametrize( - "request_json, error_detail", - [ - pytest.param( - {"slots": 3}, - "Missing required property(ies): ['name']", - id="for missing pool name", - ), - pytest.param( - {"name": "invalid_pool"}, - "Missing required property(ies): ['slots']", - id="for missing slots", - ), - pytest.param( - {}, - "Missing required property(ies): ['name', 'slots']", - id="for missing pool name AND slots AND include_deferred", - ), - pytest.param( - {"name": "invalid_pool", "slots": 3, "extra_field_1": "extra"}, - "{'extra_field_1': ['Unknown field.']}", - id="for extra fields", - ), - ], - ) - def test_response_400(self, request_json, error_detail): - response = self.client.post( - "api/v1/pools", json=request_json, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert response.json == { - "detail": error_detail, - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.post("api/v1/pools", json={"name": "test_pool_a", "slots": 3}) - - assert_401(response) - - -class TestPatchPool(TestBasePoolEndpoints): - def test_response_200(self, session): - pool = Pool(pool="test_pool", slots=2, include_deferred=True) - session.add(pool) - session.commit() - response = self.client.patch( - "api/v1/pools/test_pool", - json={"name": "test_pool_a", "slots": 3, "include_deferred": False}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "occupied_slots": 0, - "queued_slots": 0, - "name": "test_pool_a", - "open_slots": 3, - "running_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "slots": 3, - "description": None, - "include_deferred": False, - } - _check_last_log(session, dag_id=None, event="api.patch_pool", logical_date=None) - - @pytest.mark.parametrize( - "error_detail, request_json", - [ - # Missing properties - ("Missing required property(ies): ['name']", {"slots": 3}), - ("Missing required property(ies): ['slots']", {"name": "test_pool_a"}), - ("Missing required property(ies): ['name', 'slots']", {}), - # Extra properties - ( - "{'extra_field': ['Unknown field.']}", - {"name": "test_pool_a", "slots": 3, "include_deferred": True, "extra_field": "extra"}, - ), - ], - ) - @provide_session - def test_response_400(self, error_detail, request_json, session): - pool = Pool(pool="test_pool", slots=2, include_deferred=False) - session.add(pool) - session.commit() - response = self.client.patch( - "api/v1/pools/test_pool", json=request_json, environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - assert response.json == { - "detail": error_detail, - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - def test_not_found_when_no_pool_available(self): - response = self.client.patch( - "api/v1/pools/test_pool", - json={"name": "test_pool_a", "slots": 3}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json == { - "detail": "Pool with name:'test_pool' not found", - "status": 404, - "title": "Not Found", - "type": EXCEPTIONS_LINK_MAP[404], - } - - def test_should_raises_401_unauthenticated(self, session): - pool = Pool(pool="test_pool", slots=2, include_deferred=False) - session.add(pool) - session.commit() - - response = self.client.patch( - "api/v1/pools/test_pool", - json={"name": "test_pool_a", "slots": 3}, - ) - - assert_401(response) - - -class TestModifyDefaultPool(TestBasePoolEndpoints): - def test_delete_400(self): - response = self.client.delete("api/v1/pools/default_pool", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 400 - assert response.json == { - "detail": "Default Pool can't be deleted", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } - - @pytest.mark.parametrize( - "status_code, url, json, expected_response", - [ - pytest.param( - 400, - "api/v1/pools/default_pool", - {"name": "test_pool_a", "slots": 3, "include_deferred": False}, - { - "detail": "Default Pool's name can't be modified", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - }, - id="400 No update mask", - ), - pytest.param( - 400, - "api/v1/pools/default_pool?update_mask=name, slots", - {"name": "test_pool_a", "slots": 3, "include_deferred": False}, - { - "detail": "Default Pool's name can't be modified", - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - }, - id="400 Update mask with both fields", - ), - pytest.param( - 200, - "api/v1/pools/default_pool?update_mask=slots", - {"name": "test_pool_a", "slots": 3}, - { - "occupied_slots": 0, - "queued_slots": 0, - "name": "default_pool", - "open_slots": 3, - "running_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "slots": 3, - "description": "Default pool", - "include_deferred": False, - }, - id="200 Update mask with slots", - ), - pytest.param( - 200, - "api/v1/pools/default_pool?update_mask=include_deferred", - {"name": "test_pool_a", "include_deferred": True}, - { - "occupied_slots": 0, - "queued_slots": 0, - "name": "default_pool", - "open_slots": 128, - "running_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "slots": 128, - "description": "Default pool", - "include_deferred": True, - }, - id="200 Update mask with include_deferred", - ), - pytest.param( - 200, - "api/v1/pools/default_pool?update_mask=slots,include_deferred", - {"name": "test_pool_a", "slots": 3, "include_deferred": True}, - { - "occupied_slots": 0, - "queued_slots": 0, - "name": "default_pool", - "open_slots": 3, - "running_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "slots": 3, - "description": "Default pool", - "include_deferred": True, - }, - id="200 Update mask with slots AND include_deferred", - ), - pytest.param( - 200, - "api/v1/pools/default_pool?update_mask=name,slots", - {"name": "default_pool", "slots": 3}, - { - "occupied_slots": 0, - "queued_slots": 0, - "name": "default_pool", - "open_slots": 3, - "running_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "slots": 3, - "description": "Default pool", - "include_deferred": False, - }, - id="200 Update mask with slots and name", - ), - pytest.param( - 200, - "api/v1/pools/default_pool", - { - "name": "default_pool", - "slots": 3, - "include_deferred": True, - }, - { - "occupied_slots": 0, - "queued_slots": 0, - "name": "default_pool", - "open_slots": 3, - "running_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "slots": 3, - "description": "Default pool", - "include_deferred": True, - }, - id="200 no update mask", - ), - ], - ) - def test_patch(self, status_code, url, json, expected_response, session): - response = self.client.patch(url, json=json, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == status_code - assert response.json == expected_response - _check_last_log(session, dag_id=None, event="api.patch_pool", logical_date=None) - - -class TestPatchPoolWithUpdateMask(TestBasePoolEndpoints): - @pytest.mark.parametrize( - "url, patch_json, expected_name, expected_slots, expected_include_deferred", - [ - ( - "api/v1/pools/test_pool?update_mask=name, slots", - {"name": "test_pool_a", "slots": 2}, - "test_pool_a", - 2, - False, - ), - ( - "api/v1/pools/test_pool?update_mask=name", - {"name": "test_pool_a", "slots": 2}, - "test_pool_a", - 3, - False, - ), - ( - "api/v1/pools/test_pool?update_mask=slots", - {"name": "test_pool_a", "slots": 2}, - "test_pool", - 2, - False, - ), - ( - "api/v1/pools/test_pool?update_mask=slots", - {"slots": 2}, - "test_pool", - 2, - False, - ), - ( - "api/v1/pools/test_pool?update_mask=include_deferred", - {"include_deferred": True}, - "test_pool", - 3, - True, - ), - ], - ) - @provide_session - def test_response_200( - self, url, patch_json, expected_name, expected_slots, expected_include_deferred, session - ): - pool = Pool(pool="test_pool", slots=3, include_deferred=False) - session.add(pool) - session.commit() - response = self.client.patch(url, json=patch_json, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == { - "name": expected_name, - "slots": expected_slots, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": expected_slots, - "description": None, - "include_deferred": expected_include_deferred, - } - _check_last_log(session, dag_id=None, event="api.patch_pool", logical_date=None) - - @pytest.mark.parametrize( - "error_detail, url, patch_json", - [ - pytest.param( - "Property is read-only - 'occupied_slots'", - "api/v1/pools/test_pool?update_mask=slots, name, occupied_slots", - {"name": "test_pool_a", "slots": 2, "occupied_slots": 1}, - id="Patching read only field", - ), - pytest.param( - "Property is read-only - 'queued_slots'", - "api/v1/pools/test_pool?update_mask=slots, name, queued_slots", - {"name": "test_pool_a", "slots": 2, "queued_slots": 1}, - id="Patching read only field", - ), - pytest.param( - "Invalid field: names in update mask", - "api/v1/pools/test_pool?update_mask=slots, names,", - {"name": "test_pool_a", "slots": 2}, - id="Invalid update mask", - ), - pytest.param( - "Invalid field: slot in update mask", - "api/v1/pools/test_pool?update_mask=slot, name,", - {"name": "test_pool_a", "slots": 2}, - id="Invalid update mask", - ), - ], - ) - @provide_session - def test_response_400(self, error_detail, url, patch_json, session): - pool = Pool(pool="test_pool", slots=3, include_deferred=False) - session.add(pool) - session.commit() - response = self.client.patch(url, json=patch_json, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 400 - assert response.json == { - "detail": error_detail, - "status": 400, - "title": "Bad Request", - "type": EXCEPTIONS_LINK_MAP[400], - } diff --git a/tests/api_connexion/endpoints/test_provider_endpoint.py b/tests/api_connexion/endpoints/test_provider_endpoint.py deleted file mode 100644 index b02dbb61e0306..0000000000000 --- a/tests/api_connexion/endpoints/test_provider_endpoint.py +++ /dev/null @@ -1,117 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from unittest import mock - -import pytest - -from airflow.providers_manager import ProviderInfo - -from tests_common.test_utils.api_connexion_utils import create_user, delete_user - -pytestmark = pytest.mark.db_test - -MOCK_PROVIDERS = { - "apache-airflow-providers-amazon": ProviderInfo( - "1.0.0", - { - "package-name": "apache-airflow-providers-amazon", - "name": "Amazon", - "description": "`Amazon Web Services (AWS) `__.\n", - "versions": ["1.0.0"], - }, - ), - "apache-airflow-providers-apache-cassandra": ProviderInfo( - "1.0.0", - { - "package-name": "apache-airflow-providers-apache-cassandra", - "name": "Apache Cassandra", - "description": "`Apache Cassandra `__.\n", - "versions": ["1.0.0"], - }, - ), -} - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestBaseProviderEndpoint: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app, cleanup_providers_manager) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - - -class TestGetProviders(TestBaseProviderEndpoint): - @mock.patch( - "airflow.providers_manager.ProvidersManager.providers", - new_callable=mock.PropertyMock, - return_value={}, - ) - def test_response_200_empty_list(self, mock_providers): - response = self.client.get("/api/v1/providers", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == {"providers": [], "total_entries": 0} - - @mock.patch( - "airflow.providers_manager.ProvidersManager.providers", - new_callable=mock.PropertyMock, - return_value=MOCK_PROVIDERS, - ) - def test_response_200(self, mock_providers): - response = self.client.get("/api/v1/providers", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == { - "providers": [ - { - "description": "Amazon Web Services (AWS) https://aws.amazon.com/", - "package_name": "apache-airflow-providers-amazon", - "version": "1.0.0", - }, - { - "description": "Apache Cassandra http://cassandra.apache.org/", - "package_name": "apache-airflow-providers-apache-cassandra", - "version": "1.0.0", - }, - ], - "total_entries": 2, - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.get("/api/v1/providers") - assert response.status_code == 401 - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "/api/v1/providers", environ_overrides={"REMOTE_USER": "test_no_permissions"} - ) - assert response.status_code == 403 diff --git a/tests/api_connexion/endpoints/test_task_endpoint.py b/tests/api_connexion/endpoints/test_task_endpoint.py deleted file mode 100644 index f2d3ac70c22f2..0000000000000 --- a/tests/api_connexion/endpoints/test_task_endpoint.py +++ /dev/null @@ -1,589 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os -import unittest.mock -from datetime import datetime - -import pytest - -from airflow.dag_processing.bundles.manager import DagBundlesManager -from airflow.models import DagBag -from airflow.models.dag import DAG -from airflow.models.expandinput import EXPAND_INPUT_EMPTY -from airflow.models.serialized_dag import SerializedDagModel -from airflow.providers.standard.operators.empty import EmptyOperator - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.db import clear_db_dags, clear_db_runs, clear_db_serialized_dags - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestTaskEndpoint: - dag_id = "test_dag" - mapped_dag_id = "test_mapped_task" - unscheduled_dag_id = "test_unscheduled_dag" - task_id = "op1" - task_id2 = "op2" - task_id3 = "op3" - mapped_task_id = "mapped_task" - unscheduled_task_id1 = "unscheduled_task_1" - unscheduled_task_id2 = "unscheduled_task_2" - task1_start_date = datetime(2020, 6, 15) - task2_start_date = datetime(2020, 6, 16) - - @pytest.fixture(scope="class") - def setup_dag(self, configured_app): - with DAG(self.dag_id, schedule=None, start_date=self.task1_start_date, doc_md="details") as dag: - task1 = EmptyOperator(task_id=self.task_id, params={"foo": "bar"}) - task2 = EmptyOperator(task_id=self.task_id2, start_date=self.task2_start_date) - - with DAG(self.mapped_dag_id, schedule=None, start_date=self.task1_start_date) as mapped_dag: - EmptyOperator(task_id=self.task_id3) - # Use the private _expand() method to avoid the empty kwargs check. - # We don't care about how the operator runs here, only its presence. - EmptyOperator.partial(task_id=self.mapped_task_id)._expand(EXPAND_INPUT_EMPTY, strict=False) - - with DAG(self.unscheduled_dag_id, start_date=None, schedule=None) as unscheduled_dag: - task4 = EmptyOperator(task_id=self.unscheduled_task_id1, params={"is_unscheduled": True}) - task5 = EmptyOperator(task_id=self.unscheduled_task_id2, params={"is_unscheduled": True}) - task1 >> task2 - task4 >> task5 - - DagBundlesManager().sync_bundles_to_db() - dag_bag = DagBag(os.devnull, include_examples=False) - dag_bag.dags = { - dag.dag_id: dag, - mapped_dag.dag_id: mapped_dag, - unscheduled_dag.dag_id: unscheduled_dag, - } - dag_bag.sync_to_db("dags-folder", None) - configured_app.dag_bag = dag_bag # type:ignore - - @staticmethod - def clean_db(): - clear_db_runs() - clear_db_dags() - clear_db_serialized_dags() - - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app, setup_dag) -> None: - self.clean_db() - self.app = configured_app - self.client = self.app.test_client() # type:ignore - - def teardown_method(self) -> None: - self.clean_db() - - -class TestGetTask(TestTaskEndpoint): - def test_should_respond_200(self): - expected = { - "class_ref": { - "class_name": "EmptyOperator", - "module_path": "airflow.providers.standard.operators.empty", - }, - "depends_on_past": False, - "downstream_task_ids": [self.task_id2], - "end_date": None, - "execution_timeout": None, - "extra_links": [], - "operator_name": "EmptyOperator", - "owner": "airflow", - "params": { - "foo": { - "__class": "airflow.sdk.definitions.param.Param", - "value": "bar", - "description": None, - "schema": {}, - } - }, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "seconds": 300, "microseconds": 0}, - "retry_exponential_backoff": False, - "start_date": "2020-06-15T00:00:00+00:00", - "task_id": "op1", - "task_display_name": "op1", - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "is_mapped": False, - "doc_md": None, - } - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/tasks/{self.task_id}", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json == expected - - def test_mapped_task(self): - expected = { - "class_ref": { - "class_name": "EmptyOperator", - "module_path": "airflow.providers.standard.operators.empty", - }, - "depends_on_past": False, - "downstream_task_ids": [], - "end_date": None, - "execution_timeout": None, - "extra_links": [], - "is_mapped": True, - "operator_name": "EmptyOperator", - "owner": "airflow", - "params": {}, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "microseconds": 0, "seconds": 300}, - "retry_exponential_backoff": False, - "start_date": "2020-06-15T00:00:00+00:00", - "task_id": "mapped_task", - "task_display_name": "mapped_task", - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "doc_md": None, - } - response = self.client.get( - f"/api/v1/dags/{self.mapped_dag_id}/tasks/{self.mapped_task_id}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == expected - - def test_unscheduled_task(self): - expected = { - "class_ref": { - "class_name": "EmptyOperator", - "module_path": "airflow.providers.standard.operators.empty", - }, - "depends_on_past": False, - "downstream_task_ids": [], - "end_date": None, - "execution_timeout": None, - "extra_links": [], - "operator_name": "EmptyOperator", - "owner": "airflow", - "params": { - "is_unscheduled": { - "__class": "airflow.sdk.definitions.param.Param", - "value": True, - "description": None, - "schema": {}, - } - }, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "seconds": 300, "microseconds": 0}, - "retry_exponential_backoff": False, - "start_date": None, - "task_id": None, - "task_display_name": None, - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "is_mapped": False, - "doc_md": None, - } - downstream_dict = { - self.unscheduled_task_id1: self.unscheduled_task_id2, - self.unscheduled_task_id2: None, - } - for task_id, downstream_task_id in downstream_dict.items(): - response = self.client.get( - f"/api/v1/dags/{self.unscheduled_dag_id}/tasks/{task_id}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - expected["downstream_task_ids"] = [downstream_task_id] if downstream_task_id else [] - expected["task_id"] = task_id - expected["task_display_name"] = task_id - assert response.json == expected - - def test_should_respond_200_serialized(self, testing_dag_bundle): - # Get the dag out of the dagbag before we patch it to an empty one - dag = self.app.dag_bag.get_dag(self.dag_id) - dag.sync_to_db() - SerializedDagModel.write_dag(dag, bundle_name="test_bundle") - - dag_bag = DagBag(os.devnull, include_examples=False, read_dags_from_db=True) - patcher = unittest.mock.patch.object(self.app, "dag_bag", dag_bag) - patcher.start() - - expected = { - "class_ref": { - "class_name": "EmptyOperator", - "module_path": "airflow.providers.standard.operators.empty", - }, - "depends_on_past": False, - "downstream_task_ids": [self.task_id2], - "end_date": None, - "execution_timeout": None, - "extra_links": [], - "operator_name": "EmptyOperator", - "owner": "airflow", - "params": { - "foo": { - "__class": "airflow.sdk.definitions.param.Param", - "value": "bar", - "description": None, - "schema": {}, - } - }, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "seconds": 300, "microseconds": 0}, - "retry_exponential_backoff": False, - "start_date": "2020-06-15T00:00:00+00:00", - "task_id": "op1", - "task_display_name": "op1", - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "is_mapped": False, - "doc_md": None, - } - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/tasks/{self.task_id}", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json == expected - patcher.stop() - - def test_should_respond_404(self): - task_id = "xxxx_not_existing" - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/tasks/{task_id}", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - - def test_should_respond_404_when_dag_not_found(self): - dag_id = "xxxx_not_existing" - response = self.client.get( - f"/api/v1/dags/{dag_id}/tasks/{self.task_id}", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - assert response.json["title"] == "DAG not found" - - def test_should_raises_401_unauthenticated(self): - response = self.client.get(f"/api/v1/dags/{self.dag_id}/tasks/{self.task_id}") - - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/tasks", environ_overrides={"REMOTE_USER": "test_no_permissions"} - ) - assert response.status_code == 403 - - -class TestGetTasks(TestTaskEndpoint): - def test_should_respond_200(self): - expected = { - "tasks": [ - { - "class_ref": { - "class_name": "EmptyOperator", - "module_path": "airflow.providers.standard.operators.empty", - }, - "depends_on_past": False, - "downstream_task_ids": [self.task_id2], - "end_date": None, - "execution_timeout": None, - "extra_links": [], - "operator_name": "EmptyOperator", - "owner": "airflow", - "params": { - "foo": { - "__class": "airflow.sdk.definitions.param.Param", - "value": "bar", - "description": None, - "schema": {}, - } - }, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "seconds": 300, "microseconds": 0}, - "retry_exponential_backoff": False, - "start_date": "2020-06-15T00:00:00+00:00", - "task_id": "op1", - "task_display_name": "op1", - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "is_mapped": False, - "doc_md": None, - }, - { - "class_ref": { - "class_name": "EmptyOperator", - "module_path": "airflow.providers.standard.operators.empty", - }, - "depends_on_past": False, - "downstream_task_ids": [], - "end_date": None, - "execution_timeout": None, - "extra_links": [], - "operator_name": "EmptyOperator", - "owner": "airflow", - "params": {}, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "seconds": 300, "microseconds": 0}, - "retry_exponential_backoff": False, - "start_date": "2020-06-16T00:00:00+00:00", - "task_id": self.task_id2, - "task_display_name": self.task_id2, - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "is_mapped": False, - "doc_md": None, - }, - ], - "total_entries": 2, - } - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/tasks", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json == expected - - def test_get_tasks_mapped(self): - expected = { - "tasks": [ - { - "class_ref": { - "class_name": "EmptyOperator", - "module_path": "airflow.providers.standard.operators.empty", - }, - "depends_on_past": False, - "downstream_task_ids": [], - "end_date": None, - "execution_timeout": None, - "extra_links": [], - "is_mapped": True, - "operator_name": "EmptyOperator", - "owner": "airflow", - "params": {}, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "microseconds": 0, "seconds": 300}, - "retry_exponential_backoff": False, - "start_date": "2020-06-15T00:00:00+00:00", - "task_id": "mapped_task", - "task_display_name": "mapped_task", - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "doc_md": None, - }, - { - "class_ref": { - "class_name": "EmptyOperator", - "module_path": "airflow.providers.standard.operators.empty", - }, - "depends_on_past": False, - "downstream_task_ids": [], - "end_date": None, - "execution_timeout": None, - "extra_links": [], - "operator_name": "EmptyOperator", - "owner": "airflow", - "params": {}, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "seconds": 300, "microseconds": 0}, - "retry_exponential_backoff": False, - "start_date": "2020-06-15T00:00:00+00:00", - "task_id": self.task_id3, - "task_display_name": self.task_id3, - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "is_mapped": False, - "doc_md": None, - }, - ], - "total_entries": 2, - } - response = self.client.get( - f"/api/v1/dags/{self.mapped_dag_id}/tasks", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json == expected - - def test_get_unscheduled_tasks(self): - downstream_dict = { - self.unscheduled_task_id1: self.unscheduled_task_id2, - self.unscheduled_task_id2: None, - } - expected = { - "tasks": [ - { - "class_ref": { - "class_name": "EmptyOperator", - "module_path": "airflow.providers.standard.operators.empty", - }, - "depends_on_past": False, - "downstream_task_ids": [downstream_task_id] if downstream_task_id else [], - "end_date": None, - "execution_timeout": None, - "extra_links": [], - "operator_name": "EmptyOperator", - "owner": "airflow", - "params": { - "is_unscheduled": { - "__class": "airflow.sdk.definitions.param.Param", - "value": True, - "description": None, - "schema": {}, - } - }, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "seconds": 300, "microseconds": 0}, - "retry_exponential_backoff": False, - "start_date": None, - "task_id": task_id, - "task_display_name": task_id, - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "is_mapped": False, - "doc_md": None, - } - for (task_id, downstream_task_id) in downstream_dict.items() - ], - "total_entries": len(downstream_dict), - } - response = self.client.get( - f"/api/v1/dags/{self.unscheduled_dag_id}/tasks", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json == expected - - def test_should_respond_200_ascending_order_by_start_date(self): - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/tasks?order_by=start_date", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert self.task1_start_date < self.task2_start_date - assert response.json["tasks"][0]["task_id"] == self.task_id - assert response.json["tasks"][1]["task_id"] == self.task_id2 - - def test_should_respond_200_descending_order_by_start_date(self): - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/tasks?order_by=-start_date", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - # - means is descending - assert self.task1_start_date < self.task2_start_date - assert response.json["tasks"][0]["task_id"] == self.task_id2 - assert response.json["tasks"][1]["task_id"] == self.task_id - - def test_should_raise_400_for_invalid_order_by_name(self): - response = self.client.get( - f"/api/v1/dags/{self.dag_id}/tasks?order_by=invalid_task_colume_name", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json["detail"] == "'EmptyOperator' object has no attribute 'invalid_task_colume_name'" - - def test_should_respond_404(self): - dag_id = "xxxx_not_existing" - response = self.client.get(f"/api/v1/dags/{dag_id}/tasks", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 404 - - def test_should_raises_401_unauthenticated(self): - response = self.client.get(f"/api/v1/dags/{self.dag_id}/tasks") - - assert_401(response) diff --git a/tests/api_connexion/endpoints/test_task_instance_endpoint.py b/tests/api_connexion/endpoints/test_task_instance_endpoint.py deleted file mode 100644 index d1b298233e18e..0000000000000 --- a/tests/api_connexion/endpoints/test_task_instance_endpoint.py +++ /dev/null @@ -1,2921 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import datetime as dt -import urllib -from unittest import mock - -import pendulum -import pytest -from sqlalchemy import select -from sqlalchemy.orm import contains_eager - -from airflow.jobs.job import Job -from airflow.jobs.triggerer_job_runner import TriggererJobRunner -from airflow.models import DagRun, TaskInstance, Trigger -from airflow.models.renderedtifields import RenderedTaskInstanceFields as RTIF -from airflow.models.taskinstancehistory import TaskInstanceHistory -from airflow.utils.platform import getuser -from airflow.utils.session import provide_session -from airflow.utils.state import State -from airflow.utils.timezone import datetime -from airflow.utils.types import DagRunType - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.db import clear_db_runs, clear_rendered_ti_fields -from tests_common.test_utils.www import _check_last_log - -pytestmark = pytest.mark.db_test - -DEFAULT_DATETIME_1 = datetime(2020, 1, 1) -DEFAULT_DATETIME_STR_1 = "2020-01-01T00:00:00+00:00" -DEFAULT_DATETIME_STR_2 = "2020-01-02T00:00:00+00:00" - -QUOTED_DEFAULT_DATETIME_STR_1 = urllib.parse.quote(DEFAULT_DATETIME_STR_1) -QUOTED_DEFAULT_DATETIME_STR_2 = urllib.parse.quote(DEFAULT_DATETIME_STR_2) - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestTaskInstanceEndpoint: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app, dagbag) -> None: - self.default_time = DEFAULT_DATETIME_1 - self.ti_init = { - "logical_date": self.default_time, - "state": State.RUNNING, - } - self.ti_extras = { - "start_date": self.default_time + dt.timedelta(days=1), - "end_date": self.default_time + dt.timedelta(days=2), - "pid": 100, - "duration": 10000, - "pool": "default_pool", - "queue": "default_queue", - } - self.app = configured_app - self.client = self.app.test_client() # type:ignore - clear_db_runs() - clear_rendered_ti_fields() - self.dagbag = dagbag - - def create_task_instances( - self, - session, - dag_id: str = "example_python_operator", - update_extras: bool = True, - task_instances=None, - dag_run_state=State.RUNNING, - with_ti_history=False, - ): - """Method to create task instances using kwargs and default arguments""" - - dag = self.dagbag.get_dag(dag_id) - tasks = dag.tasks - counter = len(tasks) - if task_instances is not None: - counter = min(len(task_instances), counter) - - run_id = "TEST_DAG_RUN_ID" - logical_date = self.ti_init.pop("logical_date", self.default_time) - dr = None - - tis = [] - for i in range(counter): - if task_instances is None: - pass - elif update_extras: - self.ti_extras.update(task_instances[i]) - else: - self.ti_init.update(task_instances[i]) - - if "logical_date" in self.ti_init: - run_id = f"TEST_DAG_RUN_ID_{i}" - logical_date = self.ti_init.pop("logical_date") - dr = None - - if not dr: - dr = DagRun( - run_id=run_id, - dag_id=dag_id, - logical_date=logical_date, - run_type=DagRunType.MANUAL, - state=dag_run_state, - ) - session.add(dr) - ti = TaskInstance(task=tasks[i], **self.ti_init) - session.add(ti) - ti.dag_run = dr - ti.note = "placeholder-note" - - for key, value in self.ti_extras.items(): - setattr(ti, key, value) - tis.append(ti) - - session.commit() - if with_ti_history: - for ti in tis: - ti.try_number = 1 - session.merge(ti) - session.commit() - dag.clear() - for ti in tis: - ti.try_number = 2 - ti.queue = "default_queue" - session.merge(ti) - session.commit() - return tis - - -class TestGetTaskInstance(TestTaskInstanceEndpoint): - def setup_method(self): - clear_db_runs() - - def teardown_method(self): - clear_db_runs() - - @provide_session - def test_should_respond_200(self, session): - self.create_task_instances(session) - # Update ti and set operator to None to - # test that operator field is nullable. - # This prevents issue when users upgrade to 2.0+ - # from 1.10.x - # https://github.com/apache/airflow/issues/14421 - session.query(TaskInstance).update({TaskInstance.operator: None}, synchronize_session="fetch") - session.commit() - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "logical_date": "2020-01-01T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": -1, - "max_tries": 0, - "note": "placeholder-note", - "operator": None, - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "running", - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": 0, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - "rendered_fields": {}, - "rendered_map_index": None, - "trigger": None, - "triggerer_job": None, - } - - def test_should_respond_200_with_task_state_in_deferred(self, session): - now = pendulum.now("UTC") - ti = self.create_task_instances( - session, task_instances=[{"state": State.DEFERRED}], update_extras=True - )[0] - ti.trigger = Trigger("none", {}) - ti.trigger.created_date = now - ti.triggerer_job = Job() - TriggererJobRunner(job=ti.triggerer_job) - ti.triggerer_job.state = "running" - session.commit() - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context", - environ_overrides={"REMOTE_USER": "test"}, - ) - data = response.json - - # this logic in effect replicates mock.ANY for these values - values_to_ignore = { - "trigger": ["created_date", "id", "triggerer_id"], - "triggerer_job": ["executor_class", "hostname", "id", "latest_heartbeat", "start_date"], - } - for k, v in values_to_ignore.items(): - for elem in v: - del data[k][elem] - - assert response.status_code == 200 - assert data == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "logical_date": "2020-01-01T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": -1, - "max_tries": 0, - "note": "placeholder-note", - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "deferred", - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": 0, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - "rendered_fields": {}, - "rendered_map_index": None, - "trigger": { - "classpath": "none", - "kwargs": "{}", - }, - "triggerer_job": { - "dag_id": None, - "end_date": None, - "job_type": "TriggererJob", - "state": "running", - "unixname": getuser(), - }, - } - - def test_should_respond_200_with_task_state_in_removed(self, session): - self.create_task_instances(session, task_instances=[{"state": State.REMOVED}], update_extras=True) - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "logical_date": "2020-01-01T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": -1, - "max_tries": 0, - "note": "placeholder-note", - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "removed", - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": 0, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - "rendered_fields": {}, - "rendered_map_index": None, - "trigger": None, - "triggerer_job": None, - } - - def test_should_respond_200_task_instance_with_rendered(self, session): - tis = self.create_task_instances(session) - session.query() - rendered_fields = RTIF(tis[0], render_templates=False) - session.add(rendered_fields) - session.commit() - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - - assert response.json == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "logical_date": "2020-01-01T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": -1, - "max_tries": 0, - "note": "placeholder-note", - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "running", - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": 0, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - "rendered_fields": {"op_args": "()", "op_kwargs": {}, "templates_dict": None}, - "rendered_map_index": None, - "trigger": None, - "triggerer_job": None, - } - - def test_should_respond_200_mapped_task_instance_with_rtif(self, session): - """Verify we don't duplicate rows through join to RTIF""" - tis = self.create_task_instances(session) - old_ti = tis[0] - for idx in (1, 2): - ti = TaskInstance(task=old_ti.task, run_id=old_ti.run_id, map_index=idx) - ti.rendered_task_instance_fields = RTIF(ti, render_templates=False) - for attr in ["duration", "end_date", "pid", "start_date", "state", "queue", "note"]: - setattr(ti, attr, getattr(old_ti, attr)) - session.add(ti) - session.commit() - - # in each loop, we should get the right mapped TI back - for map_index in (1, 2): - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances" - f"/print_the_context/{map_index}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - - assert response.json == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "logical_date": "2020-01-01T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": map_index, - "max_tries": 0, - "note": "placeholder-note", - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "running", - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": 0, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - "rendered_fields": {"op_args": "()", "op_kwargs": {}, "templates_dict": None}, - "rendered_map_index": str(map_index), - "trigger": None, - "triggerer_job": None, - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context", - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - def test_raises_404_for_nonexistent_task_instance(self): - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/nonexistent_task", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json["title"] == "Task instance not found" - - def test_unmapped_map_index_should_return_404(self, session): - self.create_task_instances(session) - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/-1", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - - def test_should_return_404_for_mapped_endpoint(self, session): - self.create_task_instances(session) - for index in ["0", "1", "2"]: - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/" - f"taskInstances/print_the_context/{index}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - - def test_should_return_404_for_list_mapped_endpoint(self, session): - self.create_task_instances(session) - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/" - "taskInstances/print_the_context/listMapped", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - - -class TestGetTaskInstances(TestTaskInstanceEndpoint): - @pytest.mark.parametrize( - "task_instances, update_extras, url, expected_ti", - [ - pytest.param( - [ - {"logical_date": DEFAULT_DATETIME_1}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1)}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2)}, - ], - False, - ( - "/api/v1/dags/example_python_operator/dagRuns/~/" - f"taskInstances?execution_date_lte={QUOTED_DEFAULT_DATETIME_STR_1}" - ), - 1, - id="test logical date filter", - ), - pytest.param( - [ - {"start_date": DEFAULT_DATETIME_1}, - {"start_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1)}, - {"start_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2)}, - ], - True, - ( - "/api/v1/dags/example_python_operator/dagRuns/~/taskInstances" - f"?start_date_gte={QUOTED_DEFAULT_DATETIME_STR_1}&" - f"start_date_lte={QUOTED_DEFAULT_DATETIME_STR_2}" - ), - 2, - id="test start date filter", - ), - pytest.param( - [ - {"end_date": DEFAULT_DATETIME_1}, - {"end_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1)}, - {"end_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2)}, - ], - True, - ( - "/api/v1/dags/example_python_operator/dagRuns/~/taskInstances?" - f"end_date_gte={QUOTED_DEFAULT_DATETIME_STR_1}&" - f"end_date_lte={QUOTED_DEFAULT_DATETIME_STR_2}" - ), - 2, - id="test end date filter", - ), - pytest.param( - [ - {"duration": 100}, - {"duration": 150}, - {"duration": 200}, - ], - True, - ( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/" - "taskInstances?duration_gte=100&duration_lte=200" - ), - 3, - id="test duration filter", - ), - pytest.param( - [ - {"duration": 100}, - {"duration": 150}, - {"duration": 200}, - ], - True, - "/api/v1/dags/~/dagRuns/~/taskInstances?duration_gte=100&duration_lte=200", - 3, - id="test duration filter ~", - ), - pytest.param( - [ - {"state": State.RUNNING}, - {"state": State.QUEUED}, - {"state": State.SUCCESS}, - {"state": State.NONE}, - ], - False, - ( - "/api/v1/dags/example_python_operator/dagRuns/" - "TEST_DAG_RUN_ID/taskInstances?state=running,queued,none" - ), - 3, - id="test state filter", - ), - pytest.param( - [ - {"state": State.NONE}, - {"state": State.NONE}, - {"state": State.NONE}, - {"state": State.NONE}, - ], - False, - ("/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances"), - 4, - id="test null states with no filter", - ), - pytest.param( - [ - {"pool": "test_pool_1"}, - {"pool": "test_pool_2"}, - {"pool": "test_pool_3"}, - ], - True, - ( - "/api/v1/dags/example_python_operator/dagRuns/" - "TEST_DAG_RUN_ID/taskInstances?pool=test_pool_1,test_pool_2" - ), - 2, - id="test pool filter", - ), - pytest.param( - [ - {"pool": "test_pool_1"}, - {"pool": "test_pool_2"}, - {"pool": "test_pool_3"}, - ], - True, - "/api/v1/dags/~/dagRuns/~/taskInstances?pool=test_pool_1,test_pool_2", - 2, - id="test pool filter ~", - ), - pytest.param( - [ - {"queue": "test_queue_1"}, - {"queue": "test_queue_2"}, - {"queue": "test_queue_3"}, - ], - True, - ( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID" - "/taskInstances?queue=test_queue_1,test_queue_2" - ), - 2, - id="test queue filter", - ), - pytest.param( - [ - {"queue": "test_queue_1"}, - {"queue": "test_queue_2"}, - {"queue": "test_queue_3"}, - ], - True, - "/api/v1/dags/~/dagRuns/~/taskInstances?queue=test_queue_1,test_queue_2", - 2, - id="test queue filter ~", - ), - pytest.param( - [ - {"executor": "test_exec_1"}, - {"executor": "test_exec_2"}, - {"executor": "test_exec_3"}, - ], - True, - ( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID" - "/taskInstances?executor=test_exec_1,test_exec_2" - ), - 2, - id="test_executor_filter", - ), - pytest.param( - [ - {"executor": "test_exec_1"}, - {"executor": "test_exec_2"}, - {"executor": "test_exec_3"}, - ], - True, - "/api/v1/dags/~/dagRuns/~/taskInstances?executor=test_exec_1,test_exec_2", - 2, - id="test executor filter ~", - ), - ], - ) - def test_should_respond_200(self, task_instances, update_extras, url, expected_ti, session): - self.create_task_instances( - session, - update_extras=update_extras, - task_instances=task_instances, - ) - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json["total_entries"] == expected_ti - assert len(response.json["task_instances"]) == expected_ti - - def test_return_TI_only_from_readable_dags(self, session): - task_instances = { - "example_python_operator": 1, - "example_skip_dag": 2, - } - for dag_id in task_instances: - self.create_task_instances( - session, - task_instances=[ - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=i)} - for i in range(task_instances[dag_id]) - ], - dag_id=dag_id, - ) - response = self.client.get( - "/api/v1/dags/~/dagRuns/~/taskInstances", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 3 - assert len(response.json["task_instances"]) == 3 - - def test_should_respond_200_for_dag_id_filter(self, session): - self.create_task_instances(session) - self.create_task_instances(session, dag_id="example_skip_dag") - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/~/taskInstances", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - count = session.query(TaskInstance).filter(TaskInstance.dag_id == "example_python_operator").count() - assert count == response.json["total_entries"] - assert count == len(response.json["task_instances"]) - - def test_should_respond_200_for_order_by(self, session): - dag_id = "example_python_operator" - self.create_task_instances( - session, - task_instances=[ - {"start_date": DEFAULT_DATETIME_1 + dt.timedelta(minutes=(i + 1))} for i in range(10) - ], - dag_id=dag_id, - ) - - ti_count = session.query(TaskInstance).filter(TaskInstance.dag_id == dag_id).count() - - # Ascending order - response_asc = self.client.get( - "/api/v1/dags/~/dagRuns/~/taskInstances?order_by=start_date", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response_asc.status_code == 200 - assert response_asc.json["total_entries"] == ti_count - assert len(response_asc.json["task_instances"]) == ti_count - - # Descending order - response_desc = self.client.get( - "/api/v1/dags/~/dagRuns/~/taskInstances?order_by=-start_date", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response_desc.status_code == 200 - assert response_desc.json["total_entries"] == ti_count - assert len(response_desc.json["task_instances"]) == ti_count - - # Compare - start_dates_asc = [ti["start_date"] for ti in response_asc.json["task_instances"]] - assert len(start_dates_asc) == ti_count - start_dates_desc = [ti["start_date"] for ti in response_desc.json["task_instances"]] - assert len(start_dates_desc) == ti_count - assert start_dates_asc == list(reversed(start_dates_desc)) - - def test_should_respond_200_for_pagination(self, session): - dag_id = "example_python_operator" - self.create_task_instances( - session, - task_instances=[ - {"start_date": DEFAULT_DATETIME_1 + dt.timedelta(minutes=(i + 1))} for i in range(10) - ], - dag_id=dag_id, - ) - - # First 5 items - response_batch1 = self.client.get( - "/api/v1/dags/~/dagRuns/~/taskInstances?limit=5&offset=0", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response_batch1.status_code == 200, response_batch1.json - num_entries_batch1 = len(response_batch1.json["task_instances"]) - assert num_entries_batch1 == 5 - assert len(response_batch1.json["task_instances"]) == 5 - - # 5 items after that - response_batch2 = self.client.get( - "/api/v1/dags/~/dagRuns/~/taskInstances?limit=5&offset=5", - environ_overrides={"REMOTE_USER": "test"}, - json={"limit": 5, "offset": 0, "dag_ids": [dag_id]}, - ) - assert response_batch2.status_code == 200, response_batch2.json - num_entries_batch2 = len(response_batch2.json["task_instances"]) - assert num_entries_batch2 > 0 - assert len(response_batch2.json["task_instances"]) > 0 - - # Match - ti_count = session.query(TaskInstance).filter(TaskInstance.dag_id == dag_id).count() - assert response_batch1.json["total_entries"] == response_batch2.json["total_entries"] == ti_count - assert (num_entries_batch1 + num_entries_batch2) == ti_count - assert response_batch1 != response_batch2 - - def test_should_raises_401_unauthenticated(self): - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/~/taskInstances", - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/~/taskInstances", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - -class TestGetTaskInstancesBatch(TestTaskInstanceEndpoint): - @pytest.mark.parametrize( - "task_instances, update_extras, payload, expected_ti_count, username", - [ - pytest.param( - [ - {"queue": "test_queue_1"}, - {"queue": "test_queue_2"}, - {"queue": "test_queue_3"}, - ], - True, - {"queue": ["test_queue_1", "test_queue_2"]}, - 2, - "test", - id="test queue filter", - ), - pytest.param( - [ - {"executor": "test_exec_1"}, - {"executor": "test_exec_2"}, - {"executor": "test_exec_3"}, - ], - True, - {"executor": ["test_exec_1", "test_exec_2"]}, - 2, - "test", - id="test executor filter", - ), - pytest.param( - [ - {"duration": 100}, - {"duration": 150}, - {"duration": 200}, - ], - True, - {"duration_gte": 100, "duration_lte": 200}, - 3, - "test", - id="test duration filter", - ), - pytest.param( - [ - {"logical_date": DEFAULT_DATETIME_1}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1)}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2)}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=3)}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=4)}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=5)}, - ], - False, - { - "execution_date_gte": DEFAULT_DATETIME_1.isoformat(), - "execution_date_lte": (DEFAULT_DATETIME_1 + dt.timedelta(days=2)).isoformat(), - }, - 3, - "test", - id="with logical date filter", - ), - pytest.param( - [ - {"logical_date": DEFAULT_DATETIME_1}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1)}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2)}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=3)}, - ], - False, - { - "dag_run_ids": ["TEST_DAG_RUN_ID_0", "TEST_DAG_RUN_ID_1"], - }, - 2, - "test", - id="test dag run id filter", - ), - pytest.param( - [ - {"logical_date": DEFAULT_DATETIME_1}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1)}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2)}, - {"logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=3)}, - ], - False, - { - "task_ids": ["print_the_context", "log_sql_query"], - }, - 2, - "test", - id="test task id filter", - ), - ], - ) - def test_should_respond_200( - self, task_instances, update_extras, payload, expected_ti_count, username, session - ): - self.create_task_instances( - session, - update_extras=update_extras, - task_instances=task_instances, - ) - response = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": username}, - json=payload, - ) - assert response.status_code == 200, response.json - assert expected_ti_count == response.json["total_entries"] - assert expected_ti_count == len(response.json["task_instances"]) - - def test_should_respond_200_for_order_by(self, session): - dag_id = "example_python_operator" - self.create_task_instances( - session, - task_instances=[ - {"start_date": DEFAULT_DATETIME_1 + dt.timedelta(minutes=(i + 1))} for i in range(10) - ], - dag_id=dag_id, - ) - - ti_count = session.query(TaskInstance).filter(TaskInstance.dag_id == dag_id).count() - - # Ascending order - response_asc = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test"}, - json={"order_by": "start_date", "dag_ids": [dag_id]}, - ) - assert response_asc.status_code == 200, response_asc.json - assert response_asc.json["total_entries"] == ti_count - assert len(response_asc.json["task_instances"]) == ti_count - - # Descending order - response_desc = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test"}, - json={"order_by": "-start_date", "dag_ids": [dag_id]}, - ) - assert response_desc.status_code == 200, response_desc.json - assert response_desc.json["total_entries"] == ti_count - assert len(response_desc.json["task_instances"]) == ti_count - - # Compare - start_dates_asc = [ti["start_date"] for ti in response_asc.json["task_instances"]] - assert len(start_dates_asc) == ti_count - start_dates_desc = [ti["start_date"] for ti in response_desc.json["task_instances"]] - assert len(start_dates_desc) == ti_count - assert start_dates_asc == list(reversed(start_dates_desc)) - - @pytest.mark.parametrize( - "task_instances, payload, expected_ti_count", - [ - pytest.param( - [ - {"task": "test_1"}, - {"task": "test_2"}, - ], - {"dag_ids": ["latest_only"]}, - 2, - id="task_instance properties", - ), - ], - ) - @provide_session - def test_should_respond_200_when_task_instance_properties_are_none( - self, task_instances, payload, expected_ti_count, session - ): - self.ti_extras.update( - { - "start_date": None, - "end_date": None, - "state": None, - } - ) - self.create_task_instances( - session, - dag_id="latest_only", - task_instances=task_instances, - ) - response = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == 200, response.json - assert expected_ti_count == response.json["total_entries"] - assert expected_ti_count == len(response.json["task_instances"]) - - @pytest.mark.parametrize( - "payload, expected_ti, total_ti", - [ - pytest.param( - {"dag_ids": ["example_python_operator", "example_skip_dag"]}, - 17, - 17, - id="with dag filter", - ), - ], - ) - @provide_session - def test_should_respond_200_dag_ids_filter(self, payload, expected_ti, total_ti, session): - self.create_task_instances(session) - self.create_task_instances(session, dag_id="example_skip_dag") - response = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == 200 - assert len(response.json["task_instances"]) == expected_ti - assert response.json["total_entries"] == total_ti - - def test_should_raises_401_unauthenticated(self): - response = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - json={"dag_ids": ["example_python_operator", "example_skip_dag"]}, - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - json={"dag_ids": ["example_python_operator", "example_skip_dag"]}, - ) - assert response.status_code == 403 - - def test_should_raise_400_for_no_json(self): - response = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json["detail"] == "Request body must not be empty" - - def test_should_raise_400_for_unknown_fields(self): - response = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test"}, - json={"unknown_field": "unknown_value"}, - ) - assert response.status_code == 400 - assert response.json["detail"] == "{'unknown_field': ['Unknown field.']}" - - @pytest.mark.parametrize( - "payload, expected", - [ - ({"end_date_lte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), - ({"end_date_gte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), - ({"start_date_lte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), - ({"start_date_gte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), - ({"execution_date_gte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), - ({"execution_date_lte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), - ], - ) - @provide_session - def test_should_raise_400_for_naive_and_bad_datetime(self, payload, expected, session): - self.create_task_instances(session) - response = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == 400 - assert expected in response.json["detail"] - - def test_should_respond_200_for_pagination(self, session): - dag_id = "example_python_operator" - - self.create_task_instances( - session, - task_instances=[ - {"start_date": DEFAULT_DATETIME_1 + dt.timedelta(minutes=(i + 1))} for i in range(10) - ], - dag_id=dag_id, - ) - - # First 5 items - response_batch1 = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test"}, - json={"page_limit": 5, "page_offset": 0}, - ) - assert response_batch1.status_code == 200, response_batch1.json - num_entries_batch1 = len(response_batch1.json["task_instances"]) - assert num_entries_batch1 == 5 - assert len(response_batch1.json["task_instances"]) == 5 - - # 5 items after that - response_batch2 = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test"}, - json={"page_limit": 5, "page_offset": 5}, - ) - assert response_batch2.status_code == 200, response_batch2.json - num_entries_batch2 = len(response_batch2.json["task_instances"]) - assert num_entries_batch2 > 0 - assert len(response_batch2.json["task_instances"]) > 0 - - # Match - ti_count = 9 - assert response_batch1.json["total_entries"] == response_batch2.json["total_entries"] == ti_count - assert (num_entries_batch1 + num_entries_batch2) == ti_count - assert response_batch1 != response_batch2 - - # default limit and offset - response_batch3 = self.client.post( - "/api/v1/dags/~/dagRuns/~/taskInstances/list", - environ_overrides={"REMOTE_USER": "test"}, - json={}, - ) - - num_entries_batch3 = len(response_batch3.json["task_instances"]) - assert num_entries_batch3 == ti_count - assert len(response_batch3.json["task_instances"]) == ti_count - - -class TestPostClearTaskInstances(TestTaskInstanceEndpoint): - @pytest.mark.parametrize( - "main_dag, task_instances, request_dag, payload, expected_ti", - [ - pytest.param( - "example_python_operator", - [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.FAILED}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.FAILED, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.FAILED, - }, - ], - "example_python_operator", - { - "dry_run": True, - "start_date": DEFAULT_DATETIME_STR_2, - "only_failed": True, - }, - 2, - id="clear start date filter", - ), - pytest.param( - "example_python_operator", - [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.FAILED}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.FAILED, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.FAILED, - }, - ], - "example_python_operator", - { - "dry_run": True, - "end_date": DEFAULT_DATETIME_STR_2, - "only_failed": True, - }, - 2, - id="clear end date filter", - ), - pytest.param( - "example_python_operator", - [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.RUNNING}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.FAILED, - }, - ], - "example_python_operator", - {"dry_run": True, "only_running": True, "only_failed": False}, - 2, - id="clear only running", - ), - pytest.param( - "example_python_operator", - [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.FAILED}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.FAILED, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.RUNNING, - }, - ], - "example_python_operator", - { - "dry_run": True, - "only_failed": True, - }, - 2, - id="clear only failed", - ), - pytest.param( - "example_python_operator", - [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.FAILED}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.FAILED, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.FAILED, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=3), - "state": State.FAILED, - }, - ], - "example_python_operator", - { - "dry_run": True, - "task_ids": ["print_the_context", "sleep_for_1"], - }, - 2, - id="clear by task ids", - ), - pytest.param( - "example_python_operator", - [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.FAILED}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.FAILED, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.RUNNING, - }, - ], - "example_python_operator", - { - "only_failed": True, - }, - 2, - id="dry_run default", - ), - ], - ) - @provide_session - def test_should_respond_200(self, main_dag, task_instances, request_dag, payload, expected_ti, session): - self.create_task_instances( - session, - dag_id=main_dag, - task_instances=task_instances, - update_extras=False, - ) - self.app.dag_bag.sync_to_db("dags-folder", None) - response = self.client.post( - f"/api/v1/dags/{request_dag}/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == 200 - assert len(response.json["task_instances"]) == expected_ti - _check_last_log( - session, - dag_id=request_dag, - event="api.post_clear_task_instances", - logical_date=None, - expected_extra=payload, - ) - - @mock.patch("airflow.api_connexion.endpoints.task_instance_endpoint.clear_task_instances") - def test_clear_taskinstance_is_called_with_queued_dr_state(self, mock_clearti, session): - """Test that if reset_dag_runs is True, then clear_task_instances is called with State.QUEUED""" - self.create_task_instances(session) - dag_id = "example_python_operator" - payload = {"reset_dag_runs": True, "dry_run": False} - self.app.dag_bag.sync_to_db("dags-folder", None) - response = self.client.post( - f"/api/v1/dags/{dag_id}/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == 200 - mock_clearti.assert_called_once_with( - [], session, dag=self.app.dag_bag.get_dag(dag_id), dag_run_state=State.QUEUED - ) - _check_last_log(session, dag_id=dag_id, event="api.post_clear_task_instances", logical_date=None) - - def test_clear_taskinstance_is_called_with_invalid_task_ids(self, session): - """Test that dagrun is running when invalid task_ids are passed to clearTaskInstances API.""" - dag_id = "example_python_operator" - tis = self.create_task_instances(session) - dagrun = tis[0].get_dagrun() - assert dagrun.state == "running" - - payload = {"dry_run": False, "reset_dag_runs": True, "task_ids": [""]} - self.app.dag_bag.sync_to_db("dags-folder", None) - response = self.client.post( - f"/api/v1/dags/{dag_id}/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == 200 - - dagrun.refresh_from_db() - assert dagrun.state == "running" - assert all(ti.state == "running" for ti in tis) - _check_last_log(session, dag_id=dag_id, event="api.post_clear_task_instances", logical_date=None) - - def test_should_respond_200_with_reset_dag_run(self, session): - dag_id = "example_python_operator" - payload = { - "dry_run": False, - "reset_dag_runs": True, - "only_failed": False, - "only_running": True, - } - task_instances = [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.RUNNING}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=3), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=4), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=5), - "state": State.RUNNING, - }, - ] - - self.create_task_instances( - session, - dag_id=dag_id, - task_instances=task_instances, - update_extras=False, - dag_run_state=State.FAILED, - ) - response = self.client.post( - f"/api/v1/dags/{dag_id}/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - - failed_dag_runs = session.query(DagRun).filter(DagRun.state == "failed").count() - assert response.status_code == 200 - expected_response = [ - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_0", - "logical_date": "2020-01-01T00:00:00+00:00", - "task_id": "print_the_context", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_1", - "logical_date": "2020-01-02T00:00:00+00:00", - "task_id": "log_sql_query", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_2", - "logical_date": "2020-01-03T00:00:00+00:00", - "task_id": "sleep_for_0", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_3", - "logical_date": "2020-01-04T00:00:00+00:00", - "task_id": "sleep_for_1", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_4", - "logical_date": "2020-01-05T00:00:00+00:00", - "task_id": "sleep_for_2", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_5", - "logical_date": "2020-01-06T00:00:00+00:00", - "task_id": "sleep_for_3", - }, - ] - for task_instance in expected_response: - assert task_instance in response.json["task_instances"] - assert len(response.json["task_instances"]) == 6 - assert failed_dag_runs == 0 - _check_last_log(session, dag_id=dag_id, event="api.post_clear_task_instances", logical_date=None) - - def test_should_respond_200_with_dag_run_id(self, session): - dag_id = "example_python_operator" - payload = { - "dry_run": False, - "reset_dag_runs": False, - "only_failed": False, - "only_running": True, - "dag_run_id": "TEST_DAG_RUN_ID_0", - } - task_instances = [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.RUNNING}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=3), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=4), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=5), - "state": State.RUNNING, - }, - ] - - self.create_task_instances( - session, - dag_id=dag_id, - task_instances=task_instances, - update_extras=False, - dag_run_state=State.FAILED, - ) - response = self.client.post( - f"/api/v1/dags/{dag_id}/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == 200 - expected_response = [ - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_0", - "logical_date": "2020-01-01T00:00:00+00:00", - "task_id": "print_the_context", - }, - ] - assert response.json["task_instances"] == expected_response - assert len(response.json["task_instances"]) == 1 - _check_last_log(session, dag_id=dag_id, event="api.post_clear_task_instances", logical_date=None) - - def test_should_respond_200_with_include_past(self, session): - dag_id = "example_python_operator" - payload = { - "dry_run": False, - "reset_dag_runs": False, - "only_failed": False, - "include_past": True, - "only_running": True, - } - task_instances = [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.RUNNING}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=3), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=4), - "state": State.RUNNING, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=5), - "state": State.RUNNING, - }, - ] - - self.create_task_instances( - session, - dag_id=dag_id, - task_instances=task_instances, - update_extras=False, - dag_run_state=State.FAILED, - ) - response = self.client.post( - f"/api/v1/dags/{dag_id}/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == 200 - expected_response = [ - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_0", - "logical_date": "2020-01-01T00:00:00+00:00", - "task_id": "print_the_context", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_1", - "logical_date": "2020-01-02T00:00:00+00:00", - "task_id": "log_sql_query", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_2", - "logical_date": "2020-01-03T00:00:00+00:00", - "task_id": "sleep_for_0", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_3", - "logical_date": "2020-01-04T00:00:00+00:00", - "task_id": "sleep_for_1", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_4", - "logical_date": "2020-01-05T00:00:00+00:00", - "task_id": "sleep_for_2", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_5", - "logical_date": "2020-01-06T00:00:00+00:00", - "task_id": "sleep_for_3", - }, - ] - for task_instance in expected_response: - assert task_instance in response.json["task_instances"] - assert len(response.json["task_instances"]) == 6 - _check_last_log(session, dag_id=dag_id, event="api.post_clear_task_instances", logical_date=None) - - def test_should_respond_200_with_include_future(self, session): - dag_id = "example_python_operator" - payload = { - "dry_run": False, - "reset_dag_runs": False, - "only_failed": False, - "include_future": True, - "only_running": False, - } - task_instances = [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.SUCCESS}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.SUCCESS, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.SUCCESS, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=3), - "state": State.SUCCESS, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=4), - "state": State.SUCCESS, - }, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=5), - "state": State.SUCCESS, - }, - ] - - self.create_task_instances( - session, - dag_id=dag_id, - task_instances=task_instances, - update_extras=False, - dag_run_state=State.FAILED, - ) - response = self.client.post( - f"/api/v1/dags/{dag_id}/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - - assert response.status_code == 200 - expected_response = [ - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_0", - "logical_date": "2020-01-01T00:00:00+00:00", - "task_id": "print_the_context", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_1", - "logical_date": "2020-01-02T00:00:00+00:00", - "task_id": "log_sql_query", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_2", - "logical_date": "2020-01-03T00:00:00+00:00", - "task_id": "sleep_for_0", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_3", - "logical_date": "2020-01-04T00:00:00+00:00", - "task_id": "sleep_for_1", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_4", - "logical_date": "2020-01-05T00:00:00+00:00", - "task_id": "sleep_for_2", - }, - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID_5", - "logical_date": "2020-01-06T00:00:00+00:00", - "task_id": "sleep_for_3", - }, - ] - for task_instance in expected_response: - assert task_instance in response.json["task_instances"] - assert len(response.json["task_instances"]) == 6 - _check_last_log(session, dag_id=dag_id, event="api.post_clear_task_instances", logical_date=None) - - def test_should_respond_404_for_nonexistent_dagrun_id(self, session): - dag_id = "example_python_operator" - payload = { - "dry_run": False, - "reset_dag_runs": False, - "only_failed": False, - "only_running": True, - "dag_run_id": "TEST_DAG_RUN_ID_100", - } - task_instances = [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.RUNNING}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.RUNNING, - }, - ] - - self.create_task_instances( - session, - dag_id=dag_id, - task_instances=task_instances, - update_extras=False, - dag_run_state=State.FAILED, - ) - response = self.client.post( - f"/api/v1/dags/{dag_id}/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - - assert response.status_code == 404 - assert ( - response.json["title"] - == "Dag Run id TEST_DAG_RUN_ID_100 not found in dag example_python_operator" - ) - _check_last_log(session, dag_id=dag_id, event="api.post_clear_task_instances", logical_date=None) - - def test_should_raises_401_unauthenticated(self): - response = self.client.post( - "/api/v1/dags/example_python_operator/clearTaskInstances", - json={ - "dry_run": False, - "reset_dag_runs": True, - "only_failed": False, - "only_running": True, - }, - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.post( - "/api/v1/dags/example_python_operator/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - json={ - "dry_run": False, - "reset_dag_runs": True, - "only_failed": False, - "only_running": True, - }, - ) - assert response.status_code == 403 - - @pytest.mark.parametrize( - "payload, expected", - [ - ({"end_date": "2020-11-10T12:42:39.442973"}, "Naive datetime is disallowed"), - ({"end_date": "2020-11-10T12:4po"}, "{'end_date': ['Not a valid datetime.']}"), - ({"start_date": "2020-11-10T12:42:39.442973"}, "Naive datetime is disallowed"), - ({"start_date": "2020-11-10T12:4po"}, "{'start_date': ['Not a valid datetime.']}"), - ], - ) - @provide_session - def test_should_raise_400_for_naive_and_bad_datetime(self, payload, expected, session): - task_instances = [ - {"logical_date": DEFAULT_DATETIME_1, "state": State.RUNNING}, - { - "logical_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.RUNNING, - }, - ] - self.create_task_instances( - session, - dag_id="example_python_operator", - task_instances=task_instances, - update_extras=False, - ) - self.app.dag_bag.sync_to_db("dags-folder", None) - response = self.client.post( - "/api/v1/dags/example_python_operator/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == 400 - assert response.json["detail"] == expected - - def test_raises_404_for_non_existent_dag(self): - response = self.client.post( - "/api/v1/dags/non-existent-dag/clearTaskInstances", - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": False, - "reset_dag_runs": True, - "only_failed": False, - "only_running": True, - }, - ) - assert response.status_code == 404 - assert response.json["title"] == "Dag id non-existent-dag not found" - - -class TestPostSetTaskInstanceState(TestTaskInstanceEndpoint): - @mock.patch("airflow.models.dag.DAG.set_task_instance_state") - def test_should_assert_call_mocked_api(self, mock_set_task_instance_state, session): - self.create_task_instances(session) - run_id = "TEST_DAG_RUN_ID" - mock_set_task_instance_state.return_value = ( - session.query(TaskInstance) - .join(TaskInstance.dag_run) - .options(contains_eager(TaskInstance.dag_run)) - .filter(TaskInstance.task_id == "print_the_context") - .all() - ) - response = self.client.post( - "/api/v1/dags/example_python_operator/updateTaskInstancesState", - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": True, - "task_id": "print_the_context", - "dag_run_id": run_id, - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ) - assert response.status_code == 200 - assert response.json == { - "task_instances": [ - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID", - "logical_date": "2020-01-01T00:00:00+00:00", - "task_id": "print_the_context", - } - ] - } - - mock_set_task_instance_state.assert_called_once_with( - commit=False, - downstream=True, - run_id=run_id, - future=True, - past=True, - state="failed", - task_id="print_the_context", - upstream=True, - session=session, - ) - - @mock.patch("airflow.models.dag.DAG.set_task_instance_state") - def test_should_assert_call_mocked_api_when_run_id(self, mock_set_task_instance_state, session): - self.create_task_instances(session) - run_id = "TEST_DAG_RUN_ID" - mock_set_task_instance_state.return_value = ( - session.query(TaskInstance) - .join(TaskInstance.dag_run) - .filter(TaskInstance.task_id == "print_the_context") - .all() - ) - response = self.client.post( - "/api/v1/dags/example_python_operator/updateTaskInstancesState", - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": True, - "task_id": "print_the_context", - "dag_run_id": run_id, - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ) - assert response.status_code == 200 - assert response.json == { - "task_instances": [ - { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID", - "logical_date": "2020-01-01T00:00:00+00:00", - "task_id": "print_the_context", - } - ] - } - - mock_set_task_instance_state.assert_called_once_with( - commit=False, - downstream=True, - run_id=run_id, - future=True, - past=True, - state="failed", - task_id="print_the_context", - upstream=True, - session=session, - ) - - @pytest.mark.parametrize( - "error, code, payload", - [ - [ - "{'dag_run_id': ['Missing data for required field.']}", - 400, - { - "dry_run": True, - "task_id": "print_the_context", - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ], - [ - "{'dag_run_id': ['Missing data for required field.'], 'logical_date': ['Unknown field.']}", - 400, - { - "dry_run": True, - "task_id": "print_the_context", - "logical_date": "2021-01-01T00:00:00+00:00", - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ], - [ - "Task instance not found for task 'print_the_context' on DAG run with ID 'TEST_DAG_RUN_'", - 404, - { - "dry_run": True, - "task_id": "print_the_context", - "dag_run_id": "TEST_DAG_RUN_", - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ], - [ - "{'logical_date': ['Unknown field.']}", - 400, - { - "dry_run": True, - "task_id": "print_the_context", - "dag_run_id": "TEST_DAG_RUN_", - "logical_date": "2020-01-01T00:00:00+00:00", - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ], - ], - ) - def test_should_handle_errors(self, error, code, payload, session): - self.create_task_instances(session) - response = self.client.post( - "/api/v1/dags/example_python_operator/updateTaskInstancesState", - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == code - assert response.json["detail"] == error - - def test_should_raises_401_unauthenticated(self): - response = self.client.post( - "/api/v1/dags/example_python_operator/updateTaskInstancesState", - json={ - "dry_run": True, - "task_id": "print_the_context", - "logical_date": DEFAULT_DATETIME_1.isoformat(), - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.post( - "/api/v1/dags/example_python_operator/updateTaskInstancesState", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - json={ - "dry_run": True, - "task_id": "print_the_context", - "logical_date": DEFAULT_DATETIME_1.isoformat(), - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ) - assert response.status_code == 403 - - def test_should_raise_404_not_found_dag(self): - response = self.client.post( - "/api/v1/dags/INVALID_DAG/updateTaskInstancesState", - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": True, - "task_id": "print_the_context", - "dag_run_id": "random_run_id", - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ) - assert response.status_code == 404 - - @mock.patch("airflow.models.dag.DAG.set_task_instance_state") - def test_should_raise_not_found_if_run_id_is_wrong(self, mock_set_task_instance_state, session): - self.create_task_instances(session) - run_id = "random_run_id" - response = self.client.post( - "/api/v1/dags/example_python_operator/updateTaskInstancesState", - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": True, - "task_id": "print_the_context", - "dag_run_id": run_id, - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ) - assert response.status_code == 404 - assert response.json["detail"] == ( - f"Task instance not found for task 'print_the_context' on DAG run with ID '{run_id}'" - ) - assert mock_set_task_instance_state.call_count == 0 - - def test_should_raise_404_not_found_task(self): - response = self.client.post( - "/api/v1/dags/example_python_operator/updateTaskInstancesState", - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": True, - "task_id": "INVALID_TASK", - "dag_run_id": "TEST_DAG_RUN_ID", - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - }, - ) - assert response.status_code == 404 - - -class TestPatchTaskInstance(TestTaskInstanceEndpoint): - ENDPOINT_URL = ( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context" - ) - - @mock.patch("airflow.models.dag.DAG.set_task_instance_state") - def test_should_call_mocked_api(self, mock_set_task_instance_state, session): - self.create_task_instances(session) - - NEW_STATE = "failed" - mock_set_task_instance_state.return_value = session.scalars( - select(TaskInstance).where( - TaskInstance.dag_id == "example_python_operator", - TaskInstance.task_id == "print_the_context", - TaskInstance.run_id == "TEST_DAG_RUN_ID", - TaskInstance.map_index == -1, - ) - ).one_or_none() - - response = self.client.patch( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": False, - "new_state": NEW_STATE, - }, - ) - assert response.status_code == 200 - assert response.json == { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID", - "logical_date": "2020-01-01T00:00:00+00:00", - "task_id": "print_the_context", - } - - mock_set_task_instance_state.assert_called_once_with( - task_id="print_the_context", - run_id="TEST_DAG_RUN_ID", - map_indexes=[-1], - state=NEW_STATE, - commit=True, - session=session, - ) - _check_last_log( - session, - dag_id="example_python_operator", - event="api.post_set_task_instances_state", - logical_date=None, - ) - - @mock.patch("airflow.models.dag.DAG.set_task_instance_state") - def test_should_not_call_mocked_api_for_dry_run(self, mock_set_task_instance_state, session): - self.create_task_instances(session) - - NEW_STATE = "failed" - mock_set_task_instance_state.return_value = session.scalars( - select(TaskInstance).where( - TaskInstance.dag_id == "example_python_operator", - TaskInstance.task_id == "print_the_context", - TaskInstance.run_id == "TEST_DAG_RUN_ID", - TaskInstance.map_index == -1, - ) - ).one_or_none() - - response = self.client.patch( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": True, - "new_state": NEW_STATE, - }, - ) - assert response.status_code == 200 - assert response.json == { - "dag_id": "example_python_operator", - "dag_run_id": "TEST_DAG_RUN_ID", - "logical_date": "2020-01-01T00:00:00+00:00", - "task_id": "print_the_context", - } - - mock_set_task_instance_state.assert_not_called() - - def test_should_update_task_instance_state(self, session): - self.create_task_instances(session) - - NEW_STATE = "failed" - - self.client.patch( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": False, - "new_state": NEW_STATE, - }, - ) - - response2 = self.client.get( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json={}, - ) - assert response2.status_code == 200 - assert response2.json["state"] == NEW_STATE - - def test_should_update_task_instance_state_default_dry_run_to_true(self, session): - self.create_task_instances(session) - - NEW_STATE = "running" - - self.client.patch( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json={ - "new_state": NEW_STATE, - }, - ) - - response2 = self.client.get( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json={}, - ) - assert response2.status_code == 200 - assert response2.json["state"] == NEW_STATE - - def test_should_update_mapped_task_instance_state(self, session): - NEW_STATE = "failed" - map_index = 1 - tis = self.create_task_instances(session) - ti = TaskInstance(task=tis[0].task, run_id=tis[0].run_id, map_index=map_index) - ti.rendered_task_instance_fields = RTIF(ti, render_templates=False) - session.add(ti) - session.commit() - - self.client.patch( - f"{self.ENDPOINT_URL}/{map_index}", - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": False, - "new_state": NEW_STATE, - }, - ) - - response2 = self.client.get( - f"{self.ENDPOINT_URL}/{map_index}", - environ_overrides={"REMOTE_USER": "test"}, - json={}, - ) - assert response2.status_code == 200 - assert response2.json["state"] == NEW_STATE - - @pytest.mark.parametrize( - "error, code, payload", - [ - [ - "Task instance not found for task 'print_the_context' on DAG run with ID 'TEST_DAG_RUN_ID'", - 404, - { - "dry_run": True, - "new_state": "failed", - }, - ] - ], - ) - def test_should_handle_errors(self, error, code, payload, session): - response = self.client.patch( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == code - assert response.json["detail"] == error - - def test_should_raise_400_for_unknown_fields(self, session): - self.create_task_instances(session) - response = self.client.patch( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dryrun": True, - "new_state": "failed", - }, - ) - assert response.status_code == 400 - assert response.json["detail"] == "{'dryrun': ['Unknown field.']}" - - def test_should_raise_404_for_non_existent_dag(self): - response = self.client.patch( - "/api/v1/dags/non-existent-dag/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context", - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": False, - "new_state": "failed", - }, - ) - assert response.status_code == 404 - assert response.json["title"] == "DAG not found" - assert response.json["detail"] == "DAG 'non-existent-dag' not found" - - def test_should_raise_404_for_non_existent_task_in_dag(self): - response = self.client.patch( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/non_existent_task", - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": False, - "new_state": "failed", - }, - ) - assert response.status_code == 404 - assert response.json["title"] == "Task not found" - assert ( - response.json["detail"] == "Task 'non_existent_task' not found in DAG 'example_python_operator'" - ) - - def test_should_raises_401_unauthenticated(self): - response = self.client.patch( - self.ENDPOINT_URL, - json={ - "dry_run": False, - "new_state": "failed", - }, - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.patch( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - json={ - "dry_run": True, - "new_state": "failed", - }, - ) - assert response.status_code == 403 - - def test_should_raise_404_not_found_dag(self): - response = self.client.patch( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": True, - "new_state": "failed", - }, - ) - assert response.status_code == 404 - - def test_should_raise_404_not_found_task(self): - response = self.client.patch( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json={ - "dry_run": True, - "new_state": "failed", - }, - ) - assert response.status_code == 404 - - @pytest.mark.parametrize( - "payload, expected", - [ - ( - { - "dry_run": True, - "new_state": "failede", - }, - f"'failede' is not one of ['{State.SUCCESS}', '{State.FAILED}', '{State.SKIPPED}']" - " - 'new_state'", - ), - ( - { - "dry_run": True, - "new_state": "queued", - }, - f"'queued' is not one of ['{State.SUCCESS}', '{State.FAILED}', '{State.SKIPPED}']" - " - 'new_state'", - ), - ], - ) - @provide_session - def test_should_raise_400_for_invalid_task_instance_state(self, payload, expected, session): - self.create_task_instances(session) - response = self.client.patch( - self.ENDPOINT_URL, - environ_overrides={"REMOTE_USER": "test"}, - json=payload, - ) - assert response.status_code == 400 - assert response.json["detail"] == expected - assert response.json["detail"] == expected - - -class TestSetTaskInstanceNote(TestTaskInstanceEndpoint): - def setup_method(self): - clear_db_runs() - - def teardown_method(self): - clear_db_runs() - - @provide_session - def test_should_respond_200(self, session): - self.create_task_instances(session) - new_note_value = "My super cool TaskInstance note." - response = self.client.patch( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - "print_the_context/setNote", - json={"note": new_note_value}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200, response.text - assert response.json == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "logical_date": "2020-01-01T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": -1, - "max_tries": 0, - "note": new_note_value, - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "running", - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": 0, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - "rendered_fields": {}, - "rendered_map_index": None, - "trigger": None, - "triggerer_job": None, - } - ti = session.scalars(select(TaskInstance).where(TaskInstance.task_id == "print_the_context")).one() - assert ti.task_instance_note.user_id is not None - _check_last_log( - session, dag_id="example_python_operator", event="api.set_task_instance_note", logical_date=None - ) - - def test_should_respond_200_mapped_task_instance_with_rtif(self, session): - """Verify we don't duplicate rows through join to RTIF""" - tis = self.create_task_instances(session) - old_ti = tis[0] - for idx in (1, 2): - ti = TaskInstance(task=old_ti.task, run_id=old_ti.run_id, map_index=idx) - ti.rendered_task_instance_fields = RTIF(ti, render_templates=False) - for attr in ["duration", "end_date", "pid", "start_date", "state", "queue", "note"]: - setattr(ti, attr, getattr(old_ti, attr)) - session.add(ti) - session.commit() - - # in each loop, we should get the right mapped TI back - for map_index in (1, 2): - new_note_value = f"My super cool TaskInstance note {map_index}" - response = self.client.patch( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - f"print_the_context/{map_index}/setNote", - json={"note": new_note_value}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200, response.text - - assert response.json == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "logical_date": "2020-01-01T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": map_index, - "max_tries": 0, - "note": new_note_value, - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "running", - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": 0, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - "rendered_fields": {"op_args": "()", "op_kwargs": {}, "templates_dict": None}, - "rendered_map_index": str(map_index), - "trigger": None, - "triggerer_job": None, - } - - def test_should_respond_200_when_note_is_empty(self, session): - tis = self.create_task_instances(session) - for ti in tis: - ti.task_instance_note = None - session.add(ti) - session.commit() - new_note_value = "My super cool TaskInstance note." - response = self.client.patch( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - "print_the_context/setNote", - json={"note": new_note_value}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200, response.text - assert response.json["note"] == new_note_value - - def test_should_raise_400_for_unknown_fields(self, session): - self.create_task_instances(session) - response = self.client.patch( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - "print_the_context/setNote", - json={"note": "a valid field", "not": "an unknown field"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json["detail"] == "{'not': ['Unknown field.']}" - - def test_should_raises_401_unauthenticated(self): - for map_index in ["", "/0"]: - url = ( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - f"print_the_context{map_index}/setNote" - ) - response = self.client.patch( - url, - json={"note": "I am setting a note while being unauthenticated."}, - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - for map_index in ["", "/0"]: - response = self.client.patch( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - f"print_the_context{map_index}/setNote", - json={"note": "I am setting a note without the proper permissions."}, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - def test_should_respond_404(self, session): - self.create_task_instances(session) - for map_index in ["", "/0"]: - response = self.client.patch( - f"api/v1/dags/INVALID_DAG_ID/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context" - f"{map_index}/setNote", - json={"note": "I am setting a note on a DAG that doesn't exist."}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - - -class TestGetTaskDependencies(TestTaskInstanceEndpoint): - def setup_method(self): - clear_db_runs() - - def teardown_method(self): - clear_db_runs() - - @provide_session - def test_should_respond_empty_non_scheduled(self, session): - self.create_task_instances(session) - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - "print_the_context/dependencies", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200, response.text - assert response.json == {"dependencies": []} - - @pytest.mark.parametrize( - "state, dependencies", - [ - ( - State.SCHEDULED, - { - "dependencies": [ - { - "name": "Logical Date", - "reason": "The logical date is 2020-01-01T00:00:00+00:00 but this is " - "before the task's start date 2021-01-01T00:00:00+00:00.", - }, - { - "name": "Logical Date", - "reason": "The logical date is 2020-01-01T00:00:00+00:00 but this is " - "before the task's DAG's start date 2021-01-01T00:00:00+00:00.", - }, - ], - }, - ), - ( - State.NONE, - { - "dependencies": [ - { - "name": "Logical Date", - "reason": "The logical date is 2020-01-01T00:00:00+00:00 but this is before the task's start date 2021-01-01T00:00:00+00:00.", - }, - { - "name": "Logical Date", - "reason": "The logical date is 2020-01-01T00:00:00+00:00 but this is before the task's DAG's start date 2021-01-01T00:00:00+00:00.", - }, - {"name": "Task Instance State", "reason": "Task is in the 'None' state."}, - ] - }, - ), - ], - ) - @provide_session - def test_should_respond_dependencies(self, session, state, dependencies): - self.create_task_instances(session, task_instances=[{"state": state}], update_extras=True) - - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - "print_the_context/dependencies", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200, response.text - assert response.json == dependencies - - def test_should_respond_dependencies_mapped(self, session): - tis = self.create_task_instances( - session, task_instances=[{"state": State.SCHEDULED}], update_extras=True - ) - old_ti = tis[0] - - ti = TaskInstance(task=old_ti.task, run_id=old_ti.run_id, map_index=0, state=old_ti.state) - session.add(ti) - session.commit() - - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - "print_the_context/0/dependencies", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200, response.text - - def test_should_raises_401_unauthenticated(self): - for map_index in ["", "/0"]: - url = ( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - f"print_the_context{map_index}/dependencies" - ) - response = self.client.get( - url, - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - for map_index in ["", "/0"]: - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/" - f"print_the_context{map_index}/dependencies", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - def test_should_respond_404(self, session): - self.create_task_instances(session) - for map_index in ["", "/0"]: - response = self.client.get( - f"api/v1/dags/INVALID_DAG_ID/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context" - f"{map_index}/dependencies", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - - -class TestGetTaskInstanceTry(TestTaskInstanceEndpoint): - def setup_method(self): - clear_db_runs() - - def teardown_method(self): - clear_db_runs() - - @provide_session - def test_should_respond_200(self, session): - self.create_task_instances(session, task_instances=[{"state": State.SUCCESS}], with_ti_history=True) - - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/tries/1", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": -1, - "max_tries": 0, - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "success", - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": 1, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - } - - @pytest.mark.parametrize("try_number", [1, 2]) - @provide_session - def test_should_respond_200_with_different_try_numbers(self, try_number, session): - self.create_task_instances(session, task_instances=[{"state": State.SUCCESS}], with_ti_history=True) - - response = self.client.get( - f"/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/tries/{try_number}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": -1, - "max_tries": 0 if try_number == 1 else 1, - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "success" if try_number == 1 else None, - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": try_number, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - } - - @pytest.mark.parametrize("try_number", [1, 2]) - @provide_session - def test_should_respond_200_with_mapped_task_at_different_try_numbers(self, try_number, session): - tis = self.create_task_instances(session, task_instances=[{"state": State.FAILED}]) - old_ti = tis[0] - for idx in (1, 2): - ti = TaskInstance(task=old_ti.task, run_id=old_ti.run_id, map_index=idx) - ti.rendered_task_instance_fields = RTIF(ti, render_templates=False) - ti.try_number = 1 - for attr in ["duration", "end_date", "pid", "start_date", "state", "queue", "note"]: - setattr(ti, attr, getattr(old_ti, attr)) - session.add(ti) - session.commit() - tis = session.query(TaskInstance).all() - - # Record the task instance history - from airflow.models.taskinstance import clear_task_instances - - clear_task_instances(tis, session) - # Simulate the try_number increasing to new values in TI - for ti in tis: - if ti.map_index > 0: - ti.try_number += 1 - ti.queue = "default_queue" - session.merge(ti) - session.commit() - - # in each loop, we should get the right mapped TI back - for map_index in (1, 2): - # Get the info from TIHistory: try_number 1, try_number 2 is TI table(latest) - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances" - f"/print_the_context/{map_index}/tries/{try_number}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - - assert response.json == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": map_index, - "max_tries": 0 if try_number == 1 else 1, - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "failed" if try_number == 1 else None, - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": try_number, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - } - - def test_should_respond_200_with_task_state_in_deferred(self, session): - now = pendulum.now("UTC") - ti = self.create_task_instances( - session, - task_instances=[{"state": State.DEFERRED}], - update_extras=True, - )[0] - ti.trigger = Trigger("none", {}) - ti.trigger.created_date = now - ti.triggerer_job = Job() - TriggererJobRunner(job=ti.triggerer_job) - ti.triggerer_job.state = "running" - ti.try_number = 1 - session.merge(ti) - session.flush() - # Record the TaskInstanceHistory - TaskInstanceHistory.record_ti(ti, session=session) - session.flush() - # Change TaskInstance try_number to 2, ensuring api checks TIHistory - ti = session.query(TaskInstance).one_or_none() - ti.try_number = 2 - session.merge(ti) - # Set duration and end_date in TaskInstanceHistory for easy testing - tih = session.query(TaskInstanceHistory).all()[0] - tih.duration = 10000 - tih.end_date = self.default_time + dt.timedelta(days=2) - session.merge(tih) - session.flush() - # Get the task instance details from TIHistory: - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/tries/1", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - data = response.json - - assert response.status_code == 200 - assert data == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": -1, - "max_tries": 0, - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "failed", - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": 1, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - } - - def test_should_respond_200_with_task_state_in_removed(self, session): - self.create_task_instances( - session, task_instances=[{"state": State.REMOVED}], update_extras=True, with_ti_history=True - ) - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/tries/1", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - - assert response.json == { - "dag_id": "example_python_operator", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": -1, - "max_tries": 0, - "operator": "PythonOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 9, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "removed", - "task_id": "print_the_context", - "task_display_name": "print_the_context", - "try_number": 1, - "unixname": getuser(), - "dag_run_id": "TEST_DAG_RUN_ID", - } - - def test_should_raises_401_unauthenticated(self): - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/tries/0", - ) - assert_401(response) - - def test_should_raise_403_forbidden(self): - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/tries/0", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - def test_raises_404_for_nonexistent_task_instance(self): - response = self.client.get( - "api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/nonexistent_task/tries/0", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json["title"] == "Task instance not found" - - -class TestGetTaskInstanceTries(TestTaskInstanceEndpoint): - def setup_method(self): - clear_db_runs() - - def teardown_method(self): - clear_db_runs() - - def test_should_respond_200(self, session): - self.create_task_instances( - session=session, task_instances=[{"state": State.SUCCESS}], with_ti_history=True - ) - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/tries", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 2 # The task instance and its history - assert len(response.json["task_instances"]) == 2 - - def test_ti_in_retry_state_not_returned(self, session): - self.create_task_instances( - session=session, task_instances=[{"state": State.SUCCESS}], with_ti_history=True - ) - ti = session.query(TaskInstance).one() - ti.state = State.UP_FOR_RETRY - session.merge(ti) - session.commit() - - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/tries", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json["total_entries"] == 1 - assert len(response.json["task_instances"]) == 1 - - def test_mapped_task_should_respond_200(self, session): - tis = self.create_task_instances(session, task_instances=[{"state": State.FAILED}]) - old_ti = tis[0] - for idx in (1, 2): - ti = TaskInstance(task=old_ti.task, run_id=old_ti.run_id, map_index=idx) - ti.try_number = 1 - session.add(ti) - session.commit() - tis = session.query(TaskInstance).all() - - # Record the task instance history - from airflow.models.taskinstance import clear_task_instances - - clear_task_instances(tis, session) - # Simulate the try_number increasing to new values in TI - for ti in tis: - if ti.map_index > 0: - ti.try_number += 1 - ti.queue = "default_queue" - session.merge(ti) - session.commit() - - # in each loop, we should get the right mapped TI back - for map_index in (1, 2): - # Get the info from TIHistory: try_number 1, try_number 2 is TI table(latest) - response = self.client.get( - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances" - f"/print_the_context/{map_index}/tries", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert ( - response.json["total_entries"] == 2 - ) # the mapped task was cleared. So both the task instance and its history - assert len(response.json["task_instances"]) == 2 - - @pytest.mark.parametrize( - "url", - [ - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/tries", - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/0/tries", - ], - ) - def test_should_raises_401_unauthenticated(self, url): - response = self.client.get(url) - assert_401(response) - - @pytest.mark.parametrize( - "url", - [ - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/tries", - "/api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/print_the_context/0/tries", - ], - ) - def test_should_raise_403_forbidden(self, url): - response = self.client.get( - url, - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 diff --git a/tests/api_connexion/endpoints/test_update_mask.py b/tests/api_connexion/endpoints/test_update_mask.py deleted file mode 100644 index 4221f11a1319a..0000000000000 --- a/tests/api_connexion/endpoints/test_update_mask.py +++ /dev/null @@ -1,56 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from airflow.api_connexion.endpoints.update_mask import extract_update_mask_data -from airflow.api_connexion.exceptions import BadRequest - - -class TestUpdateMask: - def test_should_extract_data(self): - non_update_fields = ["field_1"] - update_mask = ["field_2"] - data = { - "field_1": "value_1", - "field_2": "value_2", - "field_3": "value_3", - } - data = extract_update_mask_data(update_mask, non_update_fields, data) - assert data == {"field_2": "value_2"} - - def test_update_forbid_field_should_raise_exception(self): - non_update_fields = ["field_1"] - update_mask = ["field_1", "field_2"] - data = { - "field_1": "value_1", - "field_2": "value_2", - "field_3": "value_3", - } - with pytest.raises(BadRequest): - extract_update_mask_data(update_mask, non_update_fields, data) - - def test_update_unknown_field_should_raise_exception(self): - non_update_fields = ["field_1"] - update_mask = ["field_2", "field_3"] - data = { - "field_1": "value_1", - "field_2": "value_2", - } - with pytest.raises(BadRequest): - extract_update_mask_data(update_mask, non_update_fields, data) diff --git a/tests/api_connexion/endpoints/test_variable_endpoint.py b/tests/api_connexion/endpoints/test_variable_endpoint.py deleted file mode 100644 index 0f39458d594a5..0000000000000 --- a/tests/api_connexion/endpoints/test_variable_endpoint.py +++ /dev/null @@ -1,393 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import urllib - -import pytest - -from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP -from airflow.models import Variable - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_variables -from tests_common.test_utils.www import _check_last_log - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestVariableEndpoint: - @pytest.fixture(autouse=True) - def setup_method(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - clear_db_variables() - - def teardown_method(self) -> None: - clear_db_variables() - - -class TestDeleteVariable(TestVariableEndpoint): - def test_should_delete_variable(self, session): - Variable.set("delete_var1", 1) - # make sure variable is added - response = self.client.get("/api/v1/variables/delete_var1", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - - response = self.client.delete( - "/api/v1/variables/delete_var1", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 204 - - # make sure variable is deleted - response = self.client.get("/api/v1/variables/delete_var1", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 404 - _check_last_log(session, dag_id=None, event="api.variable.delete", logical_date=None) - - def test_should_respond_404_if_key_does_not_exist(self): - response = self.client.delete( - "/api/v1/variables/NONEXIST_VARIABLE_KEY", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - - def test_should_raises_401_unauthenticated(self): - Variable.set("delete_var1", 1) - # make sure variable is added - response = self.client.delete("/api/v1/variables/delete_var1") - - assert_401(response) - - # make sure variable is not deleted - response = self.client.get("/api/v1/variables/delete_var1", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - - def test_should_raise_403_forbidden(self): - expected_value = '{"foo": 1}' - Variable.set("TEST_VARIABLE_KEY", expected_value) - response = self.client.get( - "/api/v1/variables/TEST_VARIABLE_KEY", environ_overrides={"REMOTE_USER": "test_no_permissions"} - ) - assert response.status_code == 403 - - -class TestGetVariable(TestVariableEndpoint): - @pytest.mark.parametrize( - "user, expected_status_code", - [ - ("test", 200), - ("test_no_permissions", 403), - ], - ) - def test_read_variable(self, user, expected_status_code): - expected_value = '{"foo": 1}' - Variable.set("TEST_VARIABLE_KEY", expected_value) - response = self.client.get( - "/api/v1/variables/TEST_VARIABLE_KEY", environ_overrides={"REMOTE_USER": user} - ) - assert response.status_code == expected_status_code - if expected_status_code == 200: - assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value, "description": None} - - def test_should_respond_404_if_not_found(self): - response = self.client.get( - "/api/v1/variables/NONEXIST_VARIABLE_KEY", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 404 - - def test_should_raises_401_unauthenticated(self): - Variable.set("TEST_VARIABLE_KEY", '{"foo": 1}') - - response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY") - - assert_401(response) - - def test_should_handle_slashes_in_keys(self): - expected_value = "hello" - Variable.set("foo/bar", expected_value) - response = self.client.get( - f"/api/v1/variables/{urllib.parse.quote('foo/bar', safe='')}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == {"key": "foo/bar", "value": expected_value, "description": None} - - -class TestGetVariables(TestVariableEndpoint): - @pytest.mark.parametrize( - "query, expected", - [ - ( - "/api/v1/variables?limit=2&offset=0", - { - "variables": [ - {"key": "var1", "value": "1", "description": "I am a variable"}, - {"key": "var2", "value": "foo", "description": "Another variable"}, - ], - "total_entries": 3, - }, - ), - ( - "/api/v1/variables?limit=2&offset=1", - { - "variables": [ - {"key": "var2", "value": "foo", "description": "Another variable"}, - {"key": "var3", "value": "[100, 101]", "description": None}, - ], - "total_entries": 3, - }, - ), - ( - "/api/v1/variables?limit=1&offset=2", - { - "variables": [ - {"key": "var3", "value": "[100, 101]", "description": None}, - ], - "total_entries": 3, - }, - ), - ], - ) - def test_should_get_list_variables(self, query, expected): - Variable.set("var1", 1, "I am a variable") - Variable.set("var2", "foo", "Another variable") - Variable.set("var3", "[100, 101]") - response = self.client.get(query, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json == expected - - def test_should_respect_page_size_limit_default(self): - for i in range(101): - Variable.set(f"var{i}", i) - response = self.client.get("/api/v1/variables", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json["total_entries"] == 101 - assert len(response.json["variables"]) == 100 - - def test_should_raise_400_for_invalid_order_by(self): - for i in range(101): - Variable.set(f"var{i}", i) - response = self.client.get( - "/api/v1/variables?order_by=invalid", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 400 - msg = "Ordering with 'invalid' is disallowed or the attribute does not exist on the model" - assert response.json["detail"] == msg - - @conf_vars({("api", "maximum_page_limit"): "150"}) - def test_should_return_conf_max_if_req_max_above_conf(self): - for i in range(200): - Variable.set(f"var{i}", i) - response = self.client.get("/api/v1/variables?limit=180", environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert len(response.json["variables"]) == 150 - - def test_should_raises_401_unauthenticated(self): - Variable.set("var1", 1) - - response = self.client.get("/api/v1/variables?limit=2&offset=0") - - assert_401(response) - - -class TestPatchVariable(TestVariableEndpoint): - def test_should_update_variable(self, session): - Variable.set("var1", "foo") - payload = { - "key": "var1", - "value": "updated", - } - response = self.client.patch( - "/api/v1/variables/var1", - json=payload, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == {"key": "var1", "value": "updated", "description": None} - _check_last_log( - session, dag_id=None, event="api.variable.edit", logical_date=None, expected_extra=payload - ) - - def test_should_update_variable_with_mask(self, session): - Variable.set("var1", "foo", description="before update") - response = self.client.patch( - "/api/v1/variables/var1?update_mask=description", - json={"key": "var1", "value": "updated", "description": "after_update"}, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == {"key": "var1", "value": "foo", "description": "after_update"} - _check_last_log(session, dag_id=None, event="api.variable.edit", logical_date=None) - - def test_should_reject_invalid_update(self): - response = self.client.patch( - "/api/v1/variables/var1", - json={ - "key": "var1", - "value": "foo", - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json == { - "title": "Variable not found", - "status": 404, - "type": EXCEPTIONS_LINK_MAP[404], - "detail": "Variable does not exist", - } - Variable.set("var1", "foo") - response = self.client.patch( - "/api/v1/variables/var1", - json={ - "key": "var2", - "value": "updated", - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json == { - "title": "Invalid post body", - "status": 400, - "type": EXCEPTIONS_LINK_MAP[400], - "detail": "key from request body doesn't match uri parameter", - } - - response = self.client.patch( - "/api/v1/variables/var1", - json={ - "key": "var2", - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.json == { - "title": "Invalid Variable schema", - "status": 400, - "type": EXCEPTIONS_LINK_MAP[400], - "detail": "{'value': ['Missing data for required field.']}", - } - - def test_should_raises_401_unauthenticated(self): - Variable.set("var1", "foo") - - response = self.client.patch( - "/api/v1/variables/var1", - json={ - "key": "var1", - "value": "updated", - }, - ) - - assert_401(response) - - -class TestPostVariables(TestVariableEndpoint): - @pytest.mark.parametrize( - "description", - [ - pytest.param(None, id="not-set"), - pytest.param("", id="empty"), - pytest.param("Spam Egg", id="desc-set"), - ], - ) - def test_should_create_variable(self, description, session): - payload = {"key": "var_create", "value": "{}"} - if description is not None: - payload["description"] = description - response = self.client.post( - "/api/v1/variables", - json=payload, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - _check_last_log( - session, dag_id=None, event="api.variable.create", logical_date=None, expected_extra=payload - ) - response = self.client.get("/api/v1/variables/var_create", environ_overrides={"REMOTE_USER": "test"}) - assert response.json == { - "key": "var_create", - "value": "{}", - "description": description, - } - - @pytest.mark.enable_redact - def test_should_create_masked_variable(self, session): - payload = {"key": "api_key", "value": "secret_key", "description": "secret"} - response = self.client.post( - "/api/v1/variables", - json=payload, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - expected_extra = { - **payload, - "value": "***", - } - _check_last_log( - session, - dag_id=None, - event="api.variable.create", - logical_date=None, - expected_extra=expected_extra, - ) - response = self.client.get("/api/v1/variables/api_key", environ_overrides={"REMOTE_USER": "test"}) - assert response.json == payload - - def test_should_reject_invalid_request(self, session): - response = self.client.post( - "/api/v1/variables", - json={ - "key": "var_create", - "v": "{}", - }, - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 400 - assert response.json == { - "title": "Invalid Variable schema", - "status": 400, - "type": EXCEPTIONS_LINK_MAP[400], - "detail": "{'value': ['Missing data for required field.'], 'v': ['Unknown field.']}", - } - _check_last_log(session, dag_id=None, event="api.variable.create", logical_date=None) - - def test_should_raises_401_unauthenticated(self): - response = self.client.post( - "/api/v1/variables", - json={ - "key": "var_create", - "value": "{}", - }, - ) - - assert_401(response) diff --git a/tests/api_connexion/endpoints/test_version_endpoint.py b/tests/api_connexion/endpoints/test_version_endpoint.py deleted file mode 100644 index fd068cb291bee..0000000000000 --- a/tests/api_connexion/endpoints/test_version_endpoint.py +++ /dev/null @@ -1,44 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from unittest import mock - -import pytest - -pytestmark = pytest.mark.db_test - - -class TestGetHealthTest: - @pytest.fixture(autouse=True) - def setup_attrs(self, minimal_app_for_api) -> None: - """ - Setup For XCom endpoint TC - """ - self.app = minimal_app_for_api - self.client = self.app.test_client() # type:ignore - - @mock.patch("airflow.api_connexion.endpoints.version_endpoint.airflow.__version__", "MOCK_VERSION") - @mock.patch( - "airflow.api_connexion.endpoints.version_endpoint.get_airflow_git_version", return_value="GIT_COMMIT" - ) - def test_should_respond_200(self, mock_get_airflow_get_commit): - response = self.client.get("/api/v1/version") - - assert response.status_code == 200 - assert response.json == {"git_version": "GIT_COMMIT", "version": "MOCK_VERSION"} - mock_get_airflow_get_commit.assert_called_once_with() diff --git a/tests/api_connexion/endpoints/test_xcom_endpoint.py b/tests/api_connexion/endpoints/test_xcom_endpoint.py deleted file mode 100644 index 598d3a99546ad..0000000000000 --- a/tests/api_connexion/endpoints/test_xcom_endpoint.py +++ /dev/null @@ -1,716 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from datetime import timedelta -from unittest import mock - -import pytest - -from airflow.models.dag import DagModel -from airflow.models.dagrun import DagRun -from airflow.models.taskinstance import TaskInstance -from airflow.models.xcom import BaseXCom, XCom, resolve_xcom_backend -from airflow.providers.standard.operators.empty import EmptyOperator -from airflow.utils import timezone -from airflow.utils.session import create_session -from airflow.utils.timezone import utcnow -from airflow.utils.types import DagRunType - -from tests_common.test_utils.api_connexion_utils import assert_401, create_user, delete_user -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_dags, clear_db_runs, clear_db_xcom - -pytestmark = pytest.mark.db_test - - -class CustomXCom(BaseXCom): - @classmethod - def deserialize_value(cls, xcom: XCom): - return f"real deserialized {super().deserialize_value(xcom)}" - - def orm_deserialize_value(self): - return f"orm deserialized {super().orm_deserialize_value()}" - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -def _compare_xcom_collections(collection1: dict, collection_2: dict): - assert collection1.get("total_entries") == collection_2.get("total_entries") - - def sort_key(record): - return ( - record.get("dag_id"), - record.get("task_id"), - record.get("logical_date"), - record.get("map_index"), - record.get("key"), - ) - - assert sorted(collection1.get("xcom_entries", []), key=sort_key) == sorted( - collection_2.get("xcom_entries", []), key=sort_key - ) - - -class TestXComEndpoint: - @staticmethod - def clean_db(): - clear_db_dags() - clear_db_runs() - clear_db_xcom() - - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - """ - Setup For XCom endpoint TC - """ - self.app = configured_app - self.client = self.app.test_client() # type:ignore - # clear existing xcoms - self.clean_db() - - def teardown_method(self) -> None: - """ - Clear Hanging XComs - """ - self.clean_db() - - -class TestGetXComEntry(TestXComEndpoint): - def test_should_respond_200_stringify(self): - dag_id = "test-dag-id" - task_id = "test-task-id" - logical_date = "2005-04-02T00:00:00+00:00" - run_after = "2005-04-02T00:00:00+00:00" - xcom_key = "test-xcom-key" - logical_date_parsed = timezone.parse(logical_date) - run_after_parsed = timezone.parse(run_after) - run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - self._create_xcom_entry(dag_id, run_id, logical_date_parsed, task_id, xcom_key, {"key": "value"}) - response = self.client.get( - f"/api/v1/dags/{dag_id}/dagRuns/{run_id}/taskInstances/{task_id}/xcomEntries/{xcom_key}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - - current_data = response.json - current_data["timestamp"] = "TIMESTAMP" - assert current_data == { - "dag_id": dag_id, - "logical_date": logical_date, - "key": xcom_key, - "task_id": task_id, - "map_index": -1, - "timestamp": "TIMESTAMP", - "value": "{'key': 'value'}", - } - - def test_should_respond_200_native(self): - dag_id = "test-dag-id" - task_id = "test-task-id" - logical_date = "2005-04-02T00:00:00+00:00" - run_after = "2005-04-02T00:00:00+00:00" - xcom_key = "test-xcom-key" - logical_date_parsed = timezone.parse(logical_date) - run_after_parsed = timezone.parse(run_after) - run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - self._create_xcom_entry(dag_id, run_id, logical_date_parsed, task_id, xcom_key, {"key": "value"}) - response = self.client.get( - f"/api/v1/dags/{dag_id}/dagRuns/{run_id}/taskInstances/{task_id}/xcomEntries/{xcom_key}?stringify=false", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - - current_data = response.json - current_data["timestamp"] = "TIMESTAMP" - assert current_data == { - "dag_id": dag_id, - "logical_date": logical_date, - "key": xcom_key, - "task_id": task_id, - "map_index": -1, - "timestamp": "TIMESTAMP", - "value": {"key": "value"}, - } - - def test_should_raise_404_for_non_existent_xcom(self): - dag_id = "test-dag-id" - task_id = "test-task-id" - logical_date = "2005-04-02T00:00:00+00:00" - run_after = "2005-04-02T00:00:00+00:00" - xcom_key = "test-xcom-key" - logical_date_parsed = timezone.parse(logical_date) - run_after_parsed = timezone.parse(run_after) - run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - self._create_xcom_entry(dag_id, run_id, logical_date_parsed, task_id, xcom_key) - response = self.client.get( - f"/api/v1/dags/nonexistentdagid/dagRuns/{run_id}/taskInstances/{task_id}/xcomEntries/{xcom_key}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json["title"] == "XCom entry not found" - - def test_should_raises_401_unauthenticated(self): - dag_id = "test-dag-id" - task_id = "test-task-id" - logical_date = "2005-04-02T00:00:00+00:00" - run_after = "2005-04-02T00:00:00+00:00" - xcom_key = "test-xcom-key" - logical_date_parsed = timezone.parse(logical_date) - run_after_parsed = timezone.parse(run_after) - run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - self._create_xcom_entry(dag_id, run_id, logical_date_parsed, task_id, xcom_key) - response = self.client.get( - f"/api/v1/dags/{dag_id}/dagRuns/{run_id}/taskInstances/{task_id}/xcomEntries/{xcom_key}" - ) - - assert_401(response) - - def test_should_raise_403_forbidden(self): - dag_id = "test-dag-id" - task_id = "test-task-id" - logical_date = "2005-04-02T00:00:00+00:00" - run_after = "2005-04-02T00:00:00+00:00" - xcom_key = "test-xcom-key" - logical_date_parsed = timezone.parse(logical_date) - run_after_parsed = timezone.parse(run_after) - run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - - self._create_xcom_entry(dag_id, run_id, logical_date_parsed, task_id, xcom_key) - response = self.client.get( - f"/api/v1/dags/{dag_id}/dagRuns/{run_id}/taskInstances/{task_id}/xcomEntries/{xcom_key}", - environ_overrides={"REMOTE_USER": "test_no_permissions"}, - ) - assert response.status_code == 403 - - def _create_xcom_entry( - self, dag_id, run_id, logical_date, task_id, xcom_key, xcom_value="TEST_VALUE", *, backend=XCom - ): - with create_session() as session: - dagrun = DagRun( - dag_id=dag_id, - run_id=run_id, - logical_date=logical_date, - start_date=logical_date, - run_type=DagRunType.MANUAL, - ) - session.add(dagrun) - ti = TaskInstance(EmptyOperator(task_id=task_id), run_id=run_id) - ti.dag_id = dag_id - session.add(ti) - backend.set( - key=xcom_key, - value=xcom_value, - run_id=run_id, - task_id=task_id, - dag_id=dag_id, - ) - - @pytest.mark.parametrize( - "allowed, query, expected_status_or_value, key", - [ - pytest.param( - True, - "?deserialize=true", - "real deserialized TEST_VALUE", - "key", - id="true", - ), - pytest.param( - False, - "?deserialize=true", - 400, - "key", - id="disallowed", - ), - pytest.param( - True, - "?deserialize=false", - "orm deserialized TEST_VALUE", - "key", - id="false-irrelevant", - ), - pytest.param( - False, - "?deserialize=false", - "orm deserialized TEST_VALUE", - "key", - id="false", - ), - pytest.param( - True, - "", - "orm deserialized TEST_VALUE", - "key", - id="default-irrelevant", - ), - pytest.param( - False, - "", - "orm deserialized TEST_VALUE", - "key", - id="default", - ), - pytest.param( - False, - "", - "orm deserialized TEST_VALUE", - "key/with/slashes", - id="key-with-slashes", - ), - ], - ) - @conf_vars({("core", "xcom_backend"): "tests.api_connexion.endpoints.test_xcom_endpoint.CustomXCom"}) - def test_custom_xcom_deserialize( - self, allowed: bool, query: str, expected_status_or_value: int | str, key: str - ): - XCom = resolve_xcom_backend() - self._create_xcom_entry("dag", "run", utcnow(), "task", key, backend=XCom) - - url = f"/api/v1/dags/dag/dagRuns/run/taskInstances/task/xcomEntries/{key}{query}" - with mock.patch("airflow.api_connexion.endpoints.xcom_endpoint.XCom", XCom): - with conf_vars({("api", "enable_xcom_deserialize_support"): str(allowed)}): - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - - if isinstance(expected_status_or_value, int): - assert response.status_code == expected_status_or_value - else: - assert response.status_code == 200 - assert response.json["value"] == expected_status_or_value - - -class TestGetXComEntries(TestXComEndpoint): - def test_should_respond_200(self): - dag_id = "test-dag-id" - task_id = "test-task-id" - logical_date = "2005-04-02T00:00:00+00:00" - run_after = "2005-04-02T00:00:00+00:00" - logical_date_parsed = timezone.parse(logical_date) - run_after_parsed = timezone.parse(run_after) - run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - - self._create_xcom_entries(dag_id, run_id, logical_date_parsed, task_id) - response = self.client.get( - f"/api/v1/dags/{dag_id}/dagRuns/{run_id}/taskInstances/{task_id}/xcomEntries", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - response_data = response.json - for xcom_entry in response_data["xcom_entries"]: - xcom_entry["timestamp"] = "TIMESTAMP" - _compare_xcom_collections( - response_data, - { - "xcom_entries": [ - { - "dag_id": dag_id, - "logical_date": logical_date, - "key": "test-xcom-key-1", - "task_id": task_id, - "timestamp": "TIMESTAMP", - "map_index": -1, - }, - { - "dag_id": dag_id, - "logical_date": logical_date, - "key": "test-xcom-key-2", - "task_id": task_id, - "timestamp": "TIMESTAMP", - "map_index": -1, - }, - ], - "total_entries": 2, - }, - ) - - def test_should_respond_200_with_tilde_and_access_to_all_dags(self): - dag_id_1 = "test-dag-id-1" - task_id_1 = "test-task-id-1" - logical_date = "2005-04-02T00:00:00+00:00" - run_after = "2005-04-02T00:00:00+00:00" - logical_date_parsed = timezone.parse(logical_date) - run_after_parsed = timezone.parse(run_after) - run_id_1 = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - self._create_xcom_entries(dag_id_1, run_id_1, logical_date_parsed, task_id_1) - - dag_id_2 = "test-dag-id-2" - task_id_2 = "test-task-id-2" - run_id_2 = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - self._create_xcom_entries(dag_id_2, run_id_2, logical_date_parsed, task_id_2) - - response = self.client.get( - "/api/v1/dags/~/dagRuns/~/taskInstances/~/xcomEntries", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - response_data = response.json - for xcom_entry in response_data["xcom_entries"]: - xcom_entry["timestamp"] = "TIMESTAMP" - _compare_xcom_collections( - response_data, - { - "xcom_entries": [ - { - "dag_id": dag_id_1, - "logical_date": logical_date, - "key": "test-xcom-key-1", - "task_id": task_id_1, - "timestamp": "TIMESTAMP", - "map_index": -1, - }, - { - "dag_id": dag_id_1, - "logical_date": logical_date, - "key": "test-xcom-key-2", - "task_id": task_id_1, - "timestamp": "TIMESTAMP", - "map_index": -1, - }, - { - "dag_id": dag_id_2, - "logical_date": logical_date, - "key": "test-xcom-key-1", - "task_id": task_id_2, - "timestamp": "TIMESTAMP", - "map_index": -1, - }, - { - "dag_id": dag_id_2, - "logical_date": logical_date, - "key": "test-xcom-key-2", - "task_id": task_id_2, - "timestamp": "TIMESTAMP", - "map_index": -1, - }, - ], - "total_entries": 4, - }, - ) - - def test_should_respond_200_with_map_index(self): - dag_id = "test-dag-id" - task_id = "test-task-id" - logical_date = "2005-04-02T00:00:00+00:00" - logical_date_parsed = timezone.parse(logical_date) - run_after = "2005-04-02T00:00:00+00:00" - run_after_parsed = timezone.parse(run_after) - dag_run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - self._create_xcom_entries(dag_id, dag_run_id, logical_date_parsed, task_id, mapped_ti=True) - - def assert_expected_result(expected_entries, map_index=None): - response = self.client.get( - "/api/v1/dags/~/dagRuns/~/taskInstances/~/xcomEntries" - f"{('?map_index=' + str(map_index)) if map_index is not None else ''}", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - response_data = response.json - for xcom_entry in response_data["xcom_entries"]: - xcom_entry["timestamp"] = "TIMESTAMP" - assert response_data == { - "xcom_entries": expected_entries, - "total_entries": len(expected_entries), - } - - expected_entry1 = { - "dag_id": dag_id, - "logical_date": logical_date, - "key": "test-xcom-key", - "task_id": task_id, - "timestamp": "TIMESTAMP", - "map_index": 0, - } - expected_entry2 = { - "dag_id": dag_id, - "logical_date": logical_date, - "key": "test-xcom-key", - "task_id": task_id, - "timestamp": "TIMESTAMP", - "map_index": 1, - } - assert_expected_result([expected_entry1], map_index=0) - assert_expected_result([expected_entry2], map_index=1) - assert_expected_result([expected_entry1, expected_entry2], map_index=None) - - def test_should_respond_200_with_xcom_key(self): - dag_id = "test-dag-id" - task_id = "test-task-id" - logical_date = "2005-04-02T00:00:00+00:00" - logical_date_parsed = timezone.parse(logical_date) - run_after = "2005-04-02T00:00:00+00:00" - run_after_parsed = timezone.parse(run_after) - dag_run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - self._create_xcom_entries(dag_id, dag_run_id, logical_date_parsed, task_id, mapped_ti=True) - - def assert_expected_result(expected_entries, key=None): - response = self.client.get( - f"/api/v1/dags/~/dagRuns/~/taskInstances/~/xcomEntries?xcom_key={key}", - environ_overrides={"REMOTE_USER": "test"}, - ) - - assert response.status_code == 200 - response_data = response.json - for xcom_entry in response_data["xcom_entries"]: - xcom_entry["timestamp"] = "TIMESTAMP" - assert response_data == { - "xcom_entries": expected_entries, - "total_entries": len(expected_entries), - } - - expected_entry1 = { - "dag_id": dag_id, - "logical_date": logical_date, - "key": "test-xcom-key", - "task_id": task_id, - "timestamp": "TIMESTAMP", - "map_index": 0, - } - expected_entry2 = { - "dag_id": dag_id, - "logical_date": logical_date, - "key": "test-xcom-key", - "task_id": task_id, - "timestamp": "TIMESTAMP", - "map_index": 1, - } - assert_expected_result([expected_entry1, expected_entry2], key="test-xcom-key") - assert_expected_result([], key="test-xcom-key-1") - - def test_should_raises_401_unauthenticated(self): - dag_id = "test-dag-id" - task_id = "test-task-id" - logical_date = "2005-04-02T00:00:00+00:00" - logical_date_parsed = timezone.parse(logical_date) - run_after = "2005-04-02T00:00:00+00:00" - run_after_parsed = timezone.parse(run_after) - run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date_parsed, run_after=run_after_parsed - ) - self._create_xcom_entries(dag_id, run_id, logical_date_parsed, task_id) - - response = self.client.get( - f"/api/v1/dags/{dag_id}/dagRuns/{run_id}/taskInstances/{task_id}/xcomEntries" - ) - - assert_401(response) - - def _create_xcom_entries(self, dag_id, run_id, logical_date, task_id, mapped_ti=False): - with create_session() as session: - dag = DagModel(dag_id=dag_id) - session.add(dag) - dagrun = DagRun( - dag_id=dag_id, - run_id=run_id, - logical_date=logical_date, - start_date=logical_date, - run_type=DagRunType.MANUAL, - ) - session.add(dagrun) - if mapped_ti: - for i in [0, 1]: - ti = TaskInstance(EmptyOperator(task_id=task_id), run_id=run_id, map_index=i) - ti.dag_id = dag_id - session.add(ti) - else: - ti = TaskInstance(EmptyOperator(task_id=task_id), run_id=run_id) - ti.dag_id = dag_id - session.add(ti) - - for i in [1, 2]: - if mapped_ti: - key = "test-xcom-key" - map_index = i - 1 - else: - key = f"test-xcom-key-{i}" - map_index = -1 - - XCom.set( - key=key, value="TEST", run_id=run_id, task_id=task_id, dag_id=dag_id, map_index=map_index - ) - - def _create_invalid_xcom_entries(self, logical_date): - """ - Invalid XCom entries to test join query - """ - with create_session() as session: - dag = DagModel(dag_id="invalid_dag") - session.add(dag) - dagrun = DagRun( - dag_id="invalid_dag", - run_id="invalid_run_id", - logical_date=logical_date + timedelta(days=1), - start_date=logical_date, - run_type=DagRunType.MANUAL, - ) - session.add(dagrun) - dagrun1 = DagRun( - dag_id="invalid_dag", - run_id="not_this_run_id", - logical_date=logical_date, - start_date=logical_date, - run_type=DagRunType.MANUAL, - ) - session.add(dagrun1) - ti = TaskInstance(EmptyOperator(task_id="invalid_task"), run_id="not_this_run_id") - ti.dag_id = "invalid_dag" - session.add(ti) - for i in [1, 2]: - XCom.set( - key=f"invalid-xcom-key-{i}", - value="TEST", - run_id="not_this_run_id", - task_id="invalid_task", - dag_id="invalid_dag", - ) - - -class TestPaginationGetXComEntries(TestXComEndpoint): - def setup_method(self): - self.dag_id = "test-dag-id" - self.task_id = "test-task-id" - self.logical_date = "2005-04-02T00:00:00+00:00" - self.logical_date_parsed = timezone.parse(self.logical_date) - run_after = "2005-04-02T00:00:00+00:00" - run_after_parsed = timezone.parse(run_after) - self.run_id = DagRun.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=self.logical_date_parsed, run_after=run_after_parsed - ) - - @pytest.mark.parametrize( - "query_params, expected_xcom_ids", - [ - ( - "limit=1", - ["TEST_XCOM_KEY1"], - ), - ( - "limit=2", - ["TEST_XCOM_KEY1", "TEST_XCOM_KEY10"], - ), - ( - "offset=5", - [ - "TEST_XCOM_KEY5", - "TEST_XCOM_KEY6", - "TEST_XCOM_KEY7", - "TEST_XCOM_KEY8", - "TEST_XCOM_KEY9", - ], - ), - ( - "offset=0", - [ - "TEST_XCOM_KEY1", - "TEST_XCOM_KEY10", - "TEST_XCOM_KEY2", - "TEST_XCOM_KEY3", - "TEST_XCOM_KEY4", - "TEST_XCOM_KEY5", - "TEST_XCOM_KEY6", - "TEST_XCOM_KEY7", - "TEST_XCOM_KEY8", - "TEST_XCOM_KEY9", - ], - ), - ( - "limit=1&offset=5", - ["TEST_XCOM_KEY5"], - ), - ( - "limit=1&offset=1", - ["TEST_XCOM_KEY10"], - ), - ( - "limit=2&offset=2", - ["TEST_XCOM_KEY2", "TEST_XCOM_KEY3"], - ), - ], - ) - def test_handle_limit_offset(self, query_params, expected_xcom_ids): - url = ( - f"/api/v1/dags/{self.dag_id}/dagRuns/{self.run_id}/taskInstances/{self.task_id}/xcomEntries" - f"?{query_params}" - ) - with create_session() as session: - dagrun = DagRun( - dag_id=self.dag_id, - run_id=self.run_id, - logical_date=self.logical_date_parsed, - start_date=self.logical_date_parsed, - run_type=DagRunType.MANUAL, - ) - session.add(dagrun) - ti = TaskInstance(EmptyOperator(task_id=self.task_id), run_id=self.run_id) - ti.dag_id = self.dag_id - session.add(ti) - - with create_session() as session: - for i in range(1, 11): - xcom = XCom( - dag_run_id=dagrun.id, - key=f"TEST_XCOM_KEY{i}", - value="null", - run_id=self.run_id, - task_id=self.task_id, - dag_id=self.dag_id, - timestamp=self.logical_date_parsed, - ) - session.add(xcom) - - response = self.client.get(url, environ_overrides={"REMOTE_USER": "test"}) - assert response.status_code == 200 - assert response.json["total_entries"] == 10 - conn_ids = [conn["key"] for conn in response.json["xcom_entries"] if conn] - assert conn_ids == expected_xcom_ids diff --git a/tests/api_connexion/schemas/__init__.py b/tests/api_connexion/schemas/__init__.py deleted file mode 100644 index 13a83393a9124..0000000000000 --- a/tests/api_connexion/schemas/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. diff --git a/tests/api_connexion/schemas/test_asset_schema.py b/tests/api_connexion/schemas/test_asset_schema.py deleted file mode 100644 index 732aa9147d7d1..0000000000000 --- a/tests/api_connexion/schemas/test_asset_schema.py +++ /dev/null @@ -1,241 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest -import time_machine - -from airflow.api_connexion.schemas.asset_schema import ( - AssetCollection, - AssetEventCollection, - asset_collection_schema, - asset_event_collection_schema, - asset_event_schema, - asset_schema, -) -from airflow.models.asset import AssetAliasModel, AssetEvent, AssetModel -from airflow.providers.standard.operators.empty import EmptyOperator -from airflow.sdk.definitions.asset import Asset - -from tests_common.test_utils.db import clear_db_assets, clear_db_dags - -pytestmark = pytest.mark.db_test - - -class TestAssetSchemaBase: - def setup_method(self) -> None: - clear_db_dags() - clear_db_assets() - self.timestamp = "2022-06-10T12:02:44+00:00" - self.freezer = time_machine.travel(self.timestamp, tick=False) - self.freezer.start() - - def teardown_method(self) -> None: - self.freezer.stop() - clear_db_dags() - clear_db_assets() - - -class TestAssetSchema(TestAssetSchemaBase): - def test_serialize(self, dag_maker, session): - asset = Asset( - uri="s3://bucket/key", - name="test_asset", - group="test-group", - extra={"foo": "bar"}, - ) - with dag_maker(dag_id="test_asset_upstream_schema", serialized=True, session=session): - EmptyOperator(task_id="task1", outlets=[asset]) - with dag_maker( - dag_id="test_asset_downstream_schema", schedule=[asset], serialized=True, session=session - ): - EmptyOperator(task_id="task2") - - asset_model = session.query(AssetModel).filter_by(uri=asset.uri).one() - - serialized_data = asset_schema.dump(asset_model) - serialized_data["id"] = 1 - assert serialized_data == { - "id": 1, - "uri": "s3://bucket/key", - "name": "test_asset", - "group": "test-group", - "extra": {"foo": "bar"}, - "created_at": self.timestamp, - "updated_at": self.timestamp, - "consuming_dags": [ - { - "dag_id": "test_asset_downstream_schema", - "created_at": self.timestamp, - "updated_at": self.timestamp, - } - ], - "producing_tasks": [ - { - "task_id": "task1", - "dag_id": "test_asset_upstream_schema", - "created_at": self.timestamp, - "updated_at": self.timestamp, - } - ], - "aliases": [], - } - - -class TestAssetCollectionSchema(TestAssetSchemaBase): - def test_serialize(self, session): - assets = [ - AssetModel( - uri=f"s3://bucket/key/{i}", - name=f"asset_{i}", - group="test-group", - extra={"foo": "bar"}, - ) - for i in range(1, 3) - ] - asset_aliases = [AssetAliasModel(name=f"alias_{i}", group="test-alias-group") for i in range(2)] - for asset_alias in asset_aliases: - asset_alias.assets.append(assets[0]) - session.add_all(assets) - session.add_all(asset_aliases) - session.flush() - serialized_data = asset_collection_schema.dump(AssetCollection(assets=assets, total_entries=2)) - serialized_data["assets"][0]["id"] = 1 - serialized_data["assets"][1]["id"] = 2 - serialized_data["assets"][0]["aliases"][0]["id"] = 1 - serialized_data["assets"][0]["aliases"][1]["id"] = 2 - assert serialized_data == { - "assets": [ - { - "id": 1, - "uri": "s3://bucket/key/1", - "name": "asset_1", - "group": "test-group", - "extra": {"foo": "bar"}, - "created_at": self.timestamp, - "updated_at": self.timestamp, - "consuming_dags": [], - "producing_tasks": [], - "aliases": [ - {"id": 1, "name": "alias_0", "group": "test-alias-group"}, - {"id": 2, "name": "alias_1", "group": "test-alias-group"}, - ], - }, - { - "id": 2, - "uri": "s3://bucket/key/2", - "name": "asset_2", - "group": "test-group", - "extra": {"foo": "bar"}, - "created_at": self.timestamp, - "updated_at": self.timestamp, - "consuming_dags": [], - "producing_tasks": [], - "aliases": [], - }, - ], - "total_entries": 2, - } - - -class TestAssetEventSchema(TestAssetSchemaBase): - def test_serialize(self, session): - assetssetsset = AssetModel("s3://abc") - session.add(assetssetsset) - session.commit() - event = AssetEvent( - id=1, - asset_id=assetssetsset.id, - extra={"foo": "bar"}, - source_dag_id="foo", - source_task_id="bar", - source_run_id="custom", - source_map_index=-1, - ) - session.add(event) - session.flush() - serialized_data = asset_event_schema.dump(event) - assert serialized_data == { - "id": 1, - "asset_id": assetssetsset.id, - "asset_uri": "s3://abc", - "extra": {"foo": "bar"}, - "source_dag_id": "foo", - "source_task_id": "bar", - "source_run_id": "custom", - "source_map_index": -1, - "timestamp": self.timestamp, - "created_dagruns": [], - } - - -class TestAssetEventCreateSchema(TestAssetSchemaBase): - def test_serialize(self, session): - asset = AssetModel("s3://abc") - session.add(asset) - session.commit() - event = AssetEvent( - id=1, - asset_id=asset.id, - extra={"foo": "bar"}, - source_dag_id=None, - source_task_id=None, - source_run_id=None, - source_map_index=-1, - ) - session.add(event) - session.flush() - serialized_data = asset_event_schema.dump(event) - assert serialized_data == { - "id": 1, - "asset_id": asset.id, - "asset_uri": "s3://abc", - "extra": {"foo": "bar"}, - "source_dag_id": None, - "source_task_id": None, - "source_run_id": None, - "source_map_index": -1, - "timestamp": self.timestamp, - "created_dagruns": [], - } - - -class TestAssetEventCollectionSchema(TestAssetSchemaBase): - def test_serialize(self, session): - common = { - "asset_id": 10, - "extra": {"foo": "bar"}, - "source_dag_id": "foo", - "source_task_id": "bar", - "source_run_id": "custom", - "source_map_index": -1, - "created_dagruns": [], - } - - events = [AssetEvent(id=i, **common) for i in [1, 2]] - session.add_all(events) - session.flush() - serialized_data = asset_event_collection_schema.dump( - AssetEventCollection(asset_events=events, total_entries=2) - ) - assert serialized_data == { - "asset_events": [ - {"id": 1, "timestamp": self.timestamp, **common}, - {"id": 2, "timestamp": self.timestamp, **common}, - ], - "total_entries": 2, - } diff --git a/tests/api_connexion/schemas/test_common_schema.py b/tests/api_connexion/schemas/test_common_schema.py deleted file mode 100644 index 7b5b51d06c04e..0000000000000 --- a/tests/api_connexion/schemas/test_common_schema.py +++ /dev/null @@ -1,84 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import datetime - -from dateutil import relativedelta - -from airflow.api_connexion.schemas.common_schema import ( - CronExpression, - CronExpressionSchema, - RelativeDeltaSchema, - TimeDeltaSchema, -) - - -class TestTimeDeltaSchema: - def test_should_serialize(self): - instance = datetime.timedelta(days=12) - schema_instance = TimeDeltaSchema() - result = schema_instance.dump(instance) - assert result == {"__type": "TimeDelta", "days": 12, "seconds": 0, "microseconds": 0} - - def test_should_deserialize(self): - instance = {"__type": "TimeDelta", "days": 12, "seconds": 0, "microseconds": 0} - schema_instance = TimeDeltaSchema() - result = schema_instance.load(instance) - expected_instance = datetime.timedelta(days=12) - assert expected_instance == result - - -class TestRelativeDeltaSchema: - def test_should_serialize(self): - instance = relativedelta.relativedelta(days=+12) - schema_instance = RelativeDeltaSchema() - result = schema_instance.dump(instance) - assert result == { - "__type": "RelativeDelta", - "day": None, - "days": 12, - "hour": None, - "hours": 0, - "leapdays": 0, - "microsecond": None, - "microseconds": 0, - "minute": None, - "minutes": 0, - "month": None, - "months": 0, - "second": None, - "seconds": 0, - "year": None, - "years": 0, - } - - def test_should_deserialize(self): - instance = {"__type": "RelativeDelta", "days": 12, "seconds": 0} - schema_instance = RelativeDeltaSchema() - result = schema_instance.load(instance) - expected_instance = relativedelta.relativedelta(days=+12) - assert expected_instance == result - - -class TestCronExpressionSchema: - def test_should_deserialize(self): - instance = {"__type": "CronExpression", "value": "5 4 * * *"} - schema_instance = CronExpressionSchema() - result = schema_instance.load(instance) - expected_instance = CronExpression("5 4 * * *") - assert expected_instance == result diff --git a/tests/api_connexion/schemas/test_config_schema.py b/tests/api_connexion/schemas/test_config_schema.py deleted file mode 100644 index 0c5447f74b1ce..0000000000000 --- a/tests/api_connexion/schemas/test_config_schema.py +++ /dev/null @@ -1,59 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from airflow.api_connexion.schemas.config_schema import Config, ConfigOption, ConfigSection, config_schema - - -class TestConfigSchema: - def test_serialize(self): - config = Config( - sections=[ - ConfigSection( - name="sec1", - options=[ - ConfigOption(key="apache", value="airflow"), - ConfigOption(key="hello", value="world"), - ], - ), - ConfigSection( - name="sec2", - options=[ - ConfigOption(key="foo", value="bar"), - ], - ), - ] - ) - result = config_schema.dump(config) - expected = { - "sections": [ - { - "name": "sec1", - "options": [ - {"key": "apache", "value": "airflow"}, - {"key": "hello", "value": "world"}, - ], - }, - { - "name": "sec2", - "options": [ - {"key": "foo", "value": "bar"}, - ], - }, - ] - } - assert result == expected diff --git a/tests/api_connexion/schemas/test_connection_schema.py b/tests/api_connexion/schemas/test_connection_schema.py deleted file mode 100644 index 42bb737c9e8c3..0000000000000 --- a/tests/api_connexion/schemas/test_connection_schema.py +++ /dev/null @@ -1,220 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import re - -import marshmallow -import pytest - -from airflow.api_connexion.schemas.connection_schema import ( - ConnectionCollection, - connection_collection_item_schema, - connection_collection_schema, - connection_schema, - connection_test_schema, -) -from airflow.models import Connection -from airflow.utils.session import create_session, provide_session - -from tests_common.test_utils.db import clear_db_connections - -pytestmark = pytest.mark.db_test - - -class TestConnectionCollectionItemSchema: - def setup_method(self) -> None: - with create_session() as session: - session.query(Connection).delete() - - def teardown_method(self) -> None: - clear_db_connections() - - @provide_session - def test_serialize(self, session): - connection_model = Connection( - conn_id="mysql_default", - conn_type="mysql", - host="mysql", - login="login", - schema="testschema", - port=80, - ) - session.add(connection_model) - session.commit() - connection_model = session.query(Connection).first() - deserialized_connection = connection_collection_item_schema.dump(connection_model) - assert deserialized_connection == { - "connection_id": "mysql_default", - "conn_type": "mysql", - "description": None, - "host": "mysql", - "login": "login", - "schema": "testschema", - "port": 80, - } - - def test_deserialize(self): - connection_dump_1 = { - "connection_id": "mysql_default_1", - "conn_type": "mysql", - "host": "mysql", - "login": "login", - "schema": "testschema", - "port": 80, - } - connection_dump_2 = { - "connection_id": "mysql_default_2", - "conn_type": "postgres", - } - result_1 = connection_collection_item_schema.load(connection_dump_1) - result_2 = connection_collection_item_schema.load(connection_dump_2) - - assert result_1 == { - "conn_id": "mysql_default_1", - "conn_type": "mysql", - "host": "mysql", - "login": "login", - "schema": "testschema", - "port": 80, - } - assert result_2 == { - "conn_id": "mysql_default_2", - "conn_type": "postgres", - } - - def test_deserialize_required_fields(self): - connection_dump_1 = { - "connection_id": "mysql_default_2", - } - with pytest.raises( - marshmallow.exceptions.ValidationError, - match=re.escape("{'conn_type': ['Missing data for required field.']}"), - ): - connection_collection_item_schema.load(connection_dump_1) - - -class TestConnectionCollectionSchema: - def setup_method(self) -> None: - with create_session() as session: - session.query(Connection).delete() - - def teardown_method(self) -> None: - clear_db_connections() - - @provide_session - def test_serialize(self, session): - connection_model_1 = Connection(conn_id="mysql_default_1", conn_type="test-type") - connection_model_2 = Connection(conn_id="mysql_default_2", conn_type="test-type2") - connections = [connection_model_1, connection_model_2] - session.add_all(connections) - session.commit() - instance = ConnectionCollection(connections=connections, total_entries=2) - deserialized_connections = connection_collection_schema.dump(instance) - assert deserialized_connections == { - "connections": [ - { - "connection_id": "mysql_default_1", - "conn_type": "test-type", - "description": None, - "host": None, - "login": None, - "schema": None, - "port": None, - }, - { - "connection_id": "mysql_default_2", - "conn_type": "test-type2", - "description": None, - "host": None, - "login": None, - "schema": None, - "port": None, - }, - ], - "total_entries": 2, - } - - -class TestConnectionSchema: - def setup_method(self) -> None: - with create_session() as session: - session.query(Connection).delete() - - def teardown_method(self) -> None: - clear_db_connections() - - @provide_session - def test_serialize(self, session): - connection_model = Connection( - conn_id="mysql_default", - conn_type="mysql", - host="mysql", - login="login", - schema="testschema", - port=80, - password="test-password", - extra='{"key": "string"}', - ) - session.add(connection_model) - session.commit() - connection_model = session.query(Connection).first() - deserialized_connection = connection_schema.dump(connection_model) - assert deserialized_connection == { - "connection_id": "mysql_default", - "conn_type": "mysql", - "description": None, - "host": "mysql", - "login": "login", - "schema": "testschema", - "port": 80, - "extra": '{"key": "string"}', - } - - def test_deserialize(self): - den = { - "connection_id": "mysql_default", - "conn_type": "mysql", - "host": "mysql", - "login": "login", - "schema": "testschema", - "port": 80, - "extra": "{'key':'string'}", - } - result = connection_schema.load(den) - assert result == { - "conn_id": "mysql_default", - "conn_type": "mysql", - "host": "mysql", - "login": "login", - "schema": "testschema", - "port": 80, - "extra": "{'key':'string'}", - } - - -class TestConnectionTestSchema: - def test_response(self): - data = { - "status": True, - "message": "Connection tested successful", - } - result = connection_test_schema.load(data) - assert result == { - "status": True, - "message": "Connection tested successful", - } diff --git a/tests/api_connexion/schemas/test_dag_run_schema.py b/tests/api_connexion/schemas/test_dag_run_schema.py deleted file mode 100644 index 04dcd7036999e..0000000000000 --- a/tests/api_connexion/schemas/test_dag_run_schema.py +++ /dev/null @@ -1,217 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest -from dateutil.parser import parse - -from airflow.api_connexion.exceptions import BadRequest -from airflow.api_connexion.schemas.dag_run_schema import ( - DAGRunCollection, - dagrun_collection_schema, - dagrun_schema, -) -from airflow.models import DagRun -from airflow.utils import timezone -from airflow.utils.session import provide_session -from airflow.utils.types import DagRunType - -from tests_common.test_utils.db import clear_db_runs -from tests_common.test_utils.version_compat import AIRFLOW_V_3_0_PLUS - -if AIRFLOW_V_3_0_PLUS: - from airflow.utils.types import DagRunTriggeredByType - -DEFAULT_TIME = "2020-06-09T13:59:56.336000+00:00" - -SECOND_TIME = "2020-06-10T13:59:56.336000+00:00" - -pytestmark = pytest.mark.db_test - - -class TestDAGRunBase: - def setup_method(self) -> None: - clear_db_runs() - self.default_time = DEFAULT_TIME - self.second_time = SECOND_TIME - - def teardown_method(self) -> None: - clear_db_runs() - - -class TestDAGRunSchema(TestDAGRunBase): - @provide_session - def test_serialize(self, session): - triggered_by_kwargs = {"triggered_by": DagRunTriggeredByType.TEST} if AIRFLOW_V_3_0_PLUS else {} - dagrun_model = DagRun( - dag_id="my-dag-run", - run_id="my-dag-run", - state="running", - run_type=DagRunType.MANUAL.value, - logical_date=timezone.parse(self.default_time), - run_after=timezone.parse(self.default_time), - start_date=timezone.parse(self.default_time), - conf='{"start": "stop"}', - **triggered_by_kwargs, - ) - session.add(dagrun_model) - session.commit() - dagrun_model = session.query(DagRun).first() - deserialized_dagrun = dagrun_schema.dump(dagrun_model) - expected_deserialized_dagrun = { - "dag_id": "my-dag-run", - "dag_run_id": "my-dag-run", - "end_date": None, - "state": "running", - "logical_date": self.default_time, - "start_date": self.default_time, - "conf": {"start": "stop"}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - "run_after": self.default_time, - } - expected_deserialized_dagrun.update({"triggered_by": "test"} if AIRFLOW_V_3_0_PLUS else {}) - - assert deserialized_dagrun == expected_deserialized_dagrun - - @pytest.mark.parametrize( - "serialized_dagrun, expected_result", - [ - ( # Conf not provided - {"dag_run_id": "my-dag-run", "run_after": DEFAULT_TIME}, - {"run_id": "my-dag-run", "run_after": parse(DEFAULT_TIME)}, - ), - ( - { - "dag_run_id": "my-dag-run", - "run_after": DEFAULT_TIME, - "conf": {"start": "stop"}, - }, - { - "run_id": "my-dag-run", - "run_after": parse(DEFAULT_TIME), - "conf": {"start": "stop"}, - }, - ), - ( - { - "dag_run_id": "my-dag-run", - "run_after": DEFAULT_TIME, - "conf": '{"start": "stop"}', - }, - { - "run_id": "my-dag-run", - "run_after": parse(DEFAULT_TIME), - "conf": {"start": "stop"}, - }, - ), - ], - ) - def test_deserialize(self, serialized_dagrun, expected_result): - result = dagrun_schema.load(serialized_dagrun) - assert result == expected_result - - def test_autofill_fields(self): - """Dag_run_id and logical_date fields are autogenerated if missing""" - serialized_dagrun = {} - result = dagrun_schema.load(serialized_dagrun) - assert result == {"run_after": result["run_after"], "run_id": result["run_id"]} - - def test_invalid_logical_date_raises(self): - serialized_dagrun = {"logical_date": "mydate"} - with pytest.raises(BadRequest) as ctx: - dagrun_schema.load(serialized_dagrun) - assert str(ctx.value) == "Incorrect datetime argument" - - -class TestDagRunCollection(TestDAGRunBase): - @provide_session - def test_serialize(self, session): - triggered_by_rest_api_kwargs = ( - {"triggered_by": DagRunTriggeredByType.REST_API} if AIRFLOW_V_3_0_PLUS else {} - ) - dagrun_model_1 = DagRun( - dag_id="my-dag-run", - run_id="my-dag-run", - state="running", - logical_date=timezone.parse(self.default_time), - run_after=timezone.parse(self.default_time), - run_type=DagRunType.MANUAL.value, - start_date=timezone.parse(self.default_time), - conf='{"start": "stop"}', - **triggered_by_rest_api_kwargs, - ) - triggered_by_ui_kwargs = {"triggered_by": DagRunTriggeredByType.UI} if AIRFLOW_V_3_0_PLUS else {} - dagrun_model_2 = DagRun( - dag_id="my-dag-run", - run_id="my-dag-run-2", - state="running", - logical_date=timezone.parse(self.second_time), - start_date=timezone.parse(self.default_time), - run_type=DagRunType.MANUAL.value, - **triggered_by_ui_kwargs, - ) - dagruns = [dagrun_model_1, dagrun_model_2] - session.add_all(dagruns) - session.commit() - instance = DAGRunCollection(dag_runs=dagruns, total_entries=2) - deserialized_dagruns = dagrun_collection_schema.dump(instance) - expected_deserialized_dagruns_rest_api = { - "dag_id": "my-dag-run", - "dag_run_id": "my-dag-run", - "end_date": None, - "logical_date": self.default_time, - "run_after": self.default_time, - "state": "running", - "start_date": self.default_time, - "conf": {"start": "stop"}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_deserialized_dagruns_rest_api.update( - {"triggered_by": "rest_api"} if AIRFLOW_V_3_0_PLUS else {} - ) - expected_deserialized_dagruns_ui = { - "dag_id": "my-dag-run", - "dag_run_id": "my-dag-run-2", - "end_date": None, - "state": "running", - "logical_date": self.second_time, - "run_after": self.second_time, - "start_date": self.default_time, - "conf": {}, - "data_interval_end": None, - "data_interval_start": None, - "last_scheduling_decision": None, - "run_type": "manual", - "note": None, - } - expected_deserialized_dagruns_ui.update({"triggered_by": "ui"} if AIRFLOW_V_3_0_PLUS else {}) - - assert deserialized_dagruns == { - "dag_runs": [ - expected_deserialized_dagruns_rest_api, - expected_deserialized_dagruns_ui, - ], - "total_entries": 2, - } diff --git a/tests/api_connexion/schemas/test_dag_schema.py b/tests/api_connexion/schemas/test_dag_schema.py deleted file mode 100644 index 800b512f993bc..0000000000000 --- a/tests/api_connexion/schemas/test_dag_schema.py +++ /dev/null @@ -1,258 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from datetime import datetime, timedelta - -import pendulum -import pytest - -from airflow.api_connexion.schemas.dag_schema import ( - DAGCollection, - DAGCollectionSchema, - DAGDetailSchema, - DAGSchema, -) -from airflow.models import DagModel, DagTag -from airflow.models.dag import DAG -from airflow.sdk.definitions.asset import Asset - -UTC_JSON_REPR = "UTC" if pendulum.__version__.startswith("3") else "Timezone('UTC')" - - -def test_serialize_test_dag_schema(url_safe_serializer): - dag_model = DagModel( - dag_id="test_dag_id", - is_paused=True, - is_active=True, - fileloc="/root/airflow/dags/my_dag.py", - owners="airflow1,airflow2", - description="The description", - timetable_summary="5 4 * * *", - tags=[DagTag(name="tag-1"), DagTag(name="tag-2")], - ) - serialized_dag = DAGSchema().dump(dag_model) - - assert serialized_dag == { - "dag_id": "test_dag_id", - "dag_display_name": "test_dag_id", - "description": "The description", - "fileloc": "/root/airflow/dags/my_dag.py", - "file_token": url_safe_serializer.dumps("/root/airflow/dags/my_dag.py"), - "is_paused": True, - "is_active": True, - "owners": ["airflow1", "airflow2"], - "timetable_summary": "5 4 * * *", - "tags": [{"name": "tag-1"}, {"name": "tag-2"}], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": None, - } - - -def test_serialize_test_dag_collection_schema(url_safe_serializer): - dag_model_a = DagModel(dag_id="test_dag_id_a", fileloc="/tmp/a.py") - dag_model_b = DagModel(dag_id="test_dag_id_b", fileloc="/tmp/a.py") - schema = DAGCollectionSchema() - instance = DAGCollection(dags=[dag_model_a, dag_model_b], total_entries=2) - assert schema.dump(instance) == { - "dags": [ - { - "dag_id": "test_dag_id_a", - "dag_display_name": "test_dag_id_a", - "description": None, - "fileloc": "/tmp/a.py", - "file_token": url_safe_serializer.dumps("/tmp/a.py"), - "is_paused": None, - "is_active": None, - "owners": [], - "timetable_summary": None, - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "max_consecutive_failed_dag_runs": 0, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": None, - }, - { - "dag_id": "test_dag_id_b", - "dag_display_name": "test_dag_id_b", - "description": None, - "fileloc": "/tmp/a.py", - "file_token": url_safe_serializer.dumps("/tmp/a.py"), - "is_active": None, - "is_paused": None, - "owners": [], - "timetable_summary": None, - "tags": [], - "next_dagrun": None, - "has_task_concurrency_limits": True, - "next_dagrun_data_interval_start": None, - "next_dagrun_data_interval_end": None, - "max_active_runs": 16, - "next_dagrun_create_after": None, - "last_expired": None, - "max_active_tasks": 16, - "max_consecutive_failed_dag_runs": 0, - "default_view": None, - "last_parsed_time": None, - "timetable_description": None, - "has_import_errors": None, - }, - ], - "total_entries": 2, - } - - -@pytest.mark.db_test -def test_serialize_test_dag_detail_schema(url_safe_serializer): - dag = DAG( - dag_id="test_dag", - schedule=timedelta(days=1), - start_date=datetime(2020, 6, 19), - doc_md="docs", - orientation="LR", - default_view="duration", - params={"foo": 1}, - tags=["example1", "example2"], - ) - schema = DAGDetailSchema() - - expected = { - "catchup": True, - "max_active_tasks": 16, - "dag_id": "test_dag", - "dag_display_name": "test_dag", - "dag_run_timeout": None, - "default_view": "duration", - "description": None, - "doc_md": "docs", - "fileloc": __file__, - "file_token": url_safe_serializer.dumps(__file__), - "is_active": None, - "is_paused": None, - "orientation": "LR", - "owners": [], - "params": { - "foo": { - "__class": "airflow.sdk.definitions.param.Param", - "value": 1, - "description": None, - "schema": {}, - } - }, - "start_date": "2020-06-19T00:00:00+00:00", - "tags": sorted( - [{"name": "example1"}, {"name": "example2"}], - key=lambda val: val["name"], - ), - "template_searchpath": None, - "timetable_summary": "1 day, 0:00:00", - "timezone": UTC_JSON_REPR, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "end_date": None, - "is_paused_upon_creation": None, - "render_template_as_native_obj": False, - } - obj = schema.dump(dag) - expected.update({"last_parsed": obj["last_parsed"]}) - obj["tags"] = sorted( - obj["tags"], - key=lambda val: val["name"], - ) - assert obj == expected - - -@pytest.mark.db_test -def test_serialize_test_dag_with_asset_schedule_detail_schema(url_safe_serializer): - asset1 = Asset(uri="s3://bucket/obj1", name="asset1") - asset2 = Asset(uri="s3://bucket/obj2", name="asset2") - dag = DAG( - dag_id="test_dag", - start_date=datetime(2020, 6, 19), - doc_md="docs", - orientation="LR", - default_view="duration", - params={"foo": 1}, - schedule=asset1 & asset2, - tags=["example1", "example2"], - ) - schema = DAGDetailSchema() - - expected = { - "catchup": True, - "max_active_tasks": 16, - "dag_id": "test_dag", - "dag_display_name": "test_dag", - "dag_run_timeout": None, - "default_view": "duration", - "description": None, - "doc_md": "docs", - "fileloc": __file__, - "file_token": url_safe_serializer.dumps(__file__), - "is_active": None, - "is_paused": None, - "orientation": "LR", - "owners": [], - "params": { - "foo": { - "__class": "airflow.sdk.definitions.param.Param", - "value": 1, - "description": None, - "schema": {}, - } - }, - "start_date": "2020-06-19T00:00:00+00:00", - "tags": sorted( - [{"name": "example1"}, {"name": "example2"}], - key=lambda val: val["name"], - ), - "template_searchpath": None, - "timetable_summary": "Asset", - "timezone": UTC_JSON_REPR, - "max_active_runs": 16, - "max_consecutive_failed_dag_runs": 0, - "end_date": None, - "is_paused_upon_creation": None, - "render_template_as_native_obj": False, - } - obj = schema.dump(dag) - expected.update({"last_parsed": obj["last_parsed"]}) - obj["tags"] = sorted( - obj["tags"], - key=lambda val: val["name"], - ) - assert obj == expected diff --git a/tests/api_connexion/schemas/test_dag_stats_schema.py b/tests/api_connexion/schemas/test_dag_stats_schema.py deleted file mode 100644 index 8899b3fb1227a..0000000000000 --- a/tests/api_connexion/schemas/test_dag_stats_schema.py +++ /dev/null @@ -1,87 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from airflow.api_connexion.schemas.dag_stats_schema import ( - dag_stats_collection_schema, - dag_stats_schema, - dag_stats_state_schema, -) -from airflow.utils.state import DagRunState - - -class TestDagStatsStateSchema: - def test_dag_stats_state_schema(self): - payload = { - "state": DagRunState.RUNNING, - "count": 2, - } - serialized_data = dag_stats_state_schema.dump(payload) - assert serialized_data == payload - - -class TestDagStatsSchema: - def test_dag_stats_schema(self): - payload = { - "dag_id": "test_dag_id", - "stats": [ - { - "state": DagRunState.QUEUED, - "count": 2, - }, - { - "state": DagRunState.FAILED, - "count": 1, - }, - ], - } - serialized_data = dag_stats_schema.dump(payload) - assert serialized_data == payload - - -class TestDagStatsCollectionSchema: - def test_dag_stats_collection_schema(self): - payload = { - "dags": [ - { - "dag_id": "test_dag_id", - "stats": [ - { - "state": DagRunState.RUNNING, - "count": 2, - }, - { - "state": DagRunState.SUCCESS, - "count": 1, - }, - ], - }, - { - "dag_id": "test_dag_id_2", - "stats": [ - { - "state": DagRunState.RUNNING, - "count": 2, - } - ], - }, - {"dag_id": "test_dag_id_3", "stats": []}, - ], - "total_entries": 3, - } - serialized_data = dag_stats_collection_schema.dump(payload) - assert serialized_data == payload diff --git a/tests/api_connexion/schemas/test_error_schema.py b/tests/api_connexion/schemas/test_error_schema.py deleted file mode 100644 index 29d4e222900ec..0000000000000 --- a/tests/api_connexion/schemas/test_error_schema.py +++ /dev/null @@ -1,104 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from airflow.api_connexion.schemas.error_schema import ( - ImportErrorCollection, - import_error_collection_schema, - import_error_schema, -) -from airflow.utils import timezone -from airflow.utils.session import provide_session - -from tests_common.test_utils.compat import ParseImportError -from tests_common.test_utils.db import clear_db_import_errors - -pytestmark = pytest.mark.db_test - - -class TestErrorSchemaBase: - def setup_method(self) -> None: - clear_db_import_errors() - self.timestamp = "2020-06-10T12:02:44" - - def teardown_method(self) -> None: - clear_db_import_errors() - - -class TestErrorSchema(TestErrorSchemaBase): - @provide_session - def test_serialize(self, session): - import_error = ParseImportError( - filename="lorem.py", - stacktrace="Lorem Ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC"), - ) - session.add(import_error) - session.commit() - serialized_data = import_error_schema.dump(import_error) - serialized_data["import_error_id"] = 1 - assert serialized_data == { - "filename": "lorem.py", - "bundle_name": None, - "import_error_id": 1, - "stack_trace": "Lorem Ipsum", - "timestamp": "2020-06-10T12:02:44+00:00", - } - - -class TestErrorCollectionSchema(TestErrorSchemaBase): - @provide_session - def test_serialize(self, session): - import_error = [ - ParseImportError( - filename="Lorem_ipsum.py", - stacktrace="Lorem ipsum", - timestamp=timezone.parse(self.timestamp, timezone="UTC"), - ) - for i in range(2) - ] - session.add_all(import_error) - session.commit() - query = session.query(ParseImportError) - query_list = query.all() - serialized_data = import_error_collection_schema.dump( - ImportErrorCollection(import_errors=query_list, total_entries=2) - ) - # To maintain consistency in the key sequence across the db in tests - serialized_data["import_errors"][0]["import_error_id"] = 1 - serialized_data["import_errors"][1]["import_error_id"] = 2 - assert serialized_data == { - "import_errors": [ - { - "filename": "Lorem_ipsum.py", - "bundle_name": None, - "import_error_id": 1, - "stack_trace": "Lorem ipsum", - "timestamp": "2020-06-10T12:02:44+00:00", - }, - { - "filename": "Lorem_ipsum.py", - "bundle_name": None, - "import_error_id": 2, - "stack_trace": "Lorem ipsum", - "timestamp": "2020-06-10T12:02:44+00:00", - }, - ], - "total_entries": 2, - } diff --git a/tests/api_connexion/schemas/test_event_log_schema.py b/tests/api_connexion/schemas/test_event_log_schema.py deleted file mode 100644 index f44346f7f755d..0000000000000 --- a/tests/api_connexion/schemas/test_event_log_schema.py +++ /dev/null @@ -1,109 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from airflow.api_connexion.schemas.event_log_schema import ( - EventLogCollection, - event_log_collection_schema, - event_log_schema, -) -from airflow.models import Log -from airflow.utils import timezone - -pytestmark = pytest.mark.db_test - - -@pytest.fixture -def task_instance(session, create_task_instance, request): - return create_task_instance( - session=session, - dag_id="TEST_DAG_ID", - task_id="TEST_TASK_ID", - run_id="TEST_RUN_ID", - logical_date=request.instance.default_time, - ) - - -class TestEventLogSchemaBase: - @pytest.fixture(autouse=True) - def set_attrs(self): - self.default_time = timezone.parse("2020-06-09T13:00:00+00:00") - self.default_time2 = timezone.parse("2020-06-11T07:00:00+00:00") - - -class TestEventLogSchema(TestEventLogSchemaBase): - def test_serialize(self, task_instance): - event_log_model = Log(event="TEST_EVENT", task_instance=task_instance) - event_log_model.dttm = self.default_time - deserialized_log = event_log_schema.dump(event_log_model) - assert deserialized_log == { - "event_log_id": event_log_model.id, - "event": "TEST_EVENT", - "dag_id": "TEST_DAG_ID", - "task_id": "TEST_TASK_ID", - "run_id": "TEST_RUN_ID", - "map_index": -1, - "try_number": 0, - "logical_date": self.default_time.isoformat(), - "owner": "airflow", - "when": self.default_time.isoformat(), - "extra": None, - } - - -class TestEventLogCollection(TestEventLogSchemaBase): - def test_serialize(self, task_instance): - event_log_model_1 = Log(event="TEST_EVENT_1", task_instance=task_instance) - event_log_model_2 = Log(event="TEST_EVENT_2", task_instance=task_instance) - event_logs = [event_log_model_1, event_log_model_2] - event_log_model_1.dttm = self.default_time - event_log_model_2.dttm = self.default_time2 - instance = EventLogCollection(event_logs=event_logs, total_entries=2) - deserialized_event_logs = event_log_collection_schema.dump(instance) - assert deserialized_event_logs == { - "event_logs": [ - { - "event_log_id": event_log_model_1.id, - "event": "TEST_EVENT_1", - "dag_id": "TEST_DAG_ID", - "task_id": "TEST_TASK_ID", - "run_id": "TEST_RUN_ID", - "map_index": -1, - "try_number": 0, - "logical_date": self.default_time.isoformat(), - "owner": "airflow", - "when": self.default_time.isoformat(), - "extra": None, - }, - { - "event_log_id": event_log_model_2.id, - "event": "TEST_EVENT_2", - "dag_id": "TEST_DAG_ID", - "task_id": "TEST_TASK_ID", - "run_id": "TEST_RUN_ID", - "map_index": -1, - "try_number": 0, - "logical_date": self.default_time.isoformat(), - "owner": "airflow", - "when": self.default_time2.isoformat(), - "extra": None, - }, - ], - "total_entries": 2, - } diff --git a/tests/api_connexion/schemas/test_health_schema.py b/tests/api_connexion/schemas/test_health_schema.py deleted file mode 100644 index fe0c83e261572..0000000000000 --- a/tests/api_connexion/schemas/test_health_schema.py +++ /dev/null @@ -1,35 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from airflow.api_connexion.schemas.health_schema import health_schema - - -class TestHealthSchema: - def setup_method(self): - self.default_datetime = "2020-06-10T12:02:44+00:00" - - def test_serialize(self): - payload = { - "metadatabase": {"status": "healthy"}, - "scheduler": { - "status": "healthy", - "latest_scheduler_heartbeat": self.default_datetime, - }, - } - serialized_data = health_schema.dump(payload) - assert serialized_data == payload diff --git a/tests/api_connexion/schemas/test_plugin_schema.py b/tests/api_connexion/schemas/test_plugin_schema.py deleted file mode 100644 index 87d25dfb3f87a..0000000000000 --- a/tests/api_connexion/schemas/test_plugin_schema.py +++ /dev/null @@ -1,137 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from unittest.mock import MagicMock - -from flask import Blueprint -from flask_appbuilder import BaseView - -from airflow.api_connexion.schemas.plugin_schema import ( - PluginCollection, - plugin_collection_schema, - plugin_schema, -) -from airflow.plugins_manager import AirflowPlugin - -from tests_common.test_utils.compat import BaseOperatorLink - - -def plugin_macro(): ... - - -class MockOperatorLink(BaseOperatorLink): - name = "mock_operator_link" - - def get_link(self, operator, *, ti_key) -> str: - return "mock_operator_link" - - -bp = Blueprint("mock_blueprint", __name__, url_prefix="/mock_blueprint") - - -class MockView(BaseView): ... - - -appbuilder_menu_items = { - "name": "mock_plugin", - "href": "https://example.com", -} - -app = MagicMock() - - -class MockPlugin(AirflowPlugin): - name = "mock_plugin" - flask_blueprints = [bp] - fastapi_apps = [{"app": app, "name": "App name", "url_prefix": "/some_prefix"}] - appbuilder_views = [{"view": MockView()}] - appbuilder_menu_items = [appbuilder_menu_items] - global_operator_extra_links = [MockOperatorLink()] - operator_extra_links = [MockOperatorLink()] - macros = [plugin_macro] - - -class TestPluginBase: - def setup_method(self) -> None: - self.mock_plugin = MockPlugin() - self.mock_plugin.name = "test_plugin" - - self.mock_plugin_2 = MockPlugin() - self.mock_plugin_2.name = "test_plugin_2" - - -class TestPluginSchema(TestPluginBase): - def test_serialize(self): - deserialized_plugin = plugin_schema.dump(self.mock_plugin) - assert deserialized_plugin == { - "appbuilder_menu_items": [appbuilder_menu_items], - "appbuilder_views": [{"view": self.mock_plugin.appbuilder_views[0]["view"]}], - "flask_blueprints": [str(bp)], - "fastapi_apps": [ - {"app": app, "name": "App name", "url_prefix": "/some_prefix"}, - ], - "global_operator_extra_links": [str(MockOperatorLink())], - "macros": [str(plugin_macro)], - "operator_extra_links": [str(MockOperatorLink())], - "source": None, - "name": "test_plugin", - "listeners": [], - "timetables": [], - } - - -class TestPluginCollectionSchema(TestPluginBase): - def test_serialize(self): - plugins = [self.mock_plugin, self.mock_plugin_2] - - deserialized = plugin_collection_schema.dump(PluginCollection(plugins=plugins, total_entries=2)) - assert deserialized == { - "plugins": [ - { - "appbuilder_menu_items": [appbuilder_menu_items], - "appbuilder_views": [{"view": self.mock_plugin.appbuilder_views[0]["view"]}], - "flask_blueprints": [str(bp)], - "fastapi_apps": [ - {"app": app, "name": "App name", "url_prefix": "/some_prefix"}, - ], - "global_operator_extra_links": [str(MockOperatorLink())], - "macros": [str(plugin_macro)], - "operator_extra_links": [str(MockOperatorLink())], - "source": None, - "name": "test_plugin", - "listeners": [], - "timetables": [], - }, - { - "appbuilder_menu_items": [appbuilder_menu_items], - "appbuilder_views": [{"view": self.mock_plugin.appbuilder_views[0]["view"]}], - "flask_blueprints": [str(bp)], - "fastapi_apps": [ - {"app": app, "name": "App name", "url_prefix": "/some_prefix"}, - ], - "global_operator_extra_links": [str(MockOperatorLink())], - "macros": [str(plugin_macro)], - "operator_extra_links": [str(MockOperatorLink())], - "source": None, - "name": "test_plugin_2", - "listeners": [], - "timetables": [], - }, - ], - "total_entries": 2, - } diff --git a/tests/api_connexion/schemas/test_pool_schemas.py b/tests/api_connexion/schemas/test_pool_schemas.py deleted file mode 100644 index 255f2c1ee6eac..0000000000000 --- a/tests/api_connexion/schemas/test_pool_schemas.py +++ /dev/null @@ -1,103 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from airflow.api_connexion.schemas.pool_schema import PoolCollection, pool_collection_schema, pool_schema -from airflow.models.pool import Pool -from airflow.utils.session import provide_session - -from tests_common.test_utils.db import clear_db_pools - -pytestmark = pytest.mark.db_test - - -class TestPoolSchema: - def setup_method(self) -> None: - clear_db_pools() - - def teardown_method(self) -> None: - clear_db_pools() - - @provide_session - def test_serialize(self, session): - pool_model = Pool(pool="test_pool", slots=2, include_deferred=False) - session.add(pool_model) - session.commit() - pool_instance = session.query(Pool).filter(Pool.pool == pool_model.pool).first() - serialized_pool = pool_schema.dump(pool_instance) - assert serialized_pool == { - "name": "test_pool", - "slots": 2, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": 2, - "description": None, - "include_deferred": False, - } - - @provide_session - def test_deserialize(self, session): - pool_dict = {"name": "test_pool", "slots": 3, "include_deferred": True} - deserialized_pool = pool_schema.load(pool_dict, session=session) - assert not isinstance(deserialized_pool, Pool) # Checks if load_instance is set to True - - -class TestPoolCollectionSchema: - def setup_method(self) -> None: - clear_db_pools() - - def teardown_method(self) -> None: - clear_db_pools() - - def test_serialize(self): - pool_model_a = Pool(pool="test_pool_a", slots=3, include_deferred=False) - pool_model_b = Pool(pool="test_pool_b", slots=3, include_deferred=True) - instance = PoolCollection(pools=[pool_model_a, pool_model_b], total_entries=2) - assert pool_collection_schema.dump(instance) == { - "pools": [ - { - "name": "test_pool_a", - "slots": 3, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": 3, - "description": None, - "include_deferred": False, - }, - { - "name": "test_pool_b", - "slots": 3, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": 3, - "description": None, - "include_deferred": True, - }, - ], - "total_entries": 2, - } diff --git a/tests/api_connexion/schemas/test_task_instance_schema.py b/tests/api_connexion/schemas/test_task_instance_schema.py deleted file mode 100644 index 7dc2af3a116e1..0000000000000 --- a/tests/api_connexion/schemas/test_task_instance_schema.py +++ /dev/null @@ -1,220 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import datetime as dt - -import pytest -from marshmallow import ValidationError - -from airflow.api_connexion.schemas.task_instance_schema import ( - clear_task_instance_form, - set_task_instance_state_form, - task_instance_schema, -) -from airflow.models import TaskInstance as TI -from airflow.providers.standard.operators.empty import EmptyOperator -from airflow.utils.platform import getuser -from airflow.utils.state import State -from airflow.utils.timezone import datetime - - -@pytest.mark.db_test -class TestTaskInstanceSchema: - @pytest.fixture(autouse=True) - def set_attrs(self, session, dag_maker): - self.default_time = datetime(2020, 1, 1) - with dag_maker(dag_id="TEST_DAG_ID", session=session): - self.task = EmptyOperator(task_id="TEST_TASK_ID", start_date=self.default_time) - - self.dr = dag_maker.create_dagrun(logical_date=self.default_time) - session.flush() - - self.default_ti_init = { - "run_id": None, - "state": State.RUNNING, - } - self.default_ti_extras = { - "dag_run": self.dr, - "start_date": self.default_time + dt.timedelta(days=1), - "end_date": self.default_time + dt.timedelta(days=2), - "pid": 100, - "duration": 10000, - "pool": "default_pool", - "queue": "default_queue", - "note": "added some notes", - } - - yield - - session.rollback() - - def test_task_instance_schema_without_rendered(self, session): - ti = TI(task=self.task, **self.default_ti_init) - session.add(ti) - for key, value in self.default_ti_extras.items(): - setattr(ti, key, value) - serialized_ti = task_instance_schema.dump(ti) - expected_json = { - "dag_id": "TEST_DAG_ID", - "duration": 10000.0, - "end_date": "2020-01-03T00:00:00+00:00", - "logical_date": "2020-01-01T00:00:00+00:00", - "executor": None, - "executor_config": "{}", - "hostname": "", - "map_index": -1, - "max_tries": 0, - "note": "added some notes", - "operator": "EmptyOperator", - "pid": 100, - "pool": "default_pool", - "pool_slots": 1, - "priority_weight": 1, - "queue": "default_queue", - "queued_when": None, - "scheduled_when": None, - "start_date": "2020-01-02T00:00:00+00:00", - "state": "running", - "task_id": "TEST_TASK_ID", - "task_display_name": "TEST_TASK_ID", - "try_number": 0, - "unixname": getuser(), - "dag_run_id": None, - "rendered_fields": {}, - "rendered_map_index": None, - "trigger": None, - "triggerer_job": None, - } - assert serialized_ti == expected_json - - -class TestClearTaskInstanceFormSchema: - @pytest.mark.parametrize( - "payload", - [ - ( - { - "dry_run": False, - "reset_dag_runs": True, - "only_failed": True, - "only_running": True, - } - ), - ( - { - "dry_run": False, - "reset_dag_runs": True, - "end_date": "2020-01-01T00:00:00+00:00", - "start_date": "2020-01-02T00:00:00+00:00", - } - ), - ( - { - "dry_run": False, - "reset_dag_runs": True, - "task_ids": [], - } - ), - ( - { - "dry_run": False, - "reset_dag_runs": True, - "dag_run_id": "scheduled__2022-06-19T00:00:00+00:00", - "start_date": "2022-08-03T00:00:00+00:00", - } - ), - ( - { - "dry_run": False, - "reset_dag_runs": True, - "dag_run_id": "scheduled__2022-06-19T00:00:00+00:00", - "end_date": "2022-08-03T00:00:00+00:00", - } - ), - ( - { - "dry_run": False, - "reset_dag_runs": True, - "dag_run_id": "scheduled__2022-06-19T00:00:00+00:00", - "end_date": "2022-08-04T00:00:00+00:00", - "start_date": "2022-08-03T00:00:00+00:00", - } - ), - ], - ) - def test_validation_error(self, payload): - with pytest.raises(ValidationError): - clear_task_instance_form.load(payload) - - -class TestSetTaskInstanceStateFormSchema: - current_input = { - "dry_run": True, - "task_id": "print_the_context", - "dag_run_id": "test_run_id", - "include_upstream": True, - "include_downstream": True, - "include_future": True, - "include_past": True, - "new_state": "failed", - } - - def test_success(self): - result = set_task_instance_state_form.load(self.current_input) - expected_result = { - "dry_run": True, - "dag_run_id": "test_run_id", - "include_downstream": True, - "include_future": True, - "include_past": True, - "include_upstream": True, - "new_state": "failed", - "task_id": "print_the_context", - } - assert expected_result == result - - def test_dry_run_is_optional(self): - data = self.current_input.copy() - data.pop("dry_run") - result = set_task_instance_state_form.load(self.current_input) - expected_result = { - "dry_run": True, - "dag_run_id": "test_run_id", - "include_downstream": True, - "include_future": True, - "include_past": True, - "include_upstream": True, - "new_state": "failed", - "task_id": "print_the_context", - } - assert expected_result == result - - @pytest.mark.parametrize( - "override_data", - [ - {"task_id": None}, - {"include_future": "foo"}, - {"new_state": "INVALID_STATE"}, - {"logical_date": "2020-01-01T00:00:00+00:00", "dag_run_id": "some-run-id"}, - ], - ) - def test_validation_error(self, override_data): - self.current_input.update(override_data) - - with pytest.raises(ValidationError): - set_task_instance_state_form.load(self.current_input) diff --git a/tests/api_connexion/schemas/test_task_schema.py b/tests/api_connexion/schemas/test_task_schema.py deleted file mode 100644 index 70d37edf251ca..0000000000000 --- a/tests/api_connexion/schemas/test_task_schema.py +++ /dev/null @@ -1,117 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from datetime import datetime - -from airflow.api_connexion.schemas.task_schema import TaskCollection, task_collection_schema, task_schema -from airflow.providers.standard.operators.empty import EmptyOperator - - -class TestTaskSchema: - def test_serialize(self): - op = EmptyOperator( - task_id="task_id", - start_date=datetime(2020, 6, 16), - end_date=datetime(2020, 6, 26), - ) - result = task_schema.dump(op) - expected = { - "class_ref": { - "module_path": "airflow.providers.standard.operators.empty", - "class_name": "EmptyOperator", - }, - "depends_on_past": False, - "downstream_task_ids": [], - "end_date": "2020-06-26T00:00:00+00:00", - "execution_timeout": None, - "extra_links": [], - "owner": "airflow", - "operator_name": "EmptyOperator", - "params": {}, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "seconds": 300, "microseconds": 0}, - "retry_exponential_backoff": False, - "start_date": "2020-06-16T00:00:00+00:00", - "task_id": "task_id", - "task_display_name": "task_id", - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "is_mapped": False, - "doc_md": None, - } - assert expected == result - - -class TestTaskCollectionSchema: - def test_serialize(self): - tasks = [EmptyOperator(task_id="task_id1", params={"foo": "bar"})] - collection = TaskCollection(tasks, 1) - result = task_collection_schema.dump(collection) - expected = { - "tasks": [ - { - "class_ref": { - "class_name": "EmptyOperator", - "module_path": "airflow.providers.standard.operators.empty", - }, - "depends_on_past": False, - "downstream_task_ids": [], - "end_date": None, - "execution_timeout": None, - "extra_links": [], - "operator_name": "EmptyOperator", - "owner": "airflow", - "params": { - "foo": { - "__class": "airflow.sdk.definitions.param.Param", - "value": "bar", - "description": None, - "schema": {}, - } - }, - "pool": "default_pool", - "pool_slots": 1.0, - "priority_weight": 1.0, - "queue": "default", - "retries": 0.0, - "retry_delay": {"__type": "TimeDelta", "days": 0, "seconds": 300, "microseconds": 0}, - "retry_exponential_backoff": False, - "start_date": None, - "task_id": "task_id1", - "task_display_name": "task_id1", - "template_fields": [], - "trigger_rule": "all_success", - "ui_color": "#e8f7e4", - "ui_fgcolor": "#000", - "wait_for_downstream": False, - "weight_rule": "downstream", - "is_mapped": False, - "doc_md": None, - } - ], - "total_entries": 1, - } - assert expected == result diff --git a/tests/api_connexion/schemas/test_xcom_schema.py b/tests/api_connexion/schemas/test_xcom_schema.py deleted file mode 100644 index 084de6a4ef718..0000000000000 --- a/tests/api_connexion/schemas/test_xcom_schema.py +++ /dev/null @@ -1,181 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest -from sqlalchemy import or_, select - -from airflow.api_connexion.schemas.xcom_schema import ( - XComCollection, - xcom_collection_item_schema, - xcom_collection_schema, -) -from airflow.models import DagRun, XCom -from airflow.utils import timezone -from airflow.utils.session import create_session - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module", autouse=True) -def clean_xcom(): - """Ensure there's no XCom littered by other modules.""" - with create_session() as session: - session.query(XCom).delete() - - -def _compare_xcom_collections(collection1: dict, collection_2: dict): - assert collection1.get("total_entries") == collection_2.get("total_entries") - - def sort_key(record): - return ( - record.get("dag_id"), - record.get("task_id"), - record.get("logical_date"), - record.get("map_index"), - record.get("key"), - ) - - assert sorted(collection1.get("xcom_entries", []), key=sort_key) == sorted( - collection_2.get("xcom_entries", []), key=sort_key - ) - - -@pytest.fixture -def create_xcom(create_task_instance, session): - def maker(dag_id, task_id, logical_date, key, map_index=-1, value=None): - ti = create_task_instance( - dag_id=dag_id, - task_id=task_id, - logical_date=logical_date, - map_index=map_index, - session=session, - ) - run: DagRun = ti.dag_run - xcom = XCom( - dag_run_id=run.id, - task_id=ti.task_id, - map_index=map_index, - key=key, - value=value, - timestamp=run.logical_date, - dag_id=run.dag_id, - run_id=run.run_id, - ) - session.add(xcom) - session.commit() - return xcom - - return maker - - -class TestXComCollectionItemSchema: - default_time = "2016-04-02T21:00:00+00:00" - default_time_parsed = timezone.parse(default_time) - - def test_serialize(self, create_xcom, session): - create_xcom( - dag_id="test_dag", - task_id="test_task_id", - logical_date=self.default_time_parsed, - key="test_key", - ) - xcom_model = session.query(XCom).first() - deserialized_xcom = xcom_collection_item_schema.dump(xcom_model) - assert deserialized_xcom == { - "key": "test_key", - "timestamp": self.default_time, - "logical_date": self.default_time, - "task_id": "test_task_id", - "dag_id": "test_dag", - "map_index": -1, - } - - def test_deserialize(self): - xcom_dump = { - "key": "test_key", - "timestamp": self.default_time, - "logical_date": self.default_time, - "task_id": "test_task_id", - "dag_id": "test_dag", - "map_index": 2, - } - result = xcom_collection_item_schema.load(xcom_dump) - assert result == { - "key": "test_key", - "timestamp": self.default_time_parsed, - "logical_date": self.default_time_parsed, - "task_id": "test_task_id", - "dag_id": "test_dag", - "map_index": 2, - } - - -class TestXComCollectionSchema: - default_time_1 = "2016-04-02T21:00:00+00:00" - default_time_2 = "2016-04-02T21:01:00+00:00" - time_1 = timezone.parse(default_time_1) - time_2 = timezone.parse(default_time_2) - - def test_serialize(self, create_xcom, session): - create_xcom( - dag_id="test_dag_1", - task_id="test_task_id_1", - logical_date=self.time_1, - key="test_key_1", - ) - create_xcom( - dag_id="test_dag_2", - task_id="test_task_id_2", - logical_date=self.time_2, - key="test_key_2", - ) - xcom_models = session.scalars( - select(XCom) - .where(or_(XCom.logical_date == self.time_1, XCom.logical_date == self.time_2)) - .order_by(XCom.dag_run_id) - ).all() - deserialized_xcoms = xcom_collection_schema.dump( - XComCollection( - xcom_entries=xcom_models, - total_entries=len(xcom_models), - ) - ) - _compare_xcom_collections( - deserialized_xcoms, - { - "xcom_entries": [ - { - "key": "test_key_1", - "timestamp": self.default_time_1, - "logical_date": self.default_time_1, - "task_id": "test_task_id_1", - "dag_id": "test_dag_1", - "map_index": -1, - }, - { - "key": "test_key_2", - "timestamp": self.default_time_2, - "logical_date": self.default_time_2, - "task_id": "test_task_id_2", - "dag_id": "test_dag_2", - "map_index": -1, - }, - ], - "total_entries": 2, - }, - ) diff --git a/tests/api_connexion/test_auth.py b/tests/api_connexion/test_auth.py deleted file mode 100644 index 3fd41d35393db..0000000000000 --- a/tests/api_connexion/test_auth.py +++ /dev/null @@ -1,88 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from unittest.mock import patch - -import pytest - -from airflow.auth.managers.simple.simple_auth_manager import SimpleAuthManager -from airflow.auth.managers.simple.user import SimpleAuthManagerUser - -from tests_common.test_utils.api_connexion_utils import assert_401 -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_pools - -pytestmark = pytest.mark.db_test - - -class BaseTestAuth: - @pytest.fixture(autouse=True) - def set_attrs(self, minimal_app_for_api): - self.app = minimal_app_for_api - - -class TestSessionAuth(BaseTestAuth): - @pytest.fixture(autouse=True, scope="class") - def with_session_backend(self, minimal_app_for_api): - from airflow.www.extensions.init_security import init_api_auth - - old_auth = getattr(minimal_app_for_api, "api_auth") - - try: - with conf_vars( - {("api", "auth_backends"): "airflow.providers.fab.auth_manager.api.auth.backend.session"} - ): - init_api_auth(minimal_app_for_api) - yield - finally: - setattr(minimal_app_for_api, "api_auth", old_auth) - - @patch.object(SimpleAuthManager, "is_logged_in", return_value=True) - @patch.object( - SimpleAuthManager, "get_user", return_value=SimpleAuthManagerUser(username="test", role="admin") - ) - def test_success(self, *args): - clear_db_pools() - - with self.app.test_client() as test_client: - response = test_client.get("/api/v1/pools") - assert response.status_code == 200 - assert response.json == { - "pools": [ - { - "name": "default_pool", - "slots": 128, - "occupied_slots": 0, - "running_slots": 0, - "queued_slots": 0, - "scheduled_slots": 0, - "deferred_slots": 0, - "open_slots": 128, - "description": "Default pool", - "include_deferred": False, - }, - ], - "total_entries": 1, - } - - def test_failure(self): - with self.app.test_client() as test_client: - response = test_client.get("/api/v1/pools") - assert response.status_code == 401 - assert response.headers["Content-Type"] == "application/problem+json" - assert_401(response) diff --git a/tests/api_connexion/test_error_handling.py b/tests/api_connexion/test_error_handling.py deleted file mode 100644 index 36752fac60373..0000000000000 --- a/tests/api_connexion/test_error_handling.py +++ /dev/null @@ -1,79 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -pytestmark = pytest.mark.db_test - - -def test_incorrect_endpoint_should_return_json(minimal_app_for_api): - client = minimal_app_for_api.test_client() - - # Given we have application with Connexion added - # When we are hitting incorrect endpoint in API path - - resp = client.get("/api/v1/incorrect_endpoint") - - # Then we have parsable JSON as output - - assert resp.json["title"] == "Not Found" - assert resp.json["status"] == 404 - assert resp.status_code == 404 - - -def test_incorrect_endpoint_should_return_html(minimal_app_for_api): - client = minimal_app_for_api.test_client() - - # When we are hitting non-api incorrect endpoint - - resp = client.get("/incorrect_endpoint") - - # Then we do not have JSON as response, rather standard HTML - - assert resp.json is None - assert resp.mimetype == "text/html" - assert resp.status_code == 404 - - -def test_incorrect_method_should_return_json(minimal_app_for_api): - client = minimal_app_for_api.test_client() - - # Given we have application with Connexion added - # When we are hitting incorrect HTTP method in API path - - resp = client.put("/api/v1/version") - - # Then we have parsable JSON as output - - assert resp.json["title"] == "Method Not Allowed" - assert resp.json["status"] == 405 - assert resp.status_code == 405 - - -def test_incorrect_method_should_return_html(minimal_app_for_api): - client = minimal_app_for_api.test_client() - - # When we are hitting non-api incorrect HTTP method - - resp = client.put("/") - - # Then we do not have JSON as response, rather standard HTML - - assert resp.json is None - assert resp.mimetype == "text/html" - assert resp.status_code == 405 diff --git a/tests/api_connexion/test_parameters.py b/tests/api_connexion/test_parameters.py deleted file mode 100644 index e64a32c4e65f1..0000000000000 --- a/tests/api_connexion/test_parameters.py +++ /dev/null @@ -1,124 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from unittest import mock - -import pendulum -import pytest - -from airflow.api_connexion.exceptions import BadRequest -from airflow.api_connexion.parameters import ( - check_limit, - format_datetime, - format_parameters, - validate_istimezone, -) -from airflow.utils import timezone - -from tests_common.test_utils.config import conf_vars - - -class TestValidateIsTimezone: - def setup_method(self) -> None: - from datetime import datetime - - self.naive = datetime.now() - self.timezoned = datetime.now(tz=timezone.utc) - - def test_gives_400_for_naive(self): - with pytest.raises(BadRequest): - validate_istimezone(self.naive) - - def test_timezone_passes(self): - assert validate_istimezone(self.timezoned) is None - - -class TestDateTimeParser: - def setup_method(self) -> None: - self.default_time = "2020-06-13T22:44:00+00:00" - self.default_time_2 = "2020-06-13T22:44:00Z" - - def test_works_with_datestring_ending_00_00(self): - datetime = format_datetime(self.default_time) - datetime2 = timezone.parse(self.default_time) - assert datetime == datetime2 - assert datetime.isoformat() == self.default_time - - def test_works_with_datestring_ending_with_zed(self): - datetime = format_datetime(self.default_time_2) - datetime2 = timezone.parse(self.default_time_2) - assert datetime == datetime2 - assert datetime.isoformat() == self.default_time # python uses +00:00 instead of Z - - def test_raises_400_for_invalid_arg(self): - invalid_datetime = "2020-06-13T22:44:00P" - with pytest.raises(BadRequest): - format_datetime(invalid_datetime) - - -class TestMaximumPagelimit: - @conf_vars({("api", "maximum_page_limit"): "320"}) - def test_maximum_limit_return_val(self): - limit = check_limit(300) - assert limit == 300 - - @conf_vars({("api", "maximum_page_limit"): "320"}) - def test_maximum_limit_returns_configured_if_limit_above_conf(self): - limit = check_limit(350) - assert limit == 320 - - @conf_vars({("api", "maximum_page_limit"): "1000"}) - def test_limit_returns_set_max_if_give_limit_is_exceeded(self): - limit = check_limit(1500) - assert limit == 1000 - - @conf_vars({("api", "fallback_page_limit"): "100"}) - def test_limit_of_zero_returns_default(self): - limit = check_limit(0) - assert limit == 100 - - @conf_vars({("api", "maximum_page_limit"): "1500"}) - def test_negative_limit_raises(self): - with pytest.raises(BadRequest): - check_limit(-1) - - -class TestFormatParameters: - def test_should_works_with_datetime_formatter(self): - decorator = format_parameters({"param_a": format_datetime}) - endpoint = mock.MagicMock() - decorated_endpoint = decorator(endpoint) - - decorated_endpoint(param_a="2020-01-01T0:0:00+00:00") - - endpoint.assert_called_once_with(param_a=pendulum.datetime(2020, 1, 1, 0, tz="UTC")) - - def test_should_propagate_exceptions(self): - decorator = format_parameters({"param_a": format_datetime}) - endpoint = mock.MagicMock() - decorated_endpoint = decorator(endpoint) - with pytest.raises(BadRequest): - decorated_endpoint(param_a="XXXXX") - - @conf_vars({("api", "maximum_page_limit"): "100"}) - def test_should_work_with_limit(self): - decorator = format_parameters({"limit": check_limit}) - endpoint = mock.MagicMock() - decorated_endpoint = decorator(endpoint) - decorated_endpoint(limit=89) - endpoint.assert_called_once_with(limit=89) diff --git a/tests/api_connexion/test_security.py b/tests/api_connexion/test_security.py deleted file mode 100644 index bdf6f3baa4484..0000000000000 --- a/tests/api_connexion/test_security.py +++ /dev/null @@ -1,52 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from tests_common.test_utils.api_connexion_utils import create_user, delete_user - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - create_user( - app, - username="test", - role_name="admin", - ) - - yield minimal_app_for_api - - delete_user(app, username="test") - - -class TestSession: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - - def test_session_not_created_on_api_request(self): - self.client.get("api/v1/dags", environ_overrides={"REMOTE_USER": "test"}) - assert all(cookie.name != "session" for cookie in self.client.cookie_jar) - - def test_session_not_created_on_health_endpoint_request(self): - self.client.get("health") - assert all(cookie.name != "session" for cookie in self.client.cookie_jar) diff --git a/tests/api_fastapi/core_api/routes/public/test_assets.py b/tests/api_fastapi/core_api/routes/public/test_assets.py index 68c0026f8ca8a..764d16b5e8326 100644 --- a/tests/api_fastapi/core_api/routes/public/test_assets.py +++ b/tests/api_fastapi/core_api/routes/public/test_assets.py @@ -39,10 +39,10 @@ from airflow.utils.state import DagRunState from airflow.utils.types import DagRunType -from tests_common.test_utils.api_fastapi import _check_last_log from tests_common.test_utils.asserts import assert_queries_count from tests_common.test_utils.db import clear_db_assets, clear_db_logs, clear_db_runs from tests_common.test_utils.format_datetime import from_datetime_to_zulu_without_ms +from tests_common.test_utils.logs import check_last_log DEFAULT_DATE = datetime(2020, 6, 11, 18, 0, 0, tzinfo=timezone.utc) @@ -891,7 +891,7 @@ def test_should_respond_204(self, test_client, session, create_dummy_dag): assert response.status_code == 204 adrqs = session.query(AssetDagRunQueue).all() assert len(adrqs) == 0 - _check_last_log(session, dag_id=dag_id, event="delete_dag_asset_queued_events", logical_date=None) + check_last_log(session, dag_id=dag_id, event="delete_dag_asset_queued_events", logical_date=None) def test_should_respond_404_invalid_dag(self, test_client): dag_id = "not_exists" @@ -939,7 +939,7 @@ def test_should_respond_200(self, test_client, session): "created_dagruns": [], "timestamp": from_datetime_to_zulu_without_ms(DEFAULT_DATE), } - _check_last_log(session, dag_id=None, event="create_asset_event", logical_date=None) + check_last_log(session, dag_id=None, event="create_asset_event", logical_date=None) def test_invalid_attr_not_allowed(self, test_client, session): self.create_assets(session) @@ -1066,7 +1066,7 @@ def test_should_respond_204(self, test_client, session, create_dummy_dag): response = test_client.delete(f"/public/assets/{asset_id}/queuedEvents") assert response.status_code == 204 assert session.get(AssetDagRunQueue, (asset_id, dag_id)) is None - _check_last_log(session, dag_id=None, event="delete_asset_queued_events", logical_date=None) + check_last_log(session, dag_id=None, event="delete_asset_queued_events", logical_date=None) def test_should_respond_404(self, test_client): response = test_client.delete("/public/assets/1/queuedEvents") @@ -1092,7 +1092,7 @@ def test_delete_should_respond_204(self, test_client, session, create_dummy_dag) assert response.status_code == 204 adrq = session.query(AssetDagRunQueue).all() assert len(adrq) == 0 - _check_last_log(session, dag_id=dag_id, event="delete_dag_asset_queued_event", logical_date=None) + check_last_log(session, dag_id=dag_id, event="delete_dag_asset_queued_event", logical_date=None) def test_should_respond_404(self, test_client): dag_id = "not_exists" diff --git a/tests/api_fastapi/core_api/routes/public/test_backfills.py b/tests/api_fastapi/core_api/routes/public/test_backfills.py index 6860cf988c8c7..b72a7e85299b9 100644 --- a/tests/api_fastapi/core_api/routes/public/test_backfills.py +++ b/tests/api_fastapi/core_api/routes/public/test_backfills.py @@ -33,7 +33,6 @@ from airflow.utils.session import provide_session from airflow.utils.state import DagRunState -from tests_common.test_utils.api_fastapi import _check_last_log from tests_common.test_utils.db import ( clear_db_backfills, clear_db_dags, @@ -41,6 +40,7 @@ clear_db_runs, clear_db_serialized_dags, ) +from tests_common.test_utils.logs import check_last_log pytestmark = [pytest.mark.db_test, pytest.mark.need_serialized_dag] @@ -214,7 +214,7 @@ def test_create_backfill(self, repro_act, repro_exp, session, dag_maker, test_cl "to_date": to_date_iso, "updated_at": mock.ANY, } - _check_last_log(session, dag_id="TEST_DAG_1", event="create_backfill", logical_date=None) + check_last_log(session, dag_id="TEST_DAG_1", event="create_backfill", logical_date=None) def test_dag_not_exist(self, session, test_client): session.query(DagModel).all() @@ -680,7 +680,7 @@ def test_cancel_backfill(self, session, test_client): # get conflict when canceling already-canceled backfill response = test_client.put(f"/public/backfills/{backfill.id}/cancel") assert response.status_code == 409 - _check_last_log(session, dag_id=None, event="cancel_backfill", logical_date=None) + check_last_log(session, dag_id=None, event="cancel_backfill", logical_date=None) def test_cancel_backfill_end_states(self, dag_maker, session, test_client): """ diff --git a/tests/api_fastapi/core_api/routes/public/test_dag_parsing.py b/tests/api_fastapi/core_api/routes/public/test_dag_parsing.py index c6aa9b5025d11..0943684e32192 100644 --- a/tests/api_fastapi/core_api/routes/public/test_dag_parsing.py +++ b/tests/api_fastapi/core_api/routes/public/test_dag_parsing.py @@ -63,6 +63,7 @@ def test_201_and_400_requests(self, url_safe_serializer, session, test_client): assert response.status_code == 409 parsing_requests = session.scalars(select(DagPriorityParsingRequest)).all() assert parsing_requests[0].fileloc == test_dag.fileloc + _check_last_log(session, dag_id=None, event="reparse_dag_file", logical_date=None) def test_bad_file_request(self, url_safe_serializer, session, test_client): url = f"/public/parseDagFile/{url_safe_serializer.dumps('/some/random/file.py')}" diff --git a/tests/auth/managers/simple/test_simple_auth_manager.py b/tests/auth/managers/simple/test_simple_auth_manager.py index f0bcc1a73c2d3..4b9227f8aa806 100644 --- a/tests/auth/managers/simple/test_simple_auth_manager.py +++ b/tests/auth/managers/simple/test_simple_auth_manager.py @@ -27,7 +27,6 @@ class TestSimpleAuthManager: - @pytest.mark.db_test def test_get_users(self, auth_manager): with conf_vars( { @@ -37,7 +36,6 @@ def test_get_users(self, auth_manager): users = auth_manager.get_users() assert users == [{"role": "viewer", "username": "test1"}, {"role": "viewer", "username": "test2"}] - @pytest.mark.db_test def test_init_with_default_user(self, auth_manager): auth_manager.init() with open(auth_manager.get_generated_password_file()) as file: @@ -46,7 +44,6 @@ def test_init_with_default_user(self, auth_manager): assert len(user_passwords_from_file) == 1 - @pytest.mark.db_test def test_init_with_users(self, auth_manager): with conf_vars( { @@ -74,7 +71,6 @@ def test_serialize_user(self, auth_manager): result = auth_manager.serialize_user(user) assert result == {"username": "test", "role": "admin"} - @pytest.mark.db_test @pytest.mark.parametrize( "api", [ @@ -96,16 +92,12 @@ def test_serialize_user(self, auth_manager): ("VIEWER", "DELETE", False), ], ) - def test_is_authorized_methods(self, auth_manager, app, api, role, method, result): - with app.test_request_context(): - assert ( - getattr(auth_manager, api)( - method=method, user=SimpleAuthManagerUser(username="test", role=role) - ) - is result - ) - - @pytest.mark.db_test + def test_is_authorized_methods(self, auth_manager, api, role, method, result): + assert ( + getattr(auth_manager, api)(method=method, user=SimpleAuthManagerUser(username="test", role=role)) + is result + ) + @pytest.mark.parametrize( "api, kwargs", [ @@ -128,14 +120,12 @@ def test_is_authorized_methods(self, auth_manager, app, api, role, method, resul ("OP", True), ], ) - def test_is_authorized_view_methods(self, auth_manager, app, api, kwargs, role, result): - with app.test_request_context(): - assert ( - getattr(auth_manager, api)(**kwargs, user=SimpleAuthManagerUser(username="test", role=role)) - is result - ) - - @pytest.mark.db_test + def test_is_authorized_view_methods(self, auth_manager, api, kwargs, role, result): + assert ( + getattr(auth_manager, api)(**kwargs, user=SimpleAuthManagerUser(username="test", role=role)) + is result + ) + @pytest.mark.parametrize( "api", [ @@ -155,16 +145,12 @@ def test_is_authorized_view_methods(self, auth_manager, app, api, kwargs, role, ("VIEWER", "PUT", False), ], ) - def test_is_authorized_methods_op_role_required(self, auth_manager, app, api, role, method, result): - with app.test_request_context(): - assert ( - getattr(auth_manager, api)( - method=method, user=SimpleAuthManagerUser(username="test", role=role) - ) - is result - ) - - @pytest.mark.db_test + def test_is_authorized_methods_op_role_required(self, auth_manager, api, role, method, result): + assert ( + getattr(auth_manager, api)(method=method, user=SimpleAuthManagerUser(username="test", role=role)) + is result + ) + @pytest.mark.parametrize( "api", ["is_authorized_dag"], @@ -179,16 +165,12 @@ def test_is_authorized_methods_op_role_required(self, auth_manager, app, api, ro ("VIEWER", "PUT", False), ], ) - def test_is_authorized_methods_user_role_required(self, auth_manager, app, api, role, method, result): - with app.test_request_context(): - assert ( - getattr(auth_manager, api)( - method=method, user=SimpleAuthManagerUser(username="test", role=role) - ) - is result - ) - - @pytest.mark.db_test + def test_is_authorized_methods_user_role_required(self, auth_manager, api, role, method, result): + assert ( + getattr(auth_manager, api)(method=method, user=SimpleAuthManagerUser(username="test", role=role)) + is result + ) + @pytest.mark.parametrize( "api", ["is_authorized_dag", "is_authorized_asset", "is_authorized_pool"], @@ -204,12 +186,9 @@ def test_is_authorized_methods_user_role_required(self, auth_manager, app, api, ], ) def test_is_authorized_methods_viewer_role_required_for_get( - self, auth_manager, app, api, role, method, result + self, auth_manager, api, role, method, result ): - with app.test_request_context(): - assert ( - getattr(auth_manager, api)( - method=method, user=SimpleAuthManagerUser(username="test", role=role) - ) - is result - ) + assert ( + getattr(auth_manager, api)(method=method, user=SimpleAuthManagerUser(username="test", role=role)) + is result + ) diff --git a/tests/cli/commands/local_commands/test_api_server_command.py b/tests/cli/commands/local_commands/test_api_server_command.py index bc8c2ba0d1189..0693e2ef6aff7 100644 --- a/tests/cli/commands/local_commands/test_api_server_command.py +++ b/tests/cli/commands/local_commands/test_api_server_command.py @@ -63,7 +63,7 @@ class TestCliFastAPI(_CommonCLIGunicornTestClass): ), ], ) - def test_cli_fastapi_api_debug(self, app, args, expected_command): + def test_cli_fastapi_api_debug(self, args, expected_command): with ( mock.patch("subprocess.Popen") as Popen, ): @@ -75,7 +75,7 @@ def test_cli_fastapi_api_debug(self, app, args, expected_command): close_fds=True, ) - def test_cli_fastapi_api_env_var_set_unset(self, app): + def test_cli_fastapi_api_env_var_set_unset(self): """ Test that AIRFLOW_API_APPS is set and unset in the environment when calling the airflow api-server command diff --git a/tests/cli/commands/local_commands/test_webserver_command.py b/tests/cli/commands/local_commands/test_webserver_command.py deleted file mode 100644 index 428f86abd71b6..0000000000000 --- a/tests/cli/commands/local_commands/test_webserver_command.py +++ /dev/null @@ -1,380 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os -import subprocess -import sys -import time -from unittest import mock - -import psutil -import pytest -from rich.console import Console - -from airflow import settings -from airflow.cli import cli_parser -from airflow.cli.commands.local_commands import webserver_command -from airflow.cli.commands.local_commands.webserver_command import GunicornMonitor - -from tests.cli.commands._common_cli_classes import _CommonCLIGunicornTestClass -from tests_common.test_utils.config import conf_vars - -console = Console(width=400, color_system="standard") - - -class TestGunicornMonitor: - def setup_method(self) -> None: - self.monitor = GunicornMonitor( - gunicorn_master_pid=1, - num_workers_expected=4, - master_timeout=60, - worker_refresh_interval=60, - worker_refresh_batch_size=2, - reload_on_plugin_change=True, - ) - mock.patch.object(self.monitor, "_generate_plugin_state", return_value={}).start() - mock.patch.object(self.monitor, "_get_num_ready_workers_running", return_value=4).start() - mock.patch.object(self.monitor, "_get_num_workers_running", return_value=4).start() - mock.patch.object(self.monitor, "_spawn_new_workers", return_value=None).start() - mock.patch.object(self.monitor, "_kill_old_workers", return_value=None).start() - mock.patch.object(self.monitor, "_reload_gunicorn", return_value=None).start() - - @mock.patch("airflow.cli.commands.local_commands.webserver_command.sleep") - def test_should_wait_for_workers_to_start(self, mock_sleep): - self.monitor._get_num_ready_workers_running.return_value = 0 - self.monitor._get_num_workers_running.return_value = 4 - self.monitor._check_workers() - self.monitor._spawn_new_workers.assert_not_called() - self.monitor._kill_old_workers.assert_not_called() - self.monitor._reload_gunicorn.assert_not_called() - - @mock.patch("airflow.cli.commands.local_commands.webserver_command.sleep") - def test_should_kill_excess_workers(self, mock_sleep): - self.monitor._get_num_ready_workers_running.return_value = 10 - self.monitor._get_num_workers_running.return_value = 10 - self.monitor._check_workers() - self.monitor._spawn_new_workers.assert_not_called() - self.monitor._kill_old_workers.assert_called_once_with(2) - self.monitor._reload_gunicorn.assert_not_called() - - @mock.patch("airflow.cli.commands.local_commands.webserver_command.sleep") - def test_should_start_new_workers_when_missing(self, mock_sleep): - self.monitor._get_num_ready_workers_running.return_value = 3 - self.monitor._get_num_workers_running.return_value = 3 - self.monitor._check_workers() - # missing one worker, starting just 1 - self.monitor._spawn_new_workers.assert_called_once_with(1) - self.monitor._kill_old_workers.assert_not_called() - self.monitor._reload_gunicorn.assert_not_called() - - @mock.patch("airflow.cli.commands.local_commands.webserver_command.sleep") - def test_should_start_new_batch_when_missing_many_workers(self, mock_sleep): - self.monitor._get_num_ready_workers_running.return_value = 1 - self.monitor._get_num_workers_running.return_value = 1 - self.monitor._check_workers() - # missing 3 workers, but starting single batch (2) - self.monitor._spawn_new_workers.assert_called_once_with(2) - self.monitor._kill_old_workers.assert_not_called() - self.monitor._reload_gunicorn.assert_not_called() - - @mock.patch("airflow.cli.commands.local_commands.webserver_command.sleep") - def test_should_start_new_workers_when_refresh_interval_has_passed(self, mock_sleep): - self.monitor._last_refresh_time -= 200 - self.monitor._check_workers() - self.monitor._spawn_new_workers.assert_called_once_with(2) - self.monitor._kill_old_workers.assert_not_called() - self.monitor._reload_gunicorn.assert_not_called() - assert abs(self.monitor._last_refresh_time - time.monotonic()) < 5 - - @mock.patch("airflow.cli.commands.local_commands.webserver_command.sleep") - def test_should_reload_when_plugin_has_been_changed(self, mock_sleep): - self.monitor._generate_plugin_state.return_value = {"AA": 12} - - self.monitor._check_workers() - - self.monitor._spawn_new_workers.assert_not_called() - self.monitor._kill_old_workers.assert_not_called() - self.monitor._reload_gunicorn.assert_not_called() - - self.monitor._generate_plugin_state.return_value = {"AA": 32} - - self.monitor._check_workers() - - self.monitor._spawn_new_workers.assert_not_called() - self.monitor._kill_old_workers.assert_not_called() - self.monitor._reload_gunicorn.assert_not_called() - - self.monitor._generate_plugin_state.return_value = {"AA": 32} - - self.monitor._check_workers() - - self.monitor._spawn_new_workers.assert_not_called() - self.monitor._kill_old_workers.assert_not_called() - self.monitor._reload_gunicorn.assert_called_once_with() - assert abs(self.monitor._last_refresh_time - time.monotonic()) < 5 - - -class TestGunicornMonitorGeneratePluginState: - def test_should_detect_changes_in_directory(self, tmp_path): - with mock.patch( - "airflow.cli.commands.local_commands.webserver_command.settings.PLUGINS_FOLDER", - os.fspath(tmp_path), - ): - (tmp_path / "file1.txt").write_text("A" * 100) - path2 = tmp_path / "nested/nested/nested/nested/file2.txt" - path2.parent.mkdir(parents=True) - path2.write_text("A" * 200) - (tmp_path / "file3.txt").write_text("A" * 300) - - monitor = GunicornMonitor( - gunicorn_master_pid=1, - num_workers_expected=4, - master_timeout=60, - worker_refresh_interval=60, - worker_refresh_batch_size=2, - reload_on_plugin_change=True, - ) - - # When the files have not changed, the result should be constant - state_a = monitor._generate_plugin_state() - state_b = monitor._generate_plugin_state() - - assert state_a == state_b - assert len(state_a) == 3 - - # Should detect new file - (tmp_path / "file4.txt").write_text("A" * 400) - - state_c = monitor._generate_plugin_state() - - assert state_b != state_c - assert len(state_c) == 4 - - # Should detect changes in files - (tmp_path / "file4.txt").write_text("A" * 450) - - state_d = monitor._generate_plugin_state() - - assert state_c != state_d - assert len(state_d) == 4 - - # Should support large files - (tmp_path / "file4.txt").write_text("A" * 4_000_000) - - state_d = monitor._generate_plugin_state() - - assert state_c != state_d - assert len(state_d) == 4 - - -class TestCLIGetNumReadyWorkersRunning: - @classmethod - def setup_class(cls): - cls.parser = cli_parser.get_parser() - - def setup_method(self): - self.children = mock.MagicMock() - self.child = mock.MagicMock() - self.process = mock.MagicMock() - self.monitor = GunicornMonitor( - gunicorn_master_pid=1, - num_workers_expected=4, - master_timeout=60, - worker_refresh_interval=60, - worker_refresh_batch_size=2, - reload_on_plugin_change=True, - ) - - def test_ready_prefix_on_cmdline(self): - self.child.cmdline.return_value = [settings.GUNICORN_WORKER_READY_PREFIX] - self.process.children.return_value = [self.child] - - with mock.patch("psutil.Process", return_value=self.process): - assert self.monitor._get_num_ready_workers_running() == 1 - - def test_ready_prefix_on_cmdline_no_children(self): - self.process.children.return_value = [] - - with mock.patch("psutil.Process", return_value=self.process): - assert self.monitor._get_num_ready_workers_running() == 0 - - def test_ready_prefix_on_cmdline_zombie(self): - self.child.cmdline.return_value = [] - self.process.children.return_value = [self.child] - - with mock.patch("psutil.Process", return_value=self.process): - assert self.monitor._get_num_ready_workers_running() == 0 - - def test_ready_prefix_on_cmdline_dead_process(self): - self.child.cmdline.side_effect = psutil.NoSuchProcess(11347) - self.process.children.return_value = [self.child] - - with mock.patch("psutil.Process", return_value=self.process): - assert self.monitor._get_num_ready_workers_running() == 0 - - -@pytest.mark.db_test -class TestCliWebServer(_CommonCLIGunicornTestClass): - main_process_regexp = r"airflow webserver" - - @pytest.mark.execution_timeout(400) - def test_cli_webserver_background(self, tmp_path): - with mock.patch.dict( - "os.environ", - AIRFLOW__CORE__DAGS_FOLDER="/dev/null", - AIRFLOW__CORE__LOAD_EXAMPLES="False", - AIRFLOW__WEBSERVER__WORKERS="1", - ): - pidfile_webserver = tmp_path / "pidflow-webserver.pid" - pidfile_monitor = tmp_path / "pidflow-webserver-monitor.pid" - stdout = tmp_path / "airflow-webserver.out" - stderr = tmp_path / "airflow-webserver.err" - logfile = tmp_path / "airflow-webserver.log" - try: - # Run webserver as daemon in background. Note that the wait method is not called. - - proc = subprocess.Popen( - [ - "airflow", - "webserver", - "--daemon", - "--pid", - os.fspath(pidfile_webserver), - "--stdout", - os.fspath(stdout), - "--stderr", - os.fspath(stderr), - "--log-file", - os.fspath(logfile), - ] - ) - assert proc.poll() is None - - pid_monitor = self._wait_pidfile(pidfile_monitor) - console.print(f"[blue]Monitor started at {pid_monitor}") - pid_webserver = self._wait_pidfile(pidfile_webserver) - console.print(f"[blue]Webserver started at {pid_webserver}") - console.print("[blue]Running airflow webserver process:") - # Assert that the webserver and gunicorn processes are running (by name rather than pid). - assert self._find_process(r"airflow webserver", print_found_process=True) - console.print("[blue]Waiting for gunicorn processes:") - # wait for gunicorn to start - for _ in range(120): - if self._find_process(r"^gunicorn"): - break - console.print("[blue]Waiting for gunicorn to start ...") - time.sleep(1) - else: - pytest.fail("Gunicorn processes not found after 120 seconds") - console.print("[blue]Running gunicorn processes:") - assert self._find_all_processes("^gunicorn", print_found_process=True) - console.print("[magenta]Webserver process started successfully.") - console.print( - "[magenta]Terminating monitor process and expect " - "Webserver and gunicorn processes to terminate as well" - ) - proc = psutil.Process(pid_monitor) - proc.terminate() - assert proc.wait(120) in (0, None) - self._check_processes(ignore_running=False) - console.print("[magenta]All Webserver and gunicorn processes are terminated.") - except Exception: - console.print("[red]Exception occurred. Dumping all logs.") - # Dump all logs - for file in tmp_path.glob("*"): - console.print(f"Dumping {file} (size: {file.stat().st_size})") - console.print(file.read_text()) - raise - - # Patch for causing webserver timeout - @mock.patch( - "airflow.cli.commands.local_commands.webserver_command.GunicornMonitor._get_num_workers_running", - return_value=0, - ) - def test_cli_webserver_shutdown_when_gunicorn_master_is_killed(self, _): - # Shorten timeout so that this test doesn't take too long time - args = self.parser.parse_args(["webserver"]) - with conf_vars({("webserver", "web_server_master_timeout"): "10"}): - with pytest.raises(SystemExit) as ctx: - webserver_command.webserver(args) - assert ctx.value.code == 1 - - def test_cli_webserver_debug(self, app): - with ( - mock.patch("airflow.www.app.create_app", return_value=app), - mock.patch.object(app, "run") as app_run, - ): - args = self.parser.parse_args( - [ - "webserver", - "--debug", - ] - ) - webserver_command.webserver(args) - - app_run.assert_called_with( - debug=True, - use_reloader=False, - port=8080, - host="0.0.0.0", - ssl_context=None, - ) - - def test_cli_webserver_args(self): - with mock.patch("subprocess.Popen") as Popen, mock.patch.object(webserver_command, "GunicornMonitor"): - args = self.parser.parse_args( - [ - "webserver", - "--access-logformat", - "custom_log_format", - "--pid", - "/tmp/x.pid", - ] - ) - webserver_command.webserver(args) - - Popen.assert_called_with( - [ - sys.executable, - "-m", - "gunicorn", - "--workers", - "4", - "--worker-class", - "sync", - "--timeout", - "120", - "--bind", - "0.0.0.0:8080", - "--name", - "airflow-webserver", - "--pid", - "/tmp/x.pid", - "--config", - "python:airflow.www.gunicorn_config", - "--access-logfile", - "-", - "--error-logfile", - "-", - "--access-logformat", - "custom_log_format", - "airflow.www.app:cached_app()", - "--preload", - ], - close_fds=True, - ) diff --git a/tests/dag_processing/test_collection.py b/tests/dag_processing/test_collection.py index 7e7e9d3d8c339..aab0c3065b74a 100644 --- a/tests/dag_processing/test_collection.py +++ b/tests/dag_processing/test_collection.py @@ -35,7 +35,6 @@ from airflow.dag_processing.collection import ( AssetModelOperation, _get_latest_runs_stmt, - _sync_dag_perms, update_dag_parsing_results_in_db, ) from airflow.exceptions import SerializationError @@ -466,37 +465,6 @@ def test_remove_error_clears_import_error(self, testing_dag_bundle, session): assert import_errors == {("def.py", bundle_name)} - def test_sync_perm_for_dag_with_dict_access_control(self, session, spy_agency: SpyAgency): - """ - Test that dagbag._sync_perm_for_dag will call ApplessAirflowSecurityManager.sync_perm_for_dag - """ - from airflow.www.security_appless import ApplessAirflowSecurityManager - - spy = spy_agency.spy_on( - ApplessAirflowSecurityManager.sync_perm_for_dag, owner=ApplessAirflowSecurityManager - ) - - dag = DAG(dag_id="test") - - def _sync_perms(): - spy.reset_calls() - _sync_dag_perms(dag, session=session) - - # perms dont exist - _sync_perms() - spy_agency.assert_spy_called_with(spy, dag.dag_id, access_control=None) - - # perms now exist - _sync_perms() - spy_agency.assert_spy_called_with(spy, dag.dag_id, access_control=None) - - # Always sync if we have access_control - dag.access_control = {"Public": {"DAGs": {"can_read"}, "DAG Runs": {"can_create"}}} - _sync_perms() - spy_agency.assert_spy_called_with( - spy, dag.dag_id, access_control={"Public": {"DAGs": {"can_read"}, "DAG Runs": {"can_create"}}} - ) - @pytest.mark.parametrize( ("attrs", "expected"), [ diff --git a/tests/operators/test_trigger_dagrun.py b/tests/operators/test_trigger_dagrun.py index 104d10649a13c..0f65c42af457c 100644 --- a/tests/operators/test_trigger_dagrun.py +++ b/tests/operators/test_trigger_dagrun.py @@ -85,14 +85,7 @@ def teardown_method(self): # pathlib.Path(self._tmpfile).unlink() def assert_extra_link(self, triggered_dag_run, triggering_task, session): - """ - Asserts whether the correct extra links url will be created. - - Specifically it tests whether the correct dag id and run id are passed to - the method which constructs the final url. - Note: We can't run that method to generate the url itself because the Flask app context - isn't available within the test logic, so it is mocked here. - """ + """Asserts whether the correct extra links url will be created.""" triggering_ti = ( session.query(TaskInstance) .filter_by( @@ -102,20 +95,14 @@ def assert_extra_link(self, triggered_dag_run, triggering_task, session): .one() ) - with mock.patch( - "airflow.providers.standard.operators.trigger_dagrun.build_airflow_url_with_query" - ) as mock_build_url: - # This is equivalent of a task run calling this and pushing to xcom - triggering_task.operator_extra_links[0].get_link( - operator=triggering_task, ti_key=triggering_ti.key - ) - assert mock_build_url.called - args, _ = mock_build_url.call_args - expected_args = { - "dag_id": triggered_dag_run.dag_id, - "dag_run_id": triggered_dag_run.run_id, - } - assert expected_args in args + # This is equivalent of a task run calling this and pushing to xcom + url = triggering_task.operator_extra_links[0].get_link( + operator=triggering_task, ti_key=triggering_ti.key + ) + expected_url = ( + f"http://localhost:9091/webapp/dags/{triggered_dag_run.dag_id}/runs/{triggered_dag_run.run_id}" + ) + assert url == expected_url def test_trigger_dagrun(self, dag_maker): """Test TriggerDagRunOperator.""" diff --git a/tests/plugins/test_plugins_manager.py b/tests/plugins/test_plugins_manager.py index 54ab88d2f60e0..c3e0c15609089 100644 --- a/tests/plugins/test_plugins_manager.py +++ b/tests/plugins/test_plugins_manager.py @@ -31,7 +31,6 @@ from airflow.listeners.listener import get_listener_manager from airflow.plugins_manager import AirflowPlugin from airflow.utils.module_loading import qualname -from airflow.www import app as application from tests_common.test_utils.config import conf_vars from tests_common.test_utils.mock_plugins import mock_plugin_manager @@ -73,95 +72,6 @@ def wrapper(*args, **kwargs): return wrapper -@pytest.mark.db_test -class TestPluginsRBAC: - @pytest.fixture(autouse=True) - def _set_attrs(self, app): - self.app = app - self.appbuilder = app.appbuilder - - def test_flaskappbuilder_views(self): - from tests.plugins.test_plugin import v_appbuilder_package - - appbuilder_class_name = str(v_appbuilder_package["view"].__class__.__name__) - plugin_views = [ - view for view in self.appbuilder.baseviews if view.blueprint.name == appbuilder_class_name - ] - - assert len(plugin_views) == 1 - - # view should have a menu item matching category of v_appbuilder_package - links = [ - menu_item - for menu_item in self.appbuilder.menu.menu - if menu_item.name == v_appbuilder_package["category"] - ] - - assert len(links) == 1 - - # menu link should also have a link matching the name of the package. - link = links[0] - assert link.name == v_appbuilder_package["category"] - assert link.childs[0].name == v_appbuilder_package["name"] - assert link.childs[0].label == v_appbuilder_package["label"] - - def test_flaskappbuilder_menu_links(self): - from tests.plugins.test_plugin import appbuilder_mitem, appbuilder_mitem_toplevel - - # menu item (category) should exist matching appbuilder_mitem.category - categories = [ - menu_item - for menu_item in self.appbuilder.menu.menu - if menu_item.name == appbuilder_mitem["category"] - ] - assert len(categories) == 1 - - # menu link should be a child in the category - category = categories[0] - assert category.name == appbuilder_mitem["category"] - assert category.childs[0].name == appbuilder_mitem["name"] - assert category.childs[0].href == appbuilder_mitem["href"] - - # a top level link isn't nested in a category - top_levels = [ - menu_item - for menu_item in self.appbuilder.menu.menu - if menu_item.name == appbuilder_mitem_toplevel["name"] - ] - assert len(top_levels) == 1 - link = top_levels[0] - assert link.href == appbuilder_mitem_toplevel["href"] - assert link.label == appbuilder_mitem_toplevel["label"] - - def test_app_blueprints(self): - from tests.plugins.test_plugin import bp - - # Blueprint should be present in the app - assert "test_plugin" in self.app.blueprints - assert self.app.blueprints["test_plugin"].name == bp.name - - def test_app_static_folder(self): - # Blueprint static folder should be properly set - assert Path(self.app.static_folder).resolve() == AIRFLOW_SOURCES_ROOT / "airflow" / "www" / "static" - - -@pytest.mark.db_test -def test_flaskappbuilder_nomenu_views(): - from tests.plugins.test_plugin import v_nomenu_appbuilder_package - - class AirflowNoMenuViewsPlugin(AirflowPlugin): - appbuilder_views = [v_nomenu_appbuilder_package] - - appbuilder_class_name = str(v_nomenu_appbuilder_package["view"].__class__.__name__) - - with mock_plugin_manager(plugins=[AirflowNoMenuViewsPlugin()]): - appbuilder = application.create_app(testing=True).appbuilder - - plugin_views = [view for view in appbuilder.baseviews if view.blueprint.name == appbuilder_class_name] - - assert len(plugin_views) == 1 - - class TestPluginsManager: @pytest.fixture(autouse=True) def clean_plugins(self): diff --git a/tests/sensors/test_external_task_sensor.py b/tests/sensors/test_external_task_sensor.py index eef7585f0c6a7..1b347526ec4f8 100644 --- a/tests/sensors/test_external_task_sensor.py +++ b/tests/sensors/test_external_task_sensor.py @@ -1068,7 +1068,6 @@ def test_external_task_sensor_extra_link( expected_external_dag_id, expected_external_task_id, create_task_instance_of_operator, - app, ): ti = create_task_instance_of_operator( ExternalTaskSensor, @@ -1084,11 +1083,9 @@ def test_external_task_sensor_extra_link( assert ti.task.external_task_id == expected_external_task_id assert ti.task.external_task_ids == [expected_external_task_id] - app.config["SERVER_NAME"] = "" - with app.app_context(): - url = ti.task.operator_extra_links[0].get_link(operator=ti.task, ti_key=ti.key) + url = ti.task.operator_extra_links[0].get_link(operator=ti.task, ti_key=ti.key) - assert f"/dags/{expected_external_dag_id}/grid" in url + assert f"/dags/{expected_external_dag_id}/runs" in url class TestExternalTaskMarker: diff --git a/tests/utils/test_edgemodifier.py b/tests/utils/test_edgemodifier.py index 37e599a9742ba..79749af5cc0a8 100644 --- a/tests/utils/test_edgemodifier.py +++ b/tests/utils/test_edgemodifier.py @@ -24,9 +24,9 @@ from airflow.models.xcom_arg import XComArg from airflow.providers.standard.operators.empty import EmptyOperator from airflow.providers.standard.operators.python import PythonOperator +from airflow.utils.dag_edges import dag_edges from airflow.utils.edgemodifier import Label from airflow.utils.task_group import TaskGroup -from airflow.www.views import dag_edges DEFAULT_ARGS = { "owner": "test", diff --git a/tests/utils/test_helpers.py b/tests/utils/test_helpers.py index b99dd206e75fd..4cc7ddea72c14 100644 --- a/tests/utils/test_helpers.py +++ b/tests/utils/test_helpers.py @@ -28,7 +28,7 @@ from airflow.utils import helpers, timezone from airflow.utils.helpers import ( at_most_one, - build_airflow_url_with_query, + build_airflow_dagrun_url, exactly_one, merge_dicts, prune_dict, @@ -38,7 +38,6 @@ ) from airflow.utils.types import NOTSET -from tests_common.test_utils.config import conf_vars from tests_common.test_utils.db import clear_db_dags, clear_db_runs if TYPE_CHECKING: @@ -162,23 +161,9 @@ def test_merge_dicts_recursive_right_only(self): merged = merge_dicts(dict1, dict2) assert merged == {"a": 1, "r": {"b": 0, "c": 3}} - @pytest.mark.db_test - @conf_vars( - { - ("webserver", "dag_default_view"): "graph", - } - ) - def test_build_airflow_url_with_query(self): - """ - Test query generated with dag_id and params - """ - query = {"dag_id": "test_dag", "param": "key/to.encode"} - expected_url = "/dags/test_dag/graph?param=key%2Fto.encode" - - from airflow.www.app import cached_app - - with cached_app(testing=True).test_request_context(): - assert build_airflow_url_with_query(query) == expected_url + def test_build_airflow_dagrun_url(self): + expected_url = "http://localhost:9091/webapp/dags/somedag/runs/abc123" + assert build_airflow_dagrun_url(dag_id="somedag", run_id="abc123") == expected_url @pytest.mark.parametrize( "key_id, message, exception", diff --git a/tests/www/test_app.py b/tests/www/test_app.py deleted file mode 100644 index c258b89ac3caa..0000000000000 --- a/tests/www/test_app.py +++ /dev/null @@ -1,287 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import hashlib -import sys -from datetime import timedelta -from unittest import mock - -import pytest -from werkzeug.routing import Rule -from werkzeug.test import create_environ -from werkzeug.wrappers import Response - -from airflow.exceptions import AirflowConfigException -from airflow.www import app as application - -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.decorators import dont_initialize_flask_app_submodules - -pytestmark = pytest.mark.db_test - - -class TestApp: - @classmethod - def setup_class(cls) -> None: - from airflow import settings - - settings.configure_orm() - - @conf_vars( - { - ("webserver", "enable_proxy_fix"): "True", - ("webserver", "proxy_fix_x_for"): "1", - ("webserver", "proxy_fix_x_proto"): "1", - ("webserver", "proxy_fix_x_host"): "1", - ("webserver", "proxy_fix_x_port"): "1", - ("webserver", "proxy_fix_x_prefix"): "1", - } - ) - @dont_initialize_flask_app_submodules - def test_should_respect_proxy_fix(self): - app = application.cached_app(testing=True) - app.url_map.add(Rule("/debug", endpoint="debug")) - - def debug_view(): - from flask import request - - # Should respect HTTP_X_FORWARDED_FOR - assert request.remote_addr == "192.168.0.1" - # Should respect HTTP_X_FORWARDED_PROTO, HTTP_X_FORWARDED_HOST, HTTP_X_FORWARDED_PORT, - # HTTP_X_FORWARDED_PREFIX - assert request.url == "https://valid:445/proxy-prefix/debug" - - return Response("success") - - app.view_functions["debug"] = debug_view - - new_environ = { - "PATH_INFO": "/debug", - "REMOTE_ADDR": "192.168.0.2", - "HTTP_HOST": "invalid:9000", - "HTTP_X_FORWARDED_FOR": "192.168.0.1", - "HTTP_X_FORWARDED_PROTO": "https", - "HTTP_X_FORWARDED_HOST": "valid", - "HTTP_X_FORWARDED_PORT": "445", - "HTTP_X_FORWARDED_PREFIX": "/proxy-prefix", - } - environ = create_environ(environ_overrides=new_environ) - - response = Response.from_app(app, environ) - - assert response.get_data() == b"success" - assert response.status_code == 200 - - @dont_initialize_flask_app_submodules - def test_should_respect_base_url_ignore_proxy_headers(self): - with conf_vars({("webserver", "base_url"): "http://localhost:8080/internal-client"}): - app = application.cached_app(testing=True) - app.url_map.add(Rule("/debug", endpoint="debug")) - - def debug_view(): - from flask import request - - # Should ignore HTTP_X_FORWARDED_FOR - assert request.remote_addr == "192.168.0.2" - # Should ignore HTTP_X_FORWARDED_PROTO, HTTP_X_FORWARDED_HOST, HTTP_X_FORWARDED_PORT, - # HTTP_X_FORWARDED_PREFIX - assert request.url == "http://invalid:9000/internal-client/debug" - - return Response("success") - - app.view_functions["debug"] = debug_view - - new_environ = { - "PATH_INFO": "/internal-client/debug", - "REMOTE_ADDR": "192.168.0.2", - "HTTP_HOST": "invalid:9000", - "HTTP_X_FORWARDED_FOR": "192.168.0.1", - "HTTP_X_FORWARDED_PROTO": "https", - "HTTP_X_FORWARDED_HOST": "valid", - "HTTP_X_FORWARDED_PORT": "445", - "HTTP_X_FORWARDED_PREFIX": "/proxy-prefix", - } - environ = create_environ(environ_overrides=new_environ) - - response = Response.from_app(app, environ) - - assert response.get_data() == b"success" - assert response.status_code == 200 - - @dont_initialize_flask_app_submodules - def test_base_url_contains_trailing_slash(self): - with conf_vars({("webserver", "base_url"): "http://localhost:8080/internal-client/"}): - with pytest.raises( - AirflowConfigException, match="webserver.base_url conf cannot have a trailing slash" - ): - application.cached_app(testing=True) - - @conf_vars( - { - ("webserver", "enable_proxy_fix"): "True", - ("webserver", "proxy_fix_x_for"): "1", - ("webserver", "proxy_fix_x_proto"): "1", - ("webserver", "proxy_fix_x_host"): "1", - ("webserver", "proxy_fix_x_port"): "1", - ("webserver", "proxy_fix_x_prefix"): "1", - } - ) - @dont_initialize_flask_app_submodules - def test_should_respect_base_url_when_proxy_fix_and_base_url_is_set_up_but_headers_missing(self): - with conf_vars({("webserver", "base_url"): "http://localhost:8080/internal-client"}): - app = application.cached_app(testing=True) - app.url_map.add(Rule("/debug", endpoint="debug")) - - def debug_view(): - from flask import request - - # Should use original REMOTE_ADDR - assert request.remote_addr == "192.168.0.1" - # Should respect base_url - assert request.url == "http://invalid:9000/internal-client/debug" - - return Response("success") - - app.view_functions["debug"] = debug_view - - new_environ = { - "PATH_INFO": "/internal-client/debug", - "REMOTE_ADDR": "192.168.0.1", - "HTTP_HOST": "invalid:9000", - } - environ = create_environ(environ_overrides=new_environ) - - response = Response.from_app(app, environ) - - assert response.get_data() == b"success" - assert response.status_code == 200 - - @conf_vars( - { - ("webserver", "base_url"): "http://localhost:8080/internal-client", - ("webserver", "enable_proxy_fix"): "True", - ("webserver", "proxy_fix_x_for"): "1", - ("webserver", "proxy_fix_x_proto"): "1", - ("webserver", "proxy_fix_x_host"): "1", - ("webserver", "proxy_fix_x_port"): "1", - ("webserver", "proxy_fix_x_prefix"): "1", - } - ) - @dont_initialize_flask_app_submodules - def test_should_respect_base_url_and_proxy_when_proxy_fix_and_base_url_is_set_up(self): - app = application.cached_app(testing=True) - app.url_map.add(Rule("/debug", endpoint="debug")) - - def debug_view(): - from flask import request - - # Should respect HTTP_X_FORWARDED_FOR - assert request.remote_addr == "192.168.0.1" - # Should respect HTTP_X_FORWARDED_PROTO, HTTP_X_FORWARDED_HOST, HTTP_X_FORWARDED_PORT, - # HTTP_X_FORWARDED_PREFIX and use base_url - assert request.url == "https://valid:445/proxy-prefix/internal-client/debug" - - return Response("success") - - app.view_functions["debug"] = debug_view - - new_environ = { - "PATH_INFO": "/internal-client/debug", - "REMOTE_ADDR": "192.168.0.2", - "HTTP_HOST": "invalid:9000", - "HTTP_X_FORWARDED_FOR": "192.168.0.1", - "HTTP_X_FORWARDED_PROTO": "https", - "HTTP_X_FORWARDED_HOST": "valid", - "HTTP_X_FORWARDED_PORT": "445", - "HTTP_X_FORWARDED_PREFIX": "/proxy-prefix", - } - environ = create_environ(environ_overrides=new_environ) - - response = Response.from_app(app, environ) - - assert response.get_data() == b"success" - assert response.status_code == 200 - - @conf_vars( - { - ("webserver", "session_lifetime_minutes"): "3600", - } - ) - @dont_initialize_flask_app_submodules - def test_should_set_permanent_session_timeout(self): - app = application.cached_app(testing=True) - assert app.config["PERMANENT_SESSION_LIFETIME"] == timedelta(minutes=3600) - - @conf_vars({("webserver", "cookie_samesite"): ""}) - @dont_initialize_flask_app_submodules - def test_correct_default_is_set_for_cookie_samesite(self): - """An empty 'cookie_samesite' should be corrected to 'Lax'.""" - app = application.cached_app(testing=True) - assert app.config["SESSION_COOKIE_SAMESITE"] == "Lax" - - @pytest.mark.parametrize( - "hash_method, result", - [ - pytest.param("sha512", hashlib.sha512, id="sha512"), - pytest.param("sha384", hashlib.sha384, id="sha384"), - pytest.param("sha256", hashlib.sha256, id="sha256"), - pytest.param("sha224", hashlib.sha224, id="sha224"), - pytest.param("sha1", hashlib.sha1, id="sha1"), - pytest.param("md5", hashlib.md5, id="md5"), - pytest.param(None, hashlib.md5, id="default"), - ], - ) - @dont_initialize_flask_app_submodules(skip_all_except=["init_auth_manager"]) - def test_should_respect_caching_hash_method(self, hash_method, result): - with conf_vars({("webserver", "caching_hash_method"): hash_method}): - app = application.cached_app(testing=True) - assert next(iter(app.extensions["cache"])).cache._hash_method == result - - @dont_initialize_flask_app_submodules - def test_should_respect_caching_hash_method_invalid(self): - with conf_vars({("webserver", "caching_hash_method"): "invalid"}): - with pytest.raises(expected_exception=AirflowConfigException): - application.cached_app(testing=True) - - -class TestFlaskCli: - @dont_initialize_flask_app_submodules(skip_all_except=["init_appbuilder"]) - def test_flask_cli_should_display_routes(self, capsys): - with ( - mock.patch.dict("os.environ", FLASK_APP="airflow.www.app:cached_app"), - mock.patch.object(sys, "argv", ["flask", "routes"]), - ): - # Import from flask.__main__ with a combination of mocking With mocking sys.argv - # will invoke ``flask routes`` command. - with pytest.raises(SystemExit) as ex_ctx: - from flask import __main__ # noqa: F401 - assert ex_ctx.value.code == 0 - - output = capsys.readouterr() - assert "/login/" in output.out - - -def test_app_can_json_serialize_k8s_pod(): - # This is mostly testing that we have correctly configured the JSON provider to use. Testing the k8s pos - # is a side-effect of that. - k8s = pytest.importorskip("kubernetes.client.models") - - pod = k8s.V1Pod(spec=k8s.V1PodSpec(containers=[k8s.V1Container(name="base")])) - app = application.cached_app(testing=True) - assert app.json.dumps(pod) == '{"spec": {"containers": [{"name": "base"}]}}' diff --git a/tests/www/test_security_manager.py b/tests/www/test_security_manager.py deleted file mode 100644 index a9c13b8cd7152..0000000000000 --- a/tests/www/test_security_manager.py +++ /dev/null @@ -1,168 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import json -from unittest import mock -from unittest.mock import Mock - -import pytest - -from airflow.security.permissions import ( - ACTION_CAN_READ, - RESOURCE_ADMIN_MENU, - RESOURCE_BROWSE_MENU, - RESOURCE_DOCS_MENU, - RESOURCE_VARIABLE, -) -from airflow.www import app as application - - -@pytest.fixture -def app(): - return application.create_app(testing=True) - - -@pytest.fixture -def app_builder(app): - return app.appbuilder - - -@pytest.fixture -def security_manager(app_builder): - return app_builder.sm - - -@pytest.mark.db_test -class TestAirflowSecurityManagerV2: - @pytest.mark.parametrize( - "action_name, resource_name, auth_manager_methods, expected", - [ - (ACTION_CAN_READ, RESOURCE_VARIABLE, {"is_authorized_variable": True}, True), - (ACTION_CAN_READ, RESOURCE_VARIABLE, {"is_authorized_variable": False}, False), - (ACTION_CAN_READ, RESOURCE_DOCS_MENU, {"is_authorized_view": True}, True), - (ACTION_CAN_READ, RESOURCE_DOCS_MENU, {"is_authorized_view": False}, False), - ( - ACTION_CAN_READ, - RESOURCE_ADMIN_MENU, - { - "is_authorized_view": False, - "is_authorized_variable": False, - "is_authorized_connection": True, - "is_authorized_dag": False, - "is_authorized_configuration": False, - "is_authorized_pool": False, - }, - True, - ), - ( - ACTION_CAN_READ, - RESOURCE_ADMIN_MENU, - { - "is_authorized_view": False, - "is_authorized_variable": False, - "is_authorized_connection": False, - "is_authorized_dag": False, - "is_authorized_configuration": False, - "is_authorized_pool": False, - }, - False, - ), - ( - ACTION_CAN_READ, - RESOURCE_BROWSE_MENU, - { - "is_authorized_dag": False, - "is_authorized_view": False, - }, - False, - ), - ( - ACTION_CAN_READ, - RESOURCE_BROWSE_MENU, - { - "is_authorized_dag": False, - "is_authorized_view": True, - }, - True, - ), - ( - "can_not_a_default_action", - "custom_resource", - {"is_authorized_custom_view": False}, - False, - ), - ( - "can_not_a_default_action", - "custom_resource", - {"is_authorized_custom_view": True}, - True, - ), - ], - ) - @mock.patch("airflow.providers.fab.www.security_manager.get_auth_manager") - def test_has_access( - self, - mock_get_auth_manager, - security_manager, - action_name, - resource_name, - auth_manager_methods, - expected, - ): - user = Mock() - auth_manager = Mock() - for method_name, method_return in auth_manager_methods.items(): - method = Mock(return_value=method_return) - getattr(auth_manager, method_name).side_effect = method - mock_get_auth_manager.return_value = auth_manager - result = security_manager.has_access(action_name, resource_name, user=user) - assert result == expected - if len(auth_manager_methods) > 1 and not expected: - for method_name in auth_manager_methods: - getattr(auth_manager, method_name).assert_called() - - @mock.patch("airflow.utils.session.create_session") - @mock.patch("airflow.providers.fab.www.security_manager.get_auth_manager") - def test_manager_does_not_create_extra_db_sessions( - self, - _, - mock_create_session, - security_manager, - ): - """ - Test that the Security Manager doesn't create extra DB sessions and - instead uses the session already available through the appbuilder - object that is attached to it. - """ - with mock.patch.object(security_manager.appbuilder, "session") as mock_appbuilder_session: - action_name = ACTION_CAN_READ - resource_pk = "PK" - user = Mock() - for func in security_manager._auth_manager_is_authorized_map.values(): - try: - func(action_name, resource_pk, user) - except json.JSONDecodeError: - # The resource-retrieving function expects a "composite" - # PK as a JSON string. Provide a mocked one. - func(action_name, "[1, 1, 1, 1]", user) - mock_create_session.assert_not_called() - - # The Security Manager's `appbuilder.session` object should have been - # put to use by many of the functions tested above. - assert len(mock_appbuilder_session.method_calls) > 0 diff --git a/tests/www/test_utils.py b/tests/www/test_utils.py deleted file mode 100644 index 7aa99c8f05d51..0000000000000 --- a/tests/www/test_utils.py +++ /dev/null @@ -1,702 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import itertools -import re -import time -from datetime import datetime -from unittest.mock import Mock -from urllib.parse import parse_qs - -import pendulum -import pytest -from bs4 import BeautifulSoup -from flask_appbuilder.models.sqla.filters import get_field_setup_query, set_value_to_type -from flask_wtf import FlaskForm -from markupsafe import Markup -from sqlalchemy.orm import Query -from wtforms.fields import StringField, TextAreaField - -from airflow.models import DagRun -from airflow.utils import json as utils_json -from airflow.www import utils -from airflow.www.utils import ( - CustomSQLAInterface, - DagRunCustomSQLAInterface, - encode_dag_run, - json_f, - wrapped_markdown, -) -from airflow.www.widgets import AirflowDateTimePickerROWidget, BS3TextAreaROWidget, BS3TextFieldROWidget - -from tests_common.test_utils.version_compat import AIRFLOW_V_3_0_PLUS - -if AIRFLOW_V_3_0_PLUS: - from airflow.utils.types import DagRunTriggeredByType - - -class TestUtils: - def check_generate_pages_html( - self, - current_page, - total_pages, - window=7, - check_middle=False, - sorting_key=None, - sorting_direction=None, - ): - extra_links = 4 # first, prev, next, last - search = "'>\"/>" - if sorting_key and sorting_direction: - html_str = utils.generate_pages( - current_page, - total_pages, - search=search, - sorting_key=sorting_key, - sorting_direction=sorting_direction, - ) - else: - html_str = utils.generate_pages(current_page, total_pages, search=search) - - assert search not in html_str, "The raw search string shouldn't appear in the output" - assert "search=%27%3E%22%2F%3E%3Cimg+src%3Dx+onerror%3Dalert%281%29%3E" in html_str - - assert callable(html_str.__html__), "Should return something that is HTML-escaping aware" - - dom = BeautifulSoup(html_str, "html.parser") - assert dom is not None - - ulist = dom.ul - ulist_items = ulist.find_all("li") - assert min(window, total_pages) + extra_links == len(ulist_items) - - page_items = ulist_items[2:-2] - mid = len(page_items) // 2 - all_nodes = [] - pages = [] - - if sorting_key and sorting_direction: - last_page = total_pages - 1 - - if current_page <= mid or total_pages < window: - pages = list(range(min(total_pages, window))) - elif mid < current_page < last_page - mid: - pages = list(range(current_page - mid, current_page + mid + 1)) - else: - pages = list(range(total_pages - window, last_page + 1)) - - pages.append(last_page + 1) - pages.sort(reverse=True if sorting_direction == "desc" else False) - - for i, item in enumerate(page_items): - a_node = item.a - href_link = a_node["href"] - node_text = a_node.string - all_nodes.append(node_text) - if node_text == str(current_page + 1): - if check_middle: - assert mid == i - assert href_link == "javascript:void(0)" - assert "active" in item["class"] - else: - assert re.search(r"^\?", href_link), "Link is page-relative" - query = parse_qs(href_link[1:]) - assert query["page"] == [str(int(node_text) - 1)] - assert query["search"] == [search] - - if sorting_key and sorting_direction: - if pages[0] == 0: - pages = [str(page) for page in pages[1:]] - - assert pages == all_nodes - - def test_generate_pager_current_start(self): - self.check_generate_pages_html(current_page=0, total_pages=6) - - def test_generate_pager_current_middle(self): - self.check_generate_pages_html(current_page=10, total_pages=20, check_middle=True) - - def test_generate_pager_current_end(self): - self.check_generate_pages_html(current_page=38, total_pages=39) - - def test_generate_pager_current_start_with_sorting(self): - self.check_generate_pages_html( - current_page=0, total_pages=4, sorting_key="dag_id", sorting_direction="asc" - ) - - def test_params_no_values(self): - """Should return an empty string if no params are passed""" - assert utils.get_params() == "" - - def test_params_search(self): - assert utils.get_params(search="bash_") == "search=bash_" - - def test_params_none_and_zero(self): - query_str = utils.get_params(a=0, b=None, c="true") - # The order won't be consistent, but that doesn't affect behaviour of a browser - pairs = sorted(query_str.split("&")) - assert pairs == ["a=0", "c=true"] - - def test_params_all(self): - query = utils.get_params(tags=["tag1", "tag2"], status="active", page=3, search="bash_") - assert parse_qs(query) == { - "tags": ["tag1", "tag2"], - "page": ["3"], - "search": ["bash_"], - "status": ["active"], - } - - def test_params_escape(self): - assert ( - utils.get_params(search="'>\"/>") - == "search=%27%3E%22%2F%3E%3Cimg+src%3Dx+onerror%3Dalert%281%29%3E" - ) - - def test_state_token(self): - # It's shouldn't possible to set these odd values anymore, but lets - # ensure they are escaped! - html = str(utils.state_token("")) - - assert "<script>alert(1)</script>" in html - assert "" not in html - - def test_nobr_f(self): - attr = {"attr_name": "attribute"} - f = attr.get("attr_name") - expected_markup = Markup("{}").format(f) - - nobr = utils.nobr_f("attr_name") - result_markup = nobr(attr) - - assert result_markup == expected_markup - - def test_nobr_f_empty_attr(self): - attr = {"attr_name": ""} - f = attr.get("attr_name") - expected_markup = Markup("{}").format(f) - - nobr = utils.nobr_f("attr_name") - result_markup = nobr(attr) - - assert result_markup == expected_markup - - def test_nobr_f_missing_attr(self): - attr = {} - f = None - expected_markup = Markup("{}").format(f) - - nobr = utils.nobr_f("attr_name") - result_markup = nobr(attr) - - assert result_markup == expected_markup - - def test_epoch(self): - test_datetime = datetime(2024, 6, 19, 12, 0, 0) - result = utils.epoch(test_datetime) - epoch_time = result[0] - - expected_epoch_time = int(time.mktime(test_datetime.timetuple())) * 1000 - - assert epoch_time == expected_epoch_time - - @pytest.mark.db_test - def test_make_cache_key(self): - from airflow.www.app import cached_app - - with cached_app(testing=True).test_request_context( - "/test/path", query_string={"key1": "value1", "key2": "value2"} - ): - expected_args = str(hash(frozenset({"key1": "value1", "key2": "value2"}.items()))) - expected_cache_key = ("/test/path" + expected_args).encode("ascii", "ignore") - result_cache_key = utils.make_cache_key() - assert result_cache_key == expected_cache_key - - @pytest.mark.db_test - def test_task_instance_link(self): - from airflow.www.app import cached_app - - with cached_app(testing=True).test_request_context(): - html = str( - utils.task_instance_link( - {"dag_id": "", "task_id": "", "map_index": 1, "logical_date": datetime.now()} - ) - ) - - html_map_index_none = str( - utils.task_instance_link( - {"dag_id": "", "task_id": "", "map_index": -1, "logical_date": datetime.now()} - ) - ) - - assert "%3Ca%261%3E" in html - assert "%3Cb2%3E" in html - assert "map_index" in html - assert "" not in html - assert "" not in html - - assert "%3Ca%261%3E" in html_map_index_none - assert "%3Cb2%3E" in html_map_index_none - assert "map_index" not in html_map_index_none - assert "" not in html_map_index_none - assert "" not in html_map_index_none - - @pytest.mark.db_test - def test_dag_link(self): - from airflow.www.app import cached_app - - with cached_app(testing=True).test_request_context(): - html = str(utils.dag_link({"dag_id": "", "logical_date": datetime.now()})) - - assert "%3Ca%261%3E" in html - assert "" not in html - - @pytest.mark.db_test - def test_dag_link_when_dag_is_none(self): - """Test that when there is no dag_id, dag_link does not contain hyperlink""" - from airflow.www.app import cached_app - - with cached_app(testing=True).test_request_context(): - html = str(utils.dag_link({})) - - assert "None" in html - assert "", "run_id": "", "logical_date": datetime.now()}) - ) - - assert "%3Ca%261%3E" in html - assert "%3Cb2%3E" in html - assert "" not in html - assert "" not in html - - -class TestAttrRenderer: - def setup_method(self): - self.attr_renderer = utils.get_attr_renderer() - - def test_python_callable(self): - def example_callable(unused_self): - print("example") - - rendered = self.attr_renderer["python_callable"](example_callable) - assert ""example"" in rendered - - def test_python_callable_none(self): - rendered = self.attr_renderer["python_callable"](None) - assert rendered == "" - - def test_markdown(self): - markdown = "* foo\n* bar" - rendered = self.attr_renderer["doc_md"](markdown) - assert "
  • foo
  • " in rendered - assert "
  • bar
  • " in rendered - - def test_markdown_none(self): - rendered = self.attr_renderer["doc_md"](None) - assert rendered is None - - def test_get_dag_run_conf(self): - dag_run_conf = { - "1": "string", - "2": b"bytes", - "3": 123, - "4": "à".encode("latin"), - "5": datetime(2023, 1, 1), - } - expected_encoded_dag_run_conf = ( - '{"1": "string", "2": "bytes", "3": 123, "4": "à", "5": "2023-01-01T00:00:00+00:00"}' - ) - encoded_dag_run_conf, conf_is_json = utils.get_dag_run_conf( - dag_run_conf, json_encoder=utils_json.WebEncoder - ) - assert expected_encoded_dag_run_conf == encoded_dag_run_conf - - def test_encode_dag_run_none(self): - no_dag_run_result = utils.encode_dag_run(None) - assert no_dag_run_result == (None, None) - - def test_json_f_webencoder(self): - dag_run_conf = { - "1": "string", - "2": b"bytes", - "3": 123, - "4": "à".encode("latin"), - "5": datetime(2023, 1, 1), - } - expected_encoded_dag_run_conf = ( - # HTML sanitization is insane - '{"1": "string", "2": "bytes", "3": 123, "4": "\\u00e0", "5": "2023-01-01T00:00:00+00:00"}' - ) - expected_markup = Markup("{}").format(expected_encoded_dag_run_conf) - - formatter = json_f("conf") - dagrun = Mock() - dagrun.get = Mock(return_value=dag_run_conf) - - assert formatter(dagrun) == expected_markup - - -def create_dag_run_for_markdown(): - params = dict(run_id="run_id_1", conf={}) - if AIRFLOW_V_3_0_PLUS: - params.update(triggered_by=DagRunTriggeredByType.TEST) - return DagRun(**params) - - -class TestWrappedMarkdown: - def test_wrapped_markdown_with_docstring_curly_braces(self): - rendered = wrapped_markdown("{braces}", css_class="a_class") - assert ( - rendered - == """

    {braces}

    -
    """ - ) - - def test_wrapped_markdown_with_some_markdown(self): - rendered = wrapped_markdown( - """*italic* - **bold** - """, - css_class="a_class", - ) - - assert ( - rendered - == """

    italic -bold

    -
    """ - ) - - def test_wrapped_markdown_with_table(self): - rendered = wrapped_markdown( - """ -| Job | Duration | -| ----------- | ----------- | -| ETL | 14m | -""" - ) - - assert ( - rendered - == """
    - - - - - - - - - - - - -
    JobDuration
    ETL14m
    -
    """ - ) - - def test_wrapped_markdown_with_indented_lines(self): - rendered = wrapped_markdown( - """ - # header - 1st line - 2nd line - """ - ) - - assert ( - rendered - == """

    header

    \n

    1st line\n2nd line

    -
    """ - ) - - def test_wrapped_markdown_with_raw_code_block(self): - rendered = wrapped_markdown( - """\ - # Markdown code block - - Inline `code` works well. - - Code block - does not - respect - newlines - - """ - ) - - assert ( - rendered - == """

    Markdown code block

    -

    Inline code works well.

    -
    Code block\ndoes not\nrespect\nnewlines\n
    -
    """ - ) - - def test_wrapped_markdown_with_nested_list(self): - rendered = wrapped_markdown( - """ - ### Docstring with a code block - - - And - - A nested list - """ - ) - - assert ( - rendered - == """

    Docstring with a code block

    -
      -
    • And -
        -
      • A nested list
      • -
      -
    • -
    -
    """ - ) - - @pytest.mark.parametrize( - "html", - [ - "test raw HTML", - "hidden nuggets.", - ], - ) - def test_wrapped_markdown_with_raw_html(self, html): - """Ensure that HTML code is not ending-up in markdown but is always escaped.""" - from markupsafe import escape - - rendered = wrapped_markdown(html) - assert escape(html) in rendered - - @pytest.mark.parametrize( - "dag_run,expected_val", - [ - [None, (None, None)], - [ - create_dag_run_for_markdown(), - ( - { - "conf": None, - "conf_is_json": False, - "data_interval_end": None, - "data_interval_start": None, - "end_date": None, - "logical_date": None, - "last_scheduling_decision": None, - "note": None, - "queued_at": None, - "run_id": "run_id_1", - "run_type": None, - "start_date": None, - "state": None, - "triggered_by": "test", - }, - None, - ), - ], - ], - ) - def test_encode_dag_run(self, dag_run, expected_val): - val = encode_dag_run(dag_run) - assert val == expected_val - - def test_encode_dag_run_circular_reference(self): - conf = {} - conf["a"] = conf - dr = DagRun(run_id="run_id_1", conf=conf) - encoded_dr, error = encode_dag_run(dr) - assert encoded_dr is None - assert error == ( - f"Circular reference detected in the DAG Run config (#{dr.run_id}). " - f"You should check your webserver logs for more details." - ) - - -class TestFilter: - def setup_method(self): - self.mock_datamodel = Mock() - self.mock_query = Mock(spec=Query) - self.mock_column_name = "test_column" - - def test_filter_is_null_apply(self): - filter_is_null = utils.FilterIsNull(datamodel=self.mock_datamodel, column_name=self.mock_column_name) - - self.mock_query, mock_field = get_field_setup_query( - self.mock_query, self.mock_datamodel, self.mock_column_name - ) - mock_value = set_value_to_type(self.mock_datamodel, self.mock_column_name, None) - - result_query_filter = filter_is_null.apply(self.mock_query, None) - self.mock_query.filter.assert_called_once_with(mock_field == mock_value) - - expected_query_filter = self.mock_query.filter(mock_field == mock_value) - - assert result_query_filter == expected_query_filter - - def test_filter_is_not_null_apply(self): - filter_is_not_null = utils.FilterIsNotNull( - datamodel=self.mock_datamodel, column_name=self.mock_column_name - ) - - self.mock_query, mock_field = get_field_setup_query( - self.mock_query, self.mock_datamodel, self.mock_column_name - ) - mock_value = set_value_to_type(self.mock_datamodel, self.mock_column_name, None) - - result_query_filter = filter_is_not_null.apply(self.mock_query, None) - self.mock_query.filter.assert_called_once_with(mock_field != mock_value) - - expected_query_filter = self.mock_query.filter(mock_field != mock_value) - - assert result_query_filter == expected_query_filter - - def test_filter_gte_none_value_apply(self): - filter_gte = utils.FilterGreaterOrEqual( - datamodel=self.mock_datamodel, column_name=self.mock_column_name - ) - - self.mock_query, mock_field = get_field_setup_query( - self.mock_query, self.mock_datamodel, self.mock_column_name - ) - mock_value = set_value_to_type(self.mock_datamodel, self.mock_column_name, None) - - result_query_filter = filter_gte.apply(self.mock_query, mock_value) - - assert result_query_filter == self.mock_query - - def test_filter_lte_none_value_apply(self): - filter_lte = utils.FilterSmallerOrEqual( - datamodel=self.mock_datamodel, column_name=self.mock_column_name - ) - - self.mock_query, mock_field = get_field_setup_query( - self.mock_query, self.mock_datamodel, self.mock_column_name - ) - mock_value = set_value_to_type(self.mock_datamodel, self.mock_column_name, None) - - result_query_filter = filter_lte.apply(self.mock_query, mock_value) - - assert result_query_filter == self.mock_query - - -@pytest.mark.db_test -def test_get_col_default_not_existing(session): - interface = CustomSQLAInterface(obj=DagRun, session=session) - default_value = interface.get_col_default("column_not_existing") - assert default_value is None - - -@pytest.mark.db_test -def test_dag_run_custom_sqla_interface_delete_no_collateral_damage(dag_maker, session): - interface = DagRunCustomSQLAInterface(obj=DagRun, session=session) - dag_ids = (f"test_dag_{x}" for x in range(1, 4)) - dates = (pendulum.datetime(2023, 1, x) for x in range(1, 4)) - for dag_id, date in itertools.product(dag_ids, dates): - with dag_maker(dag_id=dag_id): - pass - dag_maker.create_dagrun(logical_date=date, state="running", run_type="scheduled") - dag_runs = session.query(DagRun).all() - assert len(dag_runs) == 9 - assert len(set(x.run_id for x in dag_runs)) == 3 - run_id_for_single_delete = "scheduled__2023-01-01T00:00:00+00:00" - # we have 3 runs with this same run_id - assert sum(1 for x in dag_runs if x.run_id == run_id_for_single_delete) == 3 - # each is a different dag - - # if we delete one, it shouldn't delete the others - one_run = next(x for x in dag_runs if x.run_id == run_id_for_single_delete) - assert interface.delete(item=one_run) is True - session.commit() - dag_runs = session.query(DagRun).all() - # we should have one fewer dag run now - assert len(dag_runs) == 8 - - # now let's try multi delete - run_id_for_multi_delete = "scheduled__2023-01-02T00:00:00+00:00" - # verify we have 3 - runs_of_interest = [x for x in dag_runs if x.run_id == run_id_for_multi_delete] - assert len(runs_of_interest) == 3 - # and that each is different dag - assert len(set(x.dag_id for x in dag_runs)) == 3 - - to_delete = runs_of_interest[:2] - # now try multi delete - assert interface.delete_all(items=to_delete) is True - session.commit() - dag_runs = session.query(DagRun).all() - assert len(dag_runs) == 6 - assert len(set(x.dag_id for x in dag_runs)) == 3 - assert len(set(x.run_id for x in dag_runs)) == 3 - - -@pytest.fixture -def app(): - from flask import Flask - - app = Flask(__name__) - app.config["WTF_CSRF_ENABLED"] = False - app.config["SECRET_KEY"] = "secret" - with app.app_context(): - yield app - - -class TestWidgets: - def test_airflow_datetime_picker_ro_widget(self, app): - class TestForm(FlaskForm): - datetime_field = StringField(widget=AirflowDateTimePickerROWidget()) - - form = TestForm() - field = form.datetime_field - - html_output = field() - - assert 'readonly="true"' in html_output - assert "input-group datetime datetimepicker" in html_output - - def test_bs3_text_field_ro_widget(self, app): - class TestForm(FlaskForm): - text_field = StringField(widget=BS3TextFieldROWidget()) - - form = TestForm() - field = form.text_field - - html_output = field() - - assert 'readonly="true"' in html_output - assert "form-control" in html_output - - def test_bs3_text_area_ro_widget(self, app): - class TestForm(FlaskForm): - textarea_field = TextAreaField(widget=BS3TextAreaROWidget()) - - form = TestForm() - field = form.textarea_field - - html_output = field() - - assert 'readonly="true"' in html_output - assert "form-control" in html_output diff --git a/tests/www/views/__init__.py b/tests/www/views/__init__.py deleted file mode 100644 index 13a83393a9124..0000000000000 --- a/tests/www/views/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. diff --git a/tests/www/views/conftest.py b/tests/www/views/conftest.py deleted file mode 100644 index ea5db14df11db..0000000000000 --- a/tests/www/views/conftest.py +++ /dev/null @@ -1,138 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os - -import jinja2 -import pytest - -from airflow import settings -from airflow.models import DagBag -from airflow.www.app import create_app - -from tests_common.test_utils.api_connexion_utils import delete_user -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import parse_and_sync_to_db -from tests_common.test_utils.decorators import dont_initialize_flask_app_submodules -from tests_common.test_utils.www import ( - client_with_login, - client_without_login, - client_without_login_as_admin, -) - - -@pytest.fixture(autouse=True, scope="module") -def session(): - settings.configure_orm() - return settings.Session - - -@pytest.fixture(autouse=True, scope="module") -def examples_dag_bag(session): - parse_and_sync_to_db(os.devnull, include_examples=True) - dag_bag = DagBag(read_dags_from_db=True) - session.commit() - return dag_bag - - -@pytest.fixture(scope="module") -def app(examples_dag_bag): - @dont_initialize_flask_app_submodules( - skip_all_except=[ - "init_api_connexion", - "init_appbuilder", - "init_appbuilder_links", - "init_appbuilder_views", - "init_flash_views", - "init_jinja_globals", - "init_plugins", - "init_airflow_session_interface", - ] - ) - def factory(): - with conf_vars({("fab", "auth_rate_limited"): "False"}): - return create_app(testing=True) - - app = factory() - app.config["WTF_CSRF_ENABLED"] = False - app.dag_bag = examples_dag_bag - app.jinja_env.undefined = jinja2.StrictUndefined - - security_manager = app.appbuilder.sm - - test_users = [ - { - "username": "test_admin", - "first_name": "test_admin_first_name", - "last_name": "test_admin_last_name", - "email": "test_admin@fab.org", - "role": security_manager.find_role("Admin"), - "password": "test_admin_password", - }, - { - "username": "test_user", - "first_name": "test_user_first_name", - "last_name": "test_user_last_name", - "email": "test_user@fab.org", - "role": security_manager.find_role("User"), - "password": "test_user_password", - }, - { - "username": "test_viewer", - "first_name": "test_viewer_first_name", - "last_name": "test_viewer_last_name", - "email": "test_viewer@fab.org", - "role": security_manager.find_role("Viewer"), - "password": "test_viewer_password", - }, - ] - - for user_dict in test_users: - if not security_manager.find_user(username=user_dict["username"]): - security_manager.add_user(**user_dict) - - yield app - - for user_dict in test_users: - delete_user(app, user_dict["username"]) - - -@pytest.fixture -def admin_client(app): - return client_with_login(app, username="test_admin", password="test_admin") - - -@pytest.fixture -def viewer_client(app): - return client_with_login(app, username="test_viewer", password="test_viewer") - - -@pytest.fixture -def user_client(app): - return client_with_login(app, username="test_user", password="test_user") - - -@pytest.fixture -def anonymous_client(app): - return client_without_login(app) - - -@pytest.fixture -def anonymous_client_as_admin(app): - return client_without_login_as_admin(app) diff --git a/tests/www/views/test_anonymous_as_admin_role.py b/tests/www/views/test_anonymous_as_admin_role.py deleted file mode 100644 index fbff4cdd37409..0000000000000 --- a/tests/www/views/test_anonymous_as_admin_role.py +++ /dev/null @@ -1,66 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from urllib.parse import quote_plus - -import pytest - -from airflow.models import Pool -from airflow.utils.session import create_session - -from tests_common.test_utils.www import check_content_in_response - -pytestmark = pytest.mark.db_test - -POOL = { - "pool": "test-pool", - "slots": 777, - "description": "test-pool-description", - "include_deferred": False, -} - - -@pytest.fixture(autouse=True) -def clear_pools(): - with create_session() as session: - session.query(Pool).delete() - - -@pytest.fixture -def pool_factory(session): - def factory(**values): - pool = Pool(**{**POOL, **values}) # Passed in values override defaults. - session.add(pool) - session.commit() - return pool - - return factory - - -def test_delete_pool_anonymous_user_no_role(anonymous_client, pool_factory): - pool = pool_factory() - resp = anonymous_client.post(f"pool/delete/{pool.id}") - assert resp.status_code == 302 - assert f"/login/?next={quote_plus(f'http://localhost/pool/delete/{pool.id}')}" == resp.headers["Location"] - - -def test_delete_pool_anonymous_user_as_admin(anonymous_client_as_admin, pool_factory): - pool = pool_factory() - resp = anonymous_client_as_admin.post(f"pool/delete/{pool.id}", follow_redirects=True) - check_content_in_response("Deleted Row", resp) diff --git a/tests/www/views/test_session.py b/tests/www/views/test_session.py deleted file mode 100644 index d6f9f12454746..0000000000000 --- a/tests/www/views/test_session.py +++ /dev/null @@ -1,110 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from unittest import mock - -import pytest - -from airflow.exceptions import AirflowConfigException -from airflow.www import app - -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.decorators import dont_initialize_flask_app_submodules - -pytestmark = pytest.mark.db_test - - -def get_session_cookie(client): - return next((cookie for cookie in client.cookie_jar if cookie.name == "session"), None) - - -def test_session_cookie_created_on_login(user_client): - assert get_session_cookie(user_client) is not None - - -def test_session_inaccessible_after_logout(user_client): - session_cookie = get_session_cookie(user_client) - assert session_cookie is not None - - resp = user_client.post("/logout/") - assert resp.status_code == 302 - - # Try to access /home with the session cookie from earlier - user_client.set_cookie("session", session_cookie.value) - user_client.get("/home/") - assert resp.status_code == 302 - - -def test_invalid_session_backend_option(): - @dont_initialize_flask_app_submodules( - skip_all_except=[ - "init_api_connexion", - "init_appbuilder", - "init_appbuilder_links", - "init_appbuilder_views", - "init_flash_views", - "init_jinja_globals", - "init_plugins", - "init_airflow_session_interface", - ] - ) - def poorly_configured_app_factory(): - with conf_vars({("webserver", "session_backend"): "invalid_value_for_session_backend"}): - return app.create_app(testing=True) - - expected_exc_regex = ( - "^Unrecognized session backend specified in web_server_session_backend: " - r"'invalid_value_for_session_backend'\. Please set this to .+\.$" - ) - with pytest.raises(AirflowConfigException, match=expected_exc_regex): - poorly_configured_app_factory() - - -def test_session_id_rotates(app, user_client): - old_session_cookie = get_session_cookie(user_client) - assert old_session_cookie is not None - - resp = user_client.post("/logout/") - assert resp.status_code == 302 - - patch_path = "airflow.providers.fab.auth_manager.security_manager.override.check_password_hash" - with mock.patch(patch_path) as check_password_hash: - check_password_hash.return_value = True - resp = user_client.post("/login/", data={"username": "test_user", "password": "test_user"}) - assert resp.status_code == 302 - - new_session_cookie = get_session_cookie(user_client) - assert new_session_cookie is not None - assert old_session_cookie.value != new_session_cookie.value - - -def test_check_active_user(app, user_client): - user = app.appbuilder.sm.find_user(username="test_user") - user.active = False - resp = user_client.get("/home") - assert resp.status_code == 302 - assert "/login/?next=http%3A%2F%2Flocalhost%2Fhome" in resp.headers.get("Location") - - -def test_check_deactivated_user_redirected_to_login(app, user_client): - with app.test_request_context(): - user = app.appbuilder.sm.find_user(username="test_user") - user.active = False - resp = user_client.get("/home", follow_redirects=True) - assert resp.status_code == 200 - assert "/login" in resp.request.url diff --git a/tests/www/views/test_views.py b/tests/www/views/test_views.py deleted file mode 100644 index 871b7960200be..0000000000000 --- a/tests/www/views/test_views.py +++ /dev/null @@ -1,592 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os -import re -from unittest import mock -from unittest.mock import patch - -import pytest -from markupsafe import Markup - -from airflow.configuration import ( - initialize_config, - write_default_airflow_configuration_if_needed, - write_webserver_configuration_if_needed, -) -from airflow.plugins_manager import AirflowPlugin, EntryPointSource -from airflow.utils.docs import get_doc_url_for_provider -from airflow.utils.task_group import TaskGroup -from airflow.www.views import ( - ProviderView, - get_key_paths, - get_safe_url, - get_task_stats_from_query, - get_value_from_path, -) - -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.mock_plugins import mock_plugin_manager -from tests_common.test_utils.www import ( - capture_templates, # noqa: F401 - check_content_in_response, - check_content_not_in_response, -) - -pytestmark = pytest.mark.db_test - - -def test_configuration_do_not_expose_config(admin_client): - with conf_vars({("webserver", "expose_config"): "False"}): - resp = admin_client.get("configuration", follow_redirects=True) - check_content_in_response( - [ - "Airflow Configuration", - "Your Airflow administrator chose not to expose the configuration, " - "most likely for security reasons.", - ], - resp, - ) - - -@mock.patch.dict(os.environ, {"AIRFLOW__CORE__UNIT_TEST_MODE": "False"}) -def test_configuration_expose_config(admin_client): - # make sure config is initialized (without unit test mote) - conf = initialize_config() - conf.validate() - with conf_vars({("webserver", "expose_config"): "True"}): - resp = admin_client.get("configuration", follow_redirects=True) - check_content_in_response(["Airflow Configuration"], resp) - - -@mock.patch("airflow.configuration.WEBSERVER_CONFIG") -def test_webserver_configuration_config_file(mock_webserver_config_global, admin_client, tmp_path): - import airflow.configuration - - config_file = str(tmp_path / "my_custom_webserver_config.py") - with mock.patch.dict(os.environ, {"AIRFLOW__WEBSERVER__CONFIG_FILE": config_file}): - conf = write_default_airflow_configuration_if_needed() - write_webserver_configuration_if_needed(conf) - initialize_config() - assert config_file == airflow.configuration.WEBSERVER_CONFIG - - assert os.path.isfile(config_file) - - -def test_redoc_should_render_template( - capture_templates, # noqa: F811 - admin_client, -): - from airflow.utils.docs import get_docs_url - - with capture_templates() as templates: - resp = admin_client.get("redoc") - check_content_in_response("Redoc", resp) - - assert len(templates) == 1 - assert templates[0].name == "airflow/redoc.html" - assert templates[0].local_context == { - "config_test_connection": "Disabled", - "openapi_spec_url": "/api/v1/openapi.yaml", - "rest_api_enabled": True, - "get_docs_url": get_docs_url, - "excluded_events_raw": "", - "included_events_raw": "", - } - - -def test_plugin_should_list_on_page_with_details(admin_client): - resp = admin_client.get("/plugin") - check_content_in_response("test_plugin", resp) - check_content_in_response("Airflow Plugins", resp) - check_content_in_response("source", resp) - check_content_in_response("$PLUGINS_FOLDER/test_plugin.py", resp) - - -def test_plugin_should_list_entrypoint_on_page_with_details(admin_client): - mock_plugin = AirflowPlugin() - mock_plugin.name = "test_plugin" - mock_plugin.source = EntryPointSource( - mock.Mock(), mock.Mock(version="1.0.0", metadata={"Name": "test-entrypoint-testpluginview"}) - ) - with mock_plugin_manager(plugins=[mock_plugin]): - resp = admin_client.get("/plugin") - - check_content_in_response("test_plugin", resp) - check_content_in_response("Airflow Plugins", resp) - check_content_in_response("source", resp) - check_content_in_response("test-entrypoint-testpluginview==1.0.0: Apache Beam
    ' - ) - check_content_in_response(beam_href, resp) - check_content_in_response(beam_text, resp) - check_content_in_response(beam_description, resp) - check_content_in_response("Providers", resp) - - -@pytest.mark.parametrize( - "provider_description, expected", - [ - ( - "`Airbyte `__", - Markup('Airbyte'), - ), - ( - "Amazon integration (including `Amazon Web Services (AWS) `__).", - Markup( - 'Amazon integration (including Amazon Web Services (AWS)).' - ), - ), - ( - "`Java Database Connectivity (JDBC) `__", - Markup( - 'Java Database Connectivity (JDBC)' - ), - ), - ( - "`click me `__", - Markup("`click me <javascript:prompt(document.domain)>`__"), - ), - ], -) -def test__clean_description(admin_client, provider_description, expected): - p = ProviderView() - actual = p._clean_description(provider_description) - assert actual == expected - - -@pytest.mark.parametrize( - "provider_name, project_url, expected", - [ - ( - "apache-airflow-providers-airbyte", - "Documentation, https://airflow.apache.org/docs/apache-airflow-providers-airbyte/3.8.1/", - "https://airflow.apache.org/docs/apache-airflow-providers-airbyte/3.8.1/", - ), - ( - "apache-airflow-providers-amazon", - "Documentation, https://airflow.apache.org/docs/apache-airflow-providers-amazon/8.25.0/", - "https://airflow.apache.org/docs/apache-airflow-providers-amazon/8.25.0/", - ), - ( - "apache-airflow-providers-apache-druid", - "Documentation, javascript:prompt(document.domain)", - # the default one is returned - "https://airflow.apache.org/docs/apache-airflow-providers-apache-druid/1.0.0/", - ), - ], -) -@patch("airflow.utils.docs.get_project_url_from_metadata") -def test_get_doc_url_for_provider( - mock_get_project_url_from_metadata, admin_client, provider_name, project_url, expected -): - mock_get_project_url_from_metadata.return_value = [project_url] - actual = get_doc_url_for_provider(provider_name, "1.0.0") - assert actual == expected - - -def test_endpoint_should_not_be_unauthenticated(app): - resp = app.test_client().get("/provider", follow_redirects=True) - check_content_not_in_response("Providers", resp) - check_content_in_response("Sign In - Airflow", resp) - - -@pytest.mark.parametrize( - "url, content", - [ - ( - "/taskinstance/list/?_flt_0_logical_date=2018-10-09+22:44:31", - "List Task Instance", - ), - ( - "/taskreschedule/list/?_flt_0_logical_date=2018-10-09+22:44:31", - "List Task Reschedule", - ), - ], - ids=["instance", "reschedule"], -) -def test_task_start_date_filter(admin_client, url, content): - resp = admin_client.get(url) - # We aren't checking the logic of the date filter itself (that is built - # in to FAB) but simply that our UTC conversion was run - i.e. it - # doesn't blow up! - check_content_in_response(content, resp) - - -@pytest.mark.parametrize( - "url", - [ - "/taskinstance/list/?_flt_1_try_number=0", # greater than - "/taskinstance/list/?_flt_2_try_number=5", # less than - ], -) -def test_try_number_filter(admin_client, url): - resp = admin_client.get(url) - # Ensure that the taskInstance view can filter on gt / lt try_number - check_content_in_response("List Task Instance", resp) - - -@pytest.mark.parametrize( - "url, content", - [ - ( - "/taskinstance/list/?_flt_3_dag_id=test_dag", - "List Task Instance", - ) - ], - ids=["instance"], -) -def test_task_dag_id_equals_filter(admin_client, url, content): - resp = admin_client.get(url) - # We aren't checking the logic of the dag_id filter itself (that is built - # in to FAB) but simply that dag_id filter was run - check_content_in_response(content, resp) - - -@pytest.mark.parametrize( - "test_url, expected_url", - [ - ("", "/home"), - ("javascript:alert(1)", "/home"), - (" javascript:alert(1)", "/home"), - ("http://google.com", "/home"), - ("google.com", "http://localhost:8080/google.com"), - ("\\/google.com", "http://localhost:8080/\\/google.com"), - ("//google.com", "/home"), - ("\\/\\/google.com", "http://localhost:8080/\\/\\/google.com"), - ("36539'%3balert(1)%2f%2f166", "/home"), - ( - "http://localhost:8080/trigger?dag_id=test&origin=36539%27%3balert(1)%2f%2f166&abc=2", - "/home", - ), - ( - "http://localhost:8080/trigger?dag_id=test_dag&origin=%2Ftree%3Fdag_id%test_dag';alert(33)//", - "/home", - ), - ( - "http://localhost:8080/trigger?dag_id=test_dag&origin=%2Ftree%3Fdag_id%3Dtest_dag", - "http://localhost:8080/trigger?dag_id=test_dag&origin=%2Ftree%3Fdag_id%3Dtest_dag", - ), - ], -) -@mock.patch("airflow.www.views.url_for") -def test_get_safe_url(mock_url_for, app, test_url, expected_url): - mock_url_for.return_value = "/home" - with app.test_request_context(base_url="http://localhost:8080"): - assert get_safe_url(test_url) == expected_url - - -@pytest.fixture -def test_app(): - from airflow.www import app - - return app.create_app(testing=True) - - -def test_mark_task_instance_state(test_app, dag_maker): - """ - Test that _mark_task_instance_state() does all three things: - - Marks the given TaskInstance as SUCCESS; - - Clears downstream TaskInstances in FAILED/UPSTREAM_FAILED state; - """ - from airflow.models.dagbag import DagBag - from airflow.models.taskinstance import TaskInstance - from airflow.providers.standard.operators.empty import EmptyOperator - from airflow.utils.session import create_session - from airflow.utils.state import State - from airflow.utils.timezone import datetime - from airflow.utils.types import DagRunType - from airflow.www.views import Airflow - - from tests_common.test_utils.db import clear_db_runs - - clear_db_runs() - start_date = datetime(2020, 1, 1) - with dag_maker("test_mark_task_instance_state", start_date=start_date, schedule="0 0 * * *") as dag: - task_1 = EmptyOperator(task_id="task_1") - task_2 = EmptyOperator(task_id="task_2") - task_3 = EmptyOperator(task_id="task_3") - task_4 = EmptyOperator(task_id="task_4") - task_5 = EmptyOperator(task_id="task_5") - - task_1 >> [task_2, task_3, task_4, task_5] - - dagrun = dag_maker.create_dagrun(state=State.FAILED, run_type=DagRunType.SCHEDULED) - - def get_task_instance(session, task): - return ( - session.query(TaskInstance) - .filter( - TaskInstance.dag_id == dag.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.logical_date == start_date, - ) - .one() - ) - - with create_session() as session: - get_task_instance(session, task_1).state = State.FAILED - get_task_instance(session, task_2).state = State.SUCCESS - get_task_instance(session, task_3).state = State.UPSTREAM_FAILED - get_task_instance(session, task_4).state = State.FAILED - get_task_instance(session, task_5).state = State.SKIPPED - - session.commit() - - test_app.dag_bag = DagBag(dag_folder="/dev/null", include_examples=False) - test_app.dag_bag.bag_dag(dag=dag) - - with test_app.test_request_context(): - view = Airflow() - - view._mark_task_instance_state( - dag_id=dag.dag_id, - run_id=dagrun.run_id, - task_id=task_1.task_id, - map_indexes=None, - origin="", - upstream=False, - downstream=False, - future=False, - past=False, - state=State.SUCCESS, - ) - - with create_session() as session: - # After _mark_task_instance_state, task_1 is marked as SUCCESS - assert get_task_instance(session, task_1).state == State.SUCCESS - # task_2 remains as SUCCESS - assert get_task_instance(session, task_2).state == State.SUCCESS - # task_3 and task_4 are cleared because they were in FAILED/UPSTREAM_FAILED state - assert get_task_instance(session, task_3).state == State.NONE - assert get_task_instance(session, task_4).state == State.NONE - # task_5 remains as SKIPPED - assert get_task_instance(session, task_5).state == State.SKIPPED - dagrun.refresh_from_db(session=session) - # dagrun should be set to QUEUED - assert dagrun.get_state() == State.QUEUED - - -def test_mark_task_group_state(test_app, dag_maker): - """ - Test that _mark_task_group_state() does all three things: - - Marks the given TaskGroup as SUCCESS; - - Clears downstream TaskInstances in FAILED/UPSTREAM_FAILED state; - - Set DagRun to QUEUED. - """ - from airflow.models.dagbag import DagBag - from airflow.models.taskinstance import TaskInstance - from airflow.providers.standard.operators.empty import EmptyOperator - from airflow.utils.session import create_session - from airflow.utils.state import State - from airflow.utils.timezone import datetime - from airflow.utils.types import DagRunType - from airflow.www.views import Airflow - - from tests_common.test_utils.db import clear_db_runs - - clear_db_runs() - start_date = datetime(2020, 1, 1) - with dag_maker("test_mark_task_group_state", start_date=start_date, schedule="0 0 * * *") as dag: - start = EmptyOperator(task_id="start") - - with TaskGroup("section_1", tooltip="Tasks for section_1") as section_1: - task_1 = EmptyOperator(task_id="task_1") - task_2 = EmptyOperator(task_id="task_2") - task_3 = EmptyOperator(task_id="task_3") - - task_1 >> [task_2, task_3] - - task_4 = EmptyOperator(task_id="task_4") - task_5 = EmptyOperator(task_id="task_5") - task_6 = EmptyOperator(task_id="task_6") - task_7 = EmptyOperator(task_id="task_7") - task_8 = EmptyOperator(task_id="task_8") - - start >> section_1 >> [task_4, task_5, task_6, task_7, task_8] - - dagrun = dag_maker.create_dagrun(state=State.FAILED, run_type=DagRunType.SCHEDULED) - - def get_task_instance(session, task): - return ( - session.query(TaskInstance) - .filter( - TaskInstance.dag_id == dag.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.logical_date == start_date, - ) - .one() - ) - - with create_session() as session: - get_task_instance(session, task_1).state = State.FAILED - get_task_instance(session, task_2).state = State.SUCCESS - get_task_instance(session, task_3).state = State.UPSTREAM_FAILED - get_task_instance(session, task_4).state = State.SUCCESS - get_task_instance(session, task_5).state = State.UPSTREAM_FAILED - get_task_instance(session, task_6).state = State.FAILED - get_task_instance(session, task_7).state = State.SKIPPED - - session.commit() - - test_app.dag_bag = DagBag(dag_folder="/dev/null", include_examples=False) - test_app.dag_bag.bag_dag(dag=dag) - - with test_app.test_request_context(): - view = Airflow() - - view._mark_task_group_state( - dag_id=dag.dag_id, - run_id=dagrun.run_id, - group_id=section_1.group_id, - origin="", - upstream=False, - downstream=False, - future=False, - past=False, - state=State.SUCCESS, - ) - - with create_session() as session: - # After _mark_task_group_state, task_1 is marked as SUCCESS - assert get_task_instance(session, task_1).state == State.SUCCESS - # task_2 should remain as SUCCESS - assert get_task_instance(session, task_2).state == State.SUCCESS - # task_3 should be marked as SUCCESS - assert get_task_instance(session, task_3).state == State.SUCCESS - # task_4 should remain as SUCCESS - assert get_task_instance(session, task_4).state == State.SUCCESS - # task_5 and task_6 are cleared because they were in FAILED/UPSTREAM_FAILED state - assert get_task_instance(session, task_5).state == State.NONE - assert get_task_instance(session, task_6).state == State.NONE - # task_7 remains as SKIPPED - assert get_task_instance(session, task_7).state == State.SKIPPED - dagrun.refresh_from_db(session=session) - # dagrun should be set to QUEUED - assert dagrun.get_state() == State.QUEUED - - -TEST_CONTENT_DICT = {"key1": {"key2": "val2", "key3": "val3", "key4": {"key5": "val5"}}} - - -@pytest.mark.parametrize( - "test_content_dict, expected_paths", [(TEST_CONTENT_DICT, ("key1.key2", "key1.key3", "key1.key4.key5"))] -) -def test_generate_key_paths(test_content_dict, expected_paths): - for key_path in get_key_paths(test_content_dict): - assert key_path in expected_paths - - -@pytest.mark.parametrize( - "test_content_dict, test_key_path, expected_value", - [ - (TEST_CONTENT_DICT, "key1.key2", "val2"), - (TEST_CONTENT_DICT, "key1.key3", "val3"), - (TEST_CONTENT_DICT, "key1.key4.key5", "val5"), - ], -) -def test_get_value_from_path(test_content_dict, test_key_path, expected_value): - assert expected_value == get_value_from_path(test_key_path, test_content_dict) - - -def test_get_task_stats_from_query(): - query_data = [ - ["dag1", "queued", True, 1], - ["dag1", "running", True, 2], - ["dag1", "success", False, 3], - ["dag2", "running", True, 4], - ["dag2", "success", True, 5], - ["dag3", "success", False, 6], - ] - expected_data = { - "dag1": { - "queued": 1, - "running": 2, - }, - "dag2": { - "running": 4, - "success": 5, - }, - "dag3": { - "success": 6, - }, - } - - data = get_task_stats_from_query(query_data) - assert data == expected_data - - -INVALID_DATETIME_RESPONSE = re.compile(r"Invalid datetime: &#x?\d+;invalid&#x?\d+;") - - -@pytest.mark.parametrize( - "url, content", - [ - ( - "/rendered-templates?logical_date=invalid", - INVALID_DATETIME_RESPONSE, - ), - ( - "/log?dag_id=tutorial&logical_date=invalid", - INVALID_DATETIME_RESPONSE, - ), - ( - "/redirect_to_external_log?logical_date=invalid", - INVALID_DATETIME_RESPONSE, - ), - ( - "/task?logical_date=invalid", - INVALID_DATETIME_RESPONSE, - ), - ( - "dags/example_bash_operator/graph?logical_date=invalid", - INVALID_DATETIME_RESPONSE, - ), - ( - "dags/example_bash_operator/gantt?logical_date=invalid", - INVALID_DATETIME_RESPONSE, - ), - ( - "extra_links?logical_date=invalid", - INVALID_DATETIME_RESPONSE, - ), - ], -) -def test_invalid_dates(app, admin_client, url, content): - """Test invalid date format doesn't crash page.""" - resp = admin_client.get(url, follow_redirects=True) - - assert resp.status_code == 400 - assert re.search(content, resp.get_data().decode()) diff --git a/tests/www/views/test_views_asset.py b/tests/www/views/test_views_asset.py deleted file mode 100644 index a5d61b0f5edba..0000000000000 --- a/tests/www/views/test_views_asset.py +++ /dev/null @@ -1,413 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - - -from __future__ import annotations - -import pendulum -import pytest -from dateutil.tz import UTC - -from airflow.models.asset import AssetActive, AssetEvent, AssetModel -from airflow.providers.standard.operators.empty import EmptyOperator -from airflow.sdk.definitions.asset import Asset - -from tests_common.test_utils.asserts import assert_queries_count -from tests_common.test_utils.db import clear_db_assets - -pytestmark = pytest.mark.db_test - - -class TestAssetEndpoint: - @pytest.fixture(autouse=True) - def _cleanup(self): - clear_db_assets() - yield - clear_db_assets() - - @pytest.fixture - def create_assets(self, session): - def create(indexes): - assets = [ - AssetModel(id=i, uri=f"s3://bucket/key/{i}", name=f"asset-{i}", group="asset") - for i in indexes - ] - session.add_all(assets) - session.flush() - session.add_all(AssetActive.for_asset(a) for a in assets) - session.flush() - return assets - - return create - - -class TestGetAssets(TestAssetEndpoint): - def test_should_respond_200(self, admin_client, create_assets, session): - create_assets([1, 2]) - session.commit() - assert session.query(AssetModel).count() == 2 - - with assert_queries_count(10): - response = admin_client.get("/object/assets_summary") - - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "assets": [ - { - "id": 1, - "uri": "s3://bucket/key/1", - "last_asset_update": None, - "total_updates": 0, - }, - { - "id": 2, - "uri": "s3://bucket/key/2", - "last_asset_update": None, - "total_updates": 0, - }, - ], - "total_entries": 2, - } - - def test_order_by_raises_400_for_invalid_attr(self, admin_client, create_assets, session): - create_assets([1, 2]) - session.commit() - assert session.query(AssetModel).count() == 2 - - response = admin_client.get("/object/assets_summary?order_by=fake") - - assert response.status_code == 400 - msg = "Ordering with 'fake' is disallowed or the attribute does not exist on the model" - assert response.json["detail"] == msg - - def test_order_by_raises_400_for_invalid_datetimes(self, admin_client, create_assets, session): - create_assets([1, 2]) - session.commit() - assert session.query(AssetModel).count() == 2 - - response = admin_client.get("/object/assets_summary?updated_before=null") - - assert response.status_code == 400 - assert "Invalid datetime:" in response.text - - response = admin_client.get("/object/assets_summary?updated_after=null") - - assert response.status_code == 400 - assert "Invalid datetime:" in response.text - - def test_filter_by_datetimes(self, admin_client, create_assets, session): - today = pendulum.today("UTC") - - assets = create_assets(range(1, 4)) - # Update assets, one per day, starting with assets[0], ending with assets[2] - asset_events = [ - AssetEvent( - asset_id=a.id, - timestamp=today.add(days=-len(assets) + i + 1), - ) - for i, a in enumerate(assets) - ] - session.add_all(asset_events) - session.commit() - assert session.query(AssetModel).count() == len(assets) - - cutoff = today.add(days=-1).add(minutes=-5).to_iso8601_string() - response = admin_client.get(f"/object/assets_summary?updated_after={cutoff}") - - assert response.status_code == 200 - assert response.json["total_entries"] == 2 - assert [json_dict["id"] for json_dict in response.json["assets"]] == [2, 3] - - cutoff = today.add(days=-1).add(minutes=5).to_iso8601_string() - response = admin_client.get(f"/object/assets_summary?updated_before={cutoff}") - - assert response.status_code == 200 - assert response.json["total_entries"] == 2 - assert [json_dict["id"] for json_dict in response.json["assets"]] == [1, 2] - - @pytest.mark.parametrize( - "order_by, ordered_asset_ids", - [ - ("uri", [1, 2, 3, 4]), - ("-uri", [4, 3, 2, 1]), - ("last_asset_update", [4, 1, 3, 2]), - ("-last_asset_update", [2, 3, 1, 4]), - ], - ) - def test_order_by(self, admin_client, session, create_assets, order_by, ordered_asset_ids): - assets = create_assets(range(1, len(ordered_asset_ids) + 1)) - asset_events = [ - AssetEvent( - asset_id=assets[2].id, - timestamp=pendulum.today("UTC").add(days=-3), - ), - AssetEvent( - asset_id=assets[1].id, - timestamp=pendulum.today("UTC").add(days=-2), - ), - AssetEvent( - asset_id=assets[1].id, - timestamp=pendulum.today("UTC").add(days=-1), - ), - ] - session.add_all(asset_events) - session.commit() - assert session.query(AssetModel).count() == len(ordered_asset_ids) - - response = admin_client.get(f"/object/assets_summary?order_by={order_by}") - - assert response.status_code == 200 - assert ordered_asset_ids == [json_dict["id"] for json_dict in response.json["assets"]] - assert response.json["total_entries"] == len(ordered_asset_ids) - - def test_search_uri_pattern(self, admin_client, create_assets, session): - create_assets([1, 2]) - session.commit() - assert session.query(AssetModel).count() == 2 - - uri_pattern = "key_2" - response = admin_client.get(f"/object/assets_summary?uri_pattern={uri_pattern}") - - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "assets": [ - { - "id": 2, - "uri": "s3://bucket/key/2", - "last_asset_update": None, - "total_updates": 0, - }, - ], - "total_entries": 1, - } - - uri_pattern = "s3://bucket/key_" - response = admin_client.get(f"/object/assets_summary?uri_pattern={uri_pattern}") - - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "assets": [ - { - "id": 1, - "uri": "s3://bucket/key/1", - "last_asset_update": None, - "total_updates": 0, - }, - { - "id": 2, - "uri": "s3://bucket/key/2", - "last_asset_update": None, - "total_updates": 0, - }, - ], - "total_entries": 2, - } - - @pytest.mark.need_serialized_dag - def test_correct_counts_update(self, admin_client, session, dag_maker, app, monkeypatch): - with monkeypatch.context() as m: - assets = [Asset(uri=f"s3://bucket/key/{i}", name=f"asset-{i}") for i in range(1, 6)] - - # DAG that produces asset #1 - with dag_maker(dag_id="upstream", schedule=None, serialized=True, session=session): - EmptyOperator(task_id="task1", outlets=[assets[0]]) - - # DAG that is consumes only assets #1 and #2 - with dag_maker(dag_id="downstream", schedule=assets[:2], serialized=True, session=session): - EmptyOperator(task_id="task1") - - # We create multiple asset-producing and asset-consuming DAGs because the query requires - # COUNT(DISTINCT ...) for total_updates, or else it returns a multiple of the correct number due - # to the outer joins with DagScheduleAssetReference and TaskOutletAssetReference - # Two independent DAGs that produce asset #3 - with dag_maker(dag_id="independent_producer_1", serialized=True, session=session): - EmptyOperator(task_id="task1", outlets=[assets[2]]) - with dag_maker(dag_id="independent_producer_2", serialized=True, session=session): - EmptyOperator(task_id="task1", outlets=[assets[2]]) - # Two independent DAGs that consume asset #4 - with dag_maker( - dag_id="independent_consumer_1", - schedule=[assets[3]], - serialized=True, - session=session, - ): - EmptyOperator(task_id="task1") - with dag_maker( - dag_id="independent_consumer_2", - schedule=[assets[3]], - serialized=True, - session=session, - ): - EmptyOperator(task_id="task1") - - # Independent DAG that is produces and consumes the same asset, #5 - with dag_maker( - dag_id="independent_producer_self_consumer", - schedule=[assets[4]], - serialized=True, - session=session, - ): - EmptyOperator(task_id="task1", outlets=[assets[4]]) - - m.setattr(app, "dag_bag", dag_maker.dagbag) - - asset1_id = session.query(AssetModel.id).filter_by(uri=assets[0].uri).scalar() - asset2_id = session.query(AssetModel.id).filter_by(uri=assets[1].uri).scalar() - asset3_id = session.query(AssetModel.id).filter_by(uri=assets[2].uri).scalar() - asset4_id = session.query(AssetModel.id).filter_by(uri=assets[3].uri).scalar() - asset5_id = session.query(AssetModel.id).filter_by(uri=assets[4].uri).scalar() - - # asset 1 events - session.add_all( - [ - AssetEvent( - asset_id=asset1_id, - timestamp=pendulum.DateTime(2022, 8, 1, i, tzinfo=UTC), - ) - for i in range(3) - ] - ) - # asset 3 events - session.add_all( - [ - AssetEvent( - asset_id=asset3_id, - timestamp=pendulum.DateTime(2022, 8, 1, i, tzinfo=UTC), - ) - for i in range(3) - ] - ) - # asset 4 events - session.add_all( - [ - AssetEvent( - asset_id=asset4_id, - timestamp=pendulum.DateTime(2022, 8, 1, i, tzinfo=UTC), - ) - for i in range(4) - ] - ) - # asset 5 events - session.add_all( - [ - AssetEvent( - asset_id=asset5_id, - timestamp=pendulum.DateTime(2022, 8, 1, i, tzinfo=UTC), - ) - for i in range(5) - ] - ) - session.commit() - - response = admin_client.get("/object/assets_summary") - - assert response.status_code == 200 - response_data = response.json - assert response_data == { - "assets": [ - { - "id": asset1_id, - "uri": "s3://bucket/key/1", - "last_asset_update": "2022-08-01T02:00:00+00:00", - "total_updates": 3, - }, - { - "id": asset2_id, - "uri": "s3://bucket/key/2", - "last_asset_update": None, - "total_updates": 0, - }, - { - "id": asset3_id, - "uri": "s3://bucket/key/3", - "last_asset_update": "2022-08-01T02:00:00+00:00", - "total_updates": 3, - }, - { - "id": asset4_id, - "uri": "s3://bucket/key/4", - "last_asset_update": "2022-08-01T03:00:00+00:00", - "total_updates": 4, - }, - { - "id": asset5_id, - "uri": "s3://bucket/key/5", - "last_asset_update": "2022-08-01T04:00:00+00:00", - "total_updates": 5, - }, - ], - "total_entries": 5, - } - - -class TestGetAssetsEndpointPagination(TestAssetEndpoint): - @pytest.mark.parametrize( - "url, expected_asset_uris", - [ - # Limit test data - ("/object/assets_summary?limit=1", ["s3://bucket/key/1"]), - ("/object/assets_summary?limit=5", [f"s3://bucket/key/{i}" for i in range(1, 6)]), - # Offset test data - ("/object/assets_summary?offset=1", [f"s3://bucket/key/{i}" for i in range(2, 10)]), - ("/object/assets_summary?offset=3", [f"s3://bucket/key/{i}" for i in range(4, 10)]), - # Limit and offset test data - ("/object/assets_summary?offset=3&limit=3", [f"s3://bucket/key/{i}" for i in [4, 5, 6]]), - ], - ) - def test_limit_and_offset(self, admin_client, create_assets, session, url, expected_asset_uris): - create_assets(range(1, 10)) - session.commit() - - response = admin_client.get(url) - - assert response.status_code == 200 - asset_uris = [asset["uri"] for asset in response.json["assets"]] - assert asset_uris == expected_asset_uris - - def test_should_respect_page_size_limit_default(self, admin_client, create_assets, session): - create_assets(range(1, 60)) - session.commit() - - response = admin_client.get("/object/assets_summary") - - assert response.status_code == 200 - assert len(response.json["assets"]) == 25 - - def test_should_return_max_if_req_above(self, admin_client, create_assets, session): - create_assets(range(1, 60)) - session.commit() - - response = admin_client.get("/object/assets_summary?limit=180") - - assert response.status_code == 200 - assert len(response.json["assets"]) == 50 - - -class TestGetAssetNextRunSummary(TestAssetEndpoint): - def test_next_run_asset_summary(self, dag_maker, admin_client): - with dag_maker( - dag_id="upstream", schedule=[Asset(uri="s3://bucket/key/1", name="asset-1")], serialized=True - ): - EmptyOperator(task_id="task1") - - response = admin_client.post("/next_run_assets_summary", data={"dag_ids": ["upstream"]}) - - assert response.status_code == 200 - assert response.json == {"upstream": {"ready": 0, "total": 1, "uri": "s3://bucket/key/1"}} diff --git a/tests/www/views/test_views_base.py b/tests/www/views/test_views_base.py deleted file mode 100644 index 0c6316bf6058c..0000000000000 --- a/tests/www/views/test_views_base.py +++ /dev/null @@ -1,424 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import datetime -import json - -import pytest - -from airflow.jobs.job import Job -from airflow.jobs.scheduler_job_runner import SchedulerJobRunner -from airflow.utils import timezone -from airflow.utils.session import create_session -from airflow.www import app as application - -from tests_common.test_utils.asserts import assert_queries_count -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.www import check_content_in_response, check_content_not_in_response - -pytestmark = pytest.mark.db_test - - -def test_index_redirect(admin_client): - resp = admin_client.get("/") - assert resp.status_code == 302 - assert "/home" in resp.headers.get("Location") - - resp = admin_client.get("/", follow_redirects=True) - check_content_in_response("DAGs", resp) - - -def test_homepage_query_count(admin_client): - with assert_queries_count(20): - resp = admin_client.get("/home") - check_content_in_response("DAGs", resp) - - -def test_doc_urls(admin_client, monkeypatch): - # Mocking this way is tying the test closer to the implementation much more than I'd like. :shrug: - from airflow.www.views import AirflowBaseView - - monkeypatch.setitem(AirflowBaseView.extra_args, "get_docs_url", lambda _: "!!DOCS_URL!!") - resp = admin_client.get("/", follow_redirects=True) - - check_content_in_response("!!DOCS_URL!!", resp) - check_content_in_response("/api/v1/ui", resp) - - -@pytest.fixture -def heartbeat_healthy(): - # case-1: healthy scheduler status - last_heartbeat = timezone.utcnow() - job = Job( - state="running", - latest_heartbeat=last_heartbeat, - ) - SchedulerJobRunner(job=job) - with create_session() as session: - session.add(job) - yield "healthy", last_heartbeat.isoformat() - with create_session() as session: - session.query(Job).filter( - Job.job_type == "SchedulerJob", - Job.state == "running", - Job.latest_heartbeat == last_heartbeat, - ).delete() - - -@pytest.fixture -def heartbeat_too_slow(): - # case-2: unhealthy scheduler status - scenario 1 (SchedulerJob is running too slowly) - last_heartbeat = timezone.utcnow() - datetime.timedelta(minutes=1) - job = Job( - state="running", - latest_heartbeat=last_heartbeat, - ) - SchedulerJobRunner(job=job) - with create_session() as session: - session.query(Job).filter( - Job.job_type == "SchedulerJob", - ).update({"latest_heartbeat": last_heartbeat - datetime.timedelta(seconds=1)}) - session.add(job) - yield "unhealthy", last_heartbeat.isoformat() - with create_session() as session: - session.query(Job).filter( - Job.job_type == "SchedulerJob", - Job.state == "running", - Job.latest_heartbeat == last_heartbeat, - ).delete() - - -@pytest.fixture -def heartbeat_not_running(): - # case-3: unhealthy scheduler status - scenario 2 (no running SchedulerJob) - with create_session() as session: - session.query(Job).filter( - Job.job_type == "SchedulerJob", - Job.state == "running", - ).delete() - return "unhealthy", None - - -@pytest.mark.parametrize( - "heartbeat", - ["heartbeat_healthy", "heartbeat_too_slow", "heartbeat_not_running"], -) -def test_health(request, admin_client, heartbeat): - # Load the corresponding fixture by name. - scheduler_status, last_scheduler_heartbeat = request.getfixturevalue(heartbeat) - resp = admin_client.get("health", follow_redirects=True) - resp_json = json.loads(resp.data.decode("utf-8")) - assert resp_json["metadatabase"]["status"] == "healthy" - assert scheduler_status == resp_json["scheduler"]["status"] - assert last_scheduler_heartbeat == resp_json["scheduler"]["latest_scheduler_heartbeat"] - - -def test_users_list(admin_client): - resp = admin_client.get("users/list", follow_redirects=True) - check_content_in_response("List Users", resp) - - -@pytest.mark.parametrize( - "path, body_content", - [("roles/list", "List Roles"), ("roles/show/1", "Show Role")], -) -def test_roles_read(admin_client, path, body_content): - resp = admin_client.get(path, follow_redirects=True) - check_content_in_response(body_content, resp) - - -def test_roles_read_unauthorized(viewer_client): - resp = viewer_client.get("roles/list", follow_redirects=True) - check_content_in_response("Access is Denied", resp) - - -@pytest.fixture(scope="module") -def delete_role_if_exists(app): - def func(role_name): - if app.appbuilder.sm.find_role(role_name): - app.appbuilder.sm.delete_role(role_name) - - return func - - -@pytest.fixture -def non_exist_role_name(delete_role_if_exists): - role_name = "test_roles_create_role" - delete_role_if_exists(role_name) - yield role_name - delete_role_if_exists(role_name) - - -@pytest.fixture -def exist_role_name(app, delete_role_if_exists): - role_name = "test_roles_create_role_new" - app.appbuilder.sm.add_role(role_name) - yield role_name - delete_role_if_exists(role_name) - - -@pytest.fixture -def exist_role(app, exist_role_name): - return app.appbuilder.sm.find_role(exist_role_name) - - -def test_roles_create(app, admin_client, non_exist_role_name): - admin_client.post("roles/add", data={"name": non_exist_role_name}, follow_redirects=True) - assert app.appbuilder.sm.find_role(non_exist_role_name) is not None - - -def test_roles_create_unauthorized(app, viewer_client, non_exist_role_name): - resp = viewer_client.post("roles/add", data={"name": non_exist_role_name}, follow_redirects=True) - check_content_in_response("Access is Denied", resp) - assert app.appbuilder.sm.find_role(non_exist_role_name) is None - - -def test_roles_edit(app, admin_client, non_exist_role_name, exist_role): - admin_client.post( - f"roles/edit/{exist_role.id}", data={"name": non_exist_role_name}, follow_redirects=True - ) - updated_role = app.appbuilder.sm.find_role(non_exist_role_name) - assert exist_role.id == updated_role.id - - -def test_roles_edit_unauthorized(app, viewer_client, non_exist_role_name, exist_role_name, exist_role): - resp = viewer_client.post( - f"roles/edit/{exist_role.id}", data={"name": non_exist_role_name}, follow_redirects=True - ) - check_content_in_response("Access is Denied", resp) - assert app.appbuilder.sm.find_role(exist_role_name) - assert app.appbuilder.sm.find_role(non_exist_role_name) is None - - -def test_roles_delete(app, admin_client, exist_role_name, exist_role): - admin_client.post(f"roles/delete/{exist_role.id}", follow_redirects=True) - assert app.appbuilder.sm.find_role(exist_role_name) is None - - -def test_roles_delete_unauthorized(app, viewer_client, exist_role, exist_role_name): - resp = viewer_client.post(f"roles/delete/{exist_role.id}", follow_redirects=True) - check_content_in_response("Access is Denied", resp) - assert app.appbuilder.sm.find_role(exist_role_name) - - -@pytest.mark.parametrize( - "url, client, content", - [ - ("userstatschartview/chart/", "admin_client", "User Statistics"), - ("userstatschartview/chart/", "viewer_client", "Access is Denied"), - ("actions/list", "admin_client", "List Actions"), - ("actions/list", "viewer_client", "Access is Denied"), - ("resources/list/", "admin_client", "List Resources"), - ("resources/list/", "viewer_client", "Access is Denied"), - ("permissions/list/", "admin_client", "List Permissions"), - ("permissions/list/", "viewer_client", "Access is Denied"), - ("resetpassword/form?pk=1", "admin_client", "Reset Password Form"), - ("resetpassword/form?pk=1", "viewer_client", "Access is Denied"), - ("users/list", "admin_client", "List Users"), - ("users/list", "viewer_client", "Access is Denied"), - ], - ids=[ - "userstatschertview-admin", - "userstatschertview-viewer", - "actions-admin", - "actions-viewer", - "resources-admin", - "resources-viewer", - "permissions-admin", - "permissions-viewer", - "resetpassword-admin", - "resetpassword-viewer", - "users-admin", - "users-viewer", - ], -) -def test_views_get(request, url, client, content): - resp = request.getfixturevalue(client).get(url, follow_redirects=True) - check_content_in_response(content, resp) - - -def _check_task_stats_json(resp): - return set(next(iter(resp.json.items()))[1][0]) == {"state", "count"} - - -@pytest.mark.parametrize( - "url, check_response", - [ - ("blocked", None), - ("dag_stats", None), - ("task_stats", _check_task_stats_json), - ], -) -def test_views_post(admin_client, url, check_response): - resp = admin_client.post(url, follow_redirects=True) - assert resp.status_code == 200 - if check_response: - assert check_response(resp) - - -@pytest.mark.parametrize( - "url, client, content, username", - [ - ("resetmypassword/form", "viewer_client", "Password Changed", "test_viewer"), - ("resetpassword/form?pk={}", "admin_client", "Password Changed", "test_admin"), - ("resetpassword/form?pk={}", "viewer_client", "Access is Denied", "test_viewer"), - ], - ids=["my-viewer", "pk-admin", "pk-viewer"], -) -def test_resetmypasswordview_edit(app, request, url, client, content, username): - user = app.appbuilder.sm.find_user(username) - resp = request.getfixturevalue(client).post( - url.format(user.id), data={"password": "blah", "conf_password": "blah"}, follow_redirects=True - ) - check_content_in_response(content, resp) - - -def test_resetmypasswordview_read(viewer_client): - # Tests with viewer as all roles should have access. - resp = viewer_client.get("resetmypassword/form", follow_redirects=True) - check_content_in_response("Reset Password Form", resp) - - -def test_get_myuserinfo(admin_client): - resp = admin_client.get("users/userinfo/", follow_redirects=True) - check_content_in_response("Your user information", resp) - - -def test_edit_myuserinfo(admin_client): - resp = admin_client.post( - "userinfoeditview/form", - data={"first_name": "new_first_name", "last_name": "new_last_name"}, - follow_redirects=True, - ) - check_content_in_response("User information changed", resp) - - -@pytest.mark.parametrize( - "url", - ["users/add", "users/edit/1", "users/delete/1"], - ids=["add-user", "edit-user", "delete-user"], -) -def test_views_post_access_denied(viewer_client, url): - resp = viewer_client.get(url, follow_redirects=True) - check_content_in_response("Access is Denied", resp) - - -@pytest.fixture -def non_exist_username(app): - username = "fake_username" - user = app.appbuilder.sm.find_user(username) - if user is not None: - app.appbuilder.sm.del_register_user(user) - yield username - user = app.appbuilder.sm.find_user(username) - if user is not None: - app.appbuilder.sm.del_register_user(user) - - -def test_create_user(app, admin_client, non_exist_username): - resp = admin_client.post( - "users/add", - data={ - "first_name": "fake_first_name", - "last_name": "fake_last_name", - "username": non_exist_username, - "email": "fake_email@email.com", - "roles": [1], - "password": "test", - "conf_password": "test", - }, - follow_redirects=True, - ) - check_content_in_response("Added Row", resp) - assert app.appbuilder.sm.find_user(non_exist_username) - - -@pytest.fixture -def exist_username(app, exist_role): - username = "test_edit_user_user" - app.appbuilder.sm.add_user( - username, - "first_name", - "last_name", - "email@email.com", - exist_role, - password="password", - ) - yield username - if app.appbuilder.sm.find_user(username): - app.appbuilder.sm.del_register_user(username) - - -def test_edit_user(app, admin_client, exist_username): - user = app.appbuilder.sm.find_user(exist_username) - resp = admin_client.post( - f"users/edit/{user.id}", - data={"first_name": "new_first_name"}, - follow_redirects=True, - ) - check_content_in_response("new_first_name", resp) - - -def test_delete_user(app, admin_client, exist_username): - user = app.appbuilder.sm.find_user(exist_username) - resp = admin_client.post( - f"users/delete/{user.id}", - follow_redirects=True, - ) - check_content_in_response("Deleted Row", resp) - - -@conf_vars({("webserver", "show_recent_stats_for_completed_runs"): "False"}) -def test_task_stats_only_noncompleted(admin_client): - resp = admin_client.post("task_stats", follow_redirects=True) - assert resp.status_code == 200 - - -@conf_vars({("webserver", "instance_name"): "Site Title Test"}) -def test_page_instance_name(admin_client): - resp = admin_client.get("home", follow_redirects=True) - check_content_in_response("Site Title Test", resp) - - -def test_page_instance_name_xss_prevention(admin_client): - xss_string = "" - with conf_vars({("webserver", "instance_name"): xss_string}): - resp = admin_client.get("home", follow_redirects=True) - escaped_xss_string = "<script>alert('Give me your credit card number')</script>" - check_content_in_response(escaped_xss_string, resp) - check_content_not_in_response(xss_string, resp) - - -instance_name_with_markup_conf = { - ("webserver", "instance_name"): "Bold Site Title Test", - ("webserver", "instance_name_has_markup"): "True", -} - - -@conf_vars(instance_name_with_markup_conf) -def test_page_instance_name_with_markup(admin_client): - resp = admin_client.get("home", follow_redirects=True) - check_content_in_response("Bold Site Title Test", resp) - check_content_not_in_response("<b>Bold Site Title Test</b>", resp) - - -@conf_vars(instance_name_with_markup_conf) -def test_page_instance_name_with_markup_title(): - appbuilder = application.create_app(testing=True).appbuilder - assert appbuilder.app_name == "Bold Site Title Test" diff --git a/tests/www/views/test_views_cluster_activity.py b/tests/www/views/test_views_cluster_activity.py deleted file mode 100644 index 084f73108364d..0000000000000 --- a/tests/www/views/test_views_cluster_activity.py +++ /dev/null @@ -1,156 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from datetime import timedelta - -import pendulum -import pytest - -from airflow.models import DagBag -from airflow.providers.standard.operators.empty import EmptyOperator -from airflow.utils.state import DagRunState, TaskInstanceState -from airflow.utils.types import DagRunType - -from tests_common.test_utils.db import clear_db_runs - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(autouse=True, scope="module") -def examples_dag_bag(): - # Speed up: We don't want example dags for this module - - return DagBag(include_examples=False, read_dags_from_db=True) - - -@pytest.fixture(autouse=True) -def _clean(): - clear_db_runs() - yield - clear_db_runs() - - -# freeze time fixture so that it is applied before `_make_dag_runs` is! -@pytest.fixture -def _freeze_time_for_dagruns(time_machine): - time_machine.move_to("2023-05-02T00:00:00+00:00", tick=False) - - -@pytest.fixture -def _make_dag_runs(dag_maker, session, time_machine): - with dag_maker( - dag_id="test_dag_id", - serialized=True, - session=session, - start_date=pendulum.DateTime(2023, 2, 1, 0, 0, 0, tzinfo=pendulum.UTC), - ): - EmptyOperator(task_id="task_1") >> EmptyOperator(task_id="task_2") - - date = dag_maker.dag.start_date - - run1 = dag_maker.create_dagrun( - run_id="run_1", - state=DagRunState.SUCCESS, - run_type=DagRunType.SCHEDULED, - logical_date=date, - start_date=date, - ) - - run2 = dag_maker.create_dagrun( - run_id="run_2", - state=DagRunState.FAILED, - run_type=DagRunType.ASSET_TRIGGERED, - logical_date=date + timedelta(days=1), - start_date=date + timedelta(days=1), - ) - - run3 = dag_maker.create_dagrun( - run_id="run_3", - state=DagRunState.RUNNING, - run_type=DagRunType.SCHEDULED, - logical_date=pendulum.DateTime(2023, 2, 3, 0, 0, 0, tzinfo=pendulum.UTC), - start_date=pendulum.DateTime(2023, 2, 3, 0, 0, 0, tzinfo=pendulum.UTC), - ) - run3.end_date = None - - for ti in run1.task_instances: - ti.state = TaskInstanceState.SUCCESS - - for ti in run2.task_instances: - ti.state = TaskInstanceState.FAILED - - time_machine.move_to("2023-07-02T00:00:00+00:00", tick=False) - - session.flush() - - -@pytest.mark.usefixtures("_freeze_time_for_dagruns", "_make_dag_runs") -def test_historical_metrics_data(admin_client, session, time_machine): - resp = admin_client.get( - "/object/historical_metrics_data?start_date=2023-01-01T00:00&end_date=2023-08-02T00:00", - follow_redirects=True, - ) - assert resp.status_code == 200 - assert resp.json == { - "dag_run_states": {"failed": 1, "queued": 0, "running": 1, "success": 1}, - "dag_run_types": {"backfill": 0, "asset_triggered": 1, "manual": 0, "scheduled": 2}, - "task_instance_states": { - "deferred": 0, - "failed": 2, - "no_status": 2, - "queued": 0, - "removed": 0, - "restarting": 0, - "running": 0, - "scheduled": 0, - "skipped": 0, - "success": 2, - "up_for_reschedule": 0, - "up_for_retry": 0, - "upstream_failed": 0, - }, - } - - -@pytest.mark.usefixtures("_freeze_time_for_dagruns", "_make_dag_runs") -def test_historical_metrics_data_date_filters(admin_client, session): - resp = admin_client.get( - "/object/historical_metrics_data?start_date=2023-02-02T00:00&end_date=2023-06-02T00:00", - follow_redirects=True, - ) - assert resp.status_code == 200 - assert resp.json == { - "dag_run_states": {"failed": 1, "queued": 0, "running": 0, "success": 0}, - "dag_run_types": {"backfill": 0, "asset_triggered": 1, "manual": 0, "scheduled": 0}, - "task_instance_states": { - "deferred": 0, - "failed": 2, - "no_status": 0, - "queued": 0, - "removed": 0, - "restarting": 0, - "running": 0, - "scheduled": 0, - "skipped": 0, - "success": 0, - "up_for_reschedule": 0, - "up_for_retry": 0, - "upstream_failed": 0, - }, - } diff --git a/tests/www/views/test_views_configuration.py b/tests/www/views/test_views_configuration.py deleted file mode 100644 index 20d9c7bf7ce2b..0000000000000 --- a/tests/www/views/test_views_configuration.py +++ /dev/null @@ -1,76 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import html - -import pytest - -from airflow.configuration import conf - -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.www import check_content_in_response, check_content_not_in_response - -pytestmark = pytest.mark.db_test - - -@conf_vars({("webserver", "expose_config"): "False"}) -def test_user_cant_view_configuration(admin_client): - resp = admin_client.get("configuration", follow_redirects=True) - check_content_in_response( - "Your Airflow administrator chose not to expose the configuration, " - "most likely for security reasons.", - resp, - ) - - -@conf_vars({("webserver", "expose_config"): "True"}) -def test_user_can_view_configuration(admin_client): - resp = admin_client.get("configuration", follow_redirects=True) - for section, key in conf.sensitive_config_values: - value = conf.get(section, key, fallback="") - if value: - check_content_in_response(html.escape(value), resp) - - -@conf_vars({("webserver", "expose_config"): "non-sensitive-only"}) -def test_configuration_redacted(admin_client): - resp = admin_client.get("configuration", follow_redirects=True) - for section, key in conf.sensitive_config_values: - value = conf.get(section, key, fallback="") - if value and value != "airflow" and not value.startswith("db+postgresql"): - check_content_not_in_response(value, resp) - - -@conf_vars({("webserver", "expose_config"): "non-sensitive-only"}) -def test_configuration_redacted_in_running_configuration(admin_client): - resp = admin_client.get("configuration", follow_redirects=True) - for section, key in conf.sensitive_config_values: - value = conf.get(section, key, fallback="") - if value and value != "airflow": - check_content_not_in_response("" + html.escape(value) + " dict[str, Any]: - return {**CONNECTION, "extra": '{"x_secret": "testsecret","y_secret": "test"}'} - - -@pytest.fixture(autouse=True) -def _clear_connections(): - with create_session() as session: - session.query(Connection).delete() - - -@pytest.mark.execution_timeout(150) -def test_create_connection(admin_client, session): - resp = admin_client.post("/connection/add", data=CONNECTION, follow_redirects=True) - check_content_in_response("Added Row", resp) - _check_last_log(session, dag_id=None, event="connection.create", logical_date=None) - - -def test_connection_id_trailing_blanks(admin_client, session): - conn_id_with_blanks = "conn_id_with_trailing_blanks " - conn = {**CONNECTION, "conn_id": conn_id_with_blanks} - resp = admin_client.post("/connection/add", data=conn, follow_redirects=True) - check_content_in_response("Added Row", resp) - - conn = session.query(Connection).one() - assert conn.conn_id == "conn_id_with_trailing_blanks" - - -def test_connection_id_leading_blanks(admin_client, session): - conn_id_with_blanks = " conn_id_with_leading_blanks" - conn = {**CONNECTION, "conn_id": conn_id_with_blanks} - resp = admin_client.post("/connection/add", data=conn, follow_redirects=True) - check_content_in_response("Added Row", resp) - - conn = session.query(Connection).one() - assert conn.conn_id == "conn_id_with_leading_blanks" - - -def test_all_fields_with_blanks(admin_client, session): - connection = { - **CONNECTION, - "conn_id": " connection_id_with_space", - "description": " a sample http connection with leading and trailing blanks ", - "host": "localhost ", - "schema": " airflow ", - "port": 3306, - } - - resp = admin_client.post("/connection/add", data=connection, follow_redirects=True) - check_content_in_response("Added Row", resp) - - # validate all the fields - conn = session.query(Connection).one() - assert conn.conn_id == "connection_id_with_space" - assert conn.description == "a sample http connection with leading and trailing blanks" - assert conn.host == "localhost" - assert conn.schema == "airflow" - - -@pytest.mark.enable_redact -def test_action_logging_connection_masked_secrets(session, admin_client): - admin_client.post("/connection/add", data=conn_with_extra(), follow_redirects=True) - _check_last_log_masked_connection(session, dag_id=None, event="connection.create", logical_date=None) - - -def test_prefill_form_null_extra(): - mock_form = mock.Mock() - mock_form.data = {"conn_id": "test", "extra": None, "conn_type": "test"} - - cmv = ConnectionModelView() - cmv._iter_extra_field_names_and_sensitivity = mock.Mock(return_value=()) - cmv.prefill_form(form=mock_form, pk=1) - - -def test_prefill_form_sensitive_fields_extra(): - mock_form = mock.Mock() - mock_form.data = { - "conn_id": "test", - "extra": json.dumps({"sensitive_extra": "TEST1", "non_sensitive_extra": "TEST2"}), - "conn_type": "test", - } - - cmv = ConnectionModelView() - cmv._iter_extra_field_names_and_sensitivity = mock.Mock( - return_value=[("sensitive_extra_key", "sensitive_extra", True)] - ) - cmv.prefill_form(form=mock_form, pk=1) - assert json.loads(mock_form.extra.data) == { - "sensitive_extra": "RATHER_LONG_SENSITIVE_FIELD_PLACEHOLDER", - "non_sensitive_extra": "TEST2", - } - - -@pytest.mark.parametrize( - "extras, expected", - [ - pytest.param({"extra__test__my_param": "this_val"}, "this_val", id="conn_not_upgraded"), - pytest.param({"my_param": "my_val"}, "my_val", id="conn_upgraded"), - pytest.param( - {"extra__test__my_param": "this_val", "my_param": "my_val"}, - "my_val", - id="conn_upgraded_old_val_present", - ), - ], -) -def test_prefill_form_backcompat(extras, expected): - """ - When populating custom fields in the connection form we should first check for the non-prefixed - value (since prefixes in extra are deprecated) and then fallback to the prefixed value. - - Either way, the field is known internally to the model view as the prefixed value. - """ - mock_form = mock.Mock() - mock_form.data = {"conn_id": "test", "extra": json.dumps(extras), "conn_type": "test"} - - cmv = ConnectionModelView() - cmv._iter_extra_field_names_and_sensitivity = mock.Mock( - return_value=[("extra__test__my_param", "my_param", False)] - ) - cmv.prefill_form(form=mock_form, pk=1) - assert mock_form.extra__test__my_param.data == expected - - -@pytest.mark.parametrize("field_name", ["extra__test__custom_field", "custom_field"]) -@mock.patch("airflow.utils.module_loading.import_string") -@mock.patch("airflow.providers_manager.ProvidersManager.hooks", new_callable=PropertyMock) -def test_process_form_extras_both(mock_pm_hooks, mock_import_str, field_name): - """ - Test the handling of connection parameters set with the classic `Extra` field as well as custom fields. - The key used in the field definition returned by `get_connection_form_widgets` is stored in - attr `extra_field_name_mapping`. Whatever is defined there is what should end up in `extra` when - the form is processed. - """ - mock_pm_hooks.get.return_value = True # ensure that hook appears registered - - # Testing parameters set in both `Extra` and custom fields. - mock_form = mock.Mock() - mock_form.data = { - "conn_type": "test", - "conn_id": "extras_test", - "extra": '{"param1": "param1_val"}', - "extra__test__custom_field": "custom_field_val", - "extra__other_conn_type__custom_field": "another_field_val", - } - - cmv = ConnectionModelView() - cmv._iter_extra_field_names_and_sensitivity = mock.Mock( - return_value=[("extra__test__custom_field", field_name, False)] - ) - cmv.process_form(form=mock_form, is_created=True) - assert json.loads(mock_form.extra.data) == { - field_name: "custom_field_val", - "param1": "param1_val", - } - - -@mock.patch("airflow.utils.module_loading.import_string") -@mock.patch("airflow.providers_manager.ProvidersManager.hooks", new_callable=PropertyMock) -def test_process_form_extras_extra_only(mock_pm_hooks, mock_import_str): - """ - Test the handling of connection parameters set with the classic `Extra` field as well as custom fields. - The key used in the field definition returned by `get_connection_form_widgets` is stored in - attr `extra_field_name_mapping`. Whatever is defined there is what should end up in `extra` when - the form is processed. - """ - # Testing parameters set in `Extra` field only. - mock_form = mock.Mock() - mock_form.data = { - "conn_type": "test2", - "conn_id": "extras_test2", - "extra": '{"param2": "param2_val"}', - } - - cmv = ConnectionModelView() - cmv._iter_extra_field_names_and_sensitivity = mock.Mock(return_value=()) - cmv.process_form(form=mock_form, is_created=True) - assert json.loads(mock_form.extra.data) == {"param2": "param2_val"} - - -@pytest.mark.parametrize("field_name", ["extra__test3__custom_field", "custom_field"]) -@mock.patch("airflow.utils.module_loading.import_string") -@mock.patch("airflow.providers_manager.ProvidersManager.hooks", new_callable=PropertyMock) -def test_process_form_extras_custom_only(mock_pm_hooks, mock_import_str, field_name): - """ - Test the handling of connection parameters set with the classic `Extra` field as well as custom fields. - The key used in the field definition returned by `get_connection_form_widgets` is stored in - attr `extra_field_name_mapping`. Whatever is defined there is what should end up in `extra` when - the form is processed. - """ - - # Testing parameters set in custom fields only. - mock_form = mock.Mock() - mock_form.data = { - "conn_type": "test3", - "conn_id": "extras_test3", - "extra__test3__custom_field": False, - "extra__other_conn_type__custom_field": "another_field_val", - } - - cmv = ConnectionModelView() - cmv._iter_extra_field_names_and_sensitivity = mock.Mock( - return_value=[ - ("extra__test3__custom_field", field_name, False), - ("extra__test3__custom_bool_field", False, False), - ], - ) - cmv.process_form(form=mock_form, is_created=True) - assert json.loads(mock_form.extra.data) == {field_name: False} - - -@pytest.mark.parametrize("field_name", ["extra__test4__custom_field", "custom_field"]) -@mock.patch("airflow.utils.module_loading.import_string") -@mock.patch("airflow.providers_manager.ProvidersManager.hooks", new_callable=PropertyMock) -def test_process_form_extras_updates(mock_pm_hooks, mock_import_str, field_name): - """ - Test the handling of connection parameters set with the classic `Extra` field as well as custom fields. - The key used in the field definition returned by `get_connection_form_widgets` is stored in - attr `extra_field_name_mapping`. Whatever is defined there is what should end up in `extra` when - the form is processed. - """ - - # Testing parameters set in both extra and custom fields (connection updates). - mock_form = mock.Mock() - mock_form.data = { - "conn_type": "test4", - "conn_id": "extras_test4", - "extra": '{"extra__test4__custom_field": "custom_field_val3"}', - "extra__test4__custom_field": "custom_field_val4", - } - - cmv = ConnectionModelView() - cmv._iter_extra_field_names_and_sensitivity = mock.Mock( - return_value=[("extra__test4__custom_field", field_name, False)] - ) - cmv.process_form(form=mock_form, is_created=True) - - if field_name == "custom_field": - assert json.loads(mock_form.extra.data) == { - "custom_field": "custom_field_val4", - "extra__test4__custom_field": "custom_field_val3", - } - else: - assert json.loads(mock_form.extra.data) == {"extra__test4__custom_field": "custom_field_val4"} - - -@mock.patch("airflow.utils.module_loading.import_string") -@mock.patch("airflow.providers_manager.ProvidersManager.hooks", new_callable=PropertyMock) -@mock.patch("airflow.www.views.BaseHook") -def test_process_form_extras_updates_sensitive_placeholder_unchanged( - mock_base_hook, mock_pm_hooks, mock_import_str -): - """ - Test the handling of sensitive unchanged field (where placeholder has not been modified). - """ - - # Testing parameters set in both extra and custom fields (connection updates). - mock_form = mock.Mock() - mock_form.data = { - "conn_type": "test4", - "conn_id": "extras_test4", - "extra": '{"sensitive_extra": "RATHER_LONG_SENSITIVE_FIELD_PLACEHOLDER", "extra__custom": "value"}', - } - mock_base_hook.get_connection.return_value = Connection(extra='{"sensitive_extra": "old_value"}') - cmv = ConnectionModelView() - cmv._iter_extra_field_names_and_sensitivity = mock.Mock( - return_value=[("sensitive_extra_key", "sensitive_extra", True)] - ) - cmv.process_form(form=mock_form, is_created=True) - - assert json.loads(mock_form.extra.data) == { - "extra__custom": "value", - "sensitive_extra": "old_value", - } - - -@mock.patch("airflow.utils.module_loading.import_string") -@mock.patch("airflow.providers_manager.ProvidersManager.hooks", new_callable=PropertyMock) -def test_process_form_extras_remove(mock_pm_hooks, mock_import_str): - """ - Test the remove value from field. - """ - # Testing parameters set in both extra and custom fields (connection updates). - mock_form = mock.Mock() - mock_form.data = { - "conn_type": "test4", - "conn_id": "extras_test4", - "extra": '{"extra__test4__remove_field": "remove_field_val3"}', - "extra__test4__remove_field": "", - } - - cmv = ConnectionModelView() - cmv._iter_extra_field_names_and_sensitivity = mock.Mock( - return_value=[("extra__test4__remove_field", "remove_field", False)] - ) - cmv.process_form(form=mock_form, is_created=True) - - assert json.loads(mock_form.extra.data) == { - "extra__test4__remove_field": "remove_field_val3", - } - - -def test_duplicate_connection(admin_client): - """Test Duplicate multiple connection with suffix""" - conn1 = Connection( - conn_id="test_duplicate_gcp_connection", - conn_type="Google Cloud", - description="Google Cloud Connection", - ) - conn2 = Connection( - conn_id="test_duplicate_mysql_connection", - conn_type="FTP", - description="MongoDB2", - host="localhost", - schema="airflow", - port=3306, - ) - conn3 = Connection( - conn_id="test_duplicate_postgres_connection_copy1", - conn_type="FTP", - description="Postgres", - host="localhost", - schema="airflow", - port=3306, - ) - with create_session() as session: - session.query(Connection).delete() - session.add_all([conn1, conn2, conn3]) - session.commit() - - data = {"action": "mulduplicate", "rowid": [conn1.id, conn3.id]} - resp = admin_client.post("/connection/action_post", data=data, follow_redirects=True) - assert resp.status_code == 200 - - expected_connections_ids = { - "test_duplicate_gcp_connection", - "test_duplicate_gcp_connection_copy1", - "test_duplicate_mysql_connection", - "test_duplicate_postgres_connection_copy1", - "test_duplicate_postgres_connection_copy2", - } - connections_ids = {conn.conn_id for conn in session.query(Connection.conn_id)} - assert expected_connections_ids == connections_ids - - -def test_duplicate_connection_error(admin_client): - """Test Duplicate multiple connection with suffix - when there are already 10 copies, no new copy - should be created""" - - connection_ids = [f"test_duplicate_postgres_connection_copy{i}" for i in range(1, 11)] - connections = [ - Connection( - conn_id=connection_id, - conn_type="FTP", - description="Postgres", - host="localhost", - schema="airflow", - port=3306, - ) - for connection_id in connection_ids - ] - - with create_session() as session: - session.query(Connection).delete() - session.add_all(connections) - - data = {"action": "mulduplicate", "rowid": [connections[0].id]} - resp = admin_client.post("/connection/action_post", data=data, follow_redirects=True) - assert resp.status_code == 200 - - expected_connections_ids = {f"test_duplicate_postgres_connection_copy{i}" for i in range(1, 11)} - connections_ids = {conn.conn_id for conn in session.query(Connection.conn_id)} - assert expected_connections_ids == connections_ids - - -@pytest.fixture -def connection(): - connection = Connection( - conn_id="conn1", - conn_type="Conn 1", - description="Conn 1 description", - ) - with create_session() as session: - session.add(connection) - yield connection - with create_session() as session: - session.query(Connection).filter(Connection.conn_id == CONNECTION["conn_id"]).delete() - - -def test_connection_muldelete(admin_client, connection): - conn_id = connection.id - data = {"action": "muldelete", "rowid": [conn_id]} - resp = admin_client.post("/connection/action_post", data=data, follow_redirects=True) - assert resp.status_code == 200 - with create_session() as session: - assert session.query(Connection).filter(Connection.id == conn_id).count() == 0 - - -@mock.patch("airflow.providers_manager.ProvidersManager.hooks", new_callable=PropertyMock) -def test_connection_form_widgets_testable_types(mock_pm_hooks, admin_client): - mock_pm_hooks.return_value = { - "first": mock.MagicMock(connection_testable=True), - "second": mock.MagicMock(connection_testable=False), - "third": None, - } - - assert ConnectionFormWidget().testable_connection_types == ["first"] - - -def test_process_form_invalid_extra_removed(admin_client): - """ - Test that when an invalid json `extra` is passed in the form, it is removed and _not_ - saved over the existing extras. - """ - conn_details = {"conn_id": "test_conn", "conn_type": "http"} - conn = Connection(**conn_details, extra='{"foo": "bar"}') - conn.id = 1 - - with create_session() as session: - session.add(conn) - - data = {**conn_details, "extra": "Invalid"} - resp = admin_client.post("/connection/edit/1", data=data, follow_redirects=True) - - assert resp.status_code == 200 - with create_session() as session: - conn = session.get(Connection, 1) - - assert conn.extra == '{"foo": "bar"}' diff --git a/tests/www/views/test_views_dagrun.py b/tests/www/views/test_views_dagrun.py deleted file mode 100644 index dfd46340a7a92..0000000000000 --- a/tests/www/views/test_views_dagrun.py +++ /dev/null @@ -1,272 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from airflow.models import DagBag, DagRun, TaskInstance -from airflow.utils import timezone -from airflow.utils.session import create_session -from airflow.utils.types import DagRunTriggeredByType, DagRunType - -from tests_common.test_utils.www import ( - check_content_in_response, - check_content_not_in_response, -) - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module", autouse=True) -def _init_blank_dagrun(): - """Make sure there are no runs before we test anything. - - This really shouldn't be needed, but tests elsewhere leave the db dirty. - """ - with create_session() as session: - session.query(DagRun).delete() - session.query(TaskInstance).delete() - - -@pytest.fixture(autouse=True) -def _reset_dagrun(): - yield - with create_session() as session: - session.query(DagRun).delete() - session.query(TaskInstance).delete() - - -@pytest.fixture -def running_dag_run(session): - dag = DagBag().get_dag("example_bash_operator") - logical_date = timezone.datetime(2016, 1, 9) - dr = dag.create_dagrun( - state="running", - logical_date=logical_date, - data_interval=(logical_date, logical_date), - run_id="test_dag_runs_action", - run_type=DagRunType.MANUAL, - session=session, - run_after=logical_date, - triggered_by=DagRunTriggeredByType.TEST, - ) - session.add(dr) - tis = [ - TaskInstance(dag.get_task("runme_0"), run_id=dr.run_id, state="success"), - TaskInstance(dag.get_task("runme_1"), run_id=dr.run_id, state="failed"), - ] - session.bulk_save_objects(tis) - session.commit() - return dr - - -@pytest.fixture -def completed_dag_run_with_missing_task(session): - dag = DagBag().get_dag("example_bash_operator") - logical_date = timezone.datetime(2016, 1, 9) - dr = dag.create_dagrun( - state="success", - logical_date=logical_date, - data_interval=(logical_date, logical_date), - run_id="test_dag_runs_action", - run_type=DagRunType.MANUAL, - session=session, - run_after=logical_date, - triggered_by=DagRunTriggeredByType.TEST, - ) - session.add(dr) - tis = [ - TaskInstance(dag.get_task("runme_0"), run_id=dr.run_id, state="success"), - TaskInstance(dag.get_task("runme_1"), run_id=dr.run_id, state="success"), - TaskInstance(dag.get_task("also_run_this"), run_id=dr.run_id, state="success"), - TaskInstance(dag.get_task("run_after_loop"), run_id=dr.run_id, state="success"), - TaskInstance(dag.get_task("this_will_skip"), run_id=dr.run_id, state="success"), - TaskInstance(dag.get_task("run_this_last"), run_id=dr.run_id, state="success"), - ] - session.bulk_save_objects(tis) - session.commit() - return dag, dr - - -@pytest.mark.parametrize( - "action, expected_ti_states, expected_message", - [ - ( - "clear", - {None}, - "1 dag runs and 2 task instances were cleared", - ), - ( - "set_success", - {"success"}, - "1 dag runs and 1 task instances were set to success", - ), - ( - "set_failed", - {"success", "failed"}, # The success ti is not set to failed. - "1 dag runs and 0 task instances were set to failed", - ), - ( - "set_running", - {"success", "failed"}, # Unchanged. - "1 dag runs were set to running", - ), - ( - "set_queued", - {"success", "failed"}, - "1 dag runs were set to queued", - ), - ], - ids=["clear", "success", "failed", "running", "queued"], -) -def test_set_dag_runs_action( - session, - admin_client, - running_dag_run, - action, - expected_ti_states, - expected_message, -): - resp = admin_client.post( - "/dagrun/action_post", - data={"action": action, "rowid": [running_dag_run.id]}, - follow_redirects=True, - ) - check_content_in_response(expected_message, resp) - assert {ti.state for ti in session.query(TaskInstance).all()} == expected_ti_states - - -@pytest.mark.parametrize( - "action, expected_message", - [ - ("clear", "Failed to clear state"), - ("set_success", "Failed to set state"), - ("set_failed", "Failed to set state"), - ("set_running", "Failed to set state"), - ("set_queued", "Failed to set state"), - ], - ids=["clear", "success", "failed", "running", "queued"], -) -def test_set_dag_runs_action_fails(admin_client, action, expected_message): - resp = admin_client.post( - "/dagrun/action_post", - data={"action": action, "rowid": ["0"]}, - follow_redirects=True, - ) - check_content_in_response(expected_message, resp) - - -def test_muldelete_dag_runs_action(session, admin_client, running_dag_run): - dag_run_id = running_dag_run.id - resp = admin_client.post( - "/dagrun/action_post", - data={"action": "muldelete", "rowid": [dag_run_id]}, - follow_redirects=True, - ) - assert resp.status_code == 200 - assert session.query(TaskInstance).count() == 0 # associated TIs are deleted - assert session.query(DagRun).filter(DagRun.id == dag_run_id).count() == 0 - - -def test_dag_runs_queue_new_tasks_action(session, admin_client, completed_dag_run_with_missing_task): - dag, dag_run = completed_dag_run_with_missing_task - resp = admin_client.post( - "/dagrun_queued", - data={"dag_id": dag.dag_id, "dag_run_id": dag_run.run_id, "confirmed": False}, - ) - - check_content_in_response("runme_2", resp) - check_content_not_in_response("runme_1", resp) - assert resp.status_code == 200 - - -@pytest.fixture -def dag_run_with_all_done_task(session): - """Creates a DAG run for example_bash_decorator with tasks in various states and an ALL_DONE task not yet run.""" - dag = DagBag().get_dag("example_bash_decorator") - - # Re-sync the DAG to the DB - dag.sync_to_db() - - logical_date = timezone.datetime(2016, 1, 9) - dr = dag.create_dagrun( - state="running", - logical_date=logical_date, - data_interval=(logical_date, logical_date), - run_id="test_dagrun_failed", - run_type=DagRunType.MANUAL, - session=session, - run_after=logical_date, - triggered_by=DagRunTriggeredByType.TEST, - ) - - # Create task instances in various states to test the ALL_DONE trigger rule - tis = [ - # runme_loop tasks - TaskInstance(dag.get_task("runme_0"), run_id=dr.run_id, state="success"), - TaskInstance(dag.get_task("runme_1"), run_id=dr.run_id, state="failed"), - TaskInstance(dag.get_task("runme_2"), run_id=dr.run_id, state="running"), - # Other tasks before run_this_last - TaskInstance(dag.get_task("run_after_loop"), run_id=dr.run_id, state="success"), - TaskInstance(dag.get_task("also_run_this"), run_id=dr.run_id, state="success"), - TaskInstance(dag.get_task("also_run_this_again"), run_id=dr.run_id, state="skipped"), - TaskInstance(dag.get_task("this_will_skip"), run_id=dr.run_id, state="running"), - # The task with trigger_rule=ALL_DONE - TaskInstance(dag.get_task("run_this_last"), run_id=dr.run_id, state=None), - ] - session.bulk_save_objects(tis) - session.commit() - - return dag, dr - - -def test_dagrun_failed(session, admin_client, dag_run_with_all_done_task): - """Test marking a dag run as failed with a task having trigger_rule='all_done'""" - dag, dr = dag_run_with_all_done_task - - # Verify task instances were created - task_instances = ( - session.query(TaskInstance) - .filter(TaskInstance.dag_id == dr.dag_id, TaskInstance.run_id == dr.run_id) - .all() - ) - assert len(task_instances) > 0 - - resp = admin_client.post( - "/dagrun_failed", - data={"dag_id": dr.dag_id, "dag_run_id": dr.run_id, "confirmed": "true"}, - follow_redirects=True, - ) - - assert resp.status_code == 200 - - with create_session() as session: - updated_dr = ( - session.query(DagRun).filter(DagRun.dag_id == dr.dag_id, DagRun.run_id == dr.run_id).first() - ) - assert updated_dr.state == "failed" - - task_instances = ( - session.query(TaskInstance) - .filter(TaskInstance.dag_id == dr.dag_id, TaskInstance.run_id == dr.run_id) - .all() - ) - - done_states = {"success", "failed", "skipped", "upstream_failed"} - for ti in task_instances: - assert ti.state in done_states diff --git a/tests/www/views/test_views_decorators.py b/tests/www/views/test_views_decorators.py deleted file mode 100644 index 54e13641f9bb9..0000000000000 --- a/tests/www/views/test_views_decorators.py +++ /dev/null @@ -1,154 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import os - -import pytest - -from airflow.models import DagBag, Variable -from airflow.utils import timezone -from airflow.utils.state import State -from airflow.utils.types import DagRunTriggeredByType, DagRunType - -from tests_common.test_utils.db import clear_db_runs, clear_db_variables, parse_and_sync_to_db -from tests_common.test_utils.www import ( - _check_last_log, - _check_last_log_masked_variable, - check_content_in_response, -) - -pytestmark = pytest.mark.db_test - -EXAMPLE_DAG_DEFAULT_DATE = timezone.utcnow().replace(hour=0, minute=0, second=0, microsecond=0) - - -@pytest.fixture(scope="module") -def dagbag(): - parse_and_sync_to_db(os.devnull, include_examples=True) - return DagBag(read_dags_from_db=True) - - -@pytest.fixture(scope="module") -def bash_dag(dagbag): - return dagbag.get_dag("example_bash_operator") - - -@pytest.fixture(scope="module") -def xcom_dag(dagbag): - return dagbag.get_dag("example_xcom") - - -@pytest.fixture(autouse=True) -def dagruns(bash_dag, xcom_dag): - bash_dagrun = bash_dag.create_dagrun( - run_id="test_bash", - run_type=DagRunType.SCHEDULED, - logical_date=EXAMPLE_DAG_DEFAULT_DATE, - data_interval=(EXAMPLE_DAG_DEFAULT_DATE, EXAMPLE_DAG_DEFAULT_DATE), - start_date=timezone.utcnow(), - state=State.RUNNING, - run_after=EXAMPLE_DAG_DEFAULT_DATE, - triggered_by=DagRunTriggeredByType.TEST, - ) - - xcom_dagrun = xcom_dag.create_dagrun( - run_id="test_xcom", - run_type=DagRunType.SCHEDULED, - logical_date=EXAMPLE_DAG_DEFAULT_DATE, - data_interval=(EXAMPLE_DAG_DEFAULT_DATE, EXAMPLE_DAG_DEFAULT_DATE), - start_date=timezone.utcnow(), - state=State.RUNNING, - run_after=EXAMPLE_DAG_DEFAULT_DATE, - triggered_by=DagRunTriggeredByType.TEST, - ) - - yield bash_dagrun, xcom_dagrun - - clear_db_runs() - - -@pytest.fixture(autouse=True) -def _clean_db(): - clear_db_variables() - yield - clear_db_variables() - - -def test_action_logging_robots(session, admin_client): - url = "/robots.txt" - admin_client.get(url, follow_redirects=True) - - # In mysql backend, this commit() is needed to write down the logs - session.commit() - _check_last_log( - session, - event="robots", - dag_id=None, - logical_date=None, - ) - - -def test_action_logging_post(session, admin_client): - form = dict( - task_id="runme_1", - dag_id="example_bash_operator", - logical_date=EXAMPLE_DAG_DEFAULT_DATE, - upstream="false", - downstream="false", - future="false", - past="false", - only_failed="false", - ) - resp = admin_client.post("clear", data=form) - check_content_in_response(["example_bash_operator", "Please confirm"], resp) - # In mysql backend, this commit() is needed to write down the logs - session.commit() - _check_last_log( - session, - dag_id="example_bash_operator", - event="clear", - logical_date=EXAMPLE_DAG_DEFAULT_DATE, - expected_extra={ - "upstream": "false", - "downstream": "false", - "future": "false", - "past": "false", - "only_failed": "false", - }, - ) - - -def delete_variable(session, key): - session.query(Variable).filter(Variable.key == key).delete() - session.commit() - - -def test_action_logging_variables_post(session, admin_client): - form = dict(key="random", val="random") - admin_client.post("/variable/add", data=form) - session.commit() - _check_last_log(session, dag_id=None, event="variable.create", logical_date=None) - - -@pytest.mark.enable_redact -def test_action_logging_variables_masked_secrets(session, admin_client): - form = dict(key="x_secret", val="randomval") - admin_client.post("/variable/add", data=form) - session.commit() - _check_last_log_masked_variable(session, dag_id=None, event="variable.create", logical_date=None) diff --git a/tests/www/views/test_views_extra_links.py b/tests/www/views/test_views_extra_links.py deleted file mode 100644 index 9e49eb7cd622c..0000000000000 --- a/tests/www/views/test_views_extra_links.py +++ /dev/null @@ -1,315 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import json -import urllib.parse -from unittest import mock -from unittest.mock import patch - -import pytest - -from airflow.models.baseoperator import BaseOperator -from airflow.models.dag import DAG -from airflow.serialization.serialized_objects import SerializedBaseOperator -from airflow.utils import timezone -from airflow.utils.state import DagRunState -from airflow.utils.types import DagRunTriggeredByType, DagRunType - -from tests_common.test_utils.compat import BaseOperatorLink -from tests_common.test_utils.db import clear_db_runs -from tests_common.test_utils.mock_operators import ( - AirflowLink, - EmptyExtraLinkTestOperator, - EmptyNoExtraLinkTestOperator, -) - -pytestmark = pytest.mark.db_test - -DEFAULT_DATE = timezone.datetime(2017, 1, 1, tzinfo=timezone.utc) -STR_DEFAULT_DATE = urllib.parse.quote(DEFAULT_DATE.strftime("%Y-%m-%dT%H:%M:%S.%f%z")) - -ENDPOINT = "extra_links" - - -class RaiseErrorLink(BaseOperatorLink): - name = "raise_error" - - def get_link(self, operator, *, ti_key): - raise ValueError("This is an error") - - -class NoResponseLink(BaseOperatorLink): - name = "no_response" - - def get_link(self, operator, *, ti_key): - return None - - -class FooBarLink(BaseOperatorLink): - name = "foo-bar" - - def get_link(self, operator, *, ti_key): - return f"http://www.example.com/{operator.task_id}/foo-bar/{ti_key.run_id}" - - -class DummyTestOperator(BaseOperator): - # We need to ignore type check here due to 2.7 compatibility import - operator_extra_links = ( # type: ignore[assignment] - RaiseErrorLink(), - NoResponseLink(), - FooBarLink(), - AirflowLink(), - ) - - -@pytest.fixture(scope="module") -def dag(): - return DAG("dag", start_date=DEFAULT_DATE, schedule="0 0 * * *") - - -@pytest.fixture(scope="module") -def create_dag_run(dag): - def _create_dag_run(*, logical_date, session): - return dag.create_dagrun( - run_id=f"manual__{logical_date.isoformat()}", - state=DagRunState.RUNNING, - logical_date=logical_date, - data_interval=(logical_date, logical_date), - run_type=DagRunType.MANUAL, - session=session, - run_after=logical_date, - triggered_by=DagRunTriggeredByType.TEST, - ) - - return _create_dag_run - - -@pytest.fixture -def dag_run(create_dag_run, session): - return create_dag_run(logical_date=DEFAULT_DATE, session=session) - - -@pytest.fixture(scope="module", autouse=True) -def _patched_app(app, dag): - with mock.patch.object(app, "dag_bag") as mock_dag_bag: - mock_dag_bag.get_dag.return_value = dag - yield - - -@pytest.fixture(scope="module", autouse=True) -def task_1(dag): - return DummyTestOperator(task_id="some_dummy_task", dag=dag) - - -@pytest.fixture(scope="module", autouse=True) -def task_2(dag): - return EmptyExtraLinkTestOperator(task_id="some_dummy_task_2", dag=dag) - - -@pytest.fixture(scope="module", autouse=True) -def task_3(dag): - return EmptyNoExtraLinkTestOperator(task_id="some_dummy_task_3", dag=dag) - - -@pytest.fixture(scope="module", autouse=True) -def _init_blank_task_instances(): - """Make sure there are no runs before we test anything. - - This really shouldn't be needed, but tests elsewhere leave the db dirty. - """ - clear_db_runs() - - -@pytest.fixture(autouse=True) -def _reset_task_instances(): - yield - clear_db_runs() - - -def test_extra_links_works(dag_run, task_1, viewer_client, session): - expected_url = "http://www.example.com/some_dummy_task/foo-bar/manual__2017-01-01T00:00:00+00:00" - - with patch.object(task_1, "get_extra_links", return_value=expected_url, create=True): - response = viewer_client.get( - f"{ENDPOINT}?dag_id={task_1.dag_id}&task_id={task_1.task_id}" - f"&logical_date={STR_DEFAULT_DATE}&link_name=foo-bar", - follow_redirects=True, - ) - - assert response.status_code == 200 - assert json.loads(response.data.decode()) == { - "url": expected_url, - "error": None, - } - - -def test_global_extra_links_works(dag_run, task_1, viewer_client, session): - expected_url = "https://github.com/apache/airflow" - - with patch.object(task_1, "get_extra_links", return_value=expected_url, create=True): - response = viewer_client.get( - f"{ENDPOINT}?dag_id={dag_run.dag_id}&task_id={task_1.task_id}" - f"&logical_date={STR_DEFAULT_DATE}&link_name=github", - follow_redirects=True, - ) - - assert response.status_code == 200 - assert json.loads(response.data.decode()) == { - "url": "https://github.com/apache/airflow", - "error": None, - } - - -def test_operator_extra_link_override_global_extra_link(dag_run, task_1, viewer_client): - expected_url = "https://github.com/apache/airflow" - with patch.object(task_1, "get_extra_links", return_value=expected_url, create=True): - response = viewer_client.get( - f"{ENDPOINT}?dag_id={task_1.dag_id}&task_id={task_1.task_id}" - f"&logical_date={STR_DEFAULT_DATE}&link_name=airflow", - follow_redirects=True, - ) - - assert response.status_code == 200 - response_str = response.data - if isinstance(response.data, bytes): - response_str = response_str.decode() - assert json.loads(response_str) == {"url": expected_url, "error": None} - - -def test_extra_links_error_raised(dag_run, task_1, viewer_client): - with patch.object(task_1, "get_extra_links", side_effect=ValueError("This is an error"), create=True): - response = viewer_client.get( - f"{ENDPOINT}?dag_id={task_1.dag_id}&task_id={task_1.task_id}" - f"&logical_date={STR_DEFAULT_DATE}&link_name=raise_error", - follow_redirects=True, - ) - - assert response.status_code == 404 - response_str = response.data - if isinstance(response.data, bytes): - response_str = response_str.decode() - assert json.loads(response_str) == {"url": None, "error": "This is an error"} - - -def test_extra_links_no_response(dag_run, task_1, viewer_client): - with patch.object(task_1, "get_extra_links", return_value=None, create=True): - response = viewer_client.get( - f"{ENDPOINT}?dag_id={task_1.dag_id}&task_id={task_1.task_id}" - f"&logical_date={STR_DEFAULT_DATE}&link_name=no_response", - follow_redirects=True, - ) - - assert response.status_code == 404 - response_str = response.data - if isinstance(response.data, bytes): - response_str = response_str.decode() - assert json.loads(response_str) == {"url": None, "error": "No URL found for no_response"} - - -def test_operator_extra_link_override_plugin(dag_run, task_2, viewer_client): - """ - This tests checks if Operator Link (AirflowLink) defined in the Dummy2TestOperator - is overridden by Airflow Plugin (AirflowLink2). - - AirflowLink returns 'https://airflow.apache.org/' link - AirflowLink2 returns 'https://airflow.apache.org/1.10.5/' link - """ - endpoint = f"{ENDPOINT}?dag_id={task_2.dag_id}&task_id={task_2.task_id}&logical_date={STR_DEFAULT_DATE}&link_name=airflow" - expected_url = get_extra_links_for_task_from_endpoint(task_2, endpoint) - with patch.object(task_2, "get_extra_links", return_value=expected_url, create=True): - response = viewer_client.get( - endpoint, - follow_redirects=True, - ) - assert response.status_code == 200 - response_str = response.data - if isinstance(response.data, bytes): - response_str = response_str.decode() - assert json.loads(response_str) == {"url": expected_url, "error": None} - - -def test_operator_extra_link_multiple_operators(dag_run, task_2, task_3, viewer_client): - """ - This tests checks if Operator Link (AirflowLink2) defined in - Airflow Plugin (AirflowLink2) is attached to all the list of - operators defined in the AirflowLink2().operators property - - AirflowLink2 returns 'https://airflow.apache.org/1.10.5/' link - GoogleLink returns 'https://www.google.com' - """ - - endpoint = f"{ENDPOINT}?dag_id={task_2.dag_id}&task_id={task_2.task_id}&logical_date={STR_DEFAULT_DATE}&link_name=airflow" - expected_url = get_extra_links_for_task_from_endpoint(task_2, endpoint) - with patch.object(task_2, "get_extra_links", return_value=expected_url, create=True): - response = viewer_client.get( - endpoint, - follow_redirects=True, - ) - - assert response.status_code == 200 - response_str = response.data - if isinstance(response.data, bytes): - response_str = response_str.decode() - assert json.loads(response_str) == {"url": "https://airflow.apache.org/1.10.5/", "error": None} - - endpoint = f"{ENDPOINT}?dag_id={task_3.dag_id}&task_id={task_3.task_id}&logical_date={STR_DEFAULT_DATE}&link_name=airflow" - expected_url = get_extra_links_for_task_from_endpoint(task_3, endpoint) - with patch.object(task_3, "get_extra_links", return_value=expected_url, create=True): - response = viewer_client.get( - endpoint, - follow_redirects=True, - ) - - assert response.status_code == 200 - response_str = response.data - if isinstance(response.data, bytes): - response_str = response_str.decode() - assert json.loads(response_str) == {"url": "https://airflow.apache.org/1.10.5/", "error": None} - - # Also check that the other Operator Link defined for this operator exists - endpoint = f"{ENDPOINT}?dag_id={task_3.dag_id}&task_id={task_3.task_id}&logical_date={STR_DEFAULT_DATE}&link_name=google" - expected_url = get_extra_links_for_task_from_endpoint(task_3, endpoint) - with patch.object(task_3, "get_extra_links", return_value=expected_url, create=True): - response = viewer_client.get( - endpoint, - follow_redirects=True, - ) - - assert response.status_code == 200 - response_str = response.data - if isinstance(response.data, bytes): - response_str = response_str.decode() - assert json.loads(response_str) == {"url": "https://www.google.com", "error": None} - - -def convert_task_to_deser_task(task): - de = SerializedBaseOperator.deserialize_operator(SerializedBaseOperator.serialize_operator(task)) - return de - - -def get_extra_links_for_task_from_endpoint(task, endpoint): - import re - - match = re.search(r"[?&]link_name=([^&]+)", endpoint) - link_name = match.group(1) - de_task = convert_task_to_deser_task(task) - - for oe in de_task.operator_extra_links: - if oe.name == link_name: - return oe.get_link(operator=task, ti_key=None) diff --git a/tests/www/views/test_views_grid.py b/tests/www/views/test_views_grid.py deleted file mode 100644 index fec3b80ca340f..0000000000000 --- a/tests/www/views/test_views_grid.py +++ /dev/null @@ -1,569 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from datetime import timedelta -from typing import TYPE_CHECKING - -import pendulum -import pytest -from dateutil.tz import UTC - -from airflow.decorators import task_group -from airflow.models import DagBag -from airflow.models.asset import AssetDagRunQueue, AssetEvent, AssetModel -from airflow.providers.common.compat.lineage.entities import File -from airflow.providers.standard.operators.empty import EmptyOperator -from airflow.sdk.definitions.asset import Asset -from airflow.utils import timezone -from airflow.utils.state import DagRunState, TaskInstanceState -from airflow.utils.task_group import TaskGroup -from airflow.utils.types import DagRunType -from airflow.www.views import dag_to_grid - -from tests_common.test_utils.asserts import assert_queries_count -from tests_common.test_utils.db import clear_db_assets, clear_db_runs -from tests_common.test_utils.mock_operators import MockOperator - -pytestmark = pytest.mark.db_test - -if TYPE_CHECKING: - from airflow.models.dagrun import DagRun - -DAG_ID = "test" - - -@pytest.fixture(autouse=True, scope="module") -def examples_dag_bag(): - # Speed up: We don't want example dags for this module - return DagBag(include_examples=False, read_dags_from_db=True) - - -@pytest.fixture(autouse=True) -def _clean(): - clear_db_runs() - clear_db_assets() - yield - clear_db_runs() - clear_db_assets() - - -@pytest.fixture -def dag_without_runs(dag_maker, session, app, monkeypatch): - with monkeypatch.context() as m: - # Remove global operator links for this test - m.setattr("airflow.plugins_manager.global_operator_extra_links", []) - m.setattr("airflow.plugins_manager.operator_extra_links", []) - m.setattr("airflow.plugins_manager.registered_operator_link_classes", {}) - - with dag_maker(dag_id=DAG_ID, serialized=True, session=session): - EmptyOperator(task_id="task1") - - @task_group - def mapped_task_group(arg1): - return MockOperator(task_id="subtask2", arg1=arg1) - - mapped_task_group.expand(arg1=["a", "b", "c"]) - with TaskGroup(group_id="group"): - MockOperator.partial(task_id="mapped").expand(arg1=["a", "b", "c", "d"]) - - m.setattr(app, "dag_bag", dag_maker.dagbag) - yield dag_maker - - -@pytest.fixture -def dag_with_runs(dag_without_runs): - date = dag_without_runs.dag.start_date - run_1 = dag_without_runs.create_dagrun( - run_id="run_1", state=DagRunState.SUCCESS, run_type=DagRunType.SCHEDULED, logical_date=date - ) - run_2 = dag_without_runs.create_dagrun( - run_id="run_2", - run_type=DagRunType.SCHEDULED, - logical_date=date + timedelta(days=1), - ) - - return run_1, run_2 - - -def test_no_runs(admin_client, dag_without_runs): - resp = admin_client.get(f"/object/grid_data?dag_id={DAG_ID}", follow_redirects=True) - assert resp.status_code == 200, resp.json - assert resp.json == { - "dag_runs": [], - "groups": { - "children": [ - { - "extra_links": [], - "has_outlet_assets": False, - "id": "task1", - "instances": [], - "is_mapped": False, - "label": "task1", - "operator": "EmptyOperator", - "trigger_rule": "all_success", - }, - { - "children": [ - { - "extra_links": [], - "has_outlet_assets": False, - "id": "mapped_task_group.subtask2", - "instances": [], - "is_mapped": True, - "label": "subtask2", - "operator": "MockOperator", - "trigger_rule": "all_success", - } - ], - "is_mapped": True, - "id": "mapped_task_group", - "instances": [], - "label": "mapped_task_group", - "tooltip": "", - }, - { - "children": [ - { - "extra_links": [], - "has_outlet_assets": False, - "id": "group.mapped", - "instances": [], - "is_mapped": True, - "label": "mapped", - "operator": "MockOperator", - "trigger_rule": "all_success", - } - ], - "id": "group", - "instances": [], - "label": "group", - "tooltip": "", - }, - ], - "id": None, - "instances": [], - "label": None, - }, - "ordering": ["data_interval_end", "logical_date"], - "errors": [], - } - - -def test_grid_data_filtered_on_run_type_and_run_state(admin_client, dag_with_runs): - for uri_params, expected_run_types, expected_run_states in [ - ("run_state=success&run_state=queued", ["scheduled"], ["success"]), - ("run_state=running&run_state=failed", ["scheduled"], ["running"]), - ("run_type=scheduled&run_type=manual", ["scheduled", "scheduled"], ["success", "running"]), - ("run_type=backfill&run_type=manual", [], []), - ("run_state=running&run_type=failed&run_type=backfill&run_type=manual", [], []), - ( - "run_state=running&run_type=failed&run_type=scheduled&run_type=backfill&run_type=manual", - ["scheduled"], - ["running"], - ), - ]: - resp = admin_client.get(f"/object/grid_data?dag_id={DAG_ID}&{uri_params}", follow_redirects=True) - assert resp.status_code == 200, resp.json - actual_run_types = list(map(lambda x: x["run_type"], resp.json["dag_runs"])) - actual_run_states = list(map(lambda x: x["state"], resp.json["dag_runs"])) - assert actual_run_types == expected_run_types - assert actual_run_states == expected_run_states - - -# Create this as a fixture so that it is applied before the `dag_with_runs` fixture is! -@pytest.fixture -def _freeze_time_for_dagruns(time_machine): - time_machine.move_to("2022-01-02T00:00:00+00:00", tick=False) - - -@pytest.mark.usefixtures("_freeze_time_for_dagruns") -def test_one_run(admin_client, dag_with_runs: list[DagRun], session): - """ - Test a DAG with complex interaction of states: - - One run successful - - One run partly success, partly running - - One TI not yet finished - """ - run1, run2 = dag_with_runs - - for ti in run1.task_instances: - ti.state = TaskInstanceState.SUCCESS - for ti in sorted(run2.task_instances, key=lambda ti: (ti.task_id, ti.map_index)): - if ti.task_id == "task1": - ti.state = TaskInstanceState.SUCCESS - elif ti.task_id == "group.mapped": - if ti.map_index == 0: - ti.state = TaskInstanceState.SUCCESS - ti.start_date = pendulum.DateTime(2021, 7, 1, 1, 0, 0, tzinfo=pendulum.UTC) - ti.end_date = pendulum.DateTime(2021, 7, 1, 1, 2, 3, tzinfo=pendulum.UTC) - elif ti.map_index == 1: - ti.state = TaskInstanceState.RUNNING - ti.start_date = pendulum.DateTime(2021, 7, 1, 2, 3, 4, tzinfo=pendulum.UTC) - ti.end_date = None - - session.flush() - - resp = admin_client.get(f"/object/grid_data?dag_id={DAG_ID}", follow_redirects=True) - - assert resp.status_code == 200, resp.json - - assert resp.json == { - "dag_runs": [ - { - "conf": None, - "conf_is_json": False, - "data_interval_end": "2016-01-02T00:00:00+00:00", - "data_interval_start": "2016-01-01T00:00:00+00:00", - "end_date": timezone.utcnow().isoformat(), - "logical_date": "2016-01-01T00:00:00+00:00", - "last_scheduling_decision": None, - "note": None, - "queued_at": None, - "run_id": "run_1", - "run_type": "scheduled", - "start_date": "2016-01-01T00:00:00+00:00", - "state": "success", - "triggered_by": "test", - }, - { - "conf": None, - "conf_is_json": False, - "data_interval_end": "2016-01-03T00:00:00+00:00", - "data_interval_start": "2016-01-02T00:00:00+00:00", - "end_date": None, - "logical_date": "2016-01-02T00:00:00+00:00", - "last_scheduling_decision": None, - "note": None, - "queued_at": None, - "run_id": "run_2", - "run_type": "scheduled", - "start_date": "2016-01-01T00:00:00+00:00", - "state": "running", - "triggered_by": "test", - }, - ], - "groups": { - "children": [ - { - "extra_links": [], - "has_outlet_assets": False, - "id": "task1", - "instances": [ - { - "run_id": "run_1", - "queued_dttm": None, - "start_date": None, - "end_date": None, - "note": None, - "state": "success", - "task_id": "task1", - "try_number": 0, - }, - { - "run_id": "run_2", - "queued_dttm": None, - "start_date": None, - "end_date": None, - "note": None, - "state": "success", - "task_id": "task1", - "try_number": 0, - }, - ], - "is_mapped": False, - "label": "task1", - "operator": "EmptyOperator", - "trigger_rule": "all_success", - }, - { - "children": [ - { - "extra_links": [], - "has_outlet_assets": False, - "id": "mapped_task_group.subtask2", - "instances": [ - { - "run_id": "run_1", - "mapped_states": {"success": 3}, - "queued_dttm": None, - "start_date": None, - "end_date": None, - "state": "success", - "task_id": "mapped_task_group.subtask2", - }, - { - "run_id": "run_2", - "mapped_states": {"no_status": 3}, - "queued_dttm": None, - "start_date": None, - "end_date": None, - "state": None, - "task_id": "mapped_task_group.subtask2", - }, - ], - "is_mapped": True, - "label": "subtask2", - "operator": "MockOperator", - "trigger_rule": "all_success", - } - ], - "is_mapped": True, - "id": "mapped_task_group", - "instances": [ - { - "end_date": None, - "run_id": "run_1", - "mapped_states": {"success": 3}, - "queued_dttm": None, - "start_date": None, - "state": "success", - "task_id": "mapped_task_group", - }, - { - "run_id": "run_2", - "queued_dttm": None, - "start_date": None, - "end_date": None, - "state": None, - "mapped_states": {"no_status": 3}, - "task_id": "mapped_task_group", - }, - ], - "label": "mapped_task_group", - "tooltip": "", - }, - { - "children": [ - { - "extra_links": [], - "has_outlet_assets": False, - "id": "group.mapped", - "instances": [ - { - "run_id": "run_1", - "mapped_states": {"success": 4}, - "queued_dttm": None, - "start_date": None, - "end_date": None, - "state": "success", - "task_id": "group.mapped", - }, - { - "run_id": "run_2", - "mapped_states": {"no_status": 2, "running": 1, "success": 1}, - "queued_dttm": None, - "start_date": "2021-07-01T01:00:00+00:00", - "end_date": "2021-07-01T01:02:03+00:00", - "state": "running", - "task_id": "group.mapped", - }, - ], - "is_mapped": True, - "label": "mapped", - "operator": "MockOperator", - "trigger_rule": "all_success", - }, - ], - "id": "group", - "instances": [ - { - "end_date": None, - "run_id": "run_1", - "queued_dttm": None, - "start_date": None, - "state": "success", - "task_id": "group", - }, - { - "run_id": "run_2", - "queued_dttm": None, - "start_date": "2021-07-01T01:00:00+00:00", - "end_date": "2021-07-01T01:02:03+00:00", - "state": "running", - "task_id": "group", - }, - ], - "label": "group", - "tooltip": "", - }, - ], - "id": None, - "instances": [], - "label": None, - }, - "ordering": ["data_interval_end", "logical_date"], - "errors": [], - } - - -def test_query_count(dag_with_runs, session): - run1, run2 = dag_with_runs - with assert_queries_count(2): - dag_to_grid(run1.dag, (run1, run2), session) - - -def test_has_outlet_asset_flag(admin_client, dag_maker, session, app, monkeypatch): - with monkeypatch.context() as m: - # Remove global operator links for this test - m.setattr("airflow.plugins_manager.global_operator_extra_links", []) - m.setattr("airflow.plugins_manager.operator_extra_links", []) - m.setattr("airflow.plugins_manager.registered_operator_link_classes", {}) - - with dag_maker(dag_id=DAG_ID, serialized=True, session=session): - lineagefile = File("/tmp/does_not_exist") - EmptyOperator(task_id="task1") - EmptyOperator(task_id="task2", outlets=[lineagefile]) - EmptyOperator(task_id="task3", outlets=[Asset(name="foo", uri="s3://bucket/key"), lineagefile]) - EmptyOperator(task_id="task4", outlets=[Asset(name="foo", uri="s3://bucket/key")]) - - m.setattr(app, "dag_bag", dag_maker.dagbag) - resp = admin_client.get(f"/object/grid_data?dag_id={DAG_ID}", follow_redirects=True) - - def _expected_task_details(task_id, has_outlet_assets): - return { - "extra_links": [], - "has_outlet_assets": has_outlet_assets, - "id": task_id, - "instances": [], - "is_mapped": False, - "label": task_id, - "operator": "EmptyOperator", - "trigger_rule": "all_success", - } - - assert resp.status_code == 200, resp.json - assert resp.json == { - "dag_runs": [], - "groups": { - "children": [ - _expected_task_details("task1", False), - _expected_task_details("task2", False), - _expected_task_details("task3", True), - _expected_task_details("task4", True), - ], - "id": None, - "instances": [], - "label": None, - }, - "ordering": ["data_interval_end", "logical_date"], - "errors": [], - } - - -@pytest.mark.need_serialized_dag -def test_next_run_assets(admin_client, dag_maker, session, app, monkeypatch): - with monkeypatch.context() as m: - assets = [Asset(uri=f"s3://bucket/key/{i}", name=f"name_{i}", group="test-group") for i in [1, 2]] - - with dag_maker(dag_id=DAG_ID, schedule=assets, serialized=True, session=session): - EmptyOperator(task_id="task1") - - m.setattr(app, "dag_bag", dag_maker.dagbag) - - asset1_id = session.query(AssetModel.id).filter_by(uri=assets[0].uri).scalar() - asset2_id = session.query(AssetModel.id).filter_by(uri=assets[1].uri).scalar() - adrq = AssetDagRunQueue( - target_dag_id=DAG_ID, asset_id=asset1_id, created_at=pendulum.DateTime(2022, 8, 2, tzinfo=UTC) - ) - session.add(adrq) - asset_events = [ - AssetEvent( - asset_id=asset1_id, - extra={}, - timestamp=pendulum.DateTime(2022, 8, 1, 1, tzinfo=UTC), - ), - AssetEvent( - asset_id=asset1_id, - extra={}, - timestamp=pendulum.DateTime(2022, 8, 2, 1, tzinfo=UTC), - ), - AssetEvent( - asset_id=asset1_id, - extra={}, - timestamp=pendulum.DateTime(2022, 8, 2, 2, tzinfo=UTC), - ), - ] - session.add_all(asset_events) - session.commit() - - resp = admin_client.get(f"/object/next_run_assets/{DAG_ID}", follow_redirects=True) - - assert resp.status_code == 200, resp.json - assert resp.json == { - "asset_expression": { - "all": [ - {"asset": {"uri": "s3://bucket/key/1", "name": "name_1", "group": "test-group"}}, - {"asset": {"uri": "s3://bucket/key/2", "name": "name_2", "group": "test-group"}}, - ] - }, - "events": [ - {"id": asset1_id, "uri": "s3://bucket/key/1", "lastUpdate": "2022-08-02T02:00:00+00:00"}, - {"id": asset2_id, "uri": "s3://bucket/key/2", "lastUpdate": None}, - ], - } - - -def test_next_run_assets_404(admin_client): - resp = admin_client.get("/object/next_run_assets/missingdag", follow_redirects=True) - assert resp.status_code == 404, resp.json - assert resp.json == {"error": "can't find dag missingdag"} - - -@pytest.mark.usefixtures("_freeze_time_for_dagruns") -def test_dynamic_mapped_task_with_retries(admin_client, dag_with_runs: list[DagRun], session): - """ - Test a DAG with a dynamic mapped task with retries - """ - run1, run2 = dag_with_runs - - for ti in run1.task_instances: - ti.state = TaskInstanceState.SUCCESS - for ti in sorted(run2.task_instances, key=lambda ti: (ti.task_id, ti.map_index)): - if ti.task_id == "task1": - ti.state = TaskInstanceState.SUCCESS - elif ti.task_id == "group.mapped": - if ti.map_index == 0: - ti.state = TaskInstanceState.FAILED - ti.start_date = pendulum.DateTime(2021, 7, 1, 1, 0, 0, tzinfo=pendulum.UTC) - ti.end_date = pendulum.DateTime(2021, 7, 1, 1, 2, 3, tzinfo=pendulum.UTC) - elif ti.map_index == 1: - ti.try_number = 1 - ti.state = TaskInstanceState.SUCCESS - ti.start_date = pendulum.DateTime(2021, 7, 1, 2, 3, 4, tzinfo=pendulum.UTC) - ti.end_date = None - elif ti.map_index == 2: - ti.try_number = 2 - ti.state = TaskInstanceState.FAILED - ti.start_date = pendulum.DateTime(2021, 7, 1, 2, 3, 4, tzinfo=pendulum.UTC) - ti.end_date = None - elif ti.map_index == 3: - ti.try_number = 3 - ti.state = TaskInstanceState.SUCCESS - ti.start_date = pendulum.DateTime(2021, 7, 1, 2, 3, 4, tzinfo=pendulum.UTC) - ti.end_date = None - session.flush() - - resp = admin_client.get(f"/object/grid_data?dag_id={DAG_ID}", follow_redirects=True) - - assert resp.status_code == 200, resp.json - - assert resp.json["groups"]["children"][-1]["children"][-1]["instances"][-1]["mapped_states"] == { - "failed": 2, - "success": 2, - } diff --git a/tests/www/views/test_views_home.py b/tests/www/views/test_views_home.py deleted file mode 100644 index a27def9e79a38..0000000000000 --- a/tests/www/views/test_views_home.py +++ /dev/null @@ -1,299 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from unittest import mock - -import flask -import markupsafe -import pytest - -from airflow.utils.state import State -from airflow.www.utils import UIAlert -from airflow.www.views import FILTER_LASTRUN_COOKIE, FILTER_STATUS_COOKIE, FILTER_TAGS_COOKIE - -from tests_common.test_utils.db import clear_db_dags, clear_db_import_errors, clear_db_serialized_dags -from tests_common.test_utils.www import ( - capture_templates, # noqa: F401 - check_content_in_response, - check_content_not_in_response, -) - -pytestmark = [pytest.mark.db_test, pytest.mark.need_serialized_dag] - - -def clean_db(): - clear_db_dags() - clear_db_import_errors() - clear_db_serialized_dags() - - -@pytest.fixture(autouse=True) -def _setup(): - clean_db() - yield - clean_db() - - -def test_home( - capture_templates, # noqa: F811 - admin_client, -): - with capture_templates() as templates: - resp = admin_client.get("home", follow_redirects=True) - check_content_in_response("DAGs", resp) - val_state_color_mapping = ( - "const STATE_COLOR = {" - '"deferred": "mediumpurple", "failed": "red", ' - '"null": "lightblue", "queued": "gray", ' - '"removed": "lightgrey", "restarting": "violet", "running": "lime", ' - '"scheduled": "tan", ' - '"skipped": "hotpink", ' - '"success": "green", "up_for_reschedule": "turquoise", ' - '"up_for_retry": "gold", "upstream_failed": "orange"};' - ) - check_content_in_response(val_state_color_mapping, resp) - - assert len(templates) == 1 - assert templates[0].name == "airflow/dags.html" - state_color_mapping = State.state_color.copy() - state_color_mapping["null"] = state_color_mapping.pop(None) - assert templates[0].local_context["state_color"] == state_color_mapping - - -@mock.patch("airflow.www.views.AirflowBaseView.render_template") -def test_home_dags_count(render_template_mock, admin_client, _working_dags, session): - from sqlalchemy import update - - from airflow.models.dag import DagModel - - def call_kwargs(): - return render_template_mock.call_args.kwargs - - admin_client.get("home", follow_redirects=True) - assert call_kwargs()["status_count_all"] == 4 - - update_stmt = update(DagModel).where(DagModel.dag_id == "filter_test_1").values(is_active=False) - session.execute(update_stmt) - - admin_client.get("home", follow_redirects=True) - assert call_kwargs()["status_count_all"] == 3 - - -def test_home_status_filter_cookie(admin_client): - with admin_client: - admin_client.get("home", follow_redirects=True) - assert flask.session[FILTER_STATUS_COOKIE] == "all" - - admin_client.get("home?status=active", follow_redirects=True) - assert flask.session[FILTER_STATUS_COOKIE] == "active" - - admin_client.get("home?status=paused", follow_redirects=True) - assert flask.session[FILTER_STATUS_COOKIE] == "paused" - - admin_client.get("home?status=all", follow_redirects=True) - assert flask.session[FILTER_STATUS_COOKIE] == "all" - - admin_client.get("home?lastrun=running", follow_redirects=True) - assert flask.session[FILTER_LASTRUN_COOKIE] == "running" - - admin_client.get("home?lastrun=failed", follow_redirects=True) - assert flask.session[FILTER_LASTRUN_COOKIE] == "failed" - - admin_client.get("home?lastrun=all_states", follow_redirects=True) - assert flask.session[FILTER_LASTRUN_COOKIE] == "all_states" - - -TEST_FILTER_DAG_IDS = ["filter_test_1", "filter_test_2", "a_first_dag_id_asc", "filter.test"] -TEST_TAGS = ["example", "test", "team", "group"] - - -@pytest.fixture -def _working_dags(dag_maker): - for dag_id, tag in zip(TEST_FILTER_DAG_IDS, TEST_TAGS): - with dag_maker(dag_id=dag_id, fileloc=f"/{dag_id}.py", tags=[tag]): - # We need to enter+exit the dag maker context for it to create the dag - pass - - -@pytest.fixture -def _broken_dags(session): - from airflow.models.errors import ParseImportError - - for dag_id in TEST_FILTER_DAG_IDS: - session.add( - ParseImportError( - filename=f"/{dag_id}.py", bundle_name="dag_maker", stacktrace="Some Error\nTraceback:\n" - ) - ) - session.commit() - - -def test_home_filter_tags(_working_dags, admin_client): - with admin_client: - admin_client.get("home?tags=example&tags=data", follow_redirects=True) - assert flask.session[FILTER_TAGS_COOKIE] == "example,data" - - admin_client.get("home?reset_tags", follow_redirects=True) - assert flask.session[FILTER_TAGS_COOKIE] is None - - -@pytest.mark.usefixtures("_broken_dags", "_working_dags") -def test_home_importerrors(_broken_dags, user_client): - # Users with "can read on DAGs" gets all DAG import errors - resp = user_client.get("home", follow_redirects=True) - check_content_in_response("Import Errors", resp) - for dag_id in TEST_FILTER_DAG_IDS: - check_content_in_response(f"/{dag_id}.py", resp) - - -def test_home_dag_list(_working_dags, user_client): - # Users with "can read on DAGs" gets all DAGs - resp = user_client.get("home", follow_redirects=True) - for dag_id in TEST_FILTER_DAG_IDS: - check_content_in_response(f"dag_id={dag_id}", resp) - - -def test_home_dag_list_search(_working_dags, user_client): - resp = user_client.get("home?search=filter_test", follow_redirects=True) - check_content_in_response("dag_id=filter_test_1", resp) - check_content_in_response("dag_id=filter_test_2", resp) - check_content_not_in_response("dag_id=filter.test", resp) - check_content_not_in_response("dag_id=a_first_dag_id_asc", resp) - - -def test_home_robots_header_in_response(user_client): - # Responses should include X-Robots-Tag header - resp = user_client.get("home", follow_redirects=True) - assert resp.headers["X-Robots-Tag"] == "noindex, nofollow" - - -@pytest.mark.parametrize( - "client, flash_message, expected", - [ - ("anonymous_client", UIAlert("hello world"), True), - ("anonymous_client", UIAlert("hello world", roles=["Viewer"]), True), - ("anonymous_client", UIAlert("hello world", roles=["User"]), False), - ("anonymous_client", UIAlert("hello world", roles=["Viewer", "User"]), True), - ("anonymous_client", UIAlert("hello world", roles=["Admin"]), False), - ("user_client", UIAlert("hello world"), True), - ("user_client", UIAlert("hello world", roles=["User"]), True), - ("user_client", UIAlert("hello world", roles=["User", "Admin"]), True), - ("user_client", UIAlert("hello world", roles=["Admin"]), False), - ("admin_client", UIAlert("hello world"), True), - ("admin_client", UIAlert("hello world", roles=["Admin"]), True), - ("admin_client", UIAlert("hello world", roles=["User", "Admin"]), True), - ], -) -def test_dashboard_flash_messages_role_filtering(request, client, flash_message, expected): - with mock.patch("airflow.settings.DASHBOARD_UIALERTS", [flash_message]): - resp = request.getfixturevalue(client).get("home", follow_redirects=True) - if expected: - check_content_in_response(flash_message.message, resp) - else: - check_content_not_in_response(flash_message.message, resp) - - -def test_dashboard_flash_messages_many(user_client): - messages = [ - UIAlert("hello world"), - UIAlert("im_not_here", roles=["Admin"]), - UIAlert("_hello_world_"), - ] - with mock.patch("airflow.settings.DASHBOARD_UIALERTS", messages): - resp = user_client.get("home", follow_redirects=True) - check_content_in_response("hello world", resp) - check_content_not_in_response("im_not_here", resp) - check_content_in_response("_hello_world_", resp) - - -def test_dashboard_flash_messages_markup(user_client): - link = 'hello world' - user_input = markupsafe.Markup("Hello %s") % ("foo&bar",) - messages = [ - UIAlert(link, html=True), - UIAlert(user_input), - ] - with mock.patch("airflow.settings.DASHBOARD_UIALERTS", messages): - resp = user_client.get("home", follow_redirects=True) - check_content_in_response(link, resp) - check_content_in_response(user_input, resp) - - -def test_dashboard_flash_messages_type(user_client): - messages = [ - UIAlert("hello world", category="foo"), - ] - with mock.patch("airflow.settings.DASHBOARD_UIALERTS", messages): - resp = user_client.get("home", follow_redirects=True) - check_content_in_response("hello world", resp) - check_content_in_response("alert-foo", resp) - - -@pytest.mark.parametrize( - "url, lower_key, greater_key", - [ - ("home?status=all", "a_first_dag_id_asc", "filter_test_1"), - ("home?status=all&sorting_key=dag_id&sorting_direction=asc", "filter_test_1", "filter_test_2"), - ("home?status=all&sorting_key=dag_id&sorting_direction=desc", "filter_test_2", "filter_test_1"), - ], - ids=["no_order_provided", "ascending_order_on_dag_id", "descending_order_on_dag_id"], -) -def test_sorting_home_view(url, lower_key, greater_key, user_client, _working_dags): - resp = user_client.get(url, follow_redirects=True) - resp_html = resp.data.decode("utf-8") - lower_index = resp_html.find(lower_key) - greater_index = resp_html.find(greater_key) - assert lower_index < greater_index - - -@pytest.mark.parametrize( - "url, filter_tags_cookie_val, filter_lastrun_cookie_val, expected_filter_tags, expected_filter_lastrun", - [ - ("home", None, None, [], None), - # from url only - ("home?tags=example&tags=test", None, None, ["example", "test"], None), - ("home?lastrun=running", None, None, [], "running"), - ("home?tags=example&tags=test&lastrun=running", None, None, ["example", "test"], "running"), - # from cookie only - ("home", "example,test", None, ["example", "test"], None), - ("home", None, "running", [], "running"), - ("home", "example,test", "running", ["example", "test"], "running"), - # from url and cookie - ("home?tags=example", "example,test", None, ["example"], None), - ("home?lastrun=failed", None, "running", [], "failed"), - ("home?tags=example", None, "running", ["example"], "running"), - ("home?lastrun=running", "example,test", None, ["example", "test"], "running"), - ("home?tags=example&lastrun=running", "example,test", "failed", ["example"], "running"), - ], -) -def test_filter_cookie_eval( - _working_dags, - admin_client, - url, - filter_tags_cookie_val, - filter_lastrun_cookie_val, - expected_filter_tags, - expected_filter_lastrun, -): - with admin_client.session_transaction() as flask_session: - flask_session[FILTER_TAGS_COOKIE] = filter_tags_cookie_val - flask_session[FILTER_LASTRUN_COOKIE] = filter_lastrun_cookie_val - - resp = admin_client.get(url, follow_redirects=True) - assert resp.request.args.getlist("tags") == expected_filter_tags - assert resp.request.args.get("lastrun") == expected_filter_lastrun diff --git a/tests/www/views/test_views_log.py b/tests/www/views/test_views_log.py deleted file mode 100644 index 037c97d026404..0000000000000 --- a/tests/www/views/test_views_log.py +++ /dev/null @@ -1,576 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import copy -import logging -import logging.config -import os -import pathlib -import shutil -import sys -import tempfile -import unittest.mock -import urllib.parse - -import pytest - -from airflow import settings -from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG -from airflow.models import DagBag, DagRun -from airflow.models.tasklog import LogTemplate -from airflow.utils import timezone -from airflow.utils.log.file_task_handler import FileTaskHandler -from airflow.utils.log.logging_mixin import ExternalLoggingMixin -from airflow.utils.session import create_session -from airflow.utils.state import DagRunState, TaskInstanceState -from airflow.utils.types import DagRunTriggeredByType, DagRunType -from airflow.www.app import create_app - -from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_dags, clear_db_runs -from tests_common.test_utils.decorators import dont_initialize_flask_app_submodules -from tests_common.test_utils.www import client_with_login - -pytestmark = pytest.mark.db_test - -DAG_ID = "dag_for_testing_log_view" -DAG_ID_REMOVED = "removed_dag_for_testing_log_view" -TASK_ID = "task_for_testing_log_view" -DEFAULT_DATE = timezone.datetime(2017, 9, 1) -STR_DEFAULT_DATE = urllib.parse.quote(DEFAULT_DATE.strftime("%Y-%m-%dT%H:%M:%S.%f%z")) -ENDPOINT = f"log?dag_id={DAG_ID}&task_id={TASK_ID}&logical_date={STR_DEFAULT_DATE}" - - -@pytest.fixture(scope="module", autouse=True) -def backup_modules(): - """Make sure that the configure_logging is not cached.""" - return dict(sys.modules) - - -@pytest.fixture(scope="module") -def log_path(tmp_path_factory): - return tmp_path_factory.mktemp("logs") - - -@pytest.fixture(scope="module") -def log_app(backup_modules, log_path): - @dont_initialize_flask_app_submodules( - skip_all_except=[ - "init_appbuilder", - "init_jinja_globals", - "init_appbuilder_views", - "init_api_connexion", - ] - ) - @conf_vars( - { - ("logging", "logging_config_class"): "airflow_local_settings.LOGGING_CONFIG", - ("fab", "auth_rate_limited"): "False", - } - ) - def factory(): - app = create_app(testing=True) - app.config["WTF_CSRF_ENABLED"] = False - settings.configure_orm() - security_manager = app.appbuilder.sm - if not security_manager.find_user(username="test"): - security_manager.add_user( - username="test", - first_name="test", - last_name="test", - email="test@fab.org", - role=security_manager.find_role("Admin"), - password="test", - ) - return app - - # Create a custom logging configuration - logging_config = copy.deepcopy(DEFAULT_LOGGING_CONFIG) - logging_config["handlers"]["task"]["base_log_folder"] = str(log_path) - - with tempfile.TemporaryDirectory() as settings_dir: - local_settings = pathlib.Path(settings_dir, "airflow_local_settings.py") - local_settings.write_text(f"LOGGING_CONFIG = {logging_config!r}") - sys.path.append(settings_dir) - yield factory() - sys.path.remove(settings_dir) - - logging.config.dictConfig(DEFAULT_LOGGING_CONFIG) - - -@pytest.fixture(autouse=True) -def _reset_modules_after_every_test(backup_modules): - yield - # Remove any new modules imported during the test run. - # This lets us import the same source files for more than one test. - for mod in [m for m in sys.modules if m not in backup_modules]: - del sys.modules[mod] - - -@pytest.fixture(autouse=True) -def dags(log_app, create_dummy_dag, testing_dag_bundle, session): - dag, _ = create_dummy_dag( - dag_id=DAG_ID, - task_id=TASK_ID, - start_date=DEFAULT_DATE, - with_dagrun_type=None, - session=session, - ) - dag_removed, _ = create_dummy_dag( - dag_id=DAG_ID_REMOVED, - task_id=TASK_ID, - start_date=DEFAULT_DATE, - with_dagrun_type=None, - session=session, - ) - - bag = DagBag(os.devnull, include_examples=False) - bag.bag_dag(dag=dag) - bag.bag_dag(dag=dag_removed) - bag.sync_to_db("testing", None, session=session) - log_app.dag_bag = bag - - yield dag, dag_removed - - clear_db_dags() - - -@pytest.fixture(autouse=True) -def tis(dags, session): - dag, dag_removed = dags - dagrun = dag.create_dagrun( - run_id=f"scheduled__{DEFAULT_DATE.isoformat()}", - run_type=DagRunType.SCHEDULED, - logical_date=DEFAULT_DATE, - data_interval=(DEFAULT_DATE, DEFAULT_DATE), - run_after=DEFAULT_DATE, - triggered_by=DagRunTriggeredByType.TEST, - start_date=DEFAULT_DATE, - state=DagRunState.RUNNING, - session=session, - ) - (ti,) = dagrun.task_instances - ti.try_number = 1 - ti.hostname = "localhost" - dagrun_removed = dag_removed.create_dagrun( - run_id=f"scheduled__{DEFAULT_DATE.isoformat()}", - run_type=DagRunType.SCHEDULED, - logical_date=DEFAULT_DATE, - data_interval=(DEFAULT_DATE, DEFAULT_DATE), - run_after=DEFAULT_DATE, - triggered_by=DagRunTriggeredByType.TEST, - start_date=DEFAULT_DATE, - state=DagRunState.RUNNING, - session=session, - ) - (ti_removed_dag,) = dagrun_removed.task_instances - ti_removed_dag.try_number = 1 - - yield ti, ti_removed_dag - - clear_db_runs() - - -@pytest.fixture -def create_expected_log_file(log_path, tis): - ti, _ = tis - handler = FileTaskHandler(log_path) - - def create_expected_log_file(try_number): - ti.try_number = 1 - handler.set_context(ti) - handler.emit(logging.makeLogRecord({"msg": "Log for testing."})) - handler.flush() - - yield create_expected_log_file - # log_path fixture is used in log_app, so both have "module" scope. Because of that - # log_path isn't deleted automatically by pytest between tests - # We delete created log files manually to make sure tests do not reuse logs created by other tests - for sub_path in log_path.iterdir(): - shutil.rmtree(sub_path) - - -@pytest.fixture -def log_admin_client(log_app): - return client_with_login(log_app, username="test", password="test") - - -@pytest.mark.parametrize( - "state, try_number, num_logs", - [ - (None, 0, 0), - (TaskInstanceState.UP_FOR_RETRY, 2, 2), - (TaskInstanceState.UP_FOR_RESCHEDULE, 0, 0), - (TaskInstanceState.UP_FOR_RESCHEDULE, 1, 1), - (TaskInstanceState.RUNNING, 1, 1), - (TaskInstanceState.SUCCESS, 1, 1), - (TaskInstanceState.FAILED, 3, 3), - ], - ids=[ - "none", - "up-for-retry", - "up-for-reschedule-0", - "up-for-reschedule-1", - "running", - "success", - "failed", - ], -) -def test_get_file_task_log(log_admin_client, tis, state, try_number, num_logs): - ti, _ = tis - with create_session() as session: - ti.state = state - ti.try_number = try_number - session.merge(ti) - - response = log_admin_client.get( - ENDPOINT, - data={"username": "test", "password": "test"}, - follow_redirects=True, - ) - assert response.status_code == 200 - - data = response.data.decode() - assert "Log by attempts" in data - for num in range(1, num_logs + 1): - assert f"log-group-{num}" in data - assert "log-group-0" not in data - assert f"log-group-{num_logs + 1}" not in data - - -def test_get_logs_with_metadata_as_download_file(log_admin_client, create_expected_log_file): - url_template = ( - "get_logs_with_metadata?dag_id={}&" - "task_id={}&logical_date={}&" - "try_number={}&metadata={}&format=file" - ) - try_number = 1 - create_expected_log_file(try_number) - date = DEFAULT_DATE.isoformat() - url = url_template.format( - DAG_ID, - TASK_ID, - urllib.parse.quote_plus(date), - try_number, - "{}", - ) - response = log_admin_client.get(url) - - content_disposition = response.headers["Content-Disposition"] - assert content_disposition.startswith("attachment") - assert ( - f"dag_id={DAG_ID}/run_id=scheduled__{date}/task_id={TASK_ID}/attempt={try_number}.log" - in content_disposition - ) - assert response.status_code == 200 - content = response.data.decode("utf-8") - assert "Log for testing." in content - assert "localhost\n" in content - - -DIFFERENT_LOG_FILENAME = "{{ ti.dag_id }}/{{ ti.run_id }}/{{ ti.task_id }}/{{ try_number }}.log" - - -@pytest.fixture -def dag_run_with_log_filename(tis): - run_filters = [DagRun.dag_id == DAG_ID, DagRun.logical_date == DEFAULT_DATE] - with create_session() as session: - log_template = session.merge( - LogTemplate(filename=DIFFERENT_LOG_FILENAME, elasticsearch_id="irrelevant") - ) - session.flush() # To populate 'log_template.id'. - run_query = session.query(DagRun).filter(*run_filters) - run_query.update({"log_template_id": log_template.id}) - dag_run = run_query.one() - # Dag has been updated, replace Dag in Task Instance - ti, _ = tis - ti.dag_run = dag_run - yield dag_run - with create_session() as session: - session.query(DagRun).filter(*run_filters).update({"log_template_id": None}) - session.query(LogTemplate).filter(LogTemplate.id == log_template.id).delete() - - -def test_get_logs_for_changed_filename_format_db( - log_admin_client, dag_run_with_log_filename, create_expected_log_file -): - try_number = 1 - create_expected_log_file(try_number) - url = ( - f"get_logs_with_metadata?dag_id={dag_run_with_log_filename.dag_id}&" - f"task_id={TASK_ID}&" - f"logical_date={urllib.parse.quote_plus(dag_run_with_log_filename.logical_date.isoformat())}&" - f"try_number={try_number}&metadata={{}}&format=file" - ) - response = log_admin_client.get(url) - - # Should find the log under corresponding db entry. - assert response.status_code == 200 - assert "Log for testing." in response.data.decode("utf-8") - content_disposition = response.headers["Content-Disposition"] - expected_filename = ( - f"{dag_run_with_log_filename.dag_id}/{dag_run_with_log_filename.run_id}/{TASK_ID}/{try_number}.log" - ) - assert content_disposition.startswith("attachment") - assert expected_filename in content_disposition - - -@unittest.mock.patch( - "airflow.utils.log.file_task_handler.FileTaskHandler.read", - side_effect=[ - ([[("default_log", "1st line")]], [{}]), - ([[("default_log", "2nd line")]], [{"end_of_log": False}]), - ([[("default_log", "3rd line")]], [{"end_of_log": True}]), - ([[("default_log", "should never be read")]], [{"end_of_log": True}]), - ], -) -def test_get_logs_with_metadata_as_download_large_file(_, log_admin_client): - url_template = ( - "get_logs_with_metadata?dag_id={}&" - "task_id={}&logical_date={}&" - "try_number={}&metadata={}&format=file" - ) - try_number = 1 - url = url_template.format( - DAG_ID, - TASK_ID, - urllib.parse.quote_plus(DEFAULT_DATE.isoformat()), - try_number, - "{}", - ) - response = log_admin_client.get(url) - - data = response.data.decode() - assert "1st line" in data - assert "2nd line" in data - assert "3rd line" in data - assert "should never be read" not in data - - -@pytest.mark.parametrize("metadata", ["null", "{}"]) -def test_get_logs_with_metadata(log_admin_client, metadata, create_expected_log_file): - url_template = "get_logs_with_metadata?dag_id={}&task_id={}&logical_date={}&try_number={}&metadata={}" - try_number = 1 - create_expected_log_file(try_number) - response = log_admin_client.get( - url_template.format( - DAG_ID, - TASK_ID, - urllib.parse.quote_plus(DEFAULT_DATE.isoformat()), - try_number, - metadata, - ), - data={"username": "test", "password": "test"}, - follow_redirects=True, - ) - assert response.status_code == 200 - - data = response.data.decode() - assert '"message":' in data - assert '"metadata":' in data - assert "Log for testing." in data - - -def test_get_logs_with_invalid_metadata(log_admin_client): - """Test invalid metadata JSON returns error message""" - metadata = "invalid" - url_template = "get_logs_with_metadata?dag_id={}&task_id={}&logical_date={}&try_number={}&metadata={}" - response = log_admin_client.get( - url_template.format( - DAG_ID, - TASK_ID, - urllib.parse.quote_plus(DEFAULT_DATE.isoformat()), - 1, - metadata, - ), - data={"username": "test", "password": "test"}, - follow_redirects=True, - ) - - assert response.status_code == 400 - assert response.json == {"error": "Invalid JSON metadata"} - - -@unittest.mock.patch( - "airflow.utils.log.file_task_handler.FileTaskHandler.read", - return_value=(["airflow log line"], [{"end_of_log": True}]), -) -def test_get_logs_with_metadata_for_removed_dag(_, log_admin_client): - url_template = "get_logs_with_metadata?dag_id={}&task_id={}&logical_date={}&try_number={}&metadata={}" - response = log_admin_client.get( - url_template.format( - DAG_ID_REMOVED, - TASK_ID, - urllib.parse.quote_plus(DEFAULT_DATE.isoformat()), - 1, - "{}", - ), - data={"username": "test", "password": "test"}, - follow_redirects=True, - ) - assert response.status_code == 200 - - data = response.data.decode() - assert '"message":' in data - assert '"metadata":' in data - assert "airflow log line" in data - - -def test_get_logs_response_with_ti_equal_to_none(log_admin_client): - url_template = ( - "get_logs_with_metadata?dag_id={}&" - "task_id={}&logical_date={}&" - "try_number={}&metadata={}&format=file" - ) - try_number = 1 - url = url_template.format( - DAG_ID, - "Non_Existing_ID", - urllib.parse.quote_plus(DEFAULT_DATE.isoformat()), - try_number, - "{}", - ) - response = log_admin_client.get(url) - - data = response.json - assert "message" in data - assert "error" in data - assert data["message"] == "*** Task instance did not exist in the DB\n" - - -def test_get_logs_with_json_response_format(log_admin_client, create_expected_log_file): - url_template = ( - "get_logs_with_metadata?dag_id={}&" - "task_id={}&logical_date={}&" - "try_number={}&metadata={}&format=json" - ) - try_number = 1 - create_expected_log_file(try_number) - url = url_template.format( - DAG_ID, - TASK_ID, - urllib.parse.quote_plus(DEFAULT_DATE.isoformat()), - try_number, - "{}", - ) - response = log_admin_client.get(url) - assert response.status_code == 200 - - assert "message" in response.json - assert "metadata" in response.json - assert "Log for testing." in response.json["message"][0][1] - - -def test_get_logs_invalid_execution_data_format(log_admin_client): - url_template = ( - "get_logs_with_metadata?dag_id={}&" - "task_id={}&logical_date={}&" - "try_number={}&metadata={}&format=file" - ) - try_number = 1 - url = url_template.format( - DAG_ID, - TASK_ID, - urllib.parse.quote_plus("Tuesday February 27, 2024"), - try_number, - "{}", - ) - response = log_admin_client.get(url) - assert response.status_code == 400 - assert response.json == { - "error": ( - "Given logical date 'Tuesday February 27, 2024' could not be identified as a date. " - "Example date format: 2015-11-16T14:34:15+00:00" - ) - } - - -@unittest.mock.patch("airflow.www.views.TaskLogReader") -def test_get_logs_for_handler_without_read_method(mock_reader, log_admin_client): - type(mock_reader.return_value).supports_read = unittest.mock.PropertyMock(return_value=False) - url_template = ( - "get_logs_with_metadata?dag_id={}&" - "task_id={}&logical_date={}&" - "try_number={}&metadata={}&format=json" - ) - try_number = 1 - url = url_template.format( - DAG_ID, - TASK_ID, - urllib.parse.quote_plus(DEFAULT_DATE.isoformat()), - try_number, - "{}", - ) - response = log_admin_client.get(url) - assert response.status_code == 200 - - data = response.json - assert "message" in data - assert "metadata" in data - assert "Task log handler does not support read logs." in data["message"] - - -@pytest.mark.parametrize("task_id", ["inexistent", TASK_ID]) -def test_redirect_to_external_log_with_local_log_handler(log_admin_client, task_id): - """Redirect to home if TI does not exist or if log handler is local""" - url_template = "redirect_to_external_log?dag_id={}&task_id={}&logical_date={}&try_number={}" - try_number = 1 - url = url_template.format( - DAG_ID, - task_id, - urllib.parse.quote_plus(DEFAULT_DATE.isoformat()), - try_number, - ) - response = log_admin_client.get(url) - assert response.status_code == 302 - assert response.headers["Location"] == "/home" - - -class _ExternalHandler(ExternalLoggingMixin): - EXTERNAL_URL = "http://external-service.com" - - @property - def log_name(self) -> str: - return "ExternalLog" - - def get_external_log_url(self, *args, **kwargs) -> str: - return self.EXTERNAL_URL - - @property - def supports_external_link(self) -> bool: - return True - - -@unittest.mock.patch( - "airflow.utils.log.log_reader.TaskLogReader.log_handler", - new_callable=unittest.mock.PropertyMock, - return_value=_ExternalHandler(), -) -def test_redirect_to_external_log_with_external_log_handler(_, log_admin_client): - url_template = "redirect_to_external_log?dag_id={}&task_id={}&logical_date={}&try_number={}" - try_number = 1 - url = url_template.format( - DAG_ID, - TASK_ID, - urllib.parse.quote_plus(DEFAULT_DATE.isoformat()), - try_number, - ) - response = log_admin_client.get(url) - assert response.status_code == 302 - assert response.headers["Location"] == _ExternalHandler.EXTERNAL_URL diff --git a/tests/www/views/test_views_mount.py b/tests/www/views/test_views_mount.py deleted file mode 100644 index 03115fa611047..0000000000000 --- a/tests/www/views/test_views_mount.py +++ /dev/null @@ -1,62 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest -import werkzeug.test -import werkzeug.wrappers - -from airflow.www.app import create_app - -from tests_common.test_utils.config import conf_vars - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(scope="module") -def app(): - @conf_vars({("webserver", "base_url"): "http://localhost/test"}) - def factory(): - return create_app(testing=True) - - app = factory() - app.config["WTF_CSRF_ENABLED"] = False - return app - - -@pytest.fixture -def client(app): - return werkzeug.test.Client(app, werkzeug.wrappers.response.Response) - - -def test_mount(client): - # Test an endpoint that doesn't need auth! - resp = client.get("/test/health") - assert resp.status_code == 200 - assert b"healthy" in resp.data - - -def test_not_found(client): - resp = client.get("/", follow_redirects=True) - assert resp.status_code == 404 - - -def test_index(client): - resp = client.get("/test/") - assert resp.status_code == 302 - assert resp.headers["Location"] == "/test/home" diff --git a/tests/www/views/test_views_paused.py b/tests/www/views/test_views_paused.py deleted file mode 100644 index f38aee3fabb59..0000000000000 --- a/tests/www/views/test_views_paused.py +++ /dev/null @@ -1,51 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from airflow.models.log import Log - -from tests_common.test_utils.db import clear_db_dags - -pytestmark = pytest.mark.db_test - - -@pytest.fixture(autouse=True) -def dags(create_dummy_dag): - paused_dag, _ = create_dummy_dag(dag_id="paused_dag", is_paused_upon_creation=True) - dag, _ = create_dummy_dag(dag_id="unpaused_dag") - - yield dag, paused_dag - - clear_db_dags() - - -def test_logging_pause_dag(admin_client, dags, session): - dag, _ = dags - # is_paused=false mean pause the dag - admin_client.post(f"/paused?is_paused=false&dag_id={dag.dag_id}", follow_redirects=True) - dag_query = session.query(Log).filter(Log.dag_id == dag.dag_id) - assert '{"is_paused": true}' in dag_query.first().extra - - -def test_logging_unpause_dag(admin_client, dags, session): - _, paused_dag = dags - # is_paused=true mean unpause the dag - admin_client.post(f"/paused?is_paused=true&dag_id={paused_dag.dag_id}", follow_redirects=True) - dag_query = session.query(Log).filter(Log.dag_id == paused_dag.dag_id) - assert '{"is_paused": false}' in dag_query.first().extra diff --git a/tests/www/views/test_views_pool.py b/tests/www/views/test_views_pool.py deleted file mode 100644 index 04c23a6e5eb3f..0000000000000 --- a/tests/www/views/test_views_pool.py +++ /dev/null @@ -1,143 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import flask -import markupsafe -import pytest - -from airflow.models import Pool -from airflow.utils.session import create_session - -from tests_common.test_utils.www import check_content_in_response, check_content_not_in_response - -pytestmark = pytest.mark.db_test - -POOL = { - "pool": "test-pool", - "slots": 777, - "description": "test-pool-description", - "include_deferred": False, -} - - -@pytest.fixture(autouse=True) -def _clear_pools(): - with create_session() as session: - session.query(Pool).delete() - - -@pytest.fixture -def pool_factory(session): - def factory(**values): - pool = Pool(**{**POOL, **values}) # Passed in values override defaults. - session.add(pool) - session.commit() - return pool - - return factory - - -def test_create_pool_with_same_name(admin_client): - # create test pool - resp = admin_client.post("/pool/add", data=POOL, follow_redirects=True) - check_content_in_response("Added Row", resp) - - # create pool with the same name - resp = admin_client.post("/pool/add", data=POOL, follow_redirects=True) - check_content_in_response("Already exists.", resp) - - -def test_create_pool_with_empty_name(admin_client): - resp = admin_client.post( - "/pool/add", - data={**POOL, "pool": ""}, - follow_redirects=True, - ) - check_content_in_response("This field is required.", resp) - - -def test_odd_name(admin_client, pool_factory): - pool_factory(pool="test-pool") - resp = admin_client.get("/pool/list/") - check_content_in_response("test-pool<script>", resp) - check_content_not_in_response("test-poolCancel', resp) - - -@pytest.mark.parametrize( - "request_conf, expected_conf", - [ - (None, {"example_key": "example_value"}), - ({"other": "test_data", "key": 12}, {"other": "test_data", "key": 12}), - ], -) -def test_trigger_dag_params_conf(admin_client, request_conf, expected_conf): - """ - Test that textarea in Trigger DAG UI is pre-populated - with json config when the conf URL parameter is passed, - or if a params dict is passed in the DAG - - 1. Conf is not included in URL parameters -> DAG.conf is in textarea - 2. Conf is passed as a URL parameter -> passed conf json is in textarea - """ - test_dag_id = "example_bash_operator" - doc_md = "Example Bash Operator" - - if not request_conf: - resp = admin_client.get(f"dags/{test_dag_id}/trigger") - else: - test_request_conf = json.dumps(request_conf, indent=4) - resp = admin_client.get(f"dags/{test_dag_id}/trigger?conf={test_request_conf}&doc_md={doc_md}") - for key in expected_conf.keys(): - check_content_in_response(key, resp) - check_content_in_response(str(expected_conf[key]), resp) - - -def test_trigger_dag_params_render(admin_client, dag_maker, session, app, monkeypatch): - """ - Test that textarea in Trigger DAG UI is pre-populated - with param value set in DAG. - """ - account = {"name": "account_name_1", "country": "usa"} - expected_conf = {"accounts": [account]} - expected_dag_conf = json.dumps(expected_conf, indent=4).replace('"', """) - DAG_ID = "params_dag" - param = Param( - [account], - schema={ - "type": "array", - "minItems": 1, - "items": { - "type": "object", - "default": account, - "properties": {"name": {"type": "string"}, "country": {"type": "string"}}, - "required": ["name", "country"], - }, - }, - ) - with monkeypatch.context() as m: - with dag_maker(dag_id=DAG_ID, serialized=True, session=session, params={"accounts": param}): - EmptyOperator(task_id="task1") - - m.setattr(app, "dag_bag", dag_maker.dagbag) - resp = admin_client.get(f"dags/{DAG_ID}/trigger") - - check_content_in_response( - f'', - resp, - ) - - -def test_trigger_endpoint_uses_existing_dagbag(admin_client): - """ - Test that Trigger Endpoint uses the DagBag already created in views.py - instead of creating a new one. - """ - url = "dags/example_bash_operator/trigger" - resp = admin_client.post(url, data={}, follow_redirects=True) - check_content_in_response("example_bash_operator", resp) - - -def test_trigger_dag_params_array_value_none_render(admin_client, dag_maker, session, app, monkeypatch): - """ - Test that textarea in Trigger DAG UI is pre-populated - with param value None and type ["null", "array"] set in DAG. - """ - expected_conf = {"dag_param": None} - expected_dag_conf = json.dumps(expected_conf, indent=4).replace('"', """) - DAG_ID = "params_dag" - param = Param( - None, - type=["null", "array"], - minItems=0, - ) - with monkeypatch.context() as m: - with dag_maker(dag_id=DAG_ID, serialized=True, session=session, params={"dag_param": param}): - EmptyOperator(task_id="task1") - - m.setattr(app, "dag_bag", dag_maker.dagbag) - resp = admin_client.get(f"dags/{DAG_ID}/trigger") - - check_content_in_response( - f'', - resp, - ) - - -@pytest.mark.parametrize( - "pattern, run_id, result", - [ - ["^[A-Z]", "ABC", True], - ["^[A-Z]", "abc", False], - ["^[0-9]", "123", True], - # The below params tests that user configuration does not affect internally generated - # run_ids. We use manual__ as a prefix for manually triggered DAGs due to a restriction - # in manually triggered DAGs that the run_id must not start with scheduled__. - ["", "manual__2023-01-01T00:00:00+00:00", True], - ["", "scheduled_2023-01-01T00", False], - ["", "manual_2023-01-01T00", False], - ["", "asset_triggered_2023-01-01T00", False], - ["^[0-9]", "manual__2023-01-01T00:00:00+00:00", True], - ["^[a-z]", "manual__2023-01-01T00:00:00+00:00", True], - ], -) -def test_dag_run_id_pattern(session, admin_client, pattern, run_id, result): - with conf_vars({("scheduler", "allowed_run_id_pattern"): pattern}): - test_dag_id = "example_bash_operator" - run_id = quote(run_id) - admin_client.post(f"dags/{test_dag_id}/trigger?run_id={run_id}", data={"conf": "{}"}) - run = session.query(DagRun).filter(DagRun.dag_id == test_dag_id).first() - if result: - assert run is not None - assert run.run_type == DagRunType.MANUAL - else: - assert run is None diff --git a/tests/www/views/test_views_variable.py b/tests/www/views/test_views_variable.py deleted file mode 100644 index 51d9ade5dcb1f..0000000000000 --- a/tests/www/views/test_views_variable.py +++ /dev/null @@ -1,227 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from io import BytesIO -from unittest import mock - -import pytest - -from airflow.models import Variable -from airflow.utils.session import create_session - -from tests_common.test_utils.www import ( - _check_last_log, - check_content_in_response, - check_content_not_in_response, -) - -pytestmark = pytest.mark.db_test -VARIABLE = { - "key": "test_key", - "val": "text_val", - "description": "test_description", - "is_encrypted": True, -} - - -@pytest.fixture(autouse=True) -def _clear_variables(): - with create_session() as session: - session.query(Variable).delete() - - -def test_can_handle_error_on_decrypt(session, admin_client): - # create valid variable - admin_client.post("/variable/add", data=VARIABLE, follow_redirects=True) - - # update the variable with a wrong value, given that is encrypted - session.query(Variable).filter(Variable.key == VARIABLE["key"]).update( - {"val": "failed_value_not_encrypted"}, - synchronize_session=False, - ) - session.commit() - - # retrieve Variables page, should not fail and contain the Invalid - # label for the variable - resp = admin_client.get("/variable/list", follow_redirects=True) - check_content_in_response( - 'Invalid', - resp, - ) - - -def test_xss_prevention(admin_client): - xss = "/variable/list/" - resp = admin_client.get(xss, follow_redirects=True) - check_content_not_in_response("", resp, resp_code=404) - - -def test_import_variables_no_file(admin_client): - resp = admin_client.post("/variable/varimport", follow_redirects=True) - check_content_in_response("Missing file or syntax error.", resp) - - -def test_import_variables_failed(session, admin_client): - content = '{"str_key": "str_value"}' - - with mock.patch("airflow.models.Variable.set") as set_mock: - set_mock.side_effect = UnicodeEncodeError - assert session.query(Variable).count() == 0 - - bytes_content = BytesIO(bytes(content, encoding="utf-8")) - - resp = admin_client.post( - "/variable/varimport", data={"file": (bytes_content, "test.json")}, follow_redirects=True - ) - check_content_in_response("1 variable(s) failed to be updated.", resp) - - -def test_import_variables_success(session, admin_client): - assert session.query(Variable).count() == 0 - - content = '{"str_key": "str_value", "int_key": 60, "list_key": [1, 2], "dict_key": {"k_a": 2, "k_b": 3}}' - bytes_content = BytesIO(bytes(content, encoding="utf-8")) - - resp = admin_client.post( - "/variable/varimport", data={"file": (bytes_content, "test.json")}, follow_redirects=True - ) - check_content_in_response("4 variable(s) successfully updated.", resp) - _check_last_log(session, dag_id=None, event="variables.varimport", logical_date=None) - - -def test_import_variables_override_existing_variables_if_set(session, admin_client, caplog): - assert session.query(Variable).count() == 0 - Variable.set("str_key", "str_value") - content = '{"str_key": "str_value", "int_key": 60}' # str_key already exists - bytes_content = BytesIO(bytes(content, encoding="utf-8")) - - resp = admin_client.post( - "/variable/varimport", - data={"file": (bytes_content, "test.json"), "action_if_exist": "overwrite"}, - follow_redirects=True, - ) - check_content_in_response("2 variable(s) successfully updated.", resp) - _check_last_log(session, dag_id=None, event="variables.varimport", logical_date=None) - - -def test_import_variables_skips_update_if_set(session, admin_client, caplog): - assert session.query(Variable).count() == 0 - Variable.set("str_key", "str_value") - content = '{"str_key": "str_value", "int_key": 60}' # str_key already exists - bytes_content = BytesIO(bytes(content, encoding="utf-8")) - - resp = admin_client.post( - "/variable/varimport", - data={"file": (bytes_content, "test.json"), "action_if_exists": "skip"}, - follow_redirects=True, - ) - check_content_in_response("1 variable(s) successfully updated.", resp) - - check_content_in_response( - "The variables with these keys: 'str_key' were skipped because they already exists", resp - ) - _check_last_log(session, dag_id=None, event="variables.varimport", logical_date=None) - assert "Variable: str_key already exists, skipping." in caplog.text - - -def test_import_variables_fails_if_action_if_exists_is_fail(session, admin_client, caplog): - assert session.query(Variable).count() == 0 - Variable.set("str_key", "str_value") - content = '{"str_key": "str_value", "int_key": 60}' # str_key already exists - bytes_content = BytesIO(bytes(content, encoding="utf-8")) - - admin_client.post( - "/variable/varimport", - data={"file": (bytes_content, "test.json"), "action_if_exists": "fail"}, - follow_redirects=True, - ) - assert "Failed. The variables with these keys: 'str_key' already exists." in caplog.text - - -def test_import_variables_anon(session, app): - assert session.query(Variable).count() == 0 - - content = '{"str_key": "str_value}' - bytes_content = BytesIO(bytes(content, encoding="utf-8")) - - resp = app.test_client().post( - "/variable/varimport", data={"file": (bytes_content, "test.json")}, follow_redirects=True - ) - check_content_not_in_response("variable(s) successfully updated.", resp) - check_content_in_response("Sign In", resp) - - -def test_import_variables_access_denied(session, app, viewer_client): - content = '{"str_key": "str_value}' - bytes_content = BytesIO(bytes(content, encoding="utf-8")) - - resp = viewer_client.post( - "/variable/varimport", data={"file": (bytes_content, "test.json")}, follow_redirects=True - ) - check_content_in_response("Access is Denied", resp) - - -def test_import_variables_form_shown(app, admin_client): - resp = admin_client.get("/variable/list/") - check_content_in_response("Import Variables", resp) - - -def test_description_retrieval(session, admin_client): - # create valid variable - admin_client.post("/variable/add", data=VARIABLE, follow_redirects=True) - - row = session.query(Variable.key, Variable.description).first() - assert row.key == "test_key" - assert row.description == "test_description" - - -@pytest.fixture -def variable(session): - variable = Variable( - key=VARIABLE["key"], - val=VARIABLE["val"], - description=VARIABLE["description"], - ) - session.add(variable) - session.commit() - yield variable - session.query(Variable).filter(Variable.key == VARIABLE["key"]).delete() - session.commit() - - -def test_action_export(admin_client, variable): - resp = admin_client.post( - "/variable/action_post", - data={"action": "varexport", "rowid": [variable.id]}, - ) - assert resp.status_code == 200 - assert resp.headers["Content-Type"] == "application/json; charset=utf-8" - assert resp.headers["Content-Disposition"] == "attachment; filename=variables.json" - assert resp.json == {"test_key": "text_val"} - - -def test_action_muldelete(session, admin_client, variable): - var_id = variable.id - resp = admin_client.post( - "/variable/action_post", - data={"action": "muldelete", "rowid": [var_id]}, - follow_redirects=True, - ) - assert resp.status_code == 200 - assert session.query(Variable).filter(Variable.id == var_id).count() == 0 diff --git a/tests_common/pytest_plugin.py b/tests_common/pytest_plugin.py index cf6e5d5a44f33..de7e409fdc734 100644 --- a/tests_common/pytest_plugin.py +++ b/tests_common/pytest_plugin.py @@ -1664,16 +1664,6 @@ def clean_executor_loader(): ExecutorLoader.init_executors() -@pytest.fixture(scope="session") -def app(): - from tests_common.test_utils.config import conf_vars - - with conf_vars({("fab", "auth_rate_limited"): "False"}): - from airflow.www import app - - yield app.create_app(testing=True) - - @pytest.fixture def secret_key() -> str: """Return secret key configured.""" diff --git a/tests_common/test_utils/db.py b/tests_common/test_utils/db.py index 58ad46372d711..dbed341afb93d 100644 --- a/tests_common/test_utils/db.py +++ b/tests_common/test_utils/db.py @@ -85,7 +85,6 @@ def initial_db_init(): from airflow.configuration import conf from airflow.utils import db - from airflow.www.extensions.init_appbuilder import init_appbuilder from tests_common.test_utils.version_compat import AIRFLOW_V_3_0_PLUS @@ -93,18 +92,18 @@ def initial_db_init(): if AIRFLOW_V_3_0_PLUS: db.downgrade(to_revision="5f2621c13b39") db.upgradedb(to_revision="head") - _bootstrap_dagbag() - # minimal app to add roles - flask_app = Flask(__name__) - flask_app.config["SQLALCHEMY_DATABASE_URI"] = conf.get("database", "SQL_ALCHEMY_CONN") - init_appbuilder(flask_app) - - if AIRFLOW_V_3_0_PLUS: - from airflow.api_fastapi.app import get_auth_manager else: + from airflow.www.extensions.init_appbuilder import init_appbuilder from airflow.www.extensions.init_auth_manager import get_auth_manager - get_auth_manager().init() + # minimal app to add roles + flask_app = Flask(__name__) + flask_app.config["SQLALCHEMY_DATABASE_URI"] = conf.get("database", "SQL_ALCHEMY_CONN") + init_appbuilder(flask_app) + + get_auth_manager().init() + + _bootstrap_dagbag() def parse_and_sync_to_db(folder: Path | str, include_examples: bool = False): diff --git a/tests_common/test_utils/logs.py b/tests_common/test_utils/logs.py new file mode 100644 index 0000000000000..03cbe0b55b99a --- /dev/null +++ b/tests_common/test_utils/logs.py @@ -0,0 +1,71 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import json + +from airflow.models import Log +from airflow.sdk.execution_time.secrets_masker import DEFAULT_SENSITIVE_FIELDS + + +def _masked_value_check(data, sensitive_fields): + """ + Recursively check if sensitive fields are properly masked. + + :param data: JSON object (dict, list, or value) + :param sensitive_fields: Set of sensitive field names + """ + if isinstance(data, dict): + for key, value in data.items(): + if key in sensitive_fields: + assert value == "***", f"Expected masked value for {key}, but got {value}" + else: + _masked_value_check(value, sensitive_fields) + elif isinstance(data, list): + for item in data: + _masked_value_check(item, sensitive_fields) + + +def check_last_log(session, dag_id, event, logical_date, expected_extra=None, check_masked=False): + logs = ( + session.query( + Log.dag_id, + Log.task_id, + Log.event, + Log.logical_date, + Log.owner, + Log.extra, + ) + .filter( + Log.dag_id == dag_id, + Log.event == event, + Log.logical_date == logical_date, + ) + .order_by(Log.dttm.desc()) + .limit(5) + .all() + ) + assert len(logs) >= 1 + assert logs[0].extra + if expected_extra: + assert json.loads(logs[0].extra) == expected_extra + if check_masked: + extra_json = json.loads(logs[0].extra) + _masked_value_check(extra_json, DEFAULT_SENSITIVE_FIELDS) + + session.query(Log).delete()