'''
+ == rendered
+ )
+
+ def test_wrapped_markdown_with_collapsible_section(self):
+ rendered = wrapped_markdown(
+ """
+# A collapsible section with markdown
+
+ Click to expand!
+
+ ## Heading
+ 1. A numbered
+ 2. list
+ * With some
+ * Sub bullets
+
+ """
+ )
+
+ assert (
+ '''
A collapsible section with markdown
+
+ Click to expand!
+
Heading
+
+
A numbered
+
list
+
+
With some
+
Sub bullets
+
+
+
+
+
'''
+ == rendered
+ )
From 86c4540dedef05f0fd751f79feac5efd86cfbec3 Mon Sep 17 00:00:00 2001
From: Jarek Potiuk
Date: Sun, 5 Jun 2022 11:51:04 +0200
Subject: [PATCH 007/118] Remove warnings when starting breeze (#24183)
Breeze when started produced three warnings that were harmless,
but we should fix them to remove "false positives".
(cherry picked from commit ac8a790ab9dabfa747964b65aa9b93451b735ede)
---
dev/breeze/src/airflow_breeze/breeze.py | 6 ++++-
.../commands/developer_commands.py | 3 ++-
.../airflow_breeze/commands/main_command.py | 2 --
.../airflow_breeze/params/build_ci_params.py | 1 +
.../params/build_prod_params.py | 3 ++-
.../src/airflow_breeze/params/shell_params.py | 5 ++++
.../utils/docker_command_utils.py | 16 +++--------
.../src/airflow_breeze/utils/path_utils.py | 14 ++++++++++
scripts/ci/docker-compose/local.yml | 7 +----
scripts/ci/docker-compose/mypy.yml | 27 +++++++++++++++++++
scripts/ci/pre_commit/pre_commit_flake8.py | 3 ++-
.../pre_commit_migration_reference.py | 3 ++-
scripts/ci/pre_commit/pre_commit_mypy.py | 3 ++-
scripts/ci/pre_commit/pre_commit_ui_lint.py | 3 ++-
scripts/ci/pre_commit/pre_commit_www_lint.py | 3 ++-
15 files changed, 71 insertions(+), 28 deletions(-)
create mode 100644 scripts/ci/docker-compose/mypy.yml
diff --git a/dev/breeze/src/airflow_breeze/breeze.py b/dev/breeze/src/airflow_breeze/breeze.py
index a596c2be4967e..608619ae8e487 100755
--- a/dev/breeze/src/airflow_breeze/breeze.py
+++ b/dev/breeze/src/airflow_breeze/breeze.py
@@ -17,9 +17,13 @@
# under the License.
from airflow_breeze.configure_rich_click import click # isort: skip # noqa
from airflow_breeze.commands.main_command import main
-from airflow_breeze.utils.path_utils import find_airflow_sources_root_to_operate_on
+from airflow_breeze.utils.path_utils import (
+ create_directories_and_files,
+ find_airflow_sources_root_to_operate_on,
+)
find_airflow_sources_root_to_operate_on()
+create_directories_and_files()
if __name__ == '__main__':
main()
diff --git a/dev/breeze/src/airflow_breeze/commands/developer_commands.py b/dev/breeze/src/airflow_breeze/commands/developer_commands.py
index 1719bab5fc7a8..e10e79d0a8126 100644
--- a/dev/breeze/src/airflow_breeze/commands/developer_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/developer_commands.py
@@ -524,7 +524,7 @@ def stop(verbose: bool, dry_run: bool, preserve_volumes: bool):
command_to_execute = ['docker-compose', 'down', "--remove-orphans"]
if not preserve_volumes:
command_to_execute.append("--volumes")
- shell_params = ShellParams(verbose=verbose, backend="all")
+ shell_params = ShellParams(verbose=verbose, backend="all", include_mypy_volume=True)
env_variables = get_env_variables_for_docker_commands(shell_params)
run_command(command_to_execute, verbose=verbose, dry_run=dry_run, env=env_variables)
@@ -579,6 +579,7 @@ def enter_shell(**kwargs) -> RunCommandResult:
if read_from_cache_file('suppress_cheatsheet') is None:
get_console().print(CHEATSHEET, style=CHEATSHEET_STYLE)
enter_shell_params = ShellParams(**filter_out_none(**kwargs))
+ enter_shell_params.include_mypy_volume = True
rebuild_ci_image_if_needed(build_params=enter_shell_params, dry_run=dry_run, verbose=verbose)
return run_shell(verbose, dry_run, enter_shell_params)
diff --git a/dev/breeze/src/airflow_breeze/commands/main_command.py b/dev/breeze/src/airflow_breeze/commands/main_command.py
index 8ed451b0e5073..c62101a44f440 100644
--- a/dev/breeze/src/airflow_breeze/commands/main_command.py
+++ b/dev/breeze/src/airflow_breeze/commands/main_command.py
@@ -37,7 +37,6 @@
option_use_packages_from_dist,
option_verbose,
)
-from airflow_breeze.utils.path_utils import create_directories_and_files
@click.group(invoke_without_command=True, context_settings={'help_option_names': ['-h', '--help']})
@@ -64,6 +63,5 @@
def main(ctx: click.Context, **kwargs):
from airflow_breeze.commands.developer_commands import shell
- create_directories_and_files()
if not ctx.invoked_subcommand:
ctx.forward(shell, extra_args={})
diff --git a/dev/breeze/src/airflow_breeze/params/build_ci_params.py b/dev/breeze/src/airflow_breeze/params/build_ci_params.py
index aa9681fc955fe..52c242b65a77d 100644
--- a/dev/breeze/src/airflow_breeze/params/build_ci_params.py
+++ b/dev/breeze/src/airflow_breeze/params/build_ci_params.py
@@ -31,6 +31,7 @@ class BuildCiParams(CommonBuildParams):
"""
airflow_constraints_mode: str = "constraints-source-providers"
+ default_constraints_branch: str = DEFAULT_AIRFLOW_CONSTRAINTS_BRANCH
airflow_constraints_reference: str = DEFAULT_AIRFLOW_CONSTRAINTS_BRANCH
airflow_extras: str = "devel_ci"
airflow_pre_cached_pip_packages: bool = True
diff --git a/dev/breeze/src/airflow_breeze/params/build_prod_params.py b/dev/breeze/src/airflow_breeze/params/build_prod_params.py
index e81a5d6739ac8..c37569f85f136 100644
--- a/dev/breeze/src/airflow_breeze/params/build_prod_params.py
+++ b/dev/breeze/src/airflow_breeze/params/build_prod_params.py
@@ -21,7 +21,7 @@
from dataclasses import dataclass
from typing import List
-from airflow_breeze.branch_defaults import AIRFLOW_BRANCH
+from airflow_breeze.branch_defaults import AIRFLOW_BRANCH, DEFAULT_AIRFLOW_CONSTRAINTS_BRANCH
from airflow_breeze.global_constants import (
AIRFLOW_SOURCES_FROM,
AIRFLOW_SOURCES_TO,
@@ -41,6 +41,7 @@ class BuildProdParams(CommonBuildParams):
"""
airflow_constraints_mode: str = "constraints"
+ default_constraints_branch: str = DEFAULT_AIRFLOW_CONSTRAINTS_BRANCH
airflow_constraints_reference: str = ""
airflow_is_in_context: bool = False
cleanup_context: bool = False
diff --git a/dev/breeze/src/airflow_breeze/params/shell_params.py b/dev/breeze/src/airflow_breeze/params/shell_params.py
index 58107b9e27567..4ba4022651529 100644
--- a/dev/breeze/src/airflow_breeze/params/shell_params.py
+++ b/dev/breeze/src/airflow_breeze/params/shell_params.py
@@ -48,6 +48,7 @@ class ShellParams:
"""
airflow_branch: str = AIRFLOW_BRANCH
+ default_constraints_branch: str = DEFAULT_AIRFLOW_CONSTRAINTS_BRANCH
airflow_constraints_reference: str = DEFAULT_AIRFLOW_CONSTRAINTS_BRANCH
airflow_extras: str = ""
answer: Optional[str] = None
@@ -64,6 +65,7 @@ class ShellParams:
github_repository: str = "apache/airflow"
github_token: str = os.environ.get('GITHUB_TOKEN', "")
image_tag: str = "latest"
+ include_mypy_volume: bool = False
install_airflow_version: str = ""
install_providers_from_sources: bool = True
integration: Tuple[str, ...] = ()
@@ -189,6 +191,7 @@ def compose_files(self):
local_all_sources_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/local-all-sources.yml"
files_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/files.yml"
remove_sources_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/remove-sources.yml"
+ mypy_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/mypy.yml"
forward_credentials_docker_compose_file = (
f"{str(SCRIPTS_CI_DIR)}/docker-compose/forward-credentials.yml"
)
@@ -213,6 +216,8 @@ def compose_files(self):
compose_ci_file.append(forward_credentials_docker_compose_file)
if self.use_airflow_version is not None:
compose_ci_file.append(remove_sources_docker_compose_file)
+ if self.include_mypy_volume:
+ compose_ci_file.append(mypy_docker_compose_file)
if "all" in self.integration:
integrations = AVAILABLE_INTEGRATIONS
else:
diff --git a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
index 05bcc11d6cb87..1f30642edd6b0 100644
--- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
@@ -17,7 +17,6 @@
"""Various utils to prepare docker and docker compose commands."""
import os
import re
-import subprocess
import sys
from copy import deepcopy
from random import randint
@@ -97,16 +96,6 @@
]
-def create_volume_if_missing(volume_name: str):
- res_inspect = run_command(cmd=["docker", "inspect", volume_name], stdout=subprocess.DEVNULL, check=False)
- if res_inspect.returncode != 0:
- run_command(cmd=["docker", "volume", "create", volume_name], check=True)
-
-
-def create_static_check_volumes():
- create_volume_if_missing("docker-compose_mypy-cache-volume")
-
-
def get_extra_docker_flags(mount_sources: str) -> List[str]:
"""
Returns extra docker flags based on the type of mounting we want to do for sources.
@@ -124,7 +113,7 @@ def get_extra_docker_flags(mount_sources: str) -> List[str]:
["--mount", f'type=bind,src={AIRFLOW_SOURCES_ROOT / src},dst={dst}']
)
extra_docker_flags.extend(
- ['--mount', "type=volume,src=docker-compose_mypy-cache-volume,dst=/opt/airflow/.mypy_cache"]
+ ['--mount', "type=volume,src=mypy-cache-volume,dst=/opt/airflow/.mypy_cache"]
)
else: # none
extra_docker_flags.extend(
@@ -512,6 +501,7 @@ def update_expected_environment_variables(env: Dict[str, str]) -> None:
:param env: environment variables to update with missing values if not set.
"""
set_value_to_default_if_not_set(env, 'AIRFLOW_CONSTRAINTS_MODE', "constraints-source-providers")
+ set_value_to_default_if_not_set(env, 'AIRFLOW_CONSTRAINTS_REFERENCE', "constraints-source-providers")
set_value_to_default_if_not_set(env, 'AIRFLOW_EXTRAS', "")
set_value_to_default_if_not_set(env, 'ANSWER', "")
set_value_to_default_if_not_set(env, 'BREEZE', "true")
@@ -557,7 +547,9 @@ def update_expected_environment_variables(env: Dict[str, str]) -> None:
"AIRFLOW_CI_IMAGE": "airflow_image_name",
"AIRFLOW_CI_IMAGE_WITH_TAG": "airflow_image_name_with_tag",
"AIRFLOW_EXTRAS": "airflow_extras",
+ "DEFAULT_CONSTRAINTS_BRANCH": "default-constraints-branch",
"AIRFLOW_CONSTRAINTS_MODE": "airflow_constraints_mode",
+ "AIRFLOW_CONSTRAINTS_REFERENCE": "airflow_constraints_reference",
"AIRFLOW_IMAGE_KUBERNETES": "airflow_image_kubernetes",
"AIRFLOW_PROD_IMAGE": "airflow_image_name",
"AIRFLOW_SOURCES": "airflow_sources",
diff --git a/dev/breeze/src/airflow_breeze/utils/path_utils.py b/dev/breeze/src/airflow_breeze/utils/path_utils.py
index 759c3e09a7d6a..e4e79d55202ae 100644
--- a/dev/breeze/src/airflow_breeze/utils/path_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/path_utils.py
@@ -19,6 +19,7 @@
"""
import hashlib
import os
+import subprocess
import sys
import tempfile
from functools import lru_cache
@@ -249,6 +250,18 @@ def find_airflow_sources_root_to_operate_on() -> Path:
BREEZE_SOURCES_ROOT = AIRFLOW_SOURCES_ROOT / "dev" / "breeze"
+def create_volume_if_missing(volume_name: str):
+ from airflow_breeze.utils.run_utils import run_command
+
+ res_inspect = run_command(cmd=["docker", "inspect", volume_name], stdout=subprocess.DEVNULL, check=False)
+ if res_inspect.returncode != 0:
+ run_command(cmd=["docker", "volume", "create", volume_name], check=True)
+
+
+def create_static_check_volumes():
+ create_volume_if_missing("mypy-cache-volume")
+
+
def create_directories_and_files() -> None:
"""
Creates all directories and files that are needed for Breeze to work via docker-compose.
@@ -264,3 +277,4 @@ def create_directories_and_files() -> None:
(AIRFLOW_SOURCES_ROOT / ".bash_aliases").touch()
(AIRFLOW_SOURCES_ROOT / ".bash_history").touch()
(AIRFLOW_SOURCES_ROOT / ".inputrc").touch()
+ create_static_check_volumes()
diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml
index e356cb873611f..9e63ffa467d07 100644
--- a/scripts/ci/docker-compose/local.yml
+++ b/scripts/ci/docker-compose/local.yml
@@ -26,10 +26,7 @@ services:
# or those that might be useful to see in the host as output of the
# tests (such as logs)
volumes:
- - type: volume
- source: mypy-cache-volume
- target: /opt/airflow/.mypy_cache/
- # START automatically generated volumes from NECESSARY_HOST_VOLUMES in docker_command_utils.py
+ # START automatically generated volumes from NECESSARY_HOST_VOLUMES in docker_command_utils.py
- type: bind
source: ../../../.bash_aliases
target: /root/.bash_aliases
@@ -121,5 +118,3 @@ services:
source: ../../../metastore_browser
target: /opt/airflow/metastore_browser
# END automatically generated volumes from NECESSARY_HOST_VOLUMES in docker_command_utils.py
-volumes:
- mypy-cache-volume:
diff --git a/scripts/ci/docker-compose/mypy.yml b/scripts/ci/docker-compose/mypy.yml
new file mode 100644
index 0000000000000..7287738690239
--- /dev/null
+++ b/scripts/ci/docker-compose/mypy.yml
@@ -0,0 +1,27 @@
+# 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.
+---
+version: "3.7"
+services:
+ airflow:
+ volumes:
+ - type: volume
+ source: mypy-cache-volume
+ target: /opt/airflow/.mypy_cache/
+volumes:
+ mypy-cache-volume:
+ external: true
diff --git a/scripts/ci/pre_commit/pre_commit_flake8.py b/scripts/ci/pre_commit/pre_commit_flake8.py
index 59bd356d24df0..22b09978935b2 100755
--- a/scripts/ci/pre_commit/pre_commit_flake8.py
+++ b/scripts/ci/pre_commit/pre_commit_flake8.py
@@ -34,7 +34,8 @@
if __name__ == '__main__':
sys.path.insert(0, str(AIRFLOW_SOURCES / "dev" / "breeze" / "src"))
from airflow_breeze.global_constants import MOUNT_SELECTED
- from airflow_breeze.utils.docker_command_utils import create_static_check_volumes, get_extra_docker_flags
+ from airflow_breeze.utils.docker_command_utils import get_extra_docker_flags
+ from airflow_breeze.utils.path_utils import create_static_check_volumes
from airflow_breeze.utils.run_utils import get_runnable_ci_image, run_command
airflow_image = get_runnable_ci_image(verbose=VERBOSE, dry_run=DRY_RUN)
diff --git a/scripts/ci/pre_commit/pre_commit_migration_reference.py b/scripts/ci/pre_commit/pre_commit_migration_reference.py
index 154a795f3e84a..25ac796fdf4fc 100755
--- a/scripts/ci/pre_commit/pre_commit_migration_reference.py
+++ b/scripts/ci/pre_commit/pre_commit_migration_reference.py
@@ -34,7 +34,8 @@
if __name__ == '__main__':
sys.path.insert(0, str(AIRFLOW_SOURCES / "dev" / "breeze" / "src"))
from airflow_breeze.global_constants import MOUNT_SELECTED
- from airflow_breeze.utils.docker_command_utils import create_static_check_volumes, get_extra_docker_flags
+ from airflow_breeze.utils.docker_command_utils import get_extra_docker_flags
+ from airflow_breeze.utils.path_utils import create_static_check_volumes
from airflow_breeze.utils.run_utils import get_runnable_ci_image, run_command
airflow_image = get_runnable_ci_image(verbose=VERBOSE, dry_run=DRY_RUN)
diff --git a/scripts/ci/pre_commit/pre_commit_mypy.py b/scripts/ci/pre_commit/pre_commit_mypy.py
index 74d511a65b7ed..965bed9a74d3c 100755
--- a/scripts/ci/pre_commit/pre_commit_mypy.py
+++ b/scripts/ci/pre_commit/pre_commit_mypy.py
@@ -35,7 +35,8 @@
if __name__ == '__main__':
sys.path.insert(0, str(AIRFLOW_SOURCES / "dev" / "breeze" / "src"))
from airflow_breeze.global_constants import MOUNT_SELECTED
- from airflow_breeze.utils.docker_command_utils import create_static_check_volumes, get_extra_docker_flags
+ from airflow_breeze.utils.docker_command_utils import get_extra_docker_flags
+ from airflow_breeze.utils.path_utils import create_static_check_volumes
from airflow_breeze.utils.run_utils import get_runnable_ci_image, run_command
airflow_image = get_runnable_ci_image(verbose=VERBOSE, dry_run=DRY_RUN)
diff --git a/scripts/ci/pre_commit/pre_commit_ui_lint.py b/scripts/ci/pre_commit/pre_commit_ui_lint.py
index 7755e3d204bd8..5812344a45961 100755
--- a/scripts/ci/pre_commit/pre_commit_ui_lint.py
+++ b/scripts/ci/pre_commit/pre_commit_ui_lint.py
@@ -33,7 +33,8 @@
if __name__ == '__main__':
sys.path.insert(0, str(AIRFLOW_SOURCES / "dev" / "breeze" / "src"))
from airflow_breeze.global_constants import MOUNT_SELECTED
- from airflow_breeze.utils.docker_command_utils import create_static_check_volumes, get_extra_docker_flags
+ from airflow_breeze.utils.docker_command_utils import get_extra_docker_flags
+ from airflow_breeze.utils.path_utils import create_static_check_volumes
from airflow_breeze.utils.run_utils import get_runnable_ci_image, run_command
airflow_image = get_runnable_ci_image(verbose=VERBOSE, dry_run=DRY_RUN)
diff --git a/scripts/ci/pre_commit/pre_commit_www_lint.py b/scripts/ci/pre_commit/pre_commit_www_lint.py
index 65553c3d81345..b22750c56d07a 100755
--- a/scripts/ci/pre_commit/pre_commit_www_lint.py
+++ b/scripts/ci/pre_commit/pre_commit_www_lint.py
@@ -34,7 +34,8 @@
if __name__ == '__main__':
sys.path.insert(0, str(AIRFLOW_SOURCES / "dev" / "breeze" / "src"))
from airflow_breeze.global_constants import MOUNT_SELECTED
- from airflow_breeze.utils.docker_command_utils import create_static_check_volumes, get_extra_docker_flags
+ from airflow_breeze.utils.docker_command_utils import get_extra_docker_flags
+ from airflow_breeze.utils.path_utils import create_static_check_volumes
from airflow_breeze.utils.run_utils import get_runnable_ci_image, run_command
airflow_image = get_runnable_ci_image(verbose=VERBOSE, dry_run=DRY_RUN)
From d7989b60f622775fc1695e6be6ba1e0849d9581d Mon Sep 17 00:00:00 2001
From: Jarek Potiuk
Date: Tue, 7 Jun 2022 08:48:23 +0200
Subject: [PATCH 008/118] Fix choosing backend versions in breeze's command
line (#24228)
Choosing version of backend were broken when command line switches
were used. The _VERSION variables were "hard-coded" to defaults
rather than taken from command line. This is a remnant of initial
implementation and converting the parameters to "cacheable" ones.
While looking at the versions we also found that PARAM_NAME_FLAG
is not used any more so we took the opportunity to remove it.
(cherry picked from commit 4482eaefb02535317bc0d2378eba6bfcef3c45ff)
---
dev/breeze/src/airflow_breeze/global_constants.py | 15 ---------------
.../airflow_breeze/utils/docker_command_utils.py | 9 +++------
2 files changed, 3 insertions(+), 21 deletions(-)
diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py
index 4ed4c298ba230..327740f7bf12b 100644
--- a/dev/breeze/src/airflow_breeze/global_constants.py
+++ b/dev/breeze/src/airflow_breeze/global_constants.py
@@ -102,18 +102,6 @@
"MSSQL_VERSION": "MSSql version",
}
-PARAM_NAME_FLAG = {
- "BACKEND": "--backend",
- "MYSQL_VERSION": "--mysql-version",
- "KUBERNETES_MODE": "--kubernetes-mode",
- "KUBERNETES_VERSION": "--kubernetes-version",
- "KIND_VERSION": "--kind-version",
- "HELM_VERSION": "--helm-version",
- "EXECUTOR": "--executor",
- "POSTGRES_VERSION": "--postgres-version",
- "MSSQL_VERSION": "--mssql-version",
-}
-
EXCLUDE_DOCS_PACKAGE_FOLDER = [
'exts',
'integration-logos',
@@ -158,9 +146,6 @@ def get_available_packages(short_version=False) -> List[str]:
CURRENT_POSTGRES_VERSIONS = ['10', '11', '12', '13', '14']
CURRENT_MYSQL_VERSIONS = ['5.7', '8']
CURRENT_MSSQL_VERSIONS = ['2017-latest', '2019-latest']
-POSTGRES_VERSION = CURRENT_POSTGRES_VERSIONS[0]
-MYSQL_VERSION = CURRENT_MYSQL_VERSIONS[0]
-MSSQL_VERSION = CURRENT_MSSQL_VERSIONS[0]
DB_RESET = False
START_AIRFLOW = "false"
LOAD_EXAMPLES = False
diff --git a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
index 1f30642edd6b0..fb38cefabacb4 100644
--- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
@@ -45,11 +45,8 @@
MOUNT_ALL,
MOUNT_SELECTED,
MSSQL_HOST_PORT,
- MSSQL_VERSION,
MYSQL_HOST_PORT,
- MYSQL_VERSION,
POSTGRES_HOST_PORT,
- POSTGRES_VERSION,
REDIS_HOST_PORT,
SSH_PORT,
WEBSERVER_HOST_PORT,
@@ -565,9 +562,12 @@ def update_expected_environment_variables(env: Dict[str, str]) -> None:
"ISSUE_ID": "issue_id",
"LOAD_EXAMPLES": "load_example_dags",
"LOAD_DEFAULT_CONNECTIONS": "load_default_connections",
+ "MYSQL_VERSION": "mysql_version",
+ "MSSQL_VERSION": "mssql_version",
"NUM_RUNS": "num_runs",
"PACKAGE_FORMAT": "package_format",
"PYTHON_MAJOR_MINOR_VERSION": "python",
+ "POSTGRES_VERSION": "postgres_version",
"SQLITE_URL": "sqlite_url",
"START_AIRFLOW": "start_airflow",
"SKIP_ENVIRONMENT_INITIALIZATION": "skip_environment_initialization",
@@ -579,11 +579,8 @@ def update_expected_environment_variables(env: Dict[str, str]) -> None:
DOCKER_VARIABLE_CONSTANTS = {
"FLOWER_HOST_PORT": FLOWER_HOST_PORT,
"MSSQL_HOST_PORT": MSSQL_HOST_PORT,
- "MSSQL_VERSION": MSSQL_VERSION,
"MYSQL_HOST_PORT": MYSQL_HOST_PORT,
- "MYSQL_VERSION": MYSQL_VERSION,
"POSTGRES_HOST_PORT": POSTGRES_HOST_PORT,
- "POSTGRES_VERSION": POSTGRES_VERSION,
"REDIS_HOST_PORT": REDIS_HOST_PORT,
"SSH_PORT": SSH_PORT,
"WEBSERVER_HOST_PORT": WEBSERVER_HOST_PORT,
From 797fce76e3c870b08637a5c9426b00964fe623fb Mon Sep 17 00:00:00 2001
From: Jarek Potiuk
Date: Tue, 7 Jun 2022 11:29:10 +0200
Subject: [PATCH 009/118] Add command to regenerate breeze command output
images (#24216)
(cherry picked from commit 2d687e4b7b39fc5c133440de7f66f58327d33db7)
---
.gitignore | 4 +
BREEZE.rst | 11 +
.../configuration_and_maintenance_commands.py | 27 +-
images/breeze/output-commands-hash.txt | 2 +-
images/breeze/output-commands.svg | 364 +++++++++---------
.../output-regenerate-command-images.svg | 99 +++++
6 files changed, 327 insertions(+), 180 deletions(-)
create mode 100644 images/breeze/output-regenerate-command-images.svg
diff --git a/.gitignore b/.gitignore
index 9a00d53fa3bda..fe3d74aa2344a 100644
--- a/.gitignore
+++ b/.gitignore
@@ -225,3 +225,7 @@ licenses/LICENSES-ui.txt
# Packaged breeze on Windows
/breeze.exe
+
+# Generated out dir
+
+/out
diff --git a/BREEZE.rst b/BREEZE.rst
index 1ddd32dfbefce..36b10759d3444 100644
--- a/BREEZE.rst
+++ b/BREEZE.rst
@@ -426,6 +426,17 @@ of help of the commands only when they change.
:width: 100%
:alt: Breeze command-hash-export
+Regenerating images for documentation
+=====================================
+
+This documentation contains exported images with "help" of their commands and parameters. You can
+regenerate all those images (which might be needed in case new version of rich is used) via
+``regenerate-breeze-images`` command.
+
+.. image:: ./images/breeze/output-regenerate-command-images.svg
+ :width: 100%
+ :alt: Breeze regenerate-command-images
+
Starting complete Airflow installation
======================================
diff --git a/dev/breeze/src/airflow_breeze/commands/configuration_and_maintenance_commands.py b/dev/breeze/src/airflow_breeze/commands/configuration_and_maintenance_commands.py
index 818ba49ee8b83..d4ca3bcf466ca 100644
--- a/dev/breeze/src/airflow_breeze/commands/configuration_and_maintenance_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/configuration_and_maintenance_commands.py
@@ -62,7 +62,7 @@
)
from airflow_breeze.utils.recording import output_file_for_recording
from airflow_breeze.utils.reinstall import ask_to_reinstall_breeze, reinstall_breeze, warn_non_editable
-from airflow_breeze.utils.run_utils import run_command
+from airflow_breeze.utils.run_utils import assert_pre_commit_installed, run_command
from airflow_breeze.utils.visuals import ASCIIART, ASCIIART_STYLE
CONFIGURATION_AND_MAINTENANCE_COMMANDS = {
@@ -75,6 +75,7 @@
"resource-check",
"free-space",
"fix-ownership",
+ "regenerate-command-images",
"command-hash-export",
"version",
],
@@ -533,3 +534,27 @@ def remove_autogenerated_code(script_path: str):
def backup(script_path_file: Path):
shutil.copy(str(script_path_file), str(script_path_file) + ".bak")
+
+
+@main.command(name="regenerate-command-images", help="Regenerate breeze command images.")
+@option_verbose
+@option_dry_run
+def regenerate_command_images(verbose: bool, dry_run: bool):
+ assert_pre_commit_installed(verbose=verbose)
+ perform_environment_checks(verbose=verbose)
+ try:
+ (AIRFLOW_SOURCES_ROOT / "images" / "breeze" / "output-commands-hash.txt").unlink()
+ except FileNotFoundError:
+ # when we go to Python 3.8+ we can add missing_ok = True instead of try/except
+ pass
+ command_to_execute = [sys.executable, "-m", "pre_commit", 'run', 'update-breeze-file', '--all-files']
+ env = os.environ.copy()
+ run_command(
+ command_to_execute,
+ verbose=verbose,
+ dry_run=dry_run,
+ check=False,
+ no_output_dump_on_exception=True,
+ text=True,
+ env=env,
+ )
diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt
index 08e0c94ed2eef..c6b12db9fa491 100644
--- a/images/breeze/output-commands-hash.txt
+++ b/images/breeze/output-commands-hash.txt
@@ -1 +1 @@
-c227c66369d92c76c0e52939e3e4ecf1
+ed70b784364f0b604176ea142305906b
diff --git a/images/breeze/output-commands.svg b/images/breeze/output-commands.svg
index be4ea5a514d12..a592f4642e045 100644
--- a/images/breeze/output-commands.svg
+++ b/images/breeze/output-commands.svg
@@ -1,4 +1,4 @@
-
+
-
-
+
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
+
+
+
+
+
+
- Breeze commands
+ Breeze commands
-
+
-
-
-Usage: breeze [OPTIONS] COMMAND [ARGS]...
-
-╭─ Basic flags for the default (shell) command ────────────────────────────────────────────────────────────────────────╮
-│--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10)│
-│[default: 3.7] │
-│--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite]│
-│--postgres-version-PVersion of Postgres used.(>10< | 11 | 12 | 13 | 14)[default: 10]│
-│--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7]│
-│--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest]│
-│--integrationIntegration(s) to enable when running (can be more than one). │
-│(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | │
-│all) │
-│--forward-credentials-fForward local credentials to container when running.│
-│--db-reset-dReset DB when entering the container.│
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Advanced flags for the default (shell) command ─────────────────────────────────────────────────────────────────────╮
-│--use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`, `wheel`, or │
-│`sdist` if Airflow should be removed, installed from wheel packages or sdist packages │
-│available in dist folder respectively. Implies --mount-sources `none`. │
-│(none | wheel | sdist | <airflow_version>) │
-│--airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT)│
-│--use-packages-from-distInstall all found packages (--package-format determines type) from 'dist' folder when │
-│entering breeze. │
-│--package-formatFormat of packages that should be installed from dist.(wheel | sdist)[default: wheel]│
-│--force-buildForce image build no matter if it is determined as needed.│
-│--mount-sourcesChoose scope of local sources should be mounted (default = selected).│
-│(selected | all | none) │
-│[default: selected] │
-│--debian-versionDebian version used for the image.(bullseye | buster)[default: bullseye]│
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│--verbose-vPrint verbose information about performed steps.│
-│--dry-run-DIf dry-run is set, commands are only printed, not executed.│
-│--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow]│
-│--answer-aForce answer to questions.(y | n | q | yes | no | quit)│
-│--help-hShow this message and exit.│
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Developer tools ────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│shell Enter breeze.py environment. this is the default command use when no other is selected. │
-│start-airflow Enter breeze.py environment and starts all Airflow components in the tmux session. │
-│exec Joins the interactive shell of running airflow container │
-│stop Stop running breeze environment. │
-│build-docs Build documentation in the container. │
-│static-checks Run static checks. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Testing ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│docker-compose-tests Run docker-compose tests. │
-│tests Run the specified unit test targets. Multiple targets may be specified separated by spaces. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Configuration & maintenance ────────────────────────────────────────────────────────────────────────────────────────╮
-│cleanup Cleans the cache of parameters, docker cache and optionally - currently downloaded images. │
-│self-upgrade Self upgrade Breeze. │
-│setup-autocomplete Enables autocompletion of breeze commands. │
-│config Show/update configuration (Python, Backend, Cheatsheet, ASCIIART). │
-│resource-check Check if available docker resources are enough. │
-│free-space Free space for jobs run in CI. │
-│fix-ownership Fix ownership of source files to be same as host user. │
-│command-hash-exportOutputs hash of all click commands to file or stdout if `-` is used (useful to see if images │
-│should be regenerated). │
-│version Print information about version of apache-airflow-breeze. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ CI Image tools ─────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│build-image Build CI image. Include building multiple images for all python versions (sequentially). │
-│pull-image Pull and optionally verify CI images - possibly in parallel for all Python versions. │
-│verify-image Verify CI image. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Production Image tools ─────────────────────────────────────────────────────────────────────────────────────────────╮
-│build-prod-image Build Production image. Include building multiple images for all or selected Python versions │
-│sequentially. │
-│pull-prod-image Pull and optionally verify Production images - possibly in parallel for all Python versions. │
-│verify-prod-imageVerify Production image. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Release management ─────────────────────────────────────────────────────────────────────────────────────────────────╮
-│verify-provider-packages Verifies if all provider code is following expectations for providers. │
-│prepare-provider-documentation Prepare CHANGELOG, README and COMMITS information for providers. │
-│prepare-provider-packages Prepare sdist/whl packages of Airflow Providers. │
-│prepare-airflow-package Prepare sdist/whl package of Airflow. │
-│release-prod-images Release production images to DockerHub (needs DockerHub permissions). │
-│generate-constraints Generates pinned constraint files with all extras from setup.py in parallel. │
-│find-newer-dependencies Finds which dependencies are being upgraded. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+
+Usage: breeze [OPTIONS] COMMAND [ARGS]...
+
+╭─ Basic flags for the default (shell) command ────────────────────────────────────────────────────────────────────────╮
+│--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10)│
+│[default: 3.7] │
+│--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite]│
+│--postgres-version-PVersion of Postgres used.(>10< | 11 | 12 | 13 | 14)[default: 10]│
+│--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7]│
+│--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest]│
+│--integrationIntegration(s) to enable when running (can be more than one). │
+│(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | │
+│all) │
+│--forward-credentials-fForward local credentials to container when running.│
+│--db-reset-dReset DB when entering the container.│
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Advanced flags for the default (shell) command ─────────────────────────────────────────────────────────────────────╮
+│--use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`, `wheel`, or │
+│`sdist` if Airflow should be removed, installed from wheel packages or sdist packages │
+│available in dist folder respectively. Implies --mount-sources `none`. │
+│(none | wheel | sdist | <airflow_version>) │
+│--airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT)│
+│--use-packages-from-distInstall all found packages (--package-format determines type) from 'dist' folder when │
+│entering breeze. │
+│--package-formatFormat of packages that should be installed from dist.(wheel | sdist)[default: wheel]│
+│--force-buildForce image build no matter if it is determined as needed.│
+│--mount-sourcesChoose scope of local sources should be mounted (default = selected).│
+│(selected | all | none) │
+│[default: selected] │
+│--debian-versionDebian version used for the image.(bullseye | buster)[default: bullseye]│
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│--verbose-vPrint verbose information about performed steps.│
+│--dry-run-DIf dry-run is set, commands are only printed, not executed.│
+│--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow]│
+│--answer-aForce answer to questions.(y | n | q | yes | no | quit)│
+│--help-hShow this message and exit.│
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Developer tools ────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│shell Enter breeze.py environment. this is the default command use when no other is selected. │
+│start-airflow Enter breeze.py environment and starts all Airflow components in the tmux session. │
+│exec Joins the interactive shell of running airflow container │
+│stop Stop running breeze environment. │
+│build-docs Build documentation in the container. │
+│static-checks Run static checks. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Testing ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│docker-compose-tests Run docker-compose tests. │
+│tests Run the specified unit test targets. Multiple targets may be specified separated by spaces. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Configuration & maintenance ────────────────────────────────────────────────────────────────────────────────────────╮
+│cleanup Cleans the cache of parameters, docker cache and optionally - currently downloaded │
+│images. │
+│self-upgrade Self upgrade Breeze. │
+│setup-autocomplete Enables autocompletion of breeze commands. │
+│config Show/update configuration (Python, Backend, Cheatsheet, ASCIIART). │
+│resource-check Check if available docker resources are enough. │
+│free-space Free space for jobs run in CI. │
+│fix-ownership Fix ownership of source files to be same as host user. │
+│regenerate-command-imagesRegenerate breeze command images. │
+│command-hash-export Outputs hash of all click commands to file or stdout if `-` is used (useful to see if │
+│images should be regenerated). │
+│version Print information about version of apache-airflow-breeze. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ CI Image tools ─────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│build-image Build CI image. Include building multiple images for all python versions (sequentially). │
+│pull-image Pull and optionally verify CI images - possibly in parallel for all Python versions. │
+│verify-image Verify CI image. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Production Image tools ─────────────────────────────────────────────────────────────────────────────────────────────╮
+│build-prod-image Build Production image. Include building multiple images for all or selected Python versions │
+│sequentially. │
+│pull-prod-image Pull and optionally verify Production images - possibly in parallel for all Python versions. │
+│verify-prod-imageVerify Production image. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Release management ─────────────────────────────────────────────────────────────────────────────────────────────────╮
+│verify-provider-packages Verifies if all provider code is following expectations for providers. │
+│prepare-provider-documentation Prepare CHANGELOG, README and COMMITS information for providers. │
+│prepare-provider-packages Prepare sdist/whl packages of Airflow Providers. │
+│prepare-airflow-package Prepare sdist/whl package of Airflow. │
+│release-prod-images Release production images to DockerHub (needs DockerHub permissions). │
+│generate-constraints Generates pinned constraint files with all extras from setup.py in parallel. │
+│find-newer-dependencies Finds which dependencies are being upgraded. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-regenerate-command-images.svg b/images/breeze/output-regenerate-command-images.svg
new file mode 100644
index 0000000000000..0842e64fa1da3
--- /dev/null
+++ b/images/breeze/output-regenerate-command-images.svg
@@ -0,0 +1,99 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ Command: regenerate-command-images
+
+
+
+
+
+
+
+
+
+
+Usage: breeze regenerate-command-images [OPTIONS]
+
+Regenerate breeze command images.
+
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│--verbose-vPrint verbose information about performed steps.│
+│--dry-run-DIf dry-run is set, commands are only printed, not executed.│
+│--help-hShow this message and exit.│
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+
+
+
From b5cbad4d3c8c0bc8c3fd40b9993fc26c4fcec3de Mon Sep 17 00:00:00 2001
From: Jarek Potiuk
Date: Sun, 12 Jun 2022 17:14:09 +0200
Subject: [PATCH 010/118] Better diagnostics for ARM for MySQL and MSSQL
(#24185)
Until we have debian suppor tof MySQL and MSSQL ARM, runnign
those on ARM platform is not supported. However error about it
was not clear (pulling docker image failed).
This PR adds platform checking also in breeze and fails fast
without even attempting to enter breeze shell when you are on
ARM and wants to run MsSQL or MySQL breeze shell.
Also some errors with running different backend versions via
breeze have been removed.
(cherry picked from commit 00d2a3c9319af6e145b8e03f5ac33a95fa8acd66)
---
.../src/airflow_breeze/commands/developer_commands.py | 9 ++++++++-
1 file changed, 8 insertions(+), 1 deletion(-)
diff --git a/dev/breeze/src/airflow_breeze/commands/developer_commands.py b/dev/breeze/src/airflow_breeze/commands/developer_commands.py
index e10e79d0a8126..a8ad686c708c5 100644
--- a/dev/breeze/src/airflow_breeze/commands/developer_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/developer_commands.py
@@ -25,6 +25,7 @@
from airflow_breeze.commands.main_command import main
from airflow_breeze.global_constants import (
DEFAULT_PYTHON_MAJOR_MINOR_VERSION,
+ DOCKER_DEFAULT_PLATFORM,
MOUNT_SELECTED,
get_available_packages,
)
@@ -601,7 +602,13 @@ def run_shell(verbose: bool, dry_run: bool, shell_params: ShellParams) -> RunCom
env_variables = get_env_variables_for_docker_commands(shell_params)
if cmd_added is not None:
cmd.extend(['-c', cmd_added])
-
+ if "arm64" in DOCKER_DEFAULT_PLATFORM:
+ if shell_params.backend == "mysql":
+ get_console().print('\n[error]MySQL is not supported on ARM architecture.[/]\n')
+ sys.exit(1)
+ if shell_params.backend == "mssql":
+ get_console().print('\n[error]MSSQL is not supported on ARM architecture[/]\n')
+ sys.exit(1)
command_result = run_command(
cmd, verbose=verbose, dry_run=dry_run, env=env_variables, text=True, check=False
)
From 0d0f57f3bd9e8022c71fb957bfdb60cd052d5cec Mon Sep 17 00:00:00 2001
From: Jarek Potiuk
Date: Mon, 13 Jun 2022 19:38:20 +0200
Subject: [PATCH 011/118] Fix links to sources for examples (#24386)
The links to example sources in exampleinclude have been broken in a
number of providers and they were additionally broken by AIP-47.
This PR fixes it.
Fixes: #23632
Fixes: https://github.com/apache/airflow-site/issues/536
(cherry picked from commit 08b675cf6642171cb1c5ddfb09607b541db70b29)
---
.pre-commit-config.yaml | 13 +-
RELEASE_NOTES.rst | 1 +
STATIC_CODE_CHECKS.rst | 4 +
.../example_branch_datetime_operator.py | 20 +-
.../example_external_task_marker_dag.py | 26 +-
.../example_google_api_youtube_to_s3.py | 3 +-
.../amazon/aws/example_dags/example_s3.py | 5 +-
.../arangodb/example_dags/example_arangodb.py | 4 +-
.../example_dags/example_automl_tables.py | 4 +-
...mple_cloud_storage_transfer_service_aws.py | 10 +-
.../cloud/example_dags/example_pubsub.py | 8 +-
.../cloud/example_dags/example_vertex_ai.py | 5 +-
airflow/providers/mongo/hooks/mongo.py | 2 +-
.../src/airflow_breeze/pre_commit_ids.py | 2 +
docs/__init__.py | 16 ++
.../index.rst | 6 +
.../apache-airflow-providers-amazon/index.rst | 6 +
.../index.rst | 11 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 12 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../apache-airflow-providers-docker/index.rst | 6 +
.../apache-airflow-providers-google/index.rst | 6 +
.../operators/cloud/bigquery.rst | 50 ++--
.../operators/cloud/datastore.rst | 26 +-
.../operators/cloud/kubernetes_engine.rst | 10 +-
docs/apache-airflow-providers-http/index.rst | 6 +
docs/apache-airflow-providers-jdbc/index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
docs/apache-airflow-providers-mysql/index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../apache-airflow-providers-presto/index.rst | 6 +
.../apache-airflow-providers-qubole/index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
.../index.rst | 6 +
docs/apache-airflow-providers-trino/index.rst | 6 +
.../apache-airflow-providers-yandex/index.rst | 6 +
docs/apache-airflow/python-api-ref.rst | 9 +
docs/build_docs.py | 14 +-
docs/conf.py | 37 ++-
docs/exts/docs_build/docs_builder.py | 8 +-
docs/exts/provider_yaml_utils.py | 32 ++-
docs/publish_docs.py | 2 +
docs/spelling_wordlist.txt | 9 +
images/breeze/output-commands-hash.txt | 2 +-
images/breeze/output-static-checks.svg | 224 +++++++++---------
.../pre_commit_check_init_in_tests.py | 50 ++++
...e_commit_check_setup_extra_packages_ref.py | 4 +-
...mmit_check_system_tests_hidden_in_index.py | 72 ++++++
scripts/in_container/run_docs_build.sh | 3 +-
setup.cfg | 2 +-
tests/config_templates/__init__.py | 16 ++
tests/core/__init__.py | 16 ++
tests/dags/__init__.py | 16 ++
tests/dags/subdir1/__init__.py | 16 ++
tests/dags/subdir2/__init__.py | 16 ++
tests/dags/subdir2/subdir3/__init__.py | 16 ++
tests/dags_corrupted/__init__.py | 16 ++
tests/dags_with_system_exit/__init__.py | 16 ++
.../__init__.py | 16 ++
tests/jobs/test_scheduler_job.py | 1 +
tests/secrets/__init__.py | 16 ++
tests/system/README.md | 2 +-
tests/system/providers/amazon/__init__.py | 16 ++
tests/system/providers/amazon/aws/__init__.py | 16 ++
.../system/providers/apache/kylin/__init__.py | 16 ++
tests/system/providers/cncf/__init__.py | 16 ++
.../providers/elasticsearch/__init__.py | 16 ++
tests/system/providers/github/__init__.py | 16 ++
tests/system/providers/google/README.md | 2 +-
.../system/providers/google/cloud/__init__.py | 16 ++
.../providers/google/cloud/azure/__init__.py | 16 ++
.../google/cloud/bigquery/__init__.py | 16 ++
.../bigquery/example_bigquery_dataset.py | 0
.../bigquery/example_bigquery_operations.py | 0
.../example_bigquery_operations_location.py | 0
.../bigquery/example_bigquery_queries.py | 0
.../bigquery/example_bigquery_sensors.py | 0
.../bigquery/example_bigquery_tables.py | 0
.../bigquery/example_bigquery_to_bigquery.py | 0
.../bigquery/example_bigquery_to_gcs.py | 0
.../cloud/bigquery/resources/__init__.py | 16 ++
.../resources/example_bigquery_query.sql | 0
.../resources/update_table_schema.json | 0
.../bigquery/resources/us-states.csv | 0
.../google/cloud/cloud_sql/__init__.py | 16 ++
.../google/cloud/dataproc/__init__.py | 16 ++
.../dataproc/example_dataproc_gke.py | 0
.../cloud/dataproc/resources/__init__.py | 16 ++
.../google/cloud/datastore/__init__.py | 16 ++
.../datastore/example_datastore_commit.py | 0
.../example_datastore_export_import.py | 0
.../datastore/example_datastore_query.py | 0
.../datastore/example_datastore_rollback.py | 0
.../providers/google/cloud/gcs/__init__.py | 16 ++
.../gcs/example_gcs_to_bigquery.py | 0
.../google/cloud/gcs/resources/__init__.py | 16 ++
.../cloud/kubernetes_engine/__init__.py | 16 ++
.../example_kubernetes_engine.py | 0
.../google/cloud/spanner/__init__.py | 16 ++
.../google/cloud/speech_to_text/__init__.py | 16 ++
.../google/cloud/stackdriver/__init__.py | 16 ++
.../providers/google/cloud/tasks/__init__.py | 16 ++
.../google/cloud/text_to_speech/__init__.py | 16 ++
.../google/cloud/workflows/__init__.py | 16 ++
.../providers/google/workplace/__init__.py | 16 ++
tests/system/providers/presto/__init__.py | 16 ++
tests/test_utils/operators/__init__.py | 16 ++
tests/test_utils/perf/__init__.py | 16 ++
tests/test_utils/perf/dags/__init__.py | 16 ++
tests/testconfig/__init__.py | 16 ++
tests/testconfig/conf/__init__.py | 16 ++
tests/triggers/__init__.py | 16 ++
tests/www/test_logs/__init__.py | 16 ++
tests/www/views/__init__.py | 16 ++
131 files changed, 1349 insertions(+), 242 deletions(-)
create mode 100644 docs/__init__.py
create mode 100755 scripts/ci/pre_commit/pre_commit_check_init_in_tests.py
create mode 100755 scripts/ci/pre_commit/pre_commit_check_system_tests_hidden_in_index.py
create mode 100644 tests/config_templates/__init__.py
create mode 100644 tests/core/__init__.py
create mode 100644 tests/dags/__init__.py
create mode 100644 tests/dags/subdir1/__init__.py
create mode 100644 tests/dags/subdir2/__init__.py
create mode 100644 tests/dags/subdir2/subdir3/__init__.py
create mode 100644 tests/dags_corrupted/__init__.py
create mode 100644 tests/dags_with_system_exit/__init__.py
create mode 100644 tests/executors/kubernetes_executor_template_files/__init__.py
create mode 100644 tests/secrets/__init__.py
create mode 100644 tests/system/providers/amazon/__init__.py
create mode 100644 tests/system/providers/amazon/aws/__init__.py
create mode 100644 tests/system/providers/apache/kylin/__init__.py
create mode 100644 tests/system/providers/cncf/__init__.py
create mode 100644 tests/system/providers/elasticsearch/__init__.py
create mode 100644 tests/system/providers/github/__init__.py
create mode 100644 tests/system/providers/google/cloud/__init__.py
create mode 100644 tests/system/providers/google/cloud/azure/__init__.py
create mode 100644 tests/system/providers/google/cloud/bigquery/__init__.py
rename tests/system/providers/google/{ => cloud}/bigquery/example_bigquery_dataset.py (100%)
rename tests/system/providers/google/{ => cloud}/bigquery/example_bigquery_operations.py (100%)
rename tests/system/providers/google/{ => cloud}/bigquery/example_bigquery_operations_location.py (100%)
rename tests/system/providers/google/{ => cloud}/bigquery/example_bigquery_queries.py (100%)
rename tests/system/providers/google/{ => cloud}/bigquery/example_bigquery_sensors.py (100%)
rename tests/system/providers/google/{ => cloud}/bigquery/example_bigquery_tables.py (100%)
rename tests/system/providers/google/{ => cloud}/bigquery/example_bigquery_to_bigquery.py (100%)
rename tests/system/providers/google/{ => cloud}/bigquery/example_bigquery_to_gcs.py (100%)
create mode 100644 tests/system/providers/google/cloud/bigquery/resources/__init__.py
rename tests/system/providers/google/{ => cloud}/bigquery/resources/example_bigquery_query.sql (100%)
rename tests/system/providers/google/{ => cloud}/bigquery/resources/update_table_schema.json (100%)
rename tests/system/providers/google/{ => cloud}/bigquery/resources/us-states.csv (100%)
create mode 100644 tests/system/providers/google/cloud/cloud_sql/__init__.py
create mode 100644 tests/system/providers/google/cloud/dataproc/__init__.py
rename tests/system/providers/google/{ => cloud}/dataproc/example_dataproc_gke.py (100%)
create mode 100644 tests/system/providers/google/cloud/dataproc/resources/__init__.py
create mode 100644 tests/system/providers/google/cloud/datastore/__init__.py
rename tests/system/providers/google/{ => cloud}/datastore/example_datastore_commit.py (100%)
rename tests/system/providers/google/{ => cloud}/datastore/example_datastore_export_import.py (100%)
rename tests/system/providers/google/{ => cloud}/datastore/example_datastore_query.py (100%)
rename tests/system/providers/google/{ => cloud}/datastore/example_datastore_rollback.py (100%)
create mode 100644 tests/system/providers/google/cloud/gcs/__init__.py
rename tests/system/providers/google/{ => cloud}/gcs/example_gcs_to_bigquery.py (100%)
create mode 100644 tests/system/providers/google/cloud/gcs/resources/__init__.py
create mode 100644 tests/system/providers/google/cloud/kubernetes_engine/__init__.py
rename tests/system/providers/google/{ => cloud}/kubernetes_engine/example_kubernetes_engine.py (100%)
create mode 100644 tests/system/providers/google/cloud/spanner/__init__.py
create mode 100644 tests/system/providers/google/cloud/speech_to_text/__init__.py
create mode 100644 tests/system/providers/google/cloud/stackdriver/__init__.py
create mode 100644 tests/system/providers/google/cloud/tasks/__init__.py
create mode 100644 tests/system/providers/google/cloud/text_to_speech/__init__.py
create mode 100644 tests/system/providers/google/cloud/workflows/__init__.py
create mode 100644 tests/system/providers/google/workplace/__init__.py
create mode 100644 tests/system/providers/presto/__init__.py
create mode 100644 tests/test_utils/operators/__init__.py
create mode 100644 tests/test_utils/perf/__init__.py
create mode 100644 tests/test_utils/perf/dags/__init__.py
create mode 100644 tests/testconfig/__init__.py
create mode 100644 tests/testconfig/conf/__init__.py
create mode 100644 tests/triggers/__init__.py
create mode 100644 tests/www/test_logs/__init__.py
create mode 100644 tests/www/views/__init__.py
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 22d9712590add..15d18dd1b096b 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -28,7 +28,7 @@ repos:
- id: check-hooks-apply
name: Check if all hooks apply to the repository
- repo: https://github.com/thlorenz/doctoc.git
- rev: v2.1.0
+ rev: v2.2.0
hooks:
- id: doctoc
name: Add TOC for md and rst files
@@ -39,7 +39,7 @@ repos:
- "--maxlevel"
- "2"
- repo: https://github.com/Lucas-C/pre-commit-hooks
- rev: v1.1.13
+ rev: v1.2.0
hooks:
- id: forbid-tabs
name: Fail if tabs are used in the project
@@ -204,7 +204,7 @@ repos:
pass_filenames: true
# TODO: Bump to Python 3.8 when support for Python 3.7 is dropped in Airflow.
- repo: https://github.com/asottile/pyupgrade
- rev: v2.32.0
+ rev: v2.32.1
hooks:
- id: pyupgrade
name: Upgrade Python code automatically
@@ -242,7 +242,7 @@ repos:
name: Run pydocstyle
args:
- --convention=pep257
- - --add-ignore=D100,D102,D103,D104,D105,D107,D202,D205,D400,D401
+ - --add-ignore=D100,D102,D103,D104,D105,D107,D205,D400,D401
exclude: |
(?x)
^tests/.*\.py$|
@@ -422,11 +422,11 @@ repos:
- id: check-no-relative-imports
language: pygrep
name: No relative imports
- description: Airflow style is to use absolute imports only
+ description: Airflow style is to use absolute imports only (except docs building)
entry: "^\\s*from\\s+\\."
pass_filenames: true
files: \.py$
- exclude: ^tests/|^airflow/_vendor/
+ exclude: ^tests/|^airflow/_vendor/|^docs/
- id: check-for-inclusive-language
language: pygrep
name: Check for language that we do not accept as community
@@ -449,6 +449,7 @@ repos:
^airflow/www/static/|
^airflow/providers/|
^tests/providers/apache/cassandra/hooks/test_cassandra.py$|
+ ^tests/system/providers/apache/spark/example_spark_dag.py$|
^docs/apache-airflow-providers-apache-cassandra/connections/cassandra.rst$|
^docs/apache-airflow-providers-apache-hive/commits.rst$|
^airflow/api_connexion/openapi/v1.yaml$|
diff --git a/RELEASE_NOTES.rst b/RELEASE_NOTES.rst
index edc7de3df967b..66edcf9c90e90 100644
--- a/RELEASE_NOTES.rst
+++ b/RELEASE_NOTES.rst
@@ -704,6 +704,7 @@ Misc/Internal
- Remove deprecated usage of ``init_role()`` from API (#18820)
- Remove duplicate code on dbapi hook (#18821)
+
Airflow 2.2.5, (2022-04-04)
---------------------------
diff --git a/STATIC_CODE_CHECKS.rst b/STATIC_CODE_CHECKS.rst
index e2b57eae6c69b..5b7fa05908e46 100644
--- a/STATIC_CODE_CHECKS.rst
+++ b/STATIC_CODE_CHECKS.rst
@@ -205,10 +205,14 @@ require Breeze Docker image to be build locally.
+--------------------------------------------------------+------------------------------------------------------------------+---------+
| check-system-tests-present | Check if system tests have required segments of code | |
+--------------------------------------------------------+------------------------------------------------------------------+---------+
+| check-system-tests-tocs | Check that system tests is properly added | |
++--------------------------------------------------------+------------------------------------------------------------------+---------+
| check-xml | Check XML files with xmllint | |
+--------------------------------------------------------+------------------------------------------------------------------+---------+
| codespell | Run codespell to check for common misspellings in files | |
+--------------------------------------------------------+------------------------------------------------------------------+---------+
+| create-missing-init-py-files-tests | Create missing init.py files in tests | |
++--------------------------------------------------------+------------------------------------------------------------------+---------+
| debug-statements | Detect accidentally committed debug statements | |
+--------------------------------------------------------+------------------------------------------------------------------+---------+
| detect-private-key | Detect if private key is added to the repository | |
diff --git a/airflow/example_dags/example_branch_datetime_operator.py b/airflow/example_dags/example_branch_datetime_operator.py
index 3c86e40402aef..e707514c868a0 100644
--- a/airflow/example_dags/example_branch_datetime_operator.py
+++ b/airflow/example_dags/example_branch_datetime_operator.py
@@ -26,7 +26,7 @@
from airflow.operators.datetime import BranchDateTimeOperator
from airflow.operators.empty import EmptyOperator
-dag = DAG(
+dag1 = DAG(
dag_id="example_branch_datetime_operator",
start_date=pendulum.datetime(2021, 1, 1, tz="UTC"),
catchup=False,
@@ -35,8 +35,8 @@
)
# [START howto_branch_datetime_operator]
-empty_task_1 = EmptyOperator(task_id='date_in_range', dag=dag)
-empty_task_2 = EmptyOperator(task_id='date_outside_range', dag=dag)
+empty_task_11 = EmptyOperator(task_id='date_in_range', dag=dag1)
+empty_task_21 = EmptyOperator(task_id='date_outside_range', dag=dag1)
cond1 = BranchDateTimeOperator(
task_id='datetime_branch',
@@ -44,15 +44,15 @@
follow_task_ids_if_false=['date_outside_range'],
target_upper=pendulum.datetime(2020, 10, 10, 15, 0, 0),
target_lower=pendulum.datetime(2020, 10, 10, 14, 0, 0),
- dag=dag,
+ dag=dag1,
)
# Run empty_task_1 if cond1 executes between 2020-10-10 14:00:00 and 2020-10-10 15:00:00
-cond1 >> [empty_task_1, empty_task_2]
+cond1 >> [empty_task_11, empty_task_21]
# [END howto_branch_datetime_operator]
-dag = DAG(
+dag2 = DAG(
dag_id="example_branch_datetime_operator_2",
start_date=pendulum.datetime(2021, 1, 1, tz="UTC"),
catchup=False,
@@ -60,8 +60,8 @@
schedule_interval="@daily",
)
# [START howto_branch_datetime_operator_next_day]
-empty_task_1 = EmptyOperator(task_id='date_in_range', dag=dag)
-empty_task_2 = EmptyOperator(task_id='date_outside_range', dag=dag)
+empty_task_12 = EmptyOperator(task_id='date_in_range', dag=dag2)
+empty_task_22 = EmptyOperator(task_id='date_outside_range', dag=dag2)
cond2 = BranchDateTimeOperator(
task_id='datetime_branch',
@@ -69,10 +69,10 @@
follow_task_ids_if_false=['date_outside_range'],
target_upper=pendulum.time(0, 0, 0),
target_lower=pendulum.time(15, 0, 0),
- dag=dag,
+ dag=dag2,
)
# Since target_lower happens after target_upper, target_upper will be moved to the following day
# Run empty_task_1 if cond2 executes between 15:00:00, and 00:00:00 of the following day
-cond2 >> [empty_task_1, empty_task_2]
+cond2 >> [empty_task_12, empty_task_22]
# [END howto_branch_datetime_operator_next_day]
diff --git a/airflow/example_dags/example_external_task_marker_dag.py b/airflow/example_dags/example_external_task_marker_dag.py
index 0c4479a0d66f0..733b732756633 100644
--- a/airflow/example_dags/example_external_task_marker_dag.py
+++ b/airflow/example_dags/example_external_task_marker_dag.py
@@ -18,23 +18,25 @@
"""
Example DAG demonstrating setting up inter-DAG dependencies using ExternalTaskSensor and
-ExternalTaskMarker
+ExternalTaskMarker.
In this example, child_task1 in example_external_task_marker_child depends on parent_task in
-example_external_task_marker_parent. When parent_task is cleared with "Recursive" selected,
-the presence of ExternalTaskMarker tells Airflow to clear child_task1 and its
-downstream tasks.
+example_external_task_marker_parent. When parent_task is cleared with 'Recursive' selected,
+the presence of ExternalTaskMarker tells Airflow to clear child_task1 and its downstream tasks.
ExternalTaskSensor will keep poking for the status of remote ExternalTaskMarker task at a regular
interval till one of the following will happen:
-1. ExternalTaskMarker reaches the states mentioned in the allowed_states list
- In this case, ExternalTaskSensor will exit with a success status code
-2. ExternalTaskMarker reaches the states mentioned in the failed_states list
- In this case, ExternalTaskSensor will raise an AirflowException and user need to handle this
- with multiple downstream tasks
-3. ExternalTaskSensor times out
- In this case, ExternalTaskSensor will raise AirflowSkipException or AirflowSensorTimeout
- exception
+
+ExternalTaskMarker reaches the states mentioned in the allowed_states list.
+In this case, ExternalTaskSensor will exit with a success status code
+
+ExternalTaskMarker reaches the states mentioned in the failed_states list
+In this case, ExternalTaskSensor will raise an AirflowException and user need to handle this
+with multiple downstream tasks
+
+ExternalTaskSensor times out. In this case, ExternalTaskSensor will raise AirflowSkipException
+or AirflowSensorTimeout exception
+
"""
import pendulum
diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py
index 241a551320efd..2bbe4fac58e2c 100644
--- a/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py
+++ b/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py
@@ -26,7 +26,8 @@
Further information:
YOUTUBE_VIDEO_PUBLISHED_AFTER and YOUTUBE_VIDEO_PUBLISHED_BEFORE needs to be formatted
-"YYYY-MM-DDThh:mm:ss.sZ". See https://developers.google.com/youtube/v3/docs/search/list for more information.
+``YYYY-MM-DDThh:mm:ss.sZ``.
+See https://developers.google.com/youtube/v3/docs/search/list for more information.
YOUTUBE_VIDEO_PARTS depends on the fields you pass via YOUTUBE_VIDEO_FIELDS. See
https://developers.google.com/youtube/v3/docs/videos/list#parameters for more information.
YOUTUBE_CONN_ID is optional for public videos. It does only need to authenticate when there are private videos
diff --git a/airflow/providers/amazon/aws/example_dags/example_s3.py b/airflow/providers/amazon/aws/example_dags/example_s3.py
index ecd9d374cf688..83a6825d72fe0 100644
--- a/airflow/providers/amazon/aws/example_dags/example_s3.py
+++ b/airflow/providers/amazon/aws/example_dags/example_s3.py
@@ -62,12 +62,9 @@
# [START howto_sensor_s3_key_function_definition]
def check_fn(files: List) -> bool:
"""
- Example of custom check: check if all files are bigger than 1kB
+ Example of custom check: check if all files are bigger than ``1kB``
:param files: List of S3 object attributes.
- Format: [{
- 'Size': int
- }]
:return: true if the criteria is met
:rtype: bool
"""
diff --git a/airflow/providers/arangodb/example_dags/example_arangodb.py b/airflow/providers/arangodb/example_dags/example_arangodb.py
index f9da187cfb665..37a8250dfd16a 100644
--- a/airflow/providers/arangodb/example_dags/example_arangodb.py
+++ b/airflow/providers/arangodb/example_dags/example_arangodb.py
@@ -41,7 +41,7 @@
# [START howto_aql_sensor_template_file_arangodb]
-sensor = AQLSensor(
+sensor2 = AQLSensor(
task_id="aql_sensor_template_file",
query="search_judy.sql",
timeout=60,
@@ -65,7 +65,7 @@
# [START howto_aql_operator_template_file_arangodb]
-operator = AQLOperator(
+operator2 = AQLOperator(
task_id='aql_operator_template_file',
dag=dag,
result_processor=lambda cursor: print([document["name"] for document in cursor]),
diff --git a/airflow/providers/google/cloud/example_dags/example_automl_tables.py b/airflow/providers/google/cloud/example_dags/example_automl_tables.py
index 9ba0314dae777..c13de99fa8512 100644
--- a/airflow/providers/google/cloud/example_dags/example_automl_tables.py
+++ b/airflow/providers/google/cloud/example_dags/example_automl_tables.py
@@ -204,14 +204,14 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str:
catchup=False,
user_defined_macros={"extract_object_id": extract_object_id},
) as example_dag:
- create_dataset_task = AutoMLCreateDatasetOperator(
+ create_dataset_task2 = AutoMLCreateDatasetOperator(
task_id="create_dataset_task",
dataset=DATASET,
location=GCP_AUTOML_LOCATION,
project_id=GCP_PROJECT_ID,
)
- dataset_id = create_dataset_task.output['dataset_id']
+ dataset_id = create_dataset_task2.output['dataset_id']
import_dataset_task = AutoMLImportDataOperator(
task_id="import_dataset_task",
diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py
index be858c4018753..bf73959d4ff72 100644
--- a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py
+++ b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py
@@ -17,17 +17,15 @@
# under the License.
"""
-Example Airflow DAG that demonstrates interactions with Google Cloud Transfer.
+Example Airflow DAG that demonstrates interactions with Google Cloud Transfer. This DAG relies on
+the following OS environment variables
-
-This DAG relies on the following OS environment variables
+Note that you need to provide a large enough set of data so that operations do not execute too quickly.
+Otherwise, DAG will fail.
* GCP_PROJECT_ID - Google Cloud Project to use for the Google Cloud Transfer Service.
* GCP_DESCRIPTION - Description of transfer job
* GCP_TRANSFER_SOURCE_AWS_BUCKET - Amazon Web Services Storage bucket from which files are copied.
- .. warning::
- You need to provide a large enough set of data so that operations do not execute too quickly.
- Otherwise, DAG will fail.
* GCP_TRANSFER_SECOND_TARGET_BUCKET - Google Cloud Storage bucket to which files are copied
* WAIT_FOR_OPERATION_POKE_INTERVAL - interval of what to check the status of the operation
A smaller value than the default value accelerates the system test and ensures its correct execution with
diff --git a/airflow/providers/google/cloud/example_dags/example_pubsub.py b/airflow/providers/google/cloud/example_dags/example_pubsub.py
index 8e3dd1fe8f01e..05ae16bd68f35 100644
--- a/airflow/providers/google/cloud/example_dags/example_pubsub.py
+++ b/airflow/providers/google/cloud/example_dags/example_pubsub.py
@@ -56,7 +56,7 @@
catchup=False,
) as example_sensor_dag:
# [START howto_operator_gcp_pubsub_create_topic]
- create_topic = PubSubCreateTopicOperator(
+ create_topic1 = PubSubCreateTopicOperator(
task_id="create_topic", topic=TOPIC_FOR_SENSOR_DAG, project_id=GCP_PROJECT_ID, fail_if_exists=False
)
# [END howto_operator_gcp_pubsub_create_topic]
@@ -105,7 +105,7 @@
)
# [END howto_operator_gcp_pubsub_delete_topic]
- create_topic >> subscribe_task >> publish_task
+ create_topic1 >> subscribe_task >> publish_task
pull_messages >> pull_messages_result >> unsubscribe_task >> delete_topic
# Task dependencies created via `XComArgs`:
@@ -120,7 +120,7 @@
catchup=False,
) as example_operator_dag:
# [START howto_operator_gcp_pubsub_create_topic]
- create_topic = PubSubCreateTopicOperator(
+ create_topic2 = PubSubCreateTopicOperator(
task_id="create_topic", topic=TOPIC_FOR_OPERATOR_DAG, project_id=GCP_PROJECT_ID
)
# [END howto_operator_gcp_pubsub_create_topic]
@@ -170,7 +170,7 @@
# [END howto_operator_gcp_pubsub_delete_topic]
(
- create_topic
+ create_topic2
>> subscribe_task
>> publish_task
>> pull_messages_operator
diff --git a/airflow/providers/google/cloud/example_dags/example_vertex_ai.py b/airflow/providers/google/cloud/example_dags/example_vertex_ai.py
index cded48ae9b4de..a421c31a7ee2c 100644
--- a/airflow/providers/google/cloud/example_dags/example_vertex_ai.py
+++ b/airflow/providers/google/cloud/example_dags/example_vertex_ai.py
@@ -26,15 +26,16 @@
This DAG relies on the following OS environment variables:
* GCP_VERTEX_AI_BUCKET - Google Cloud Storage bucket where the model will be saved
-after training process was finished.
+ after training process was finished.
* CUSTOM_CONTAINER_URI - path to container with model.
* PYTHON_PACKAGE_GSC_URI - path to test model in archive.
* LOCAL_TRAINING_SCRIPT_PATH - path to local training script.
* DATASET_ID - ID of dataset which will be used in training process.
* MODEL_ID - ID of model which will be used in predict process.
* MODEL_ARTIFACT_URI - The artifact_uri should be the path to a GCS directory containing saved model
-artifacts.
+ artifacts.
"""
+
import os
from datetime import datetime
from uuid import uuid4
diff --git a/airflow/providers/mongo/hooks/mongo.py b/airflow/providers/mongo/hooks/mongo.py
index 96a5ec800302a..c022ec4135ebd 100644
--- a/airflow/providers/mongo/hooks/mongo.py
+++ b/airflow/providers/mongo/hooks/mongo.py
@@ -266,7 +266,7 @@ def replace_many(
:param mongo_collection: The name of the collection to update.
:param docs: The new documents.
:param filter_docs: A list of queries that match the documents to replace.
- Can be omitted; then the _id fields from docs will be used.
+ Can be omitted; then the _id fields from airflow.docs will be used.
:param mongo_db: The name of the database to use.
Can be omitted; then the database from the connection string is used.
:param upsert: If ``True``, perform an insert if no documents
diff --git a/dev/breeze/src/airflow_breeze/pre_commit_ids.py b/dev/breeze/src/airflow_breeze/pre_commit_ids.py
index 40ff16a7a03bd..2f8366b01ffd2 100644
--- a/dev/breeze/src/airflow_breeze/pre_commit_ids.py
+++ b/dev/breeze/src/airflow_breeze/pre_commit_ids.py
@@ -59,8 +59,10 @@
'check-setup-order',
'check-start-date-not-used-in-defaults',
'check-system-tests-present',
+ 'check-system-tests-tocs',
'check-xml',
'codespell',
+ 'create-missing-init-py-files-tests',
'debug-statements',
'detect-private-key',
'doctoc',
diff --git a/docs/__init__.py b/docs/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/docs/__init__.py
@@ -0,0 +1,16 @@
+# 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/docs/apache-airflow-providers-alibaba/index.rst b/docs/apache-airflow-providers-alibaba/index.rst
index fca2d993f395d..d5a1292090ae1 100644
--- a/docs/apache-airflow-providers-alibaba/index.rst
+++ b/docs/apache-airflow-providers-alibaba/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/alibaba/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/alibaba/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-amazon/index.rst b/docs/apache-airflow-providers-amazon/index.rst
index 477b6f6a7573f..d1fd9c63006f8 100644
--- a/docs/apache-airflow-providers-amazon/index.rst
+++ b/docs/apache-airflow-providers-amazon/index.rst
@@ -36,6 +36,12 @@ Content
Python API <_api/airflow/providers/amazon/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/amazon/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-apache-beam/index.rst b/docs/apache-airflow-providers-apache-beam/index.rst
index f9a7159ccb3ff..dc26b8c708522 100644
--- a/docs/apache-airflow-providers-apache-beam/index.rst
+++ b/docs/apache-airflow-providers-apache-beam/index.rst
@@ -26,6 +26,17 @@ Content
:caption: References
Python API <_api/airflow/providers/apache/beam/index>
+
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/apache/beam/index>
+
+.. toctree::
+ :maxdepth: 1
+ :caption: Resources
+
PyPI Repository
Example DAGs
diff --git a/docs/apache-airflow-providers-apache-cassandra/index.rst b/docs/apache-airflow-providers-apache-cassandra/index.rst
index 4f207a2e3f1d5..6fcf012613868 100644
--- a/docs/apache-airflow-providers-apache-cassandra/index.rst
+++ b/docs/apache-airflow-providers-apache-cassandra/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/apache/cassandra/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/apache/cassandra/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-apache-drill/index.rst b/docs/apache-airflow-providers-apache-drill/index.rst
index d3d85f780fe75..bc8b7aa1113f2 100644
--- a/docs/apache-airflow-providers-apache-drill/index.rst
+++ b/docs/apache-airflow-providers-apache-drill/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/apache/drill/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/apache/drill/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-apache-druid/index.rst b/docs/apache-airflow-providers-apache-druid/index.rst
index 4946cc1fd143a..d8558d0b49eac 100644
--- a/docs/apache-airflow-providers-apache-druid/index.rst
+++ b/docs/apache-airflow-providers-apache-druid/index.rst
@@ -32,6 +32,18 @@ Content
:caption: References
Python API <_api/airflow/providers/apache/druid/index>
+
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/apache/druid/index>
+
+
+.. toctree::
+ :maxdepth: 1
+ :caption: Resources
+
PyPI Repository
Installing from sources
Example DAGs
diff --git a/docs/apache-airflow-providers-apache-hive/index.rst b/docs/apache-airflow-providers-apache-hive/index.rst
index 0ca5ac916b47e..de8124e80cc03 100644
--- a/docs/apache-airflow-providers-apache-hive/index.rst
+++ b/docs/apache-airflow-providers-apache-hive/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/apache/hive/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/apache/hive/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-apache-kylin/index.rst b/docs/apache-airflow-providers-apache-kylin/index.rst
index 02fb13b255947..faf390c0b3fa9 100644
--- a/docs/apache-airflow-providers-apache-kylin/index.rst
+++ b/docs/apache-airflow-providers-apache-kylin/index.rst
@@ -28,6 +28,12 @@ Content
Python API <_api/airflow/providers/apache/kylin/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/apache/kylin/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-apache-livy/index.rst b/docs/apache-airflow-providers-apache-livy/index.rst
index cd2f5058c61fc..5b27808aaa716 100644
--- a/docs/apache-airflow-providers-apache-livy/index.rst
+++ b/docs/apache-airflow-providers-apache-livy/index.rst
@@ -33,6 +33,12 @@ Content
Python API <_api/airflow/providers/apache/livy/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/apache/livy/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-apache-pig/index.rst b/docs/apache-airflow-providers-apache-pig/index.rst
index 4fd1ffee28698..33dd0a1f095f2 100644
--- a/docs/apache-airflow-providers-apache-pig/index.rst
+++ b/docs/apache-airflow-providers-apache-pig/index.rst
@@ -33,6 +33,12 @@ Content
Python API <_api/airflow/providers/apache/pig/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/apache/pig/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-apache-spark/index.rst b/docs/apache-airflow-providers-apache-spark/index.rst
index 03f1449cb3d94..7bfecdc428377 100644
--- a/docs/apache-airflow-providers-apache-spark/index.rst
+++ b/docs/apache-airflow-providers-apache-spark/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/apache/spark/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/apache/spark/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-cncf-kubernetes/index.rst b/docs/apache-airflow-providers-cncf-kubernetes/index.rst
index bc5b6b7d169ff..4ead6580fbdf6 100644
--- a/docs/apache-airflow-providers-cncf-kubernetes/index.rst
+++ b/docs/apache-airflow-providers-cncf-kubernetes/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/cncf/kubernetes/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/cncf/kubernetes/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-databricks/index.rst b/docs/apache-airflow-providers-databricks/index.rst
index 1a6d32cab3df5..712eee960ba38 100644
--- a/docs/apache-airflow-providers-databricks/index.rst
+++ b/docs/apache-airflow-providers-databricks/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/databricks/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/databricks/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-dbt-cloud/index.rst b/docs/apache-airflow-providers-dbt-cloud/index.rst
index ecfa51fc0ac59..2a1a9cf199358 100644
--- a/docs/apache-airflow-providers-dbt-cloud/index.rst
+++ b/docs/apache-airflow-providers-dbt-cloud/index.rst
@@ -39,6 +39,12 @@ Content
Python API <_api/airflow/providers/dbt/cloud/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/dbt/cloud/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-dingding/index.rst b/docs/apache-airflow-providers-dingding/index.rst
index ba3be6088494b..4966b0a174073 100644
--- a/docs/apache-airflow-providers-dingding/index.rst
+++ b/docs/apache-airflow-providers-dingding/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/dingding/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/dingding/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-docker/index.rst b/docs/apache-airflow-providers-docker/index.rst
index 218968f18ef96..3fa5c2b9c0fc0 100644
--- a/docs/apache-airflow-providers-docker/index.rst
+++ b/docs/apache-airflow-providers-docker/index.rst
@@ -29,6 +29,12 @@ Content
Connection types
Python API <_api/airflow/providers/docker/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/docker/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-google/index.rst b/docs/apache-airflow-providers-google/index.rst
index f7ea7fae3149f..c22fa4bf57fac 100644
--- a/docs/apache-airflow-providers-google/index.rst
+++ b/docs/apache-airflow-providers-google/index.rst
@@ -38,6 +38,12 @@ Content
Python API <_api/airflow/providers/google/index>
Configuration
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/google/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-google/operators/cloud/bigquery.rst b/docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
index 96038ce8c191a..57e4d87ff8c03 100644
--- a/docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
+++ b/docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
@@ -42,7 +42,7 @@ Create dataset
To create an empty dataset in a BigQuery database you can use
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateEmptyDatasetOperator`.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_dataset.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_create_dataset]
@@ -58,7 +58,7 @@ To get the details of an existing dataset you can use
This operator returns a `Dataset Resource `__.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_dataset.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_get_dataset]
@@ -72,7 +72,7 @@ List tables in dataset
To retrieve the list of tables in a given dataset use
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryGetDatasetTablesOperator`.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_get_dataset_tables]
@@ -89,7 +89,7 @@ To update a table in BigQuery you can use
The update method replaces the entire Table resource, whereas the patch
method only replaces fields that are provided in the submitted Table resource.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_update_table]
@@ -106,7 +106,7 @@ To update a dataset in BigQuery you can use
The update method replaces the entire dataset resource, whereas the patch
method only replaces fields that are provided in the submitted dataset resource.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_dataset.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_update_dataset]
@@ -120,7 +120,7 @@ Delete dataset
To delete an existing dataset from a BigQuery database you can use
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryDeleteDatasetOperator`.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_dataset.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_delete_dataset]
@@ -143,7 +143,7 @@ ways. You may either directly pass the schema fields in, or you may point the
operator to a Google Cloud Storage object name. The object in Google Cloud
Storage must be a JSON file with the schema fields in it.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_create_table]
@@ -151,7 +151,7 @@ Storage must be a JSON file with the schema fields in it.
You can use this operator to create a view on top of an existing table.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_create_view]
@@ -160,7 +160,7 @@ You can use this operator to create a view on top of an existing table.
You can also use this operator to create a materialized view that periodically
cache results of a query for increased performance and efficiency.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_create_materialized_view]
@@ -179,7 +179,7 @@ Similarly to
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateEmptyTableOperator`
you can directly pass the schema fields in.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_operations.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_operations.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_create_external_table]
@@ -187,7 +187,7 @@ you can directly pass the schema fields in.
Or you may point the operator to a Google Cloud Storage object name where the schema is stored.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_create_table_schema_json]
@@ -211,7 +211,7 @@ returned list will be equal to the number of rows fetched. Each element in the
list will again be a list where elements would represent the column values for
that row.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_queries.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py
:language: python
:dedent: 8
:start-after: [START howto_operator_bigquery_get_data]
@@ -228,7 +228,7 @@ To upsert a table you can use
This operator either updates the existing table or creates a new, empty table
in the given dataset.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_upsert_table]
@@ -245,7 +245,7 @@ To update the schema of a table you can use
This operator updates the schema field values supplied, while leaving the rest unchanged. This is useful
for instance to set new field descriptions on an existing table schema.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_update_table_schema]
@@ -259,7 +259,7 @@ Delete table
To delete an existing table you can use
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryDeleteTableOperator`.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_delete_table]
@@ -267,7 +267,7 @@ To delete an existing table you can use
You can also use this operator to delete a view.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_delete_view]
@@ -275,7 +275,7 @@ You can also use this operator to delete a view.
You can also use this operator to delete a materialized view.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_tables.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
:language: python
:dedent: 4
:start-after: [START howto_operator_bigquery_delete_materialized_view]
@@ -288,7 +288,7 @@ Execute BigQuery jobs
Let's say you would like to execute the following query.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_queries.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py
:language: python
:dedent: 0
:start-after: [START howto_operator_bigquery_query]
@@ -298,7 +298,7 @@ To execute the SQL query in a specific BigQuery database you can use
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryInsertJobOperator` with
proper query job configuration that can be Jinja templated.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_queries.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py
:language: python
:dedent: 8
:start-after: [START howto_operator_bigquery_insert_job]
@@ -310,7 +310,7 @@ For more information on types of BigQuery job please check
If you want to include some files in your configuration you can use ``include`` clause of Jinja template
language as follow:
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_queries.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py
:language: python
:dedent: 8
:start-after: [START howto_operator_bigquery_select_job]
@@ -339,7 +339,7 @@ This operator expects a sql query that will return a single row. Each value on
that first row is evaluated using python ``bool`` casting. If any of the values
return ``False`` the check is failed and errors out.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_queries.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py
:language: python
:dedent: 8
:start-after: [START howto_operator_bigquery_check]
@@ -357,7 +357,7 @@ This operator expects a sql query that will return a single row. Each value on
that first row is evaluated against ``pass_value`` which can be either a string
or numeric value. If numeric, you can also specify ``tolerance``.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_queries.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py
:language: python
:dedent: 8
:start-after: [START howto_operator_bigquery_value_check]
@@ -372,7 +372,7 @@ To check that the values of metrics given as SQL expressions are within a certai
tolerance of the ones from ``days_back`` before you can use
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryIntervalCheckOperator`.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_queries.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py
:language: python
:dedent: 8
:start-after: [START howto_operator_bigquery_interval_check]
@@ -390,7 +390,7 @@ use the ``{{ ds_nodash }}`` macro as the table name suffix.
:class:`~airflow.providers.google.cloud.sensors.bigquery.BigQueryTableExistenceSensor`.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_sensors.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_sensors.py
:language: python
:dedent: 4
:start-after: [START howto_sensor_bigquery_table]
@@ -402,7 +402,7 @@ Check that a Table Partition exists
To check that a table exists and has a partition you can use.
:class:`~airflow.providers.google.cloud.sensors.bigquery.BigQueryTablePartitionExistenceSensor`.
-.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_sensors.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/bigquery/example_bigquery_sensors.py
:language: python
:dedent: 4
:start-after: [START howto_sensor_bigquery_table_partition]
diff --git a/docs/apache-airflow-providers-google/operators/cloud/datastore.rst b/docs/apache-airflow-providers-google/operators/cloud/datastore.rst
index 4a8e623d6ee14..05da36a3f83a2 100644
--- a/docs/apache-airflow-providers-google/operators/cloud/datastore.rst
+++ b/docs/apache-airflow-providers-google/operators/cloud/datastore.rst
@@ -38,7 +38,7 @@ Export Entities
To export entities from Google Cloud Datastore to Cloud Storage use
:class:`~airflow.providers.google.cloud.operators.datastore.CloudDatastoreExportEntitiesOperator`
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_export_import.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_export_import.py
:language: python
:dedent: 4
:start-after: [START how_to_export_task]
@@ -52,7 +52,7 @@ Import Entities
To import entities from Cloud Storage to Google Cloud Datastore use
:class:`~airflow.providers.google.cloud.operators.datastore.CloudDatastoreImportEntitiesOperator`
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_export_import.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_export_import.py
:language: python
:dedent: 4
:start-after: [START how_to_import_task]
@@ -66,7 +66,7 @@ Allocate Ids
To allocate IDs for incomplete keys use
:class:`~airflow.providers.google.cloud.operators.datastore.CloudDatastoreAllocateIdsOperator`
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_commit.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_commit.py
:language: python
:dedent: 4
:start-after: [START how_to_allocate_ids]
@@ -74,7 +74,7 @@ To allocate IDs for incomplete keys use
An example of a partial keys required by the operator:
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_commit.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_commit.py
:language: python
:dedent: 0
:start-after: [START how_to_keys_def]
@@ -88,7 +88,7 @@ Begin transaction
To begin a new transaction use
:class:`~airflow.providers.google.cloud.operators.datastore.CloudDatastoreBeginTransactionOperator`
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_commit.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_commit.py
:language: python
:dedent: 4
:start-after: [START how_to_begin_transaction]
@@ -96,7 +96,7 @@ To begin a new transaction use
An example of a transaction options required by the operator:
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_commit.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_commit.py
:language: python
:dedent: 0
:start-after: [START how_to_transaction_def]
@@ -110,7 +110,7 @@ Commit transaction
To commit a transaction, optionally creating, deleting or modifying some entities
use :class:`~airflow.providers.google.cloud.operators.datastore.CloudDatastoreCommitOperator`
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_commit.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_commit.py
:language: python
:dedent: 4
:start-after: [START how_to_commit_task]
@@ -118,7 +118,7 @@ use :class:`~airflow.providers.google.cloud.operators.datastore.CloudDatastoreCo
An example of a commit information required by the operator:
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_commit.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_commit.py
:language: python
:dedent: 0
:start-after: [START how_to_commit_def]
@@ -132,7 +132,7 @@ Run query
To run a query for entities use
:class:`~airflow.providers.google.cloud.operators.datastore.CloudDatastoreRunQueryOperator`
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_query.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_query.py
:language: python
:dedent: 4
:start-after: [START how_to_run_query]
@@ -140,7 +140,7 @@ To run a query for entities use
An example of a query required by the operator:
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_query.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_query.py
:language: python
:dedent: 0
:start-after: [START how_to_query_def]
@@ -154,7 +154,7 @@ Roll back transaction
To roll back a transaction
use :class:`~airflow.providers.google.cloud.operators.datastore.CloudDatastoreRollbackOperator`
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_rollback.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_rollback.py
:language: python
:dedent: 4
:start-after: [START how_to_rollback_transaction]
@@ -168,7 +168,7 @@ Get operation state
To get the current state of a long-running operation use
:class:`~airflow.providers.google.cloud.operators.datastore.CloudDatastoreGetOperationOperator`
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_export_import.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_export_import.py
:language: python
:dedent: 4
:start-after: [START get_operation_state]
@@ -182,7 +182,7 @@ Delete operation
To delete an operation use
:class:`~airflow.providers.google.cloud.operators.datastore.CloudDatastoreDeleteOperationOperator`
-.. exampleinclude:: /../../tests/system/providers/google/datastore/example_datastore_export_import.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/datastore/example_datastore_export_import.py
:language: python
:dedent: 4
:start-after: [START delete_operation]
diff --git a/docs/apache-airflow-providers-google/operators/cloud/kubernetes_engine.rst b/docs/apache-airflow-providers-google/operators/cloud/kubernetes_engine.rst
index 39da66207a965..23ed67d8aa9b1 100644
--- a/docs/apache-airflow-providers-google/operators/cloud/kubernetes_engine.rst
+++ b/docs/apache-airflow-providers-google/operators/cloud/kubernetes_engine.rst
@@ -43,7 +43,7 @@ Create GKE cluster
Here is an example of a cluster definition:
-.. exampleinclude:: /../../tests/system/providers/google/kubernetes_engine/example_kubernetes_engine.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py
:language: python
:start-after: [START howto_operator_gcp_gke_create_cluster_definition]
:end-before: [END howto_operator_gcp_gke_create_cluster_definition]
@@ -53,7 +53,7 @@ A dict object like this, or a
definition, is required when creating a cluster with
:class:`~airflow.providers.google.cloud.operators.kubernetes_engine.GKECreateClusterOperator`.
-.. exampleinclude:: /../../tests/system/providers/google/kubernetes_engine/example_kubernetes_engine.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py
:language: python
:dedent: 4
:start-after: [START howto_operator_gke_create_cluster]
@@ -68,7 +68,7 @@ To delete a cluster, use
:class:`~airflow.providers.google.cloud.operators.kubernetes_engine.GKEDeleteClusterOperator`.
This would also delete all the nodes allocated to the cluster.
-.. exampleinclude:: /../../tests/system/providers/google/kubernetes_engine/example_kubernetes_engine.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py
:language: python
:dedent: 4
:start-after: [START howto_operator_gke_delete_cluster]
@@ -117,7 +117,7 @@ is the path ``/airflow/xcom``. To provide values to the XCom, ensure your Pod wr
``return.json`` in the sidecar. The contents of this can then be used downstream in your DAG.
Here is an example of it being used:
-.. exampleinclude:: /../../tests/system/providers/google/kubernetes_engine/example_kubernetes_engine.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py
:language: python
:dedent: 4
:start-after: [START howto_operator_gke_start_pod_xcom]
@@ -125,7 +125,7 @@ Here is an example of it being used:
And then use it in other operators:
-.. exampleinclude:: /../../tests/system/providers/google/kubernetes_engine/example_kubernetes_engine.py
+.. exampleinclude:: /../../tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py
:language: python
:dedent: 4
:start-after: [START howto_operator_gke_xcom_result]
diff --git a/docs/apache-airflow-providers-http/index.rst b/docs/apache-airflow-providers-http/index.rst
index 2b7ff3930dd0f..e9bf6af3116b7 100644
--- a/docs/apache-airflow-providers-http/index.rst
+++ b/docs/apache-airflow-providers-http/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/http/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/http/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-jdbc/index.rst b/docs/apache-airflow-providers-jdbc/index.rst
index c46fc1e43a880..9cc096df79492 100644
--- a/docs/apache-airflow-providers-jdbc/index.rst
+++ b/docs/apache-airflow-providers-jdbc/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/jdbc/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/jdbc/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-jenkins/index.rst b/docs/apache-airflow-providers-jenkins/index.rst
index 9a23555000ed5..92107cd33d587 100644
--- a/docs/apache-airflow-providers-jenkins/index.rst
+++ b/docs/apache-airflow-providers-jenkins/index.rst
@@ -33,6 +33,12 @@ Content
Python API <_api/airflow/providers/jenkins/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/jenkins/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-microsoft-azure/index.rst b/docs/apache-airflow-providers-microsoft-azure/index.rst
index ddbc51d490c4a..681ee497f2adc 100644
--- a/docs/apache-airflow-providers-microsoft-azure/index.rst
+++ b/docs/apache-airflow-providers-microsoft-azure/index.rst
@@ -37,6 +37,12 @@ Content
Python API <_api/airflow/providers/microsoft/azure/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/microsoft/azure/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-microsoft-mssql/index.rst b/docs/apache-airflow-providers-microsoft-mssql/index.rst
index ac42801752ede..bb3df09b2cf85 100644
--- a/docs/apache-airflow-providers-microsoft-mssql/index.rst
+++ b/docs/apache-airflow-providers-microsoft-mssql/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/microsoft/mssql/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/microsoft/mssql/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-microsoft-winrm/index.rst b/docs/apache-airflow-providers-microsoft-winrm/index.rst
index 5f3d5d148a16e..1c15797fa605a 100644
--- a/docs/apache-airflow-providers-microsoft-winrm/index.rst
+++ b/docs/apache-airflow-providers-microsoft-winrm/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/microsoft/winrm/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/microsoft/winrm/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-mysql/index.rst b/docs/apache-airflow-providers-mysql/index.rst
index 6fb518d7e9a5f..90cfe7b235ddb 100644
--- a/docs/apache-airflow-providers-mysql/index.rst
+++ b/docs/apache-airflow-providers-mysql/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/mysql/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/mysql/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-opsgenie/index.rst b/docs/apache-airflow-providers-opsgenie/index.rst
index 3197d6d77fe56..776d096bdda58 100644
--- a/docs/apache-airflow-providers-opsgenie/index.rst
+++ b/docs/apache-airflow-providers-opsgenie/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/opsgenie/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/opsgenie/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-papermill/index.rst b/docs/apache-airflow-providers-papermill/index.rst
index 7effd4b35ca76..424ecb67398cd 100644
--- a/docs/apache-airflow-providers-papermill/index.rst
+++ b/docs/apache-airflow-providers-papermill/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/papermill/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/papermill/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-presto/index.rst b/docs/apache-airflow-providers-presto/index.rst
index b3c4b6f09f768..6c066e3602ade 100644
--- a/docs/apache-airflow-providers-presto/index.rst
+++ b/docs/apache-airflow-providers-presto/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/presto/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/presto/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-qubole/index.rst b/docs/apache-airflow-providers-qubole/index.rst
index 1e5b673d2cc80..aa687242250c3 100644
--- a/docs/apache-airflow-providers-qubole/index.rst
+++ b/docs/apache-airflow-providers-qubole/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/qubole/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/qubole/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-salesforce/index.rst b/docs/apache-airflow-providers-salesforce/index.rst
index 2da4b06569dee..37d7956ea4cf5 100644
--- a/docs/apache-airflow-providers-salesforce/index.rst
+++ b/docs/apache-airflow-providers-salesforce/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/salesforce/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/salesforce/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-singularity/index.rst b/docs/apache-airflow-providers-singularity/index.rst
index 32acfebd72098..75b63f96eb4de 100644
--- a/docs/apache-airflow-providers-singularity/index.rst
+++ b/docs/apache-airflow-providers-singularity/index.rst
@@ -28,6 +28,12 @@ Content
Python API <_api/airflow/providers/singularity/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/singularity/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-snowflake/index.rst b/docs/apache-airflow-providers-snowflake/index.rst
index 5d0a6b317cc14..2ff510af123ff 100644
--- a/docs/apache-airflow-providers-snowflake/index.rst
+++ b/docs/apache-airflow-providers-snowflake/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/snowflake/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/snowflake/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-tableau/index.rst b/docs/apache-airflow-providers-tableau/index.rst
index 41abe988f1b44..27d08fe67a153 100644
--- a/docs/apache-airflow-providers-tableau/index.rst
+++ b/docs/apache-airflow-providers-tableau/index.rst
@@ -30,6 +30,12 @@ Content
Operators
Python API <_api/airflow/providers/tableau/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/tableau/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-telegram/index.rst b/docs/apache-airflow-providers-telegram/index.rst
index 782de1e712d21..f3a9a3fd87763 100644
--- a/docs/apache-airflow-providers-telegram/index.rst
+++ b/docs/apache-airflow-providers-telegram/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/telegram/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/telegram/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-trino/index.rst b/docs/apache-airflow-providers-trino/index.rst
index 239ec03cd8002..c56a9b85897db 100644
--- a/docs/apache-airflow-providers-trino/index.rst
+++ b/docs/apache-airflow-providers-trino/index.rst
@@ -34,6 +34,12 @@ Content
Python API <_api/airflow/providers/trino/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/trino/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow-providers-yandex/index.rst b/docs/apache-airflow-providers-yandex/index.rst
index 7ad2303510141..7699d53bd2c04 100644
--- a/docs/apache-airflow-providers-yandex/index.rst
+++ b/docs/apache-airflow-providers-yandex/index.rst
@@ -35,6 +35,12 @@ Content
Python API <_api/airflow/providers/yandex/index>
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/yandex/index>
+
.. toctree::
:maxdepth: 1
:caption: Resources
diff --git a/docs/apache-airflow/python-api-ref.rst b/docs/apache-airflow/python-api-ref.rst
index e8c4dff3c499c..28a4545859db6 100644
--- a/docs/apache-airflow/python-api-ref.rst
+++ b/docs/apache-airflow/python-api-ref.rst
@@ -154,3 +154,12 @@ schedule DAG runs in ways not possible with built-in schedule expressions.
:maxdepth: 1
_api/airflow/timetables/index
+
+Example DAGs
+------------
+
+.. toctree::
+ :includehidden:
+ :maxdepth: 1
+
+ _api/airflow/example_dags/index
diff --git a/docs/build_docs.py b/docs/build_docs.py
index d7edad1cad97e..9f4217bd106c4 100755
--- a/docs/build_docs.py
+++ b/docs/build_docs.py
@@ -15,17 +15,19 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
+
+
import argparse
import multiprocessing
import os
import sys
from collections import defaultdict
-from typing import Dict, List, NamedTuple, Optional, Tuple
+from itertools import filterfalse, tee
+from typing import Callable, Dict, Iterable, List, NamedTuple, Optional, Tuple, TypeVar
from rich.console import Console
from tabulate import tabulate
-from airflow.utils.helpers import partition
from docs.exts.docs_build import dev_index_generator, lint_checks
from docs.exts.docs_build.code_utils import CONSOLE_WIDTH, PROVIDER_INIT_FILE
from docs.exts.docs_build.docs_builder import DOCS_DIR, AirflowDocsBuilder, get_available_packages
@@ -62,6 +64,14 @@
console = Console(force_terminal=True, color_system="standard", width=CONSOLE_WIDTH)
+T = TypeVar('T')
+
+
+def partition(pred: Callable[[T], bool], iterable: Iterable[T]) -> Tuple[Iterable[T], Iterable[T]]:
+ """Use a predicate to partition entries into false entries and true entries"""
+ iter_1, iter_2 = tee(iterable)
+ return filterfalse(pred, iter_1), filter(pred, iter_2)
+
def _promote_new_flags():
console.print()
diff --git a/docs/conf.py b/docs/conf.py
index b1742f0c00202..47334381fc8e4 100644
--- a/docs/conf.py
+++ b/docs/conf.py
@@ -35,20 +35,17 @@
import os
import sys
from collections import defaultdict
+from pathlib import Path
from typing import Any, Dict, List, Optional, Tuple
import yaml
-try:
- from yaml import CSafeLoader as SafeLoader
-except ImportError:
- from yaml import SafeLoader # type: ignore[misc]
-
import airflow
from airflow.configuration import AirflowConfigParser, default_config_yaml
-from docs.exts.docs_build.third_party_inventories import THIRD_PARTY_INDEXES
-sys.path.append(os.path.join(os.path.dirname(__file__), 'exts'))
+sys.path.append(str(Path(__file__).parent / 'exts'))
+
+from docs_build.third_party_inventories import THIRD_PARTY_INDEXES # noqa: E402
CONF_DIR = os.path.abspath(os.path.join(os.path.dirname(__file__)))
INVENTORY_CACHE_DIR = os.path.join(CONF_DIR, '_inventory_cache')
@@ -61,6 +58,7 @@
if PACKAGE_NAME == 'apache-airflow':
PACKAGE_DIR = os.path.join(ROOT_DIR, 'airflow')
PACKAGE_VERSION = airflow.__version__
+ SYSTEM_TESTS_DIR = None
elif PACKAGE_NAME.startswith('apache-airflow-providers-'):
from provider_yaml_utils import load_package_data
@@ -75,23 +73,27 @@
raise Exception(f"Could not find provider.yaml file for package: {PACKAGE_NAME}")
PACKAGE_DIR = CURRENT_PROVIDER['package-dir']
PACKAGE_VERSION = CURRENT_PROVIDER['versions'][0]
+ SYSTEM_TESTS_DIR = CURRENT_PROVIDER['system-tests-dir']
elif PACKAGE_NAME == 'apache-airflow-providers':
from provider_yaml_utils import load_package_data
PACKAGE_DIR = os.path.join(ROOT_DIR, 'airflow', 'providers')
PACKAGE_VERSION = 'devel'
ALL_PROVIDER_YAMLS = load_package_data()
+ SYSTEM_TESTS_DIR = None
elif PACKAGE_NAME == 'helm-chart':
PACKAGE_DIR = os.path.join(ROOT_DIR, 'chart')
CHART_YAML_FILE = os.path.join(PACKAGE_DIR, 'Chart.yaml')
with open(CHART_YAML_FILE) as chart_file:
- chart_yaml_contents = yaml.load(chart_file, SafeLoader)
+ chart_yaml_contents = yaml.safe_load(chart_file)
PACKAGE_VERSION = chart_yaml_contents['version']
+ SYSTEM_TESTS_DIR = None
else:
PACKAGE_DIR = None
PACKAGE_VERSION = 'devel'
+ SYSTEM_TESTS_DIR = None
# Adds to environment variables for easy access from other plugins like airflow_intersphinx.
os.environ['AIRFLOW_PACKAGE_NAME'] = PACKAGE_NAME
if PACKAGE_DIR:
@@ -220,6 +222,7 @@ def _get_rst_filepath_from_path(filepath: str):
exclude_patterns.append(f"_api/airflow/{name.rpartition('.')[0]}")
browsable_packages = [
"hooks",
+ "example_dags",
"executors",
"models",
"operators",
@@ -318,9 +321,12 @@ def _get_rst_filepath_from_path(filepath: str):
html_show_copyright = False
# Theme configuration
-html_theme_options: Dict[str, Any] = {
- 'hide_website_buttons': True,
-}
+if PACKAGE_NAME.startswith('apache-airflow-providers-'):
+ # Only hide hidden items for providers. For Chart and Airflow we are using the approach where
+ # TOC is hidden but sidebar still shows the content (but we are not doing it for providers).
+ html_theme_options: Dict[str, Any] = {'hide_website_buttons': True, 'sidebar_includehidden': False}
+else:
+ html_theme_options = {'hide_website_buttons': True, 'sidebar_includehidden': True}
if FOR_PRODUCTION:
html_theme_options['navbar_links'] = [
{'href': '/community/', 'text': 'Community'},
@@ -415,7 +421,7 @@ def _load_config():
return {}
with open(file_path) as f:
- return yaml.load(f, SafeLoader)
+ return yaml.safe_load(f)
config = _load_config()
jinja_contexts = {
@@ -668,6 +674,9 @@ def _get_params(root_schema: dict, prefix: str = "", default_section: str = "")
PACKAGE_DIR,
]
+if SYSTEM_TESTS_DIR and os.path.exists(SYSTEM_TESTS_DIR):
+ autoapi_dirs.append(SYSTEM_TESTS_DIR)
+
# A directory that has user-defined templates to override our default templates.
if PACKAGE_NAME == 'apache-airflow':
autoapi_template_dir = 'autoapi_templates'
@@ -675,11 +684,13 @@ def _get_params(root_schema: dict, prefix: str = "", default_section: str = "")
# A list of patterns to ignore when finding files
autoapi_ignore = [
'*/airflow/_vendor/*',
- '*/example_dags/*',
'*/_internal*',
'*/node_modules/*',
'*/migrations/*',
'*/contrib/*',
+ '**/example_sla_dag.py',
+ '**/example_taskflow_api_etl_docker_virtualenv.py',
+ '**/example_dag_decorator.py',
]
if PACKAGE_NAME == 'apache-airflow':
autoapi_ignore.append('*/airflow/providers/*')
diff --git a/docs/exts/docs_build/docs_builder.py b/docs/exts/docs_build/docs_builder.py
index cc65f3f1700fc..b8fcf6fd64ecf 100644
--- a/docs/exts/docs_build/docs_builder.py
+++ b/docs/exts/docs_build/docs_builder.py
@@ -24,7 +24,7 @@
from rich.console import Console
-from docs.exts.docs_build.code_utils import (
+from .code_utils import (
AIRFLOW_SITE_DIR,
ALL_PROVIDER_YAMLS,
CONSOLE_WIDTH,
@@ -32,9 +32,9 @@
PROCESS_TIMEOUT,
pretty_format_path,
)
-from docs.exts.docs_build.errors import DocBuildError, parse_sphinx_warnings
-from docs.exts.docs_build.helm_chart_utils import chart_version
-from docs.exts.docs_build.spelling_checks import SpellingError, parse_spelling_warnings
+from .errors import DocBuildError, parse_sphinx_warnings
+from .helm_chart_utils import chart_version
+from .spelling_checks import SpellingError, parse_spelling_warnings
console = Console(force_terminal=True, color_system="standard", width=CONSOLE_WIDTH)
diff --git a/docs/exts/provider_yaml_utils.py b/docs/exts/provider_yaml_utils.py
index a6d1ee297d560..085cd480c722f 100644
--- a/docs/exts/provider_yaml_utils.py
+++ b/docs/exts/provider_yaml_utils.py
@@ -18,19 +18,14 @@
import json
import os
from glob import glob
+from pathlib import Path
from typing import Any, Dict, List
import jsonschema
import yaml
-try:
- from yaml import CSafeLoader as SafeLoader
-except ImportError:
- from yaml import SafeLoader # type: ignore[misc]
-
-
-ROOT_DIR = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir, os.pardir))
-PROVIDER_DATA_SCHEMA_PATH = os.path.join(ROOT_DIR, "airflow", "provider.yaml.schema.json")
+ROOT_DIR = Path(__file__).parents[2].resolve()
+PROVIDER_DATA_SCHEMA_PATH = ROOT_DIR / "airflow" / "provider.yaml.schema.json"
def _load_schema() -> Dict[str, Any]:
@@ -40,8 +35,17 @@ def _load_schema() -> Dict[str, Any]:
def _filepath_to_module(filepath: str):
- filepath = os.path.relpath(os.path.abspath(filepath), ROOT_DIR)
- return filepath.replace("/", ".")
+ return str(Path(filepath).relative_to(ROOT_DIR)).replace("/", ".")
+
+
+def _filepath_to_system_tests(filepath: str):
+ return str(
+ ROOT_DIR
+ / "tests"
+ / "system"
+ / "providers"
+ / Path(filepath).relative_to(ROOT_DIR / "airflow" / "providers")
+ )
def get_provider_yaml_paths():
@@ -59,12 +63,14 @@ def load_package_data() -> List[Dict[str, Any]]:
result = []
for provider_yaml_path in get_provider_yaml_paths():
with open(provider_yaml_path) as yaml_file:
- provider = yaml.load(yaml_file, SafeLoader)
+ provider = yaml.safe_load(yaml_file)
try:
jsonschema.validate(provider, schema=schema)
except jsonschema.ValidationError:
raise Exception(f"Unable to parse: {provider_yaml_path}.")
- provider['python-module'] = _filepath_to_module(os.path.dirname(provider_yaml_path))
- provider['package-dir'] = os.path.dirname(provider_yaml_path)
+ provider_yaml_dir = os.path.dirname(provider_yaml_path)
+ provider['python-module'] = _filepath_to_module(provider_yaml_dir)
+ provider['package-dir'] = provider_yaml_dir
+ provider['system-tests-dir'] = _filepath_to_system_tests(provider_yaml_dir)
result.append(provider)
return result
diff --git a/docs/publish_docs.py b/docs/publish_docs.py
index 60c89d10e424c..7451033f16c20 100755
--- a/docs/publish_docs.py
+++ b/docs/publish_docs.py
@@ -17,6 +17,7 @@
# specific language governing permissions and limitations
# under the License.
+
import argparse
import os
@@ -26,6 +27,7 @@
AIRFLOW_SITE_DIR = os.environ.get('AIRFLOW_SITE_DIRECTORY')
+
if __name__ != "__main__":
raise SystemExit(
"This file is intended to be executed as an executable program. You cannot use it as a module."
diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt
index b9f2aad173a90..69130e7d058d6 100644
--- a/docs/spelling_wordlist.txt
+++ b/docs/spelling_wordlist.txt
@@ -193,6 +193,7 @@ Hou
Http
HttpError
HttpRequest
+IGM
IdP
ImageAnnotatorClient
Imap
@@ -281,6 +282,7 @@ OSS
Oauth
Oauthlib
Okta
+OnFailure
Oozie
Opsgenie
Optimise
@@ -794,6 +796,7 @@ evals
eventlet
evo
exasol
+executables
execvp
exitcode
explicit
@@ -819,12 +822,14 @@ filehandle
fileloc
filelocs
filepath
+fileshare
filesize
filesystem
filesystems
filetype
finalizers
findall
+firestore
firstname
fluentd
fmt
@@ -873,6 +878,7 @@ greenlet
groupId
grpc
gz
+gzipped
hadoop
hadoopcmd
hardcoded
@@ -1184,6 +1190,7 @@ preloading
prepend
prepended
preprocess
+preprocessing
presign
presigned
prestocmd
@@ -1339,6 +1346,7 @@ sourceRepository
sourceUploadUrl
sparkApplication
sparkcmd
+sparkr
sparksql
spegno
spotahome
@@ -1469,6 +1477,7 @@ tsql
tsv
ttl
tunables
+twitterHandle
txt
typeahead
tz
diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt
index c6b12db9fa491..96155ac41a84c 100644
--- a/images/breeze/output-commands-hash.txt
+++ b/images/breeze/output-commands-hash.txt
@@ -1 +1 @@
-ed70b784364f0b604176ea142305906b
+2f24dd5bade0ac4b635ef6ada358590b
diff --git a/images/breeze/output-static-checks.svg b/images/breeze/output-static-checks.svg
index edbc2fee71d47..1e8d54ca8ec49 100644
--- a/images/breeze/output-static-checks.svg
+++ b/images/breeze/output-static-checks.svg
@@ -1,4 +1,4 @@
-
+
-
-
+
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
+
+
+
- Command: static-checks
+ Command: static-checks
-
+
-
-
-Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]...
-
-Run static checks.
-
-╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮
-│--type-tType(s) of the static checks to run (multiple can be added). │
-│(all | black | blacken-docs | check-airflow-2-1-compatibility | │
-│check-airflow-config-yaml-consistent | check-airflow-providers-have-extras | │
-│check-apache-license-rat | check-base-operator-usage | check-boring-cyborg-configuration│
-│| check-breeze-top-dependencies-limited | check-builtin-literals | │
-│check-changelog-has-no-duplicates | check-daysago-import-from-utils | │
-│check-docstring-param-types | check-executables-have-shebangs | │
-│check-extra-packages-references | check-extras-order | check-for-inclusive-language | │
-│check-hooks-apply | check-incorrect-use-of-LoggingMixin | │
-│check-integrations-are-consistent | check-merge-conflict | check-newsfragments-are-valid│
-│| check-no-providers-in-core-examples | check-no-relative-imports | │
-│check-persist-credentials-disabled-in-github-workflows | │
-│check-pre-commit-information-consistent | check-provide-create-sessions-imports | │
-│check-provider-yaml-valid | check-providers-init-file-missing | │
-│check-providers-subpackages-init-file-exist | check-pydevd-left-in-code | │
-│check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order | │
-│check-start-date-not-used-in-defaults | check-system-tests-present | check-xml | │
-│codespell | debug-statements | detect-private-key | doctoc | end-of-file-fixer | │
-│fix-encoding-pragma | flynt | forbid-tabs | identity | insert-license | isort | │
-│lint-chart-schema | lint-css | lint-dockerfile | lint-helm-chart | lint-javascript | │
-│lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending | pretty-format-json│
-│| pydocstyle | python-no-log-warn | pyupgrade | rst-backticks | run-flake8 | run-mypy | │
-│run-shellcheck | static-check-autoflake | trailing-whitespace | update-breeze-file | │
-│update-breeze-readme-config-hash | update-extras | update-in-the-wild-to-be-sorted | │
-│update-inlined-dockerfile-scripts | update-local-yml-file | update-migration-references │
-│| update-providers-dependencies | update-setup-cfg-file | │
-│update-spelling-wordlist-to-be-sorted | update-supported-versions | │
-│update-vendored-in-k8s-json-schema | update-version | yamllint | yesqa) │
-│--file-fList of files to run the checks on.(PATH)│
-│--all-files-aRun checks on all files.│
-│--show-diff-on-failure-sShow diff for files modified by the checks.│
-│--last-commit-cRun checks for all files in last commit. Mutually exclusive with --commit-ref.│
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│--commit-ref-rRun checks for this commit reference only (can be any git commit-ish reference). Mutually │
-│exclusive with --last-commit. │
-│(TEXT) │
-│--verbose-vPrint verbose information about performed steps.│
-│--dry-run-DIf dry-run is set, commands are only printed, not executed.│
-│--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow]│
-│--help-hShow this message and exit.│
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+
+Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]...
+
+Run static checks.
+
+╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮
+│--type-tType(s) of the static checks to run (multiple can be added). │
+│(all | black | blacken-docs | check-airflow-2-1-compatibility | │
+│check-airflow-config-yaml-consistent | check-airflow-providers-have-extras | │
+│check-apache-license-rat | check-base-operator-usage | check-boring-cyborg-configuration│
+│| check-breeze-top-dependencies-limited | check-builtin-literals | │
+│check-changelog-has-no-duplicates | check-daysago-import-from-utils | │
+│check-docstring-param-types | check-executables-have-shebangs | │
+│check-extra-packages-references | check-extras-order | check-for-inclusive-language | │
+│check-hooks-apply | check-incorrect-use-of-LoggingMixin | │
+│check-integrations-are-consistent | check-merge-conflict | check-newsfragments-are-valid│
+│| check-no-providers-in-core-examples | check-no-relative-imports | │
+│check-persist-credentials-disabled-in-github-workflows | │
+│check-pre-commit-information-consistent | check-provide-create-sessions-imports | │
+│check-provider-yaml-valid | check-providers-init-file-missing | │
+│check-providers-subpackages-init-file-exist | check-pydevd-left-in-code | │
+│check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order | │
+│check-start-date-not-used-in-defaults | check-system-tests-present | │
+│check-system-tests-tocs | check-xml | codespell | create-missing-init-py-files-tests | │
+│debug-statements | detect-private-key | doctoc | end-of-file-fixer | fix-encoding-pragma│
+│| flynt | forbid-tabs | identity | insert-license | isort | lint-chart-schema | lint-css│
+│| lint-dockerfile | lint-helm-chart | lint-javascript | lint-json-schema | lint-markdown│
+│| lint-openapi | mixed-line-ending | pretty-format-json | pydocstyle | │
+│python-no-log-warn | pyupgrade | rst-backticks | run-flake8 | run-mypy | run-shellcheck │
+│| static-check-autoflake | trailing-whitespace | update-breeze-file | │
+│update-breeze-readme-config-hash | update-extras | update-in-the-wild-to-be-sorted | │
+│update-inlined-dockerfile-scripts | update-local-yml-file | update-migration-references │
+│| update-providers-dependencies | update-setup-cfg-file | │
+│update-spelling-wordlist-to-be-sorted | update-supported-versions | │
+│update-vendored-in-k8s-json-schema | update-version | yamllint | yesqa) │
+│--file-fList of files to run the checks on.(PATH)│
+│--all-files-aRun checks on all files.│
+│--show-diff-on-failure-sShow diff for files modified by the checks.│
+│--last-commit-cRun checks for all files in last commit. Mutually exclusive with --commit-ref.│
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│--commit-ref-rRun checks for this commit reference only (can be any git commit-ish reference). Mutually │
+│exclusive with --last-commit. │
+│(TEXT) │
+│--verbose-vPrint verbose information about performed steps.│
+│--dry-run-DIf dry-run is set, commands are only printed, not executed.│
+│--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow]│
+│--help-hShow this message and exit.│
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/scripts/ci/pre_commit/pre_commit_check_init_in_tests.py b/scripts/ci/pre_commit/pre_commit_check_init_in_tests.py
new file mode 100755
index 0000000000000..947f284e1661d
--- /dev/null
+++ b/scripts/ci/pre_commit/pre_commit_check_init_in_tests.py
@@ -0,0 +1,50 @@
+#!/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
+# 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 os
+import pathlib
+import sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+ raise SystemExit(
+ "This file is intended to be executed as an executable program. You cannot use it as a module."
+ f"To execute this script, run ./{__file__} [FILE] ..."
+ )
+
+ROOT_DIR = pathlib.Path(__file__).resolve().parents[3]
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+added = False
+
+if __name__ == '__main__':
+ for dir, sub_dirs, files in os.walk(str(ROOT_DIR / "tests")):
+ for sub_dir in sub_dirs:
+ dir_to_check = dir + os.sep + sub_dir
+ init_py_path = Path(dir_to_check) / "__init__.py"
+ if not init_py_path.exists() and "/test_logs/" not in str(init_py_path):
+ init_py_path.touch()
+ console.print(f"[yellow] Created {init_py_path}[/]")
+ added = True
+ sys.exit(1 if added else 0)
diff --git a/scripts/ci/pre_commit/pre_commit_check_setup_extra_packages_ref.py b/scripts/ci/pre_commit/pre_commit_check_setup_extra_packages_ref.py
index 6097ac22512b5..28d2bbc7c9e5b 100755
--- a/scripts/ci/pre_commit/pre_commit_check_setup_extra_packages_ref.py
+++ b/scripts/ci/pre_commit/pre_commit_check_setup_extra_packages_ref.py
@@ -59,7 +59,7 @@ def get_extras_from_setup() -> Set[str]:
def get_extras_from_docs() -> Set[str]:
"""
- Returns a list of extras from docs.
+ Returns a list of extras from airflow.docs.
"""
docs_content = get_file_content(DOCS_FILE)
extras_section_regex = re.compile(
@@ -86,7 +86,7 @@ def get_preinstalled_providers_from_docs() -> List[str]:
def get_deprecated_extras_from_docs() -> Dict[str, str]:
"""
- Returns dict of deprecated extras from docs (alias -> target extra)
+ Returns dict of deprecated extras from airflow.docs (alias -> target extra)
"""
deprecated_extras = {}
docs_content = get_file_content(DOCS_FILE)
diff --git a/scripts/ci/pre_commit/pre_commit_check_system_tests_hidden_in_index.py b/scripts/ci/pre_commit/pre_commit_check_system_tests_hidden_in_index.py
new file mode 100755
index 0000000000000..72f8b4fe78cc0
--- /dev/null
+++ b/scripts/ci/pre_commit/pre_commit_check_system_tests_hidden_in_index.py
@@ -0,0 +1,72 @@
+#!/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
+# 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 sys
+from pathlib import Path
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+ raise SystemExit(
+ "This file is intended to be executed as an executable program. You cannot use it as a module."
+ f"To run this script, run the ./{__file__} command [FILE] ..."
+ )
+
+
+console = Console(color_system="standard", width=200)
+
+AIRFLOW_SOURCES_ROOT = Path(__file__).parents[3].resolve()
+DOCS_ROOT = AIRFLOW_SOURCES_ROOT / "docs"
+
+PREFIX = "apache-airflow-providers-"
+
+
+errors = []
+
+
+def check_system_test_entry_hidden(provider_index: Path):
+ console.print(f"[bright_blue]Checking {provider_index}")
+ provider_folder = provider_index.parent.name
+ if not provider_folder.startswith(PREFIX):
+ console.print(f"[red]Bad provider index passed: {provider_index}")
+ errors.append(provider_index)
+ provider_path = provider_folder[len(PREFIX) :].replace("-", "/")
+ expected_text = f"""
+.. toctree::
+ :hidden:
+ :caption: System tests
+
+ System Tests <_api/tests/system/providers/{provider_path}/index>
+"""
+ index_text = provider_index.read_text()
+ system_tests_path = AIRFLOW_SOURCES_ROOT / "tests" / "system" / "providers" / provider_path
+ if system_tests_path.exists():
+ if expected_text not in index_text:
+ console.print(f"[red]The {provider_index} does not contain System Tests TOC.\n")
+ console.print(f"[yellow]Make sure to add those lines to {provider_index}:\n")
+ console.print(expected_text, markup=False)
+ errors.append(provider_index)
+ else:
+ console.print(f"[green]All ok. The {provider_index} contains hidden index.\n")
+ else:
+ console.print(f"[yellow]All ok. The {provider_index} does not contain system tests.\n")
+
+
+if __name__ == '__main__':
+ for file in sys.argv[1:]:
+ check_system_test_entry_hidden(Path(file))
+ sys.exit(0 if len(errors) == 0 else 1)
diff --git a/scripts/in_container/run_docs_build.sh b/scripts/in_container/run_docs_build.sh
index b96be023048c9..85e7c85dafae3 100755
--- a/scripts/in_container/run_docs_build.sh
+++ b/scripts/in_container/run_docs_build.sh
@@ -18,7 +18,8 @@
# shellcheck source=scripts/in_container/_in_container_script_init.sh
. "$( dirname "${BASH_SOURCE[0]}" )/_in_container_script_init.sh"
-sudo -E "${AIRFLOW_SOURCES}/docs/build_docs.py" "${@}"
+cd "${AIRFLOW_SOURCES}" || exit 1
+python -m docs.build_docs "${@}"
if [[ ( ${CI:="false"} == "true" || ${CI} == "True" ) && -d "${AIRFLOW_SOURCES}/docs/_build/docs/" ]]; then
diff --git a/setup.cfg b/setup.cfg
index 8d996337ebfb2..1512a6201c46e 100644
--- a/setup.cfg
+++ b/setup.cfg
@@ -229,7 +229,7 @@ no_implicit_optional = False
line_length=110
combine_as_imports = true
default_section = THIRDPARTY
-known_first_party=airflow,airflow_breeze,tests
+known_first_party=airflow,airflow_breeze,tests,docs
# Need to be consistent with the exclude config defined in pre-commit-config.yaml
skip=build,.tox,venv
profile = black
diff --git a/tests/config_templates/__init__.py b/tests/config_templates/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/config_templates/__init__.py
@@ -0,0 +1,16 @@
+# 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/core/__init__.py b/tests/core/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/core/__init__.py
@@ -0,0 +1,16 @@
+# 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/dags/__init__.py b/tests/dags/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/dags/__init__.py
@@ -0,0 +1,16 @@
+# 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/dags/subdir1/__init__.py b/tests/dags/subdir1/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/dags/subdir1/__init__.py
@@ -0,0 +1,16 @@
+# 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/dags/subdir2/__init__.py b/tests/dags/subdir2/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/dags/subdir2/__init__.py
@@ -0,0 +1,16 @@
+# 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/dags/subdir2/subdir3/__init__.py b/tests/dags/subdir2/subdir3/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/dags/subdir2/subdir3/__init__.py
@@ -0,0 +1,16 @@
+# 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/dags_corrupted/__init__.py b/tests/dags_corrupted/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/dags_corrupted/__init__.py
@@ -0,0 +1,16 @@
+# 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/dags_with_system_exit/__init__.py b/tests/dags_with_system_exit/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/dags_with_system_exit/__init__.py
@@ -0,0 +1,16 @@
+# 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/executors/kubernetes_executor_template_files/__init__.py b/tests/executors/kubernetes_executor_template_files/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/executors/kubernetes_executor_template_files/__init__.py
@@ -0,0 +1,16 @@
+# 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/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py
index 29642e493bf16..6f50b1c759a1e 100644
--- a/tests/jobs/test_scheduler_job.py
+++ b/tests/jobs/test_scheduler_job.py
@@ -2665,6 +2665,7 @@ def test_list_py_file_paths(self):
'test_ignore_this.py',
'test_invalid_param.py',
'test_nested_dag.py',
+ '__init__.py',
}
for root, _, files in os.walk(TEST_DAG_FOLDER):
for file_name in files:
diff --git a/tests/secrets/__init__.py b/tests/secrets/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/secrets/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/README.md b/tests/system/README.md
index 912bfc248ce0d..c1452bfe6e5df 100644
--- a/tests/system/README.md
+++ b/tests/system/README.md
@@ -67,7 +67,7 @@ example of command:
```commandline
# pytest --system [provider_name] [path_to_test(s)]
-pytest --system google tests/system/providers/google/bigquery/example_bigquery_queries.py
+pytest --system google tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py
```
You can specify several `--system` flags if you want to execute tests for several providers:
diff --git a/tests/system/providers/amazon/__init__.py b/tests/system/providers/amazon/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/amazon/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/amazon/aws/__init__.py b/tests/system/providers/amazon/aws/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/amazon/aws/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/apache/kylin/__init__.py b/tests/system/providers/apache/kylin/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/apache/kylin/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/cncf/__init__.py b/tests/system/providers/cncf/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/cncf/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/elasticsearch/__init__.py b/tests/system/providers/elasticsearch/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/elasticsearch/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/github/__init__.py b/tests/system/providers/github/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/github/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/README.md b/tests/system/providers/google/README.md
index 35d07c8e1e72d..da8709cb901e1 100644
--- a/tests/system/providers/google/README.md
+++ b/tests/system/providers/google/README.md
@@ -23,7 +23,7 @@
All Google-related system tests are located inside this subdirectory of system tests which is
`tests/system/providers/google/`. They are grouped in directories by the related service name, e.g. all BigQuery
-tests are stored inside `tests/system/providers/google/bigquery/` directory. In each directory you will find test files
+tests are stored inside `tests/system/providers/google/cloud/bigquery/` directory. In each directory you will find test files
as self-contained DAGs (one DAG per file). Each test may require some additional resources which should be placed in
`resources` directory found on the same level as tests. Each test file should start with prefix `example_*`. If there
is anything more needed for the test to be executed, it should be documented in the docstrings.
diff --git a/tests/system/providers/google/cloud/__init__.py b/tests/system/providers/google/cloud/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/cloud/azure/__init__.py b/tests/system/providers/google/cloud/azure/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/azure/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/cloud/bigquery/__init__.py b/tests/system/providers/google/cloud/bigquery/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/bigquery/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/bigquery/example_bigquery_dataset.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py
similarity index 100%
rename from tests/system/providers/google/bigquery/example_bigquery_dataset.py
rename to tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py
diff --git a/tests/system/providers/google/bigquery/example_bigquery_operations.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_operations.py
similarity index 100%
rename from tests/system/providers/google/bigquery/example_bigquery_operations.py
rename to tests/system/providers/google/cloud/bigquery/example_bigquery_operations.py
diff --git a/tests/system/providers/google/bigquery/example_bigquery_operations_location.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_operations_location.py
similarity index 100%
rename from tests/system/providers/google/bigquery/example_bigquery_operations_location.py
rename to tests/system/providers/google/cloud/bigquery/example_bigquery_operations_location.py
diff --git a/tests/system/providers/google/bigquery/example_bigquery_queries.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py
similarity index 100%
rename from tests/system/providers/google/bigquery/example_bigquery_queries.py
rename to tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py
diff --git a/tests/system/providers/google/bigquery/example_bigquery_sensors.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_sensors.py
similarity index 100%
rename from tests/system/providers/google/bigquery/example_bigquery_sensors.py
rename to tests/system/providers/google/cloud/bigquery/example_bigquery_sensors.py
diff --git a/tests/system/providers/google/bigquery/example_bigquery_tables.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
similarity index 100%
rename from tests/system/providers/google/bigquery/example_bigquery_tables.py
rename to tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py
diff --git a/tests/system/providers/google/bigquery/example_bigquery_to_bigquery.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_bigquery.py
similarity index 100%
rename from tests/system/providers/google/bigquery/example_bigquery_to_bigquery.py
rename to tests/system/providers/google/cloud/bigquery/example_bigquery_to_bigquery.py
diff --git a/tests/system/providers/google/bigquery/example_bigquery_to_gcs.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_gcs.py
similarity index 100%
rename from tests/system/providers/google/bigquery/example_bigquery_to_gcs.py
rename to tests/system/providers/google/cloud/bigquery/example_bigquery_to_gcs.py
diff --git a/tests/system/providers/google/cloud/bigquery/resources/__init__.py b/tests/system/providers/google/cloud/bigquery/resources/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/bigquery/resources/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/bigquery/resources/example_bigquery_query.sql b/tests/system/providers/google/cloud/bigquery/resources/example_bigquery_query.sql
similarity index 100%
rename from tests/system/providers/google/bigquery/resources/example_bigquery_query.sql
rename to tests/system/providers/google/cloud/bigquery/resources/example_bigquery_query.sql
diff --git a/tests/system/providers/google/bigquery/resources/update_table_schema.json b/tests/system/providers/google/cloud/bigquery/resources/update_table_schema.json
similarity index 100%
rename from tests/system/providers/google/bigquery/resources/update_table_schema.json
rename to tests/system/providers/google/cloud/bigquery/resources/update_table_schema.json
diff --git a/tests/system/providers/google/bigquery/resources/us-states.csv b/tests/system/providers/google/cloud/bigquery/resources/us-states.csv
similarity index 100%
rename from tests/system/providers/google/bigquery/resources/us-states.csv
rename to tests/system/providers/google/cloud/bigquery/resources/us-states.csv
diff --git a/tests/system/providers/google/cloud/cloud_sql/__init__.py b/tests/system/providers/google/cloud/cloud_sql/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/cloud_sql/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/cloud/dataproc/__init__.py b/tests/system/providers/google/cloud/dataproc/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/dataproc/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/dataproc/example_dataproc_gke.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_gke.py
similarity index 100%
rename from tests/system/providers/google/dataproc/example_dataproc_gke.py
rename to tests/system/providers/google/cloud/dataproc/example_dataproc_gke.py
diff --git a/tests/system/providers/google/cloud/dataproc/resources/__init__.py b/tests/system/providers/google/cloud/dataproc/resources/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/dataproc/resources/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/cloud/datastore/__init__.py b/tests/system/providers/google/cloud/datastore/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/datastore/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/datastore/example_datastore_commit.py b/tests/system/providers/google/cloud/datastore/example_datastore_commit.py
similarity index 100%
rename from tests/system/providers/google/datastore/example_datastore_commit.py
rename to tests/system/providers/google/cloud/datastore/example_datastore_commit.py
diff --git a/tests/system/providers/google/datastore/example_datastore_export_import.py b/tests/system/providers/google/cloud/datastore/example_datastore_export_import.py
similarity index 100%
rename from tests/system/providers/google/datastore/example_datastore_export_import.py
rename to tests/system/providers/google/cloud/datastore/example_datastore_export_import.py
diff --git a/tests/system/providers/google/datastore/example_datastore_query.py b/tests/system/providers/google/cloud/datastore/example_datastore_query.py
similarity index 100%
rename from tests/system/providers/google/datastore/example_datastore_query.py
rename to tests/system/providers/google/cloud/datastore/example_datastore_query.py
diff --git a/tests/system/providers/google/datastore/example_datastore_rollback.py b/tests/system/providers/google/cloud/datastore/example_datastore_rollback.py
similarity index 100%
rename from tests/system/providers/google/datastore/example_datastore_rollback.py
rename to tests/system/providers/google/cloud/datastore/example_datastore_rollback.py
diff --git a/tests/system/providers/google/cloud/gcs/__init__.py b/tests/system/providers/google/cloud/gcs/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/gcs/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/gcs/example_gcs_to_bigquery.py b/tests/system/providers/google/cloud/gcs/example_gcs_to_bigquery.py
similarity index 100%
rename from tests/system/providers/google/gcs/example_gcs_to_bigquery.py
rename to tests/system/providers/google/cloud/gcs/example_gcs_to_bigquery.py
diff --git a/tests/system/providers/google/cloud/gcs/resources/__init__.py b/tests/system/providers/google/cloud/gcs/resources/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/gcs/resources/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/cloud/kubernetes_engine/__init__.py b/tests/system/providers/google/cloud/kubernetes_engine/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/kubernetes_engine/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/kubernetes_engine/example_kubernetes_engine.py b/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py
similarity index 100%
rename from tests/system/providers/google/kubernetes_engine/example_kubernetes_engine.py
rename to tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py
diff --git a/tests/system/providers/google/cloud/spanner/__init__.py b/tests/system/providers/google/cloud/spanner/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/spanner/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/cloud/speech_to_text/__init__.py b/tests/system/providers/google/cloud/speech_to_text/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/speech_to_text/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/cloud/stackdriver/__init__.py b/tests/system/providers/google/cloud/stackdriver/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/stackdriver/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/cloud/tasks/__init__.py b/tests/system/providers/google/cloud/tasks/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/tasks/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/cloud/text_to_speech/__init__.py b/tests/system/providers/google/cloud/text_to_speech/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/text_to_speech/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/cloud/workflows/__init__.py b/tests/system/providers/google/cloud/workflows/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/cloud/workflows/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/google/workplace/__init__.py b/tests/system/providers/google/workplace/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/google/workplace/__init__.py
@@ -0,0 +1,16 @@
+# 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/system/providers/presto/__init__.py b/tests/system/providers/presto/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/system/providers/presto/__init__.py
@@ -0,0 +1,16 @@
+# 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/test_utils/operators/__init__.py b/tests/test_utils/operators/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/test_utils/operators/__init__.py
@@ -0,0 +1,16 @@
+# 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/test_utils/perf/__init__.py b/tests/test_utils/perf/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/test_utils/perf/__init__.py
@@ -0,0 +1,16 @@
+# 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/test_utils/perf/dags/__init__.py b/tests/test_utils/perf/dags/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/test_utils/perf/dags/__init__.py
@@ -0,0 +1,16 @@
+# 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/testconfig/__init__.py b/tests/testconfig/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/testconfig/__init__.py
@@ -0,0 +1,16 @@
+# 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/testconfig/conf/__init__.py b/tests/testconfig/conf/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/testconfig/conf/__init__.py
@@ -0,0 +1,16 @@
+# 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/triggers/__init__.py b/tests/triggers/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/triggers/__init__.py
@@ -0,0 +1,16 @@
+# 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/test_logs/__init__.py b/tests/www/test_logs/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/www/test_logs/__init__.py
@@ -0,0 +1,16 @@
+# 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/__init__.py b/tests/www/views/__init__.py
new file mode 100644
index 0000000000000..13a83393a9124
--- /dev/null
+++ b/tests/www/views/__init__.py
@@ -0,0 +1,16 @@
+# 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 a66af3b17deb5d19258342bc89ff88466a63477d Mon Sep 17 00:00:00 2001
From: Jarek Potiuk
Date: Tue, 14 Jun 2022 22:32:49 +0200
Subject: [PATCH 012/118] Add CI-friendly progress output for tests (#24236)
This is the first step to run breeze tests in parallel in CI.
This flag adds "limited progress" output when running tests
which means that the runnig tests will just print few lines with
percent progress and color status indication from last few
progress lines of Pytest output, but when it completes, the whole output is
printed in a CI group - colored depending on status.
The final version (wnen we implement parallel test execution) should
also defer writing the output to until all tests are completed, but
this should be a follow-up PR.
(cherry picked from commit 41fefa146ac64379447db503b7dba82d5121f06a)
---
TESTING.rst | 15 ++
.../configuration_and_maintenance_commands.py | 6 +-
.../commands/testing_commands.py | 163 +++++++++++++++++-
.../src/airflow_breeze/utils/ci_group.py | 8 +-
.../src/airflow_breeze/utils/console.py | 14 ++
images/breeze/output-commands-hash.txt | 2 +-
images/breeze/output-docker-compose-tests.svg | 92 +++++-----
images/breeze/output-tests.svg | 132 ++++++++------
8 files changed, 319 insertions(+), 113 deletions(-)
diff --git a/TESTING.rst b/TESTING.rst
index 12983726e1ebb..2271e73ecfd8c 100644
--- a/TESTING.rst
+++ b/TESTING.rst
@@ -182,6 +182,21 @@ You can also specify individual tests or a group of tests:
breeze tests --db-reset tests/core/test_core.py::TestCore
+You can also limit the tests to execute to specific group of tests
+
+.. code-block:: bash
+
+ breeze tests --test-type Core
+
+
+You can also write tests in "limited progress" mode (useful in the future to run CI). In this mode each
+test just prints "percentage" summary of the run as single line and only dumps full output of the test
+after it completes.
+
+.. code-block:: bash
+
+ breeze tests --test-type Core --limit-progress-output
+
Running Tests of a specified type from the Host
-----------------------------------------------
diff --git a/dev/breeze/src/airflow_breeze/commands/configuration_and_maintenance_commands.py b/dev/breeze/src/airflow_breeze/commands/configuration_and_maintenance_commands.py
index d4ca3bcf466ca..116319a2efca3 100644
--- a/dev/breeze/src/airflow_breeze/commands/configuration_and_maintenance_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/configuration_and_maintenance_commands.py
@@ -157,9 +157,9 @@ def cleanup(verbose: bool, dry_run: bool, github_repository: str, all: bool, ans
)
images = command_result.stdout.splitlines() if command_result and command_result.stdout else []
if images:
- get_console().print("[light_blue]Removing images:[/]")
+ get_console().print("[info]Removing images:[/]")
for image in images:
- get_console().print(f"[light_blue] * {image}[/]")
+ get_console().print(f"[info] * {image}[/]")
get_console().print()
docker_rmi_command_to_execute = [
'docker',
@@ -173,7 +173,7 @@ def cleanup(verbose: bool, dry_run: bool, github_repository: str, all: bool, ans
elif given_answer == Answer.QUIT:
sys.exit(0)
else:
- get_console().print("[light_blue]No locally downloaded images to remove[/]\n")
+ get_console().print("[info]No locally downloaded images to remove[/]\n")
get_console().print("Pruning docker images")
given_answer = user_confirm("Are you sure with the removal?")
if given_answer == Answer.YES:
diff --git a/dev/breeze/src/airflow_breeze/commands/testing_commands.py b/dev/breeze/src/airflow_breeze/commands/testing_commands.py
index 84bfd29d0ea5c..ebe4701b73ceb 100644
--- a/dev/breeze/src/airflow_breeze/commands/testing_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/testing_commands.py
@@ -14,10 +14,16 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
-
+import errno
import os
+import re
+import shutil
+import subprocess
import sys
-from typing import Tuple
+import tempfile
+from threading import Event, Thread
+from time import sleep
+from typing import Dict, List, Tuple
import click
@@ -25,24 +31,29 @@
from airflow_breeze.global_constants import ALLOWED_TEST_TYPES
from airflow_breeze.params.build_prod_params import BuildProdParams
from airflow_breeze.params.shell_params import ShellParams
+from airflow_breeze.utils.ci_group import ci_group
from airflow_breeze.utils.common_options import (
+ option_backend,
option_db_reset,
option_dry_run,
option_github_repository,
option_image_name,
option_image_tag,
option_integration,
+ option_mssql_version,
+ option_mysql_version,
+ option_postgres_version,
option_python,
option_verbose,
)
-from airflow_breeze.utils.console import get_console
+from airflow_breeze.utils.console import get_console, message_type_from_return_code
from airflow_breeze.utils.custom_param_types import BetterChoice
from airflow_breeze.utils.docker_command_utils import (
get_env_variables_for_docker_commands,
perform_environment_checks,
)
from airflow_breeze.utils.run_tests import run_docker_compose_tests
-from airflow_breeze.utils.run_utils import run_command
+from airflow_breeze.utils.run_utils import RunCommandResult, run_command
TESTING_COMMANDS = {
"name": "Testing",
@@ -55,8 +66,8 @@
"name": "Docker-compose tests flag",
"options": [
"--image-name",
- "--python",
"--image-tag",
+ "--python",
],
}
],
@@ -66,7 +77,13 @@
"options": [
"--integration",
"--test-type",
+ "--limit-progress-output",
"--db-reset",
+ "--backend",
+ "--python",
+ "--postgres-version",
+ "--mysql-version",
+ "--mssql-version",
],
}
],
@@ -112,6 +129,91 @@ def docker_compose_tests(
sys.exit(return_code)
+class MonitoringThread(Thread):
+ """Thread class with a stop() method. The thread itself has to check
+ regularly for the stopped() condition."""
+
+ def __init__(self, title: str, file_name: str):
+ super().__init__(target=self.peek_percent_at_last_lines_of_file, daemon=True)
+ self._stop_event = Event()
+ self.title = title
+ self.file_name = file_name
+
+ def peek_percent_at_last_lines_of_file(self) -> None:
+ max_line_length = 400
+ matcher = re.compile(r"^.*\[([^\]]*)\]$")
+ while not self.stopped():
+ if os.path.exists(self.file_name):
+ try:
+ with open(self.file_name, 'rb') as temp_f:
+ temp_f.seek(-(max_line_length * 2), os.SEEK_END)
+ tail = temp_f.read().decode()
+ try:
+ two_last_lines = tail.splitlines()[-2:]
+ previous_no_ansi_line = escape_ansi(two_last_lines[0])
+ m = matcher.match(previous_no_ansi_line)
+ if m:
+ get_console().print(f"[info]{self.title}:[/] {m.group(1).strip()}")
+ print(f"\r{two_last_lines[0]}\r")
+ print(f"\r{two_last_lines[1]}\r")
+ except IndexError:
+ pass
+ except OSError as e:
+ if e.errno == errno.EINVAL:
+ pass
+ else:
+ raise
+ sleep(5)
+
+ def stop(self):
+ self._stop_event.set()
+
+ def stopped(self):
+ return self._stop_event.is_set()
+
+
+def escape_ansi(line):
+ ansi_escape = re.compile(r'(?:\x1B[@-_]|[\x80-\x9F])[0-?]*[ -/]*[@-~]')
+ return ansi_escape.sub('', line)
+
+
+def run_with_progress(
+ cmd: List[str],
+ env_variables: Dict[str, str],
+ test_type: str,
+ python: str,
+ backend: str,
+ version: str,
+ verbose: bool,
+ dry_run: bool,
+) -> RunCommandResult:
+ title = f"Running tests: {test_type}, Python: {python}, Backend: {backend}:{version}"
+ try:
+ with tempfile.NamedTemporaryFile(mode='w+t', delete=False) as f:
+ get_console().print(f"[info]Starting test = {title}[/]")
+ thread = MonitoringThread(title=title, file_name=f.name)
+ thread.start()
+ try:
+ result = run_command(
+ cmd,
+ verbose=verbose,
+ dry_run=dry_run,
+ env=env_variables,
+ check=False,
+ stdout=f,
+ stderr=subprocess.STDOUT,
+ )
+ finally:
+ thread.stop()
+ thread.join()
+ with ci_group(f"Result of {title}", message_type=message_type_from_return_code(result.returncode)):
+ with open(f.name) as f:
+ shutil.copyfileobj(f, sys.stdout)
+ finally:
+ os.unlink(f.name)
+ return result
+
+
@main.command(
name='tests',
help="Run the specified unit test targets. Multiple targets may be specified separated by spaces.",
@@ -122,10 +224,19 @@ def docker_compose_tests(
)
@option_dry_run
@option_verbose
+@option_python
+@option_backend
+@option_postgres_version
+@option_mysql_version
+@option_mssql_version
@option_integration
+@click.option(
+ '--limit-progress-output',
+ help="Limit progress to percentage only and just show the summary when tests complete.",
+ is_flag=True,
+)
@click.argument('extra_pytest_args', nargs=-1, type=click.UNPROCESSED)
@click.option(
- "-tt",
"--test-type",
help="Type of test to run.",
default="All",
@@ -135,6 +246,12 @@ def docker_compose_tests(
def tests(
dry_run: bool,
verbose: bool,
+ python: str,
+ backend: str,
+ postgres_version: str,
+ mysql_version: str,
+ mssql_version: str,
+ limit_progress_output: bool,
integration: Tuple,
extra_pytest_args: Tuple,
test_type: str,
@@ -149,11 +266,39 @@ def tests(
os.environ["LIST_OF_INTEGRATION_TESTS_TO_RUN"] = ' '.join(list(integration))
if db_reset:
os.environ["DB_RESET"] = "true"
-
- exec_shell_params = ShellParams(verbose=verbose, dry_run=dry_run)
+ exec_shell_params = ShellParams(
+ verbose=verbose,
+ dry_run=dry_run,
+ python=python,
+ backend=backend,
+ postgres_version=postgres_version,
+ mysql_version=mysql_version,
+ mssql_version=mssql_version,
+ )
env_variables = get_env_variables_for_docker_commands(exec_shell_params)
perform_environment_checks(verbose=verbose)
cmd = ['docker-compose', 'run', '--service-ports', '--rm', 'airflow']
cmd.extend(list(extra_pytest_args))
- result = run_command(cmd, verbose=verbose, dry_run=dry_run, env=env_variables, check=False)
+ version = (
+ mssql_version
+ if backend == "mssql"
+ else mysql_version
+ if backend == "mysql"
+ else postgres_version
+ if backend == "postgres"
+ else "none"
+ )
+ if limit_progress_output:
+ result = run_with_progress(
+ cmd=cmd,
+ env_variables=env_variables,
+ test_type=test_type,
+ python=python,
+ backend=backend,
+ version=version,
+ verbose=verbose,
+ dry_run=dry_run,
+ )
+ else:
+ result = run_command(cmd, verbose=verbose, dry_run=dry_run, env=env_variables, check=False)
sys.exit(result.returncode)
diff --git a/dev/breeze/src/airflow_breeze/utils/ci_group.py b/dev/breeze/src/airflow_breeze/utils/ci_group.py
index e65751a322a2e..96525b55253a8 100644
--- a/dev/breeze/src/airflow_breeze/utils/ci_group.py
+++ b/dev/breeze/src/airflow_breeze/utils/ci_group.py
@@ -18,11 +18,11 @@
import os
from contextlib import contextmanager
-from airflow_breeze.utils.console import get_console
+from airflow_breeze.utils.console import MessageType, get_console
@contextmanager
-def ci_group(title: str, enabled: bool = True):
+def ci_group(title: str, enabled: bool = True, message_type: MessageType = MessageType.INFO):
"""
If used in GitHub Action, creates an expandable group in the GitHub Action log.
Otherwise, display simple text groups.
@@ -34,9 +34,9 @@ def ci_group(title: str, enabled: bool = True):
yield
return
if os.environ.get('GITHUB_ACTIONS', 'false') != "true":
- get_console().print(f"[info]{title}[/]")
+ get_console().print(f"[{message_type.value}]{title}[/]")
yield
return
- get_console().print(f"::group::: [info]{title}[/]")
+ get_console().print(f"::group::: [{message_type.value}]{title}[/]")
yield
get_console().print("::endgroup::")
diff --git a/dev/breeze/src/airflow_breeze/utils/console.py b/dev/breeze/src/airflow_breeze/utils/console.py
index 9a14d91eaed89..41ae65ef61158 100644
--- a/dev/breeze/src/airflow_breeze/utils/console.py
+++ b/dev/breeze/src/airflow_breeze/utils/console.py
@@ -19,6 +19,7 @@
to be only run in CI or real development terminal - in both cases we want to have colors on.
"""
import os
+from enum import Enum
from functools import lru_cache
from rich.console import Console
@@ -56,6 +57,19 @@ def get_theme() -> Theme:
)
+class MessageType(Enum):
+ SUCCESS = "success"
+ INFO = "info"
+ WARNING = "warning"
+ ERROR = "error"
+
+
+def message_type_from_return_code(return_code: int) -> MessageType:
+ if return_code == 0:
+ return MessageType.SUCCESS
+ return MessageType.ERROR
+
+
@lru_cache(maxsize=None)
def get_console() -> Console:
return Console(
diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt
index 96155ac41a84c..aedf286361b03 100644
--- a/images/breeze/output-commands-hash.txt
+++ b/images/breeze/output-commands-hash.txt
@@ -1 +1 @@
-2f24dd5bade0ac4b635ef6ada358590b
+4715fa0a006457cc3f4f6447e3fecc95
diff --git a/images/breeze/output-docker-compose-tests.svg b/images/breeze/output-docker-compose-tests.svg
index 4830ca1215289..75f5c1a31b102 100644
--- a/images/breeze/output-docker-compose-tests.svg
+++ b/images/breeze/output-docker-compose-tests.svg
@@ -19,109 +19,109 @@
font-weight: 700;
}
- .terminal-25948600-matrix {
+ .terminal-1448538552-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 24.4px;
font-variant-east-asian: full-width;
}
- .terminal-25948600-title {
+ .terminal-1448538552-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-25948600-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-25948600-r2 { fill: #c5c8c6 }
-.terminal-25948600-r3 { fill: #d0b344;font-weight: bold }
-.terminal-25948600-r4 { fill: #868887 }
-.terminal-25948600-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-25948600-r6 { fill: #98a84b;font-weight: bold }
-.terminal-25948600-r7 { fill: #8d7b39 }
+ .terminal-1448538552-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-1448538552-r2 { fill: #c5c8c6 }
+.terminal-1448538552-r3 { fill: #d0b344;font-weight: bold }
+.terminal-1448538552-r4 { fill: #868887 }
+.terminal-1448538552-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-1448538552-r6 { fill: #98a84b;font-weight: bold }
+.terminal-1448538552-r7 { fill: #8d7b39 }
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
- Command: docker-compose-tests
+ Command: docker-compose-tests
-
+
-
-
-Usage: breeze docker-compose-tests [OPTIONS] [EXTRA_PYTEST_ARGS]...
-
-Run docker-compose tests.
-
-╭─ Docker-compose tests flag ──────────────────────────────────────────────────────────────────────────────────────────╮
-│--image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT)│
-│--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10)│
-│[default: 3.7] │
-│--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT)│
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│--verbose-vPrint verbose information about performed steps.│
-│--dry-run-DIf dry-run is set, commands are only printed, not executed.│
-│--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow]│
-│--help-hShow this message and exit.│
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+
+Usage: breeze docker-compose-tests [OPTIONS] [EXTRA_PYTEST_ARGS]...
+
+Run docker-compose tests.
+
+╭─ Docker-compose tests flag ──────────────────────────────────────────────────────────────────────────────────────────╮
+│--image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT)│
+│--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT)│
+│--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10)│
+│[default: 3.7] │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│--verbose-vPrint verbose information about performed steps.│
+│--dry-run-DIf dry-run is set, commands are only printed, not executed.│
+│--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow]│
+│--help-hShow this message and exit.│
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-tests.svg b/images/breeze/output-tests.svg
index 7c02458342214..914f2c4587a0b 100644
--- a/images/breeze/output-tests.svg
+++ b/images/breeze/output-tests.svg
@@ -1,4 +1,4 @@
-
+
-
-
+
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
-
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
- Command: tests
+ Command: tests
-
+
-
-
-Usage: breeze tests [OPTIONS] [EXTRA_PYTEST_ARGS]...
-
-Run the specified unit test targets. Multiple targets may be specified separated by spaces.
-
-╭─ Basic flag for tests command ───────────────────────────────────────────────────────────────────────────────────────╮
-│--integrationIntegration(s) to enable when running (can be more than one). │
-│(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | all)│
-│--test-type-ttType of test to run. │
-│(All | Always | Core | Providers | API | CLI | Integration | Other | WWW | Postgres | MySQL | │
-│Helm | Quarantined) │
-│--db-reset-dReset DB when entering the container.│
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│--dry-run-DIf dry-run is set, commands are only printed, not executed.│
-│--verbose-vPrint verbose information about performed steps.│
-│--help-hShow this message and exit.│
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+
+Usage: breeze tests [OPTIONS] [EXTRA_PYTEST_ARGS]...
+
+Run the specified unit test targets. Multiple targets may be specified separated by spaces.
+
+╭─ Basic flag for tests command ───────────────────────────────────────────────────────────────────────────────────────╮
+│--integrationIntegration(s) to enable when running (can be more than one). │
+│(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | │
+│all) │
+│--test-typeType of test to run. │
+│(All | Always | Core | Providers | API | CLI | Integration | Other | WWW | Postgres | │
+│MySQL | Helm | Quarantined) │
+│--limit-progress-outputLimit progress to percentage only and just show the summary when tests complete.│
+│--db-reset-dReset DB when entering the container.│
+│--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite]│
+│--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10)│
+│[default: 3.7] │
+│--postgres-version-PVersion of Postgres used.(>10< | 11 | 12 | 13 | 14)[default: 10]│
+│--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7]│
+│--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest]│
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│--dry-run-DIf dry-run is set, commands are only printed, not executed.│
+│--verbose-vPrint verbose information about performed steps.│
+│--help-hShow this message and exit.│
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
From 6a05f043004482a7928712e2b4d6ef360a950018 Mon Sep 17 00:00:00 2001
From: Jarek Potiuk
Date: Wed, 15 Jun 2022 11:42:41 +0200
Subject: [PATCH 013/118] First attempt to have CI-controlled process of
releasing PROD image (#24433)
(cherry picked from commit b79790d8e5de18782a3179a8a55457eafa529a88)
---
.github/workflows/release_dockerhub_image.yml | 125 ++++++++++++++++++
dev/MANUALLY_BUILDING_IMAGES.md | 96 ++++++++++++++
dev/README_RELEASE_AIRFLOW.md | 109 +++------------
.../commands/release_management_commands.py | 7 +-
dev/images/release_prod_image.png | Bin 0 -> 112569 bytes
5 files changed, 245 insertions(+), 92 deletions(-)
create mode 100644 .github/workflows/release_dockerhub_image.yml
create mode 100644 dev/MANUALLY_BUILDING_IMAGES.md
create mode 100644 dev/images/release_prod_image.png
diff --git a/.github/workflows/release_dockerhub_image.yml b/.github/workflows/release_dockerhub_image.yml
new file mode 100644
index 0000000000000..bd9fdaec1e6a7
--- /dev/null
+++ b/.github/workflows/release_dockerhub_image.yml
@@ -0,0 +1,125 @@
+# 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.
+#
+---
+name: "Release PROD image"
+on: # yamllint disable-line rule:truthy
+ workflow_dispatch:
+ inputs:
+ airflowVersion:
+ description: 'Airflow version'
+ required: true
+ skipLatest:
+ description: 'Skip Latest: Set to true if not latest.'
+ default: ''
+ required: false
+jobs:
+ build-info:
+ timeout-minutes: 10
+ name: "Build Info"
+ runs-on: ${{ github.repository == 'apache/airflow' && 'self-hosted' || 'ubuntu-20.04' }}
+ outputs:
+ pythonVersions: ${{ steps.selective-checks.outputs.python-versions }}
+ allPythonVersions: ${{ steps.selective-checks.outputs.all-python-versions }}
+ defaultPythonVersion: ${{ steps.selective-checks.outputs.default-python-version }}
+ skipLatest: ${{ github.event.inputs.skipLatest == '' && ' ' || '--skip-latest' }}
+ limitPlatform: ${{ github.repository == 'apache/airflow' && ' ' || '--limit-platform linux/amd64' }}
+ env:
+ GITHUB_CONTEXT: ${{ toJson(github) }}
+ steps:
+ - name: Cleanup repo
+ run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*"
+ - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )"
+ uses: actions/checkout@v2
+ with:
+ persist-credentials: false
+ submodules: recursive
+ - name: Selective checks
+ id: selective-checks
+ run: ./scripts/ci/selective_ci_checks.sh
+ release-images:
+ timeout-minutes: 120
+ name: "Release images"
+ runs-on: ${{ github.repository == 'apache/airflow' && 'self-hosted' || 'ubuntu-20.04' }}
+ needs: [build-info]
+ strategy:
+ fail-fast: false
+ matrix:
+ python-version: ${{ fromJson(needs.build-info.outputs.pythonVersions) }}
+ env:
+ RUNS_ON: ${{ github.repository == 'apache/airflow' && 'self-hosted' || 'ubuntu-20.04' }}
+ if: contains(fromJSON('[
+ "ashb",
+ "ephraimbuddy",
+ "jedcunningham",
+ "kaxil",
+ "potiuk",
+ ]'), github.event.sender.login)
+ steps:
+ - name: Cleanup repo
+ run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*"
+ - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )"
+ uses: actions/checkout@v2
+ with:
+ persist-credentials: false
+ - name: "Setup python"
+ uses: actions/setup-python@v2
+ with:
+ python-version: ${{ needs.build-info.outputs.defaultPythonVersion }}
+ cache: 'pip'
+ cache-dependency-path: ./dev/breeze/setup*
+ - run: ./scripts/ci/install_breeze.sh
+ - name: "Free space"
+ run: breeze free-space
+ - name: Build CI image for PROD build ${{ needs.build-info.outputs.defaultPythonVersion }}
+ run: breeze build-image
+ env:
+ PYTHON_MAJOR_MINOR_VERSION: ${{ needs.build-info.outputs.defaultPythonVersion }}
+ - name: "Cleanup dist and context file"
+ run: rm -fv ./dist/* ./docker-context-files/*
+ - name: "Start ARM instance"
+ run: ./scripts/ci/images/ci_start_arm_instance_and_connect_to_docker.sh
+ if: github.repository == 'apache/airflow'
+ - name: "Login to docker"
+ run: >
+ echo ${{ secrets.DOCKERHUB_TOKEN }} |
+ docker login --password-stdin --username ${{ secrets.DOCKERHUB_USER }}
+ - name: "Release regular images"
+ run: >
+ breeze release-prod-images
+ --dockerhub-repo ${{ github.repository }}
+ --airflow-version ${{ github.event.inputs.airflowVersion }}
+ ${{ needs.build-info.outputs.skipLatest }}
+ ${{ needs.build-info.outputs.limitPlatform }}
+ --limit-python ${{ matrix.python-version }}
+ - name: "Release slim images"
+ run: >
+ breeze release-prod-images
+ --dockerhub-repo ${{ github.repository }}
+ --airflow-version ${{ github.event.inputs.airflowVersion }}
+ ${{ needs.build-info.outputs.skipLatest }}
+ ${{ needs.build-info.outputs.limitPlatform }}
+ --limit-python ${{ matrix.python-version }} --slim-images
+ - name: "Docker logout"
+ run: docker logout
+ if: always()
+ - name: "Stop ARM instance"
+ run: ./scripts/ci/images/ci_stop_arm_instance.sh
+ if: always() && github.repository == 'apache/airflow'
+ - name: "Fix ownership"
+ run: breeze fix-ownership
+ if: always()
diff --git a/dev/MANUALLY_BUILDING_IMAGES.md b/dev/MANUALLY_BUILDING_IMAGES.md
new file mode 100644
index 0000000000000..99cf589d691d4
--- /dev/null
+++ b/dev/MANUALLY_BUILDING_IMAGES.md
@@ -0,0 +1,96 @@
+
+
+
+
+**Table of Contents** *generated with [DocToc](https://github.com/thlorenz/doctoc)*
+
+- [Building docker images](#building-docker-images)
+- [Setting environment with emulation](#setting-environment-with-emulation)
+- [Setting up cache refreshing with hardware ARM/AMD support](#setting-up-cache-refreshing-with-hardware-armamd-support)
+
+
+
+## Building docker images
+
+In order to build images on local hardware, you need to have the buildx plugin installed to run the build.
+Also, you need to have regctl installed from https://github.com/regclient/regclient in order to tag
+the multi-platform images in DockerHub. The script to build images will refuse to work if
+you do not have those two installed.
+
+You also need to have the right permissions to push the images, so you should run
+`docker login` before and authenticate with your DockerHub token.
+
+## Setting environment with emulation
+
+According to the [official installation instructions](https://docs.docker.com/buildx/working-with-buildx/#build-multi-platform-images)
+this can be achieved via:
+
+```shell
+docker run --privileged --rm tonistiigi/binfmt --install all
+```
+
+More information can be found [here](https://docs.docker.com/engine/reference/commandline/buildx_create/).
+
+However, emulation is very slow - more than 10x slower than hardware-backed builds.
+
+## Setting up cache refreshing with hardware ARM/AMD support
+
+If you plan to build a number of images, it's probably better to set up a hardware remote builder
+for your ARM or AMD builds (depending which platform you build images on - the "other" platform should be
+remote).
+
+This can be achieved by settings build as described in
+[this blog post](https://www.docker.com/blog/speed-up-building-with-docker-buildx-and-graviton2-ec2/) and
+adding it to docker buildx `airflow_cache` builder.
+
+This usually can be done with those two commands:
+
+```bash
+docker buildx create --name airflow_cache # your local builder
+docker buildx create --name airflow_cache --append HOST:PORT # your remote builder
+```
+
+One of the ways to have HOST:PORT is to login to the remote machine via SSH and forward the port to
+the docker engine running on the remote machine.
+
+When everything is fine you should see both local and remote builder configured and reporting status:
+
+```bash
+docker buildx ls
+
+ airflow_cache docker-container
+ airflow_cache0 unix:///var/run/docker.sock
+ airflow_cache1 tcp://127.0.0.1:2375
+```
+
+Preparing regular images:
+
+```shell script
+breeze release-prod-images --airflow-version "${VERSION}"
+```
+
+Preparing slim images:
+
+```shell script
+breeze release-prod-images --airflow-version "${VERSION}" --slim-images
+```
+
+This will wipe Breeze cache and docker-context-files in order to make sure the build is "clean". It
+also performs image verification after pushing the images.
diff --git a/dev/README_RELEASE_AIRFLOW.md b/dev/README_RELEASE_AIRFLOW.md
index 6dce6e487c64a..3d242ff9d4843 100644
--- a/dev/README_RELEASE_AIRFLOW.md
+++ b/dev/README_RELEASE_AIRFLOW.md
@@ -28,9 +28,6 @@
- [[\Optional\] Prepare new release branches and cache](#%5Coptional%5C-prepare-new-release-branches-and-cache)
- [Prepare PyPI convenience "snapshot" packages](#prepare-pypi-convenience-snapshot-packages)
- [Prepare production Docker Image](#prepare-production-docker-image)
- - [Prerequisites](#prerequisites)
- - [Setting environment with emulation](#setting-environment-with-emulation)
- - [Setting up cache refreshing with hardware ARM/AMD support](#setting-up-cache-refreshing-with-hardware-armamd-support)
- [Prepare issue for testing status of rc](#prepare-issue-for-testing-status-of-rc)
- [Prepare Vote email on the Apache Airflow release candidate](#prepare-vote-email-on-the-apache-airflow-release-candidate)
- [Verify the release candidate by PMCs](#verify-the-release-candidate-by-pmcs)
@@ -493,76 +490,23 @@ is not supposed to be used by and advertised to the end-users who do not read th
Production Docker images should be manually prepared and pushed by the release manager or another committer
who has access to Airflow's DockerHub. Note that we started releasing a multi-platform build, so you need
-to have an environment prepared to build multi-platform images. You can achieve it with either emulation
-(very slow) or if you have two types of hardware (AMD64 and ARM64) you can configure Hardware builders.
+to have an environment prepared to build multi-platform images. You can achieve it with:
-## Prerequisites
+* GitHub Actions Manual Job (easiest)
+* Emulation (very slow)
+* Hardware builders if you have both AMD64 and ARM64 hardware locally
-You need to have buildx plugin installed to run the build. Also, you need to have regctl
-installed from https://github.com/regclient/regclient in order to tag the multi-platform images in
-DockerHub. The script to build images will refuse to work if you do not have those two installed.
+Building the image is triggered by running the `Release PROD image` workflow via
+[GitHub Actions](https://github.com/apache/airflow/actions).
-You also need to have the right permissions to push the images, so you should run
-`docker login` before and authenticate with your DockerHub token.
+When you trigger it you need to pass:
-## Setting environment with emulation
+* Airflow Version
+* Optional "true" in skip latest field if you do not want to retag the latest image
-According to the [official installation instructions](https://docs.docker.com/buildx/working-with-buildx/#build-multi-platform-images)
-this can be achieved via:
-
-```shell
-docker run --privileged --rm tonistiigi/binfmt --install all
-```
-
-More information can be found [here](https://docs.docker.com/engine/reference/commandline/buildx_create/)
-
-However, emulation is very slow - more than 10x slower than hardware-backed builds.
-
-## Setting up cache refreshing with hardware ARM/AMD support
-
-If you plan to build a number of images, probably better solution is to set up a hardware remote builder
-for your ARM or AMD builds (depending which platform you build images on - the "other" platform should be
-remote.
-
-This can be achieved by settings build as described in
-[this guideline](https://www.docker.com/blog/speed-up-building-with-docker-buildx-and-graviton2-ec2/) and
-adding it to docker buildx `airflow_cache` builder.
-
-This usually can be done with those two commands:
-
-```bash
-docker buildx create --name airflow_cache # your local builder
-docker buildx create --name airflow_cache --append HOST:PORT # your remote builder
-```
-
-One of the ways to have HOST:PORT is to login to the remote machine via SSH and forward the port to
-the docker engine running on the remote machine.
-
-When everything is fine you should see both local and remote builder configured and reporting status:
-
-```bash
-docker buildx ls
-
- airflow_cache docker-container
- airflow_cache0 unix:///var/run/docker.sock
- airflow_cache1 tcp://127.0.0.1:2375
-```
-
-Preparing regular images:
-
-```shell script
-breeze release-prod-images --airflow-version "${VERSION}"
-```
-
-Preparing slim images:
-
-```shell script
-breeze release-prod-images --airflow-version "${VERSION}" --slim-images
-```
-
-This will wipe Breeze cache and docker-context-files in order to make sure the build is "clean". It
-also performs image verification after pushing the images.
+![Release prod image](images/release_prod_image.png)
+The manual building is described in [MANUALLY_BUILDING_IMAGES.md](MANUALLY_BUILDING_IMAGES.md).
## Prepare issue for testing status of rc
@@ -1013,33 +957,22 @@ At this point we release an official package:
## Manually prepare production Docker Image
-Note that this scripts prepares multi-platform image, so you need to fulfill prerequisites as
-described above in the preparation of RC images.
+Building the image is triggered by running the `Release PROD image` workflow via
+[GitHub Actions](https://github.com/apache/airflow/actions).
+
+When you trigger it you need to pass:
+
+* Airflow Version
+* Optional "true" in skip latest field if you do not want to retag the latest image
+
+![Release prod image](images/release_prod_image.png)
Note that by default the `latest` images tagged are aliased to the just released image which is the usual
way we release. For example when you are releasing 2.3.N image and 2.3 is our latest branch the new image is
marked as "latest".
In case we are releasing (which almost never happens so far) a critical bugfix release in one of
-the older branches, you should add the `--skip-latest` flag.
-
-Preparing regular images:
-
-```shell script
-breeze release-prod-images --airflow-version "${VERSION}"
-```
-
-Preparing slim images:
-
-```shell script
-breeze release-prod-images --airflow-version "${VERSION}" --slim-images
-```
-
-Preparing a release that is not in the latest branch:
-
-```shell script
-breeze release-prod-images --airflow-version "${VERSION}" --slim-images --skip-latest
-```
+the older branches, you should set the "skip" field to true.
## Publish documentation
diff --git a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py
index ce80e6a1932a8..839b966faebb5 100644
--- a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py
@@ -650,12 +650,11 @@ def release_prod_images(
["docker", 'buildx', 'inspect', 'airflow_cache'], check=False, dry_run=dry_run, verbose=verbose
)
if result_inspect_builder.returncode != 0:
- get_console().print("[error]Regctl must be installed and on PATH to release the images[/]")
+ get_console().print("[error]Airflow Cache builder must be configured to release the images[/]")
get_console().print()
get_console().print(
- "See https://github.com/apache/airflow/blob/main/dev/README_RELEASE_AIRFLOW.md"
- "#setting-up-cache-refreshing-with-hardware-armamd-support for "
- "instructions on setting it up."
+ "See https://github.com/apache/airflow/blob/main/dev/MANUALLY_BUILDING_IMAGES.md"
+ " for instructions on setting it up."
)
sys.exit(1)
result_regctl = run_command(["regctl", 'version'], check=False, dry_run=dry_run, verbose=verbose)
diff --git a/dev/images/release_prod_image.png b/dev/images/release_prod_image.png
new file mode 100644
index 0000000000000000000000000000000000000000..78f941a43b4c9ac23929eda5e742eab83279fbd0
GIT binary patch
literal 112569
zcmeFYXIN9))<3G-f})6ufKo&$(m{cMbQS5nLlBTo2t|4a3q`shy(%3-=m{ORgkGg2
z0YZ@)Ak+Y%1j5bU`<(Oc^Sz`I}=!XlW=?Q8G|o
zxNw0=S?RUTg$opB7cN}7cI`LvKiJxzgBLEmzo7j3rJj%3MvK3$9=h?oY4-QyMLiwo
zndIAA*HqXqU%&I4>Et
zp{eG;lvV9yNdxF|lel}VZ<~kv9hiD>smuYi`&l>Bp6H6>F=1@@wEz4d_?&_~fq%3R
z+F)1Fi~qd4@OutpnZ>{E-dA1%-~ZR$qmADguKw%pnq+0zf4$d!>)&2m6l5v1X8(iF
zzuJpyI~FQkqc)%_7WT95Q~RPE$*9xL2B&{Mzi`24F<@r|FJ3Y5h}0jf&_(M07&`i2
z#4h}z!n535X~;%|5#nF{FFcOXF<$r-mn_LV
z``{Dc=D!}?9zC_QF4HkskhK+bz5jP1jYpil&|r#<$$=e?Sp1!|#GR-G8eO4e0af?(
zX5oJm?Dz;+<+=K7+K0bWSU)Ap;O}#>pg=wS*92Zn{99+Qf8&z*e_y+B`~P#1wB1=q
zK+!u-3-p*biL_5XzHsNds-ZJ5!@S-LQAi8wUs2^%$cOfz!+>pWt?sn{RHcvmYj)d$
zV7RC>8%KN9wO@mHQ6oeZ2;yb%P>O)8S|NmWb$v>*VKgjinR#g6(YKys7A;F7)za!+
zmG_BJ^@e0%8yMWnOLBYXyPw&aG$)M&205suH}?OvT6er^*ae&p@X_4ynYA5N63zS_
zeekKq--ZbZ&?Xglkal8<`IoFLTKdI}>n>t@3&IJ^V%zxrvZdPS0v7fG_x<1~C4qG*
z@7=xzlw9wPji|Q_9fdr??Bd)%TCdrv*qb2kfEaNVbAFC^IlyTfsl{(k$p5@9^nnrL
zL5vK>_a8pv{X5$q>$w#bJxIW~wk|I$gjXvWxEhEk`+)z<1DC&$SV{KK+ubAC`(&QLGjoNXnWIy{x
z5V3~>A1-e^WosviGxec%s`=?zB-Aawik+CalQ75pKlY*X<6ph^FCCtHfX-(FO(<{Z
zT2=k!!5+OCfzG=$GMY$y(mkw05|DFJRcT4H?pf-%g{-I-tX`lX-LICH0>^x7t4rTv
z1vJfmG@j-{IXGkCKKkLs4LmdJx4Io25~ljy%!ZJrm41epB=;iKWXq)%1rhKBW_;S~
zv{pe^(G+}WJo9L4gR3uy*!-w$-uRK1yL`A1#gej4vM{gt-5sV|7GNJz*pS7{m7stk
zX!Se!E^GGWGpv<(P5@C{3*z(T6wc`bODVW5rC
z?_!!k&U`?<2`|k<{ooRfnu;Ta)HW_>(mspap4H;`-QKafFfp^A(D`txhC+GWx-hnd
z{291dtN4nb$`|uItbMR?zD1D_D0d;ZfEUk0_$!>4U8hMIW91(svr8=S@-QC`f*zGy4eJ_J)q%`SIV`WTMA^5BSas#X>UC{#pf)3dy`Oj+al@lG}kC4NUo
zB5A17Rp$IY^Kk(Vk`C*Ayp-{0%>8|7;CBX+kD+`ZX7{ArDg-KhKYYQMvUwk
z1ap^X)jh{}{I)1Y--}-EXzW<^?Wc^?jF9Yi#0e@Xy)o2zgFzK1vek2?@XMG5_S6XV
zl2%hsOdV$-AuwMX^=?dUHdJTWv|qOJjiskd9g~vTZRXuwlqtYTot4`x`>v8z~3jGku@LJk0f*BL}O>KWgV3t{t?n+~3^W6O3^Xg>*nyV1=+z
z!g(=29@$(0Q^bk}6&3MF-ftOc-IW_FWRsC4)WG-A6$Z;ep1R&HZ54^#H?64@q0&cr
z$;_wwb?~hI{Ih4bP+j>I-=lyHd#B88?((7CZJx}hZw4(zn>Onh21QRIZO?d`E78@T
z`EjFf+*E;e%({S^pd3Rwc#Oj(nPyfKL{wknzTD9oEL6c3@5`@Z(2n^e~_
zkf65AtOyNH276h*_UqL~((i}@U`$Xj1gz`yv2e;XVkIE49;#?ftZ+y7je7zX4u_nw
zJk_&GFS4KQ=_%(5l}(NHm9X~D30e$~W;x+OYa0_MyZ21T$D4IyYo;a>X^Ol)_dI|1
z#{KFr+>1$eW~5kBW>7y3C^i^S`}3%d&v&i*uJ+I0P4?5=Q^uiUNl>KSVsmY7-U;1u
z@6VCq20g3<5T7hbaozg~EQ}2$_W9Om*;80IeR18a9mE@EZwM`6*M7Fbs_Rx7!tbv=
z&f($!0i!ZcqGlA~#x0#CbDa#@li{&ARgA=7iVwI^`=%i*EJI~9iIg?lwQ}o#@mdw9
zO@s1^#gD$b4Xl+Ox$9n?5p;myW+=}L2e+0j9w*f6{@mhtmhM3N=cK83>9T#xF7lPt
zGV1xB$TgWua@jv&VOT7DZj=e3?UJ@NI%FJFt?ABwCZ-{Qw^lDhLugH;)W5FreYrYf
zXk5E}DhE4T=}l5uJWb*x)++esv0XBy7l8Rv45T%aau^I($DUc*9Pm4hwD1n58~|^>
zgwB5!83P|?2YA0L*m1JU1T;G?`i@!11i9@Id7bFX0}jXJz8atIoSGqT0vz|dgXc<2
zkSk~Rj2t}{_hlT-DqPT^`q0K0|Dw(5_f?mK>%#c0vSkIk0*-c26>Sg1XM{AT;EWMn
zg@(sx&jJ=z+!qazoNW%zgUW@Klw^lrA3$
zUxCT^4)1Ug&=qxE*Ye7BUg9@yELVBP%%Ez(NawzRqMT=ZSOJ#A1X!8^KQ&I*m6z~HYPV5V
zxr(nBI7-+b!{7Rrn9R=Yms!A;H%cz0T~|2&%syQ1H?PwotsGhrqg*zBg*$t3y`(19
zxhk?7ZK^%#LBJ)OW}uz~XT~47Cb+#^#M6GNAodW-xMqf;2JzRh=vF9bw$uGCW$fp0
z1zo53>{y2Q+J<7~4Le+l-)N(OR_>;c$>9j=u+HW&&93XqP`uL^W|6)1;=+yCp2L#5
z_B?6Igx7#~nB|{oc+czmOtHq^uTDw7n;624+l<^FhiT4xTMWfp7=9QBNAn2VONbBaJd4R*)KJFo>
zwgTr}F{oZ=-zVAi0)A5cB+x!F?&z%vVz`TS?haA~F3}czltXgGjrPVsWHnVxns_AD
zMFULB=0o&j+EyuWk_{+ej>Qyty}GnoH+O6w;)pRcv?+c&jqxIZ+X+Cn_KiK~pI6T-
zS0p<@EeBDW*VQtVBns2u)a8zEtY=LmS1!LhU!*6Qq!6$b>MF#5&XZxAyZ~ENW#`=a
zA}X|+-G)VG8ud=KjZ08Lkj~*C3x*?YgqlBF1+3VKlqw!>#I^Q`?4kZzP9A9y048a>
z=tnBKK1M_@O`aXalK&bW|^0
z(~a(7jFHs$-|aolTGtA@Qklyf7ualN%3|Q)mL~p
zji+Cr5v|MOTH!)Vlh7Dbs3n%7Htb5(ovQ11V*5I^v#To@h5^Qv!qR6e;Q&N6ey%#hR)6i&oD@E%zjGYJN+Dlh_;E>U0AQ_%MYFZ}Cs*)M^
zIe-6@RdW{Q$Ukoi3zNj)lPCQYhPBN~y{hsaz3UtopO@gUX04%oopg225PVk~$53Xz
zvUXPFYC4^R08uP8cG0)c;!J}oo^A^-`=wEuX-ZszhD;UQ1c7~m+$>;+v9QQrOhdN5
zcY579w>AvbG@}4zz7@7*sGT1zS#`BmOoXX9#FNxQ0F(ZN?JRFM^|R;@b5Il(9?@mv
zU*XWbr5|frap!dDlyDjP{*!H<;sXuHb`N_R+?LUBD)Wq~wHDx5;o&I5$+pQmeo>vH
zG^geM+1fUhBCps=c|dSA_vJr#?T;g!sWc;zzjWRAB{Ym2R62MJD$q~y<1BpUB}Ulv
zE6KUZH-5S&1H6B&it66YRe7ZmGx2o_stwF*;<)pk8+g6q$7`QUuZ#E&6YuHQ-QT`F
z_8c=^=?ObHGyPF!;mwNTrf*NYAj6BQBye;z#KIR&VaZ}M>M>87SHl3)AbHu@p{+%I
zis}?&7v)Wqu*wg$(4J_hIv!|8&9MBv$eW(iA?{jB-^;gp;QBnCncd|O+t*%OD&Bn6
zcf)jFv9c0XkrJ9Uv)&`wmb-Vf3=?g9uc=`h>t({NL;(?9aHs~0IjhDQ3VWon>(zHG
zoc7_dC%4=LgGTwrC|FUd*_~uG*)@m}-71q>Ia3!j;BB)T=u`q!0eNrl?x#2$)8;mI
z^Y9H(gU72oWuv?+R5$QwByg^Qi
zN$$3T%+D7V6#05f=(4~!Y;CEA(>^Bc_^E}6sW|G5t`DfoH{AjtAIwqu#5s%6p53&i
z>h7|3oB{eR^pwPQRkTb+^@!NcFeD4?;#oJD!wi?uvB-YFGodkE)(u&0YNdM9{YT_j
z)yd9b3kGnsu5tB=&e)tli;8wIpOps(WVPOmxiuFM9mCNJ0E)`6A`f~yrfR#ms|pZ6
z0N14NK?+Op*Y31whHRQ3-B(^qkfh(n3R!}SI|Kx^DHV;(6~wY|3nxq((;Ty|Xwb>?
zr!)<4BW^9T+~o|ian4-JQhiWO_K|cI7Gde;PoA`;4I1q)ooZk=9au{%H@yd6Os71`
zXnWE$=3}0j0BkzV*F50bW-=*>zFoJRpj}UB(m|w;D+w>>%78F@8}cSC*X*k9cjIPm
zW|L4~aQj?j_1dkrLRa>H6t#(Oj;a*+>+@2Ed`|j*SWlkn4|i!UmLlzN7T`ZKm{
zH#!K+;XHuDWYw0NH6eZ#K*@__6IB^6;#3`0pbT5+nLD`bBpJuz#axx-^CwpP{pgi{)CrWz80@!1FxS6&*5o92j^sxGUH)0>MU>W~b|(!Qki4sxB^I
z?!Mv3YT}!%5Trb{B^88E$=HxBx*aY!c2IWyw$9AswO@qAPj3%YmV>z8o5p=AS^&Ri
z`>%{DK$$daW0W2R_x&0hs^RN3+|M{)@?Q^xHoIc)RO%c+A&dU>{J2Vw%A!hepS?)IzvY&ZJC)6XL87sJkaxo2_
zpyHLMq!F0dkNX6(q7}|`=}G$7+!lBl8Hm9pDPT=j9KpuWdGc-yX4X)E$*XGxTI2+b
z)rL{UVP+b$D)dLcjLh}Y!PM(UhPTyx+VlXyE`SQy#@^V6Pra_-Vcr7{vm?RHjByPr
z+r^=r4hB(ls}BflWJ?MM*Xaczy$(_=TH=0uEpO2{Wckb164Fc%z}Rx}<3*#nm8IC71+>fe&>#k3|e|P2?y#7{MEfSKnBcUGhh1GKT@vdsXm|)W-Bg8tm
zOu#RWxH4aHnI_SLkya>nm<=NiK=4fXLbmxJ1
zX1!rQgBSD?O@l%3#v(PXMxoYs4s>sw}}E-x$q;)_X+a#*GU(E0`K
zN7@c5%+)CicbvcL&xg|tbY;1~!W2Dq%^nYV0ByS&M5#$ZQ-v`FF;21!89&>E{}lgw
zEi6aby!f+>{-w6f7Mq6ZD#ovzJgV@AKxctVrDz7TZ6J8!C0PL$JXb2&3qKC3_zXU2
z2XhE{CGavfz?F9G_r)R_fQTF8XrnAB;8>AC+re}MWQ6F{0Q5Qt3@ZE(-
zBUMTl*(;cZX`avr@}Meh^V5TxS&mS;ZI>G-8h8=Dz2oSggVPLV2DwtK(-N38Q?QRl
zs#-)bQoavR%HKOQv!S99afIrYvMY2qeUpQD<~t>&R(qH-@}zfHa03=rR4vcxa;g(n
z3UKt2VlkVn?y25C`;?C1tv2{%OQkq$e4LcdeY)d^;pp57m}q_}Ohcbi;(6EE)IoO>
z7&RjvEQhV!V&oxfVm6oKXl9d++BvN~lXZw0
z&VD}osm^ix&@btxww;`bb@hlK?5uR7cdpL9K;J_#=xBlCAiu#t$LsbV%HXGR^asu6
zIt>C3u)D@pWXN+?6KI?B^bUyPV9c|^-RTAIgTG87BBd$}E7&W=?jM}{@pVFx#>1j=
zuKrO@2Pi(4tqwiibZ3D1U8>4Oz6kDIlj{|8uYIInTPHG!&-lV?YU~RTBf>N-4Qk_;
z?N3dTrxqNj6kQCUOwtV#ZA;pSp4J3%=Z}IsZs7_sAi0qKerr`
zA@->G#BPQr06+6mm(FP%ixi7KX`3Q0{6