diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 287d7d1c09b5e..7e3f948825522 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -463,6 +463,12 @@ repos:
^.*RELEASE_NOTES\.rst$|
^.*CHANGELOG\.txt$|^.*CHANGELOG\.rst$|
git
+ - id: check-base-operator-partial-arguments
+ name: Check BaseOperator and partial() arguments
+ language: python
+ entry: ./scripts/ci/pre_commit/pre_commit_base_operator_partial_arguments.py
+ pass_filenames: false
+ files: ^airflow/models/(?:base|mapped)operator.py$
- id: check-base-operator-usage
language: pygrep
name: Check BaseOperator[Link] core imports
diff --git a/STATIC_CODE_CHECKS.rst b/STATIC_CODE_CHECKS.rst
index e2b57eae6c69b..a6c5b259ec0cf 100644
--- a/STATIC_CODE_CHECKS.rst
+++ b/STATIC_CODE_CHECKS.rst
@@ -144,6 +144,8 @@ require Breeze Docker image to be build locally.
+--------------------------------------------------------+------------------------------------------------------------------+---------+
| check-apache-license-rat | Check if licenses are OK for Apache | |
+--------------------------------------------------------+------------------------------------------------------------------+---------+
+| check-base-operator-partial-arguments | Check BaseOperator and partial() arguments | |
++--------------------------------------------------------+------------------------------------------------------------------+---------+
| check-base-operator-usage | * Check BaseOperator[Link] core imports | |
| | * Check BaseOperator[Link] other imports | |
+--------------------------------------------------------+------------------------------------------------------------------+---------+
diff --git a/airflow/decorators/base.py b/airflow/decorators/base.py
index 1b14cd066832e..1b5b5b760bf77 100644
--- a/airflow/decorators/base.py
+++ b/airflow/decorators/base.py
@@ -45,7 +45,7 @@
from airflow.models.baseoperator import (
BaseOperator,
coerce_resources,
- coerce_retry_delay,
+ coerce_timedelta,
get_merged_defaults,
parse_retries,
)
@@ -344,8 +344,15 @@ def expand(self, **map_kwargs: "Mappable") -> XComArg:
if partial_kwargs.get("pool") is None:
partial_kwargs["pool"] = Pool.DEFAULT_POOL_NAME
partial_kwargs["retries"] = parse_retries(partial_kwargs.get("retries", DEFAULT_RETRIES))
- partial_kwargs["retry_delay"] = coerce_retry_delay(
+ partial_kwargs["retry_delay"] = coerce_timedelta(
partial_kwargs.get("retry_delay", DEFAULT_RETRY_DELAY),
+ key="retry_delay",
+ )
+ max_retry_delay = partial_kwargs.get("max_retry_delay")
+ partial_kwargs["max_retry_delay"] = (
+ max_retry_delay
+ if max_retry_delay is None
+ else coerce_timedelta(max_retry_delay, key="max_retry_delay")
)
partial_kwargs["resources"] = coerce_resources(partial_kwargs.get("resources"))
partial_kwargs.setdefault("executor_config", {})
diff --git a/airflow/models/abstractoperator.py b/airflow/models/abstractoperator.py
index 6187c37182950..cb566ea6f9557 100644
--- a/airflow/models/abstractoperator.py
+++ b/airflow/models/abstractoperator.py
@@ -61,6 +61,9 @@
DEFAULT_POOL_SLOTS: int = 1
DEFAULT_PRIORITY_WEIGHT: int = 1
DEFAULT_QUEUE: str = conf.get_mandatory_value("operators", "default_queue")
+DEFAULT_IGNORE_FIRST_DEPENDS_ON_PAST: bool = conf.getboolean(
+ "scheduler", "ignore_first_depends_on_past_by_default"
+)
DEFAULT_RETRIES: int = conf.getint("core", "default_task_retries", fallback=0)
DEFAULT_RETRY_DELAY: datetime.timedelta = datetime.timedelta(
seconds=conf.getint("core", "default_task_retry_delay", fallback=300)
diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py
index 2fea76071603a..3208c435769b3 100644
--- a/airflow/models/baseoperator.py
+++ b/airflow/models/baseoperator.py
@@ -59,6 +59,7 @@
from airflow.exceptions import AirflowException, TaskDeferred
from airflow.lineage import apply_lineage, prepare_lineage
from airflow.models.abstractoperator import (
+ DEFAULT_IGNORE_FIRST_DEPENDS_ON_PAST,
DEFAULT_OWNER,
DEFAULT_POOL_SLOTS,
DEFAULT_PRIORITY_WEIGHT,
@@ -120,11 +121,11 @@ def parse_retries(retries: Any) -> Optional[int]:
return parsed_retries
-def coerce_retry_delay(retry_delay: Union[float, timedelta]) -> timedelta:
- if isinstance(retry_delay, timedelta):
- return retry_delay
- logger.debug("retry_delay isn't a timedelta object, assuming secs")
- return timedelta(seconds=retry_delay)
+def coerce_timedelta(value: Union[float, timedelta], *, key: str) -> timedelta:
+ if isinstance(value, timedelta):
+ return value
+ logger.debug("%s isn't a timedelta object, assuming secs", key)
+ return timedelta(seconds=value)
def coerce_resources(resources: Optional[Dict[str, Any]]) -> Optional[Resources]:
@@ -197,12 +198,14 @@ def partial(
resources: Optional[Dict[str, Any]] = None,
trigger_rule: str = DEFAULT_TRIGGER_RULE,
depends_on_past: bool = False,
+ ignore_first_depends_on_past: bool = DEFAULT_IGNORE_FIRST_DEPENDS_ON_PAST,
wait_for_downstream: bool = False,
retries: Optional[int] = DEFAULT_RETRIES,
queue: str = DEFAULT_QUEUE,
pool: Optional[str] = None,
pool_slots: int = DEFAULT_POOL_SLOTS,
execution_timeout: Optional[timedelta] = DEFAULT_TASK_EXECUTION_TIMEOUT,
+ max_retry_delay: Union[None, timedelta, float] = None,
retry_delay: Union[timedelta, float] = DEFAULT_RETRY_DELAY,
retry_exponential_backoff: bool = False,
priority_weight: int = DEFAULT_PRIORITY_WEIGHT,
@@ -217,6 +220,11 @@ def partial(
executor_config: Optional[Dict] = None,
inlets: Optional[Any] = None,
outlets: Optional[Any] = None,
+ doc: Optional[str] = None,
+ doc_md: Optional[str] = None,
+ doc_json: Optional[str] = None,
+ doc_yaml: Optional[str] = None,
+ doc_rst: Optional[str] = None,
**kwargs,
) -> OperatorPartial:
from airflow.models.dag import DagContext
@@ -250,12 +258,14 @@ def partial(
partial_kwargs.setdefault("params", default_params)
partial_kwargs.setdefault("trigger_rule", trigger_rule)
partial_kwargs.setdefault("depends_on_past", depends_on_past)
+ partial_kwargs.setdefault("ignore_first_depends_on_past", ignore_first_depends_on_past)
partial_kwargs.setdefault("wait_for_downstream", wait_for_downstream)
partial_kwargs.setdefault("retries", retries)
partial_kwargs.setdefault("queue", queue)
partial_kwargs.setdefault("pool", pool)
partial_kwargs.setdefault("pool_slots", pool_slots)
partial_kwargs.setdefault("execution_timeout", execution_timeout)
+ partial_kwargs.setdefault("max_retry_delay", max_retry_delay)
partial_kwargs.setdefault("retry_delay", retry_delay)
partial_kwargs.setdefault("retry_exponential_backoff", retry_exponential_backoff)
partial_kwargs.setdefault("priority_weight", priority_weight)
@@ -271,6 +281,11 @@ def partial(
partial_kwargs.setdefault("inlets", inlets)
partial_kwargs.setdefault("outlets", outlets)
partial_kwargs.setdefault("resources", resources)
+ partial_kwargs.setdefault("doc", doc)
+ partial_kwargs.setdefault("doc_json", doc_json)
+ partial_kwargs.setdefault("doc_md", doc_md)
+ partial_kwargs.setdefault("doc_rst", doc_rst)
+ partial_kwargs.setdefault("doc_yaml", doc_yaml)
# Post-process arguments. Should be kept in sync with _TaskDecorator.expand().
if "task_concurrency" in kwargs: # Reject deprecated option.
@@ -282,7 +297,12 @@ def partial(
if partial_kwargs["pool"] is None:
partial_kwargs["pool"] = Pool.DEFAULT_POOL_NAME
partial_kwargs["retries"] = parse_retries(partial_kwargs["retries"])
- partial_kwargs["retry_delay"] = coerce_retry_delay(partial_kwargs["retry_delay"])
+ partial_kwargs["retry_delay"] = coerce_timedelta(partial_kwargs["retry_delay"], key="retry_delay")
+ if partial_kwargs["max_retry_delay"] is not None:
+ partial_kwargs["max_retry_delay"] = coerce_timedelta(
+ partial_kwargs["max_retry_delay"],
+ key="max_retry_delay",
+ )
partial_kwargs["executor_config"] = partial_kwargs["executor_config"] or {}
partial_kwargs["resources"] = coerce_resources(partial_kwargs["resources"])
@@ -694,9 +714,7 @@ def __init__(
start_date: Optional[datetime] = None,
end_date: Optional[datetime] = None,
depends_on_past: bool = False,
- ignore_first_depends_on_past: bool = conf.getboolean(
- 'scheduler', 'ignore_first_depends_on_past_by_default'
- ),
+ ignore_first_depends_on_past: bool = DEFAULT_IGNORE_FIRST_DEPENDS_ON_PAST,
wait_for_downstream: bool = False,
dag: Optional['DAG'] = None,
params: Optional[Dict] = None,
@@ -756,10 +774,7 @@ def __init__(
dag = dag or DagContext.get_current_dag()
task_group = task_group or TaskGroupContext.get_current_task_group(dag)
- if task_group:
- self.task_id = task_group.child_id(task_id)
- else:
- self.task_id = task_id
+ self.task_id = task_group.child_id(task_id) if task_group else task_id
if not self.__from_mapped and task_group:
task_group.add(self)
@@ -825,20 +840,18 @@ def __init__(
self.trigger_rule = TriggerRule(trigger_rule)
self.depends_on_past: bool = depends_on_past
- self.ignore_first_depends_on_past = ignore_first_depends_on_past
- self.wait_for_downstream = wait_for_downstream
+ self.ignore_first_depends_on_past: bool = ignore_first_depends_on_past
+ self.wait_for_downstream: bool = wait_for_downstream
if wait_for_downstream:
self.depends_on_past = True
- self.retry_delay = coerce_retry_delay(retry_delay)
+ self.retry_delay = coerce_timedelta(retry_delay, key="retry_delay")
self.retry_exponential_backoff = retry_exponential_backoff
- self.max_retry_delay = max_retry_delay
- if max_retry_delay:
- if isinstance(max_retry_delay, timedelta):
- self.max_retry_delay = max_retry_delay
- else:
- self.log.debug("max_retry_delay isn't a timedelta object, assuming secs")
- self.max_retry_delay = timedelta(seconds=max_retry_delay)
+ self.max_retry_delay = (
+ max_retry_delay
+ if max_retry_delay is None
+ else coerce_timedelta(max_retry_delay, key="max_retry_delay")
+ )
# At execution_time this becomes a normal dict
self.params: Union[ParamsDict, dict] = ParamsDict(params)
diff --git a/airflow/models/mappedoperator.py b/airflow/models/mappedoperator.py
index 663ceeece16a0..69621cadc5088 100644
--- a/airflow/models/mappedoperator.py
+++ b/airflow/models/mappedoperator.py
@@ -49,6 +49,7 @@
from airflow.compat.functools import cache, cached_property
from airflow.exceptions import AirflowException, UnmappableOperator
from airflow.models.abstractoperator import (
+ DEFAULT_IGNORE_FIRST_DEPENDS_ON_PAST,
DEFAULT_OWNER,
DEFAULT_POOL_SLOTS,
DEFAULT_PRIORITY_WEIGHT,
@@ -387,6 +388,11 @@ def trigger_rule(self) -> TriggerRule:
def depends_on_past(self) -> bool:
return bool(self.partial_kwargs.get("depends_on_past"))
+ @property
+ def ignore_first_depends_on_past(self) -> bool:
+ value = self.partial_kwargs.get("ignore_first_depends_on_past", DEFAULT_IGNORE_FIRST_DEPENDS_ON_PAST)
+ return bool(value)
+
@property
def wait_for_downstream(self) -> bool:
return bool(self.partial_kwargs.get("wait_for_downstream"))
@@ -411,6 +417,10 @@ def pool_slots(self) -> Optional[str]:
def execution_timeout(self) -> Optional[datetime.timedelta]:
return self.partial_kwargs.get("execution_timeout")
+ @property
+ def max_retry_delay(self) -> Optional[datetime.timedelta]:
+ return self.partial_kwargs.get("max_retry_delay")
+
@property
def retry_delay(self) -> datetime.timedelta:
return self.partial_kwargs.get("retry_delay", DEFAULT_RETRY_DELAY)
@@ -471,6 +481,26 @@ def inlets(self) -> Optional[Any]:
def outlets(self) -> Optional[Any]:
return self.partial_kwargs.get("outlets", None)
+ @property
+ def doc(self) -> Optional[str]:
+ return self.partial_kwargs.get("doc")
+
+ @property
+ def doc_md(self) -> Optional[str]:
+ return self.partial_kwargs.get("doc_md")
+
+ @property
+ def doc_json(self) -> Optional[str]:
+ return self.partial_kwargs.get("doc_json")
+
+ @property
+ def doc_yaml(self) -> Optional[str]:
+ return self.partial_kwargs.get("doc_yaml")
+
+ @property
+ def doc_rst(self) -> Optional[str]:
+ return self.partial_kwargs.get("doc_rst")
+
def get_dag(self) -> Optional["DAG"]:
"""Implementing Operator."""
return self.dag
@@ -480,7 +510,6 @@ def serialize_for_task_group(self) -> Tuple[DagAttributeTypes, Any]:
return DagAttributeTypes.OP, self.task_id
def _get_unmap_kwargs(self) -> Dict[str, Any]:
-
return {
"task_id": self.task_id,
"dag": self.dag,
diff --git a/airflow/ti_deps/deps/prev_dagrun_dep.py b/airflow/ti_deps/deps/prev_dagrun_dep.py
index fb7f7e436c38e..391563a12770a 100644
--- a/airflow/ti_deps/deps/prev_dagrun_dep.py
+++ b/airflow/ti_deps/deps/prev_dagrun_dep.py
@@ -34,7 +34,7 @@ class PrevDagrunDep(BaseTIDep):
IS_TASK_DEP = True
@provide_session
- def _get_dep_statuses(self, ti, session, dep_context):
+ def _get_dep_statuses(self, ti: TI, session, dep_context):
if dep_context.ignore_depends_on_past:
reason = "The context specified that the state of past DAGs could be ignored."
yield self._passing_status(reason=reason)
@@ -50,7 +50,7 @@ def _get_dep_statuses(self, ti, session, dep_context):
return
# Don't depend on the previous task instance if we are the first task.
- catchup = ti.task.dag.catchup
+ catchup = ti.task.dag and ti.task.dag.catchup
if catchup:
last_dagrun = dr.get_previous_scheduled_dagrun(session)
else:
diff --git a/dev/breeze/src/airflow_breeze/pre_commit_ids.py b/dev/breeze/src/airflow_breeze/pre_commit_ids.py
index 40ff16a7a03bd..faebc7db29a74 100644
--- a/dev/breeze/src/airflow_breeze/pre_commit_ids.py
+++ b/dev/breeze/src/airflow_breeze/pre_commit_ids.py
@@ -29,6 +29,7 @@
'check-airflow-config-yaml-consistent',
'check-airflow-providers-have-extras',
'check-apache-license-rat',
+ 'check-base-operator-partial-arguments',
'check-base-operator-usage',
'check-boring-cyborg-configuration',
'check-breeze-top-dependencies-limited',
diff --git a/images/breeze/output-build-docs.svg b/images/breeze/output-build-docs.svg
index c65181b9aeff9..86d7969351ad5 100644
--- a/images/breeze/output-build-docs.svg
+++ b/images/breeze/output-build-docs.svg
@@ -19,90 +19,90 @@
font-weight: 700;
}
- .terminal-925795226-matrix {
+ .terminal-1803387802-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-925795226-title {
+ .terminal-1803387802-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-925795226-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-925795226-r2 { fill: #c5c8c6 }
-.terminal-925795226-r3 { fill: #d0b344;font-weight: bold }
-.terminal-925795226-r4 { fill: #868887 }
-.terminal-925795226-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-925795226-r6 { fill: #98a84b;font-weight: bold }
-.terminal-925795226-r7 { fill: #8d7b39 }
+ .terminal-1803387802-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-1803387802-r2 { fill: #c5c8c6 }
+.terminal-1803387802-r3 { fill: #d0b344;font-weight: bold }
+.terminal-1803387802-r4 { fill: #868887 }
+.terminal-1803387802-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-1803387802-r6 { fill: #98a84b;font-weight: bold }
+.terminal-1803387802-r7 { fill: #8d7b39 }
- Command: build-docs
+ Command: build-docs
-
- Usage: breeze build-docs [OPTIONS]
-
- Build documentation in the container.
-
-╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --docs-only -d Only build documentation. │
-│ --spellcheck-only -s Only run spell checking. │
-│ --for-production -p Builds documentation for official release i.e. all links point to stable version. │
-│ --package-filter -p List of packages to consider. │
-│ (apache-airflow | apache-airflow-providers | apache-airflow-providers-airbyte | │
-│ apache-airflow-providers-alibaba | apache-airflow-providers-amazon | │
-│ apache-airflow-providers-apache-beam | apache-airflow-providers-apache-cassandra | │
-│ apache-airflow-providers-apache-drill | apache-airflow-providers-apache-druid | │
-│ apache-airflow-providers-apache-hdfs | apache-airflow-providers-apache-hive | │
-│ apache-airflow-providers-apache-kylin | apache-airflow-providers-apache-livy | │
-│ apache-airflow-providers-apache-pig | apache-airflow-providers-apache-pinot | │
-│ apache-airflow-providers-apache-spark | apache-airflow-providers-apache-sqoop | │
-│ apache-airflow-providers-arangodb | apache-airflow-providers-asana | │
-│ apache-airflow-providers-celery | apache-airflow-providers-cloudant | │
-│ apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-databricks | │
-│ apache-airflow-providers-datadog | apache-airflow-providers-dbt-cloud | │
-│ apache-airflow-providers-dingding | apache-airflow-providers-discord | │
-│ apache-airflow-providers-docker | apache-airflow-providers-elasticsearch | │
-│ apache-airflow-providers-exasol | apache-airflow-providers-facebook | │
-│ apache-airflow-providers-ftp | apache-airflow-providers-github | │
-│ apache-airflow-providers-google | apache-airflow-providers-grpc | │
-│ apache-airflow-providers-hashicorp | apache-airflow-providers-http | │
-│ apache-airflow-providers-imap | apache-airflow-providers-influxdb | │
-│ apache-airflow-providers-jdbc | apache-airflow-providers-jenkins | │
-│ apache-airflow-providers-jira | apache-airflow-providers-microsoft-azure | │
-│ apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp | │
-│ apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo | │
-│ apache-airflow-providers-mysql | apache-airflow-providers-neo4j | │
-│ apache-airflow-providers-odbc | apache-airflow-providers-openfaas | │
-│ apache-airflow-providers-opsgenie | apache-airflow-providers-oracle | │
-│ apache-airflow-providers-pagerduty | apache-airflow-providers-papermill | │
-│ apache-airflow-providers-plexus | apache-airflow-providers-postgres | │
-│ apache-airflow-providers-presto | apache-airflow-providers-qubole | │
-│ apache-airflow-providers-redis | apache-airflow-providers-salesforce | │
-│ apache-airflow-providers-samba | apache-airflow-providers-segment | │
-│ apache-airflow-providers-sendgrid | apache-airflow-providers-sftp | │
-│ apache-airflow-providers-singularity | apache-airflow-providers-slack | │
-│ apache-airflow-providers-snowflake | apache-airflow-providers-sqlite | │
-│ apache-airflow-providers-ssh | apache-airflow-providers-tableau | │
-│ apache-airflow-providers-telegram | apache-airflow-providers-trino | │
-│ apache-airflow-providers-vertica | apache-airflow-providers-yandex | │
-│ apache-airflow-providers-zendesk | docker-stack | helm-chart) │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze build-docs [OPTIONS]
+
+ Build documentation in the container.
+
+╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --docs-only -d Only build documentation. │
+│ --spellcheck-only -s Only run spell checking. │
+│ --for-production -p Builds documentation for official release i.e. all links point to stable version. │
+│ --package-filter -p List of packages to consider. │
+│ (apache-airflow | apache-airflow-providers | apache-airflow-providers-airbyte | │
+│ apache-airflow-providers-alibaba | apache-airflow-providers-amazon | │
+│ apache-airflow-providers-apache-beam | apache-airflow-providers-apache-cassandra | │
+│ apache-airflow-providers-apache-drill | apache-airflow-providers-apache-druid | │
+│ apache-airflow-providers-apache-hdfs | apache-airflow-providers-apache-hive | │
+│ apache-airflow-providers-apache-kylin | apache-airflow-providers-apache-livy | │
+│ apache-airflow-providers-apache-pig | apache-airflow-providers-apache-pinot | │
+│ apache-airflow-providers-apache-spark | apache-airflow-providers-apache-sqoop | │
+│ apache-airflow-providers-arangodb | apache-airflow-providers-asana | │
+│ apache-airflow-providers-celery | apache-airflow-providers-cloudant | │
+│ apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-databricks | │
+│ apache-airflow-providers-datadog | apache-airflow-providers-dbt-cloud | │
+│ apache-airflow-providers-dingding | apache-airflow-providers-discord | │
+│ apache-airflow-providers-docker | apache-airflow-providers-elasticsearch | │
+│ apache-airflow-providers-exasol | apache-airflow-providers-facebook | │
+│ apache-airflow-providers-ftp | apache-airflow-providers-github | │
+│ apache-airflow-providers-google | apache-airflow-providers-grpc | │
+│ apache-airflow-providers-hashicorp | apache-airflow-providers-http | │
+│ apache-airflow-providers-imap | apache-airflow-providers-influxdb | │
+│ apache-airflow-providers-jdbc | apache-airflow-providers-jenkins | │
+│ apache-airflow-providers-jira | apache-airflow-providers-microsoft-azure | │
+│ apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp | │
+│ apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo | │
+│ apache-airflow-providers-mysql | apache-airflow-providers-neo4j | │
+│ apache-airflow-providers-odbc | apache-airflow-providers-openfaas | │
+│ apache-airflow-providers-opsgenie | apache-airflow-providers-oracle | │
+│ apache-airflow-providers-pagerduty | apache-airflow-providers-papermill | │
+│ apache-airflow-providers-plexus | apache-airflow-providers-postgres | │
+│ apache-airflow-providers-presto | apache-airflow-providers-qubole | │
+│ apache-airflow-providers-redis | apache-airflow-providers-salesforce | │
+│ apache-airflow-providers-samba | apache-airflow-providers-segment | │
+│ apache-airflow-providers-sendgrid | apache-airflow-providers-sftp | │
+│ apache-airflow-providers-singularity | apache-airflow-providers-slack | │
+│ apache-airflow-providers-snowflake | apache-airflow-providers-sqlite | │
+│ apache-airflow-providers-ssh | apache-airflow-providers-tableau | │
+│ apache-airflow-providers-telegram | apache-airflow-providers-trino | │
+│ apache-airflow-providers-vertica | apache-airflow-providers-yandex | │
+│ apache-airflow-providers-zendesk | docker-stack | helm-chart) │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-build-image.svg b/images/breeze/output-build-image.svg
index b779367062c37..371ab8b3b68c0 100644
--- a/images/breeze/output-build-image.svg
+++ b/images/breeze/output-build-image.svg
@@ -19,93 +19,93 @@
font-weight: 700;
}
- .terminal-2884033632-matrix {
+ .terminal-1635638368-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-2884033632-title {
+ .terminal-1635638368-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-2884033632-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-2884033632-r2 { fill: #c5c8c6 }
-.terminal-2884033632-r3 { fill: #d0b344;font-weight: bold }
-.terminal-2884033632-r4 { fill: #868887 }
-.terminal-2884033632-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-2884033632-r6 { fill: #98a84b;font-weight: bold }
-.terminal-2884033632-r7 { fill: #8d7b39 }
+ .terminal-1635638368-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-1635638368-r2 { fill: #c5c8c6 }
+.terminal-1635638368-r3 { fill: #d0b344;font-weight: bold }
+.terminal-1635638368-r4 { fill: #868887 }
+.terminal-1635638368-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-1635638368-r6 { fill: #98a84b;font-weight: bold }
+.terminal-1635638368-r7 { fill: #8d7b39 }
- Command: build-image
+ Command: build-image
-
- Usage: breeze build-image [OPTIONS]
-
- Build CI image. Include building multiple images for all python versions (sequentially).
-
-╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --python -p Python major/minor version used in Airflow image for images. │
-│ (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --upgrade-to-newer-dependencies -u When set, upgrade all PIP packages to latest. │
-│ --debian-version Debian version used for the image. (bullseye | buster) [default: bullseye] │
-│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
-│ --tag-as-latest Tags the image as latest and update checksum of all files after pulling. │
-│ Useful when you build or pull image with --image-tag. │
-│ --docker-cache -c Cache option for image used during the build. (registry | local | disabled) │
-│ [default: registry] │
-│ --force-build Force image build no matter if it is determined as needed. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Building multiple images ───────────────────────────────────────────────────────────────────────────────────────────╮
-│ --build-multiple-images Run the operation sequentially on all or selected subset of Python versions. │
-│ --python-versions Space separated list of python versions used for build with multiple versions. (TEXT) │
-│ [default: 3.7 3.8 3.9 3.10] │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Advanced options (for power users) ─────────────────────────────────────────────────────────────────────────────────╮
-│ --install-providers-from-sources Install providers from sources when installing. │
-│ --airflow-constraints-mode Mode of constraints for CI image building │
-│ (constraints-source-providers | constraints | constraints-no-providers) │
-│ [default: constraints-source-providers] │
-│ --airflow-constraints-reference Constraint reference to use when building the image. (TEXT) │
-│ --additional-python-deps Additional python dependencies to use when building the images. (TEXT) │
-│ --runtime-apt-deps Apt runtime dependencies to use when building the images. (TEXT) │
-│ --runtime-apt-command Command executed before runtime apt deps are installed. (TEXT) │
-│ --additional-extras Additional extra package while installing Airflow in the image. (TEXT) │
-│ --additional-runtime-apt-deps Additional apt runtime dependencies to use when building the images. (TEXT) │
-│ --additional-runtime-apt-env Additional environment variables set when adding runtime dependencies. (TEXT) │
-│ --additional-runtime-apt-command Additional command executed before runtime apt deps are installed. (TEXT) │
-│ --additional-dev-apt-deps Additional apt dev dependencies to use when building the images. (TEXT) │
-│ --additional-dev-apt-env Additional environment variables set when adding dev dependencies. (TEXT) │
-│ --additional-dev-apt-command Additional command executed before dev apt deps are installed. (TEXT) │
-│ --dev-apt-deps Apt dev dependencies to use when building the images. (TEXT) │
-│ --dev-apt-command Command executed before dev apt deps are installed. (TEXT) │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮
-│ --github-token The token used to authenticate to GitHub. (TEXT) │
-│ --github-username The user name used to authenticate to GitHub. (TEXT) │
-│ --platform Platform for Airflow image. (linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) │
-│ --push-image Push image after building it. │
-│ --empty-image Prepare empty image tagged with the same name as the Airflow image. │
-│ --prepare-buildx-cache Prepares build cache (this is done as separate per-platform steps instead of building │
-│ the image). │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze build-image [OPTIONS]
+
+ Build CI image. Include building multiple images for all python versions (sequentially).
+
+╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --python -p Python major/minor version used in Airflow image for images. │
+│ (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --upgrade-to-newer-dependencies -u When set, upgrade all PIP packages to latest. │
+│ --debian-version Debian version used for the image. (bullseye | buster) [default: bullseye] │
+│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
+│ --tag-as-latest Tags the image as latest and update checksum of all files after pulling. Useful │
+│ when you build or pull image with --image-tag. │
+│ --docker-cache -c Cache option for image used during the build. (registry | local | disabled) │
+│ [default: registry] │
+│ --force-build Force image build no matter if it is determined as needed. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Building multiple images ───────────────────────────────────────────────────────────────────────────────────────────╮
+│ --build-multiple-images Run the operation sequentially on all or selected subset of Python versions. │
+│ --python-versions Space separated list of python versions used for build with multiple versions. (TEXT) │
+│ [default: 3.7 3.8 3.9 3.10] │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Advanced options (for power users) ─────────────────────────────────────────────────────────────────────────────────╮
+│ --install-providers-from-sources Install providers from sources when installing. │
+│ --airflow-constraints-mode Mode of constraints for CI image building │
+│ (constraints-source-providers | constraints | constraints-no-providers) │
+│ [default: constraints-source-providers] │
+│ --airflow-constraints-reference Constraint reference to use when building the image. (TEXT) │
+│ --additional-python-deps Additional python dependencies to use when building the images. (TEXT) │
+│ --runtime-apt-deps Apt runtime dependencies to use when building the images. (TEXT) │
+│ --runtime-apt-command Command executed before runtime apt deps are installed. (TEXT) │
+│ --additional-extras Additional extra package while installing Airflow in the image. (TEXT) │
+│ --additional-runtime-apt-deps Additional apt runtime dependencies to use when building the images. (TEXT) │
+│ --additional-runtime-apt-env Additional environment variables set when adding runtime dependencies. (TEXT) │
+│ --additional-runtime-apt-command Additional command executed before runtime apt deps are installed. (TEXT) │
+│ --additional-dev-apt-deps Additional apt dev dependencies to use when building the images. (TEXT) │
+│ --additional-dev-apt-env Additional environment variables set when adding dev dependencies. (TEXT) │
+│ --additional-dev-apt-command Additional command executed before dev apt deps are installed. (TEXT) │
+│ --dev-apt-deps Apt dev dependencies to use when building the images. (TEXT) │
+│ --dev-apt-command Command executed before dev apt deps are installed. (TEXT) │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮
+│ --github-token The token used to authenticate to GitHub. (TEXT) │
+│ --github-username The user name used to authenticate to GitHub. (TEXT) │
+│ --platform Platform for Airflow image. (linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) │
+│ --push-image Push image after building it. │
+│ --empty-image Prepare empty image tagged with the same name as the Airflow image. │
+│ --prepare-buildx-cache Prepares build cache (this is done as separate per-platform steps instead of building the │
+│ image). │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-build-prod-image.svg b/images/breeze/output-build-prod-image.svg
index 247461d55530c..83cbbd96c3cc9 100644
--- a/images/breeze/output-build-prod-image.svg
+++ b/images/breeze/output-build-prod-image.svg
@@ -19,110 +19,110 @@
font-weight: 700;
}
- .terminal-997154497-matrix {
+ .terminal-3136577362-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-997154497-title {
+ .terminal-3136577362-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-997154497-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-997154497-r2 { fill: #c5c8c6 }
-.terminal-997154497-r3 { fill: #d0b344;font-weight: bold }
-.terminal-997154497-r4 { fill: #868887 }
-.terminal-997154497-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-997154497-r6 { fill: #98a84b;font-weight: bold }
-.terminal-997154497-r7 { fill: #8d7b39 }
+ .terminal-3136577362-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-3136577362-r2 { fill: #c5c8c6 }
+.terminal-3136577362-r3 { fill: #d0b344;font-weight: bold }
+.terminal-3136577362-r4 { fill: #868887 }
+.terminal-3136577362-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-3136577362-r6 { fill: #98a84b;font-weight: bold }
+.terminal-3136577362-r7 { fill: #8d7b39 }
- Command: build-prod-image
+ Command: build-prod-image
-
- Usage: breeze build-prod-image [OPTIONS]
-
- Build Production image. Include building multiple images for all or selected Python versions sequentially.
-
-╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --python -p Python major/minor version used in Airflow image for images. │
-│ (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --install-airflow-version -V Install version of Airflow from PyPI. (TEXT) │
-│ --upgrade-to-newer-dependencies -u When set, upgrade all PIP packages to latest. │
-│ --debian-version Debian version used for the image. (bullseye | buster) [default: bullseye] │
-│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
-│ --tag-as-latest Tags the image as latest and update checksum of all files after pulling. │
-│ Useful when you build or pull image with --image-tag. │
-│ --docker-cache -c Cache option for image used during the build. (registry | local | disabled) │
-│ [default: registry] │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Building multiple images ───────────────────────────────────────────────────────────────────────────────────────────╮
-│ --build-multiple-images Run the operation sequentially on all or selected subset of Python versions. │
-│ --python-versions Space separated list of python versions used for build with multiple versions. (TEXT) │
-│ [default: 3.7 3.8 3.9 3.10] │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options for customizing images ─────────────────────────────────────────────────────────────────────────────────────╮
-│ --install-providers-from-sources Install providers from sources when installing. │
-│ --airflow-extras Extras to install by default. │
-│ (TEXT) │
-│ [default: │
-│ amazon,async,celery,cncf.kubernetes,dask,docker,elasticsearch,ftp,google,goog… │
-│ --airflow-constraints-mode Mode of constraints for PROD image building │
-│ (constraints | constraints-no-providers | constraints-source-providers) │
-│ [default: constraints] │
-│ --airflow-constraints-reference Constraint reference to use when building the image. (TEXT) │
-│ --additional-python-deps Additional python dependencies to use when building the images. (TEXT) │
-│ --additional-extras Additional extra package while installing Airflow in the image. (TEXT) │
-│ --additional-runtime-apt-deps Additional apt runtime dependencies to use when building the images. (TEXT) │
-│ --additional-runtime-apt-env Additional environment variables set when adding runtime dependencies. (TEXT) │
-│ --additional-runtime-apt-command Additional command executed before runtime apt deps are installed. (TEXT) │
-│ --additional-dev-apt-deps Additional apt dev dependencies to use when building the images. (TEXT) │
-│ --additional-dev-apt-env Additional environment variables set when adding dev dependencies. (TEXT) │
-│ --additional-dev-apt-command Additional command executed before dev apt deps are installed. (TEXT) │
-│ --runtime-apt-deps Apt runtime dependencies to use when building the images. (TEXT) │
-│ --runtime-apt-command Command executed before runtime apt deps are installed. (TEXT) │
-│ --dev-apt-deps Apt dev dependencies to use when building the images. (TEXT) │
-│ --dev-apt-command Command executed before dev apt deps are installed. (TEXT) │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Customization options (for specific customization needs) ───────────────────────────────────────────────────────────╮
-│ --install-packages-from-context Install wheels from local docker-context-files when building image. │
-│ --airflow-is-in-context If set Airflow is installed from docker-context-files only rather than │
-│ from PyPI or sources. │
-│ --cleanup-context Clean up docker context files before running build (cannot be used │
-│ together with --install-packages-from-context). │
-│ --disable-mysql-client-installation Do not install MySQL client. │
-│ --disable-mssql-client-installation Do not install MsSQl client. │
-│ --disable-postgres-client-installation Do not install Postgres client. │
-│ --disable-airflow-repo-cache Disable cache from Airflow repository during building. │
-│ --install-airflow-reference Install Airflow using GitHub tag or branch. (TEXT) │
-│ --installation-method Install Airflow from: sources or PyPI. (. | apache-airflow) │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮
-│ --github-token The token used to authenticate to GitHub. (TEXT) │
-│ --github-username The user name used to authenticate to GitHub. (TEXT) │
-│ --platform Platform for Airflow image. (linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) │
-│ --push-image Push image after building it. │
-│ --empty-image Prepare empty image tagged with the same name as the Airflow image. │
-│ --prepare-buildx-cache Prepares build cache (this is done as separate per-platform steps instead of building │
-│ the image). │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --verbose -v Print verbose information about performed steps. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze build-prod-image [OPTIONS]
+
+ Build Production image. Include building multiple images for all or selected Python versions sequentially.
+
+╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --python -p Python major/minor version used in Airflow image for images. │
+│ (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --install-airflow-version -V Install version of Airflow from PyPI. (TEXT) │
+│ --upgrade-to-newer-dependencies -u When set, upgrade all PIP packages to latest. │
+│ --debian-version Debian version used for the image. (bullseye | buster) [default: bullseye] │
+│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
+│ --tag-as-latest Tags the image as latest and update checksum of all files after pulling. Useful │
+│ when you build or pull image with --image-tag. │
+│ --docker-cache -c Cache option for image used during the build. (registry | local | disabled) │
+│ [default: registry] │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Building multiple images ───────────────────────────────────────────────────────────────────────────────────────────╮
+│ --build-multiple-images Run the operation sequentially on all or selected subset of Python versions. │
+│ --python-versions Space separated list of python versions used for build with multiple versions. (TEXT) │
+│ [default: 3.7 3.8 3.9 3.10] │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options for customizing images ─────────────────────────────────────────────────────────────────────────────────────╮
+│ --install-providers-from-sources Install providers from sources when installing. │
+│ --airflow-extras Extras to install by default. │
+│ (TEXT) │
+│ [default: │
+│ amazon,async,celery,cncf.kubernetes,dask,docker,elasticsearch,ftp,google,google… │
+│ --airflow-constraints-mode Mode of constraints for PROD image building │
+│ (constraints | constraints-no-providers | constraints-source-providers) │
+│ [default: constraints] │
+│ --airflow-constraints-reference Constraint reference to use when building the image. (TEXT) │
+│ --additional-python-deps Additional python dependencies to use when building the images. (TEXT) │
+│ --additional-extras Additional extra package while installing Airflow in the image. (TEXT) │
+│ --additional-runtime-apt-deps Additional apt runtime dependencies to use when building the images. (TEXT) │
+│ --additional-runtime-apt-env Additional environment variables set when adding runtime dependencies. (TEXT) │
+│ --additional-runtime-apt-command Additional command executed before runtime apt deps are installed. (TEXT) │
+│ --additional-dev-apt-deps Additional apt dev dependencies to use when building the images. (TEXT) │
+│ --additional-dev-apt-env Additional environment variables set when adding dev dependencies. (TEXT) │
+│ --additional-dev-apt-command Additional command executed before dev apt deps are installed. (TEXT) │
+│ --runtime-apt-deps Apt runtime dependencies to use when building the images. (TEXT) │
+│ --runtime-apt-command Command executed before runtime apt deps are installed. (TEXT) │
+│ --dev-apt-deps Apt dev dependencies to use when building the images. (TEXT) │
+│ --dev-apt-command Command executed before dev apt deps are installed. (TEXT) │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Customization options (for specific customization needs) ───────────────────────────────────────────────────────────╮
+│ --install-packages-from-context Install wheels from local docker-context-files when building image. │
+│ --airflow-is-in-context If set Airflow is installed from docker-context-files only rather than │
+│ from PyPI or sources. │
+│ --cleanup-context Clean up docker context files before running build (cannot be used │
+│ together with --install-packages-from-context). │
+│ --disable-mysql-client-installation Do not install MySQL client. │
+│ --disable-mssql-client-installation Do not install MsSQl client. │
+│ --disable-postgres-client-installation Do not install Postgres client. │
+│ --disable-airflow-repo-cache Disable cache from Airflow repository during building. │
+│ --install-airflow-reference Install Airflow using GitHub tag or branch. (TEXT) │
+│ --installation-method Install Airflow from: sources or PyPI. (. | apache-airflow) │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮
+│ --github-token The token used to authenticate to GitHub. (TEXT) │
+│ --github-username The user name used to authenticate to GitHub. (TEXT) │
+│ --platform Platform for Airflow image. (linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) │
+│ --push-image Push image after building it. │
+│ --empty-image Prepare empty image tagged with the same name as the Airflow image. │
+│ --prepare-buildx-cache Prepares build cache (this is done as separate per-platform steps instead of building the │
+│ image). │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --verbose -v Print verbose information about performed steps. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-cleanup.svg b/images/breeze/output-cleanup.svg
index 68395e65ebe26..b91de5f1a23a1 100644
--- a/images/breeze/output-cleanup.svg
+++ b/images/breeze/output-cleanup.svg
@@ -19,50 +19,50 @@
font-weight: 700;
}
- .terminal-2880815510-matrix {
+ .terminal-111002006-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-2880815510-title {
+ .terminal-111002006-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-2880815510-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-2880815510-r2 { fill: #c5c8c6 }
-.terminal-2880815510-r3 { fill: #d0b344;font-weight: bold }
-.terminal-2880815510-r4 { fill: #868887 }
-.terminal-2880815510-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-2880815510-r6 { fill: #98a84b;font-weight: bold }
-.terminal-2880815510-r7 { fill: #8d7b39 }
+ .terminal-111002006-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-111002006-r2 { fill: #c5c8c6 }
+.terminal-111002006-r3 { fill: #d0b344;font-weight: bold }
+.terminal-111002006-r4 { fill: #868887 }
+.terminal-111002006-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-111002006-r6 { fill: #98a84b;font-weight: bold }
+.terminal-111002006-r7 { fill: #8d7b39 }
- Command: cleanup
+ Command: cleanup
-
- Usage: breeze cleanup [OPTIONS]
-
- Cleans the cache of parameters, docker cache and optionally - currently downloaded images.
-
-╭─ Cleanup flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --all Also remove currently downloaded Breeze images. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze cleanup [OPTIONS]
+
+ Cleans the cache of parameters, docker cache and optionally - currently downloaded images.
+
+╭─ Cleanup flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --all Also remove currently downloaded Breeze images. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-command-hash-export.svg b/images/breeze/output-command-hash-export.svg
index 03d562e242f45..af2725ad9b109 100644
--- a/images/breeze/output-command-hash-export.svg
+++ b/images/breeze/output-command-hash-export.svg
@@ -19,43 +19,43 @@
font-weight: 700;
}
- .terminal-4066869648-matrix {
+ .terminal-161448336-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-4066869648-title {
+ .terminal-161448336-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-4066869648-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-4066869648-r2 { fill: #c5c8c6 }
-.terminal-4066869648-r3 { fill: #d0b344;font-weight: bold }
-.terminal-4066869648-r4 { fill: #868887 }
-.terminal-4066869648-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-4066869648-r6 { fill: #98a84b;font-weight: bold }
+ .terminal-161448336-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-161448336-r2 { fill: #c5c8c6 }
+.terminal-161448336-r3 { fill: #d0b344;font-weight: bold }
+.terminal-161448336-r4 { fill: #868887 }
+.terminal-161448336-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-161448336-r6 { fill: #98a84b;font-weight: bold }
- Command: command-hash-export
+ Command: command-hash-export
-
- Usage: breeze command-hash-export [OPTIONS] OUTPUT
-
- Outputs hash of all click commands to file or stdout if `-` is used (useful to see if images should be regenerated).
-
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze command-hash-export [OPTIONS] OUTPUT
+
+ Outputs hash of all click commands to file or stdout if `-` is used (useful to see if images should be regenerated).
+
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt
index 0a73056042294..4b5a2913c32d6 100644
--- a/images/breeze/output-commands-hash.txt
+++ b/images/breeze/output-commands-hash.txt
@@ -1 +1 @@
-dfa197bfca0465db8174b262c4b9850b
+5f6d42eb170dba8f19b375f8e7c588b3
diff --git a/images/breeze/output-commands.svg b/images/breeze/output-commands.svg
index 4fe025fcef29d..638d76251a653 100644
--- a/images/breeze/output-commands.svg
+++ b/images/breeze/output-commands.svg
@@ -1,4 +1,4 @@
-
diff --git a/images/breeze/output-docker-compose-tests.svg b/images/breeze/output-docker-compose-tests.svg
index 71d16ecda4421..a3007e838b78b 100644
--- a/images/breeze/output-docker-compose-tests.svg
+++ b/images/breeze/output-docker-compose-tests.svg
@@ -19,52 +19,52 @@
font-weight: 700;
}
- .terminal-2246963640-matrix {
+ .terminal-25948600-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-2246963640-title {
+ .terminal-25948600-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-2246963640-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-2246963640-r2 { fill: #c5c8c6 }
-.terminal-2246963640-r3 { fill: #d0b344;font-weight: bold }
-.terminal-2246963640-r4 { fill: #868887 }
-.terminal-2246963640-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-2246963640-r6 { fill: #98a84b;font-weight: bold }
-.terminal-2246963640-r7 { fill: #8d7b39 }
+ .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 }
- 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 -n Name of the image to verify (overrides --python and --image-tag). (TEXT) │
-│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze docker-compose-tests [OPTIONS] [EXTRA_PYTEST_ARGS]...
+
+ Run docker-compose tests.
+
+╭─ Docker-compose tests flag ──────────────────────────────────────────────────────────────────────────────────────────╮
+│ --image-name -n Name of the image to verify (overrides --python and --image-tag). (TEXT) │
+│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-exec.svg b/images/breeze/output-exec.svg
index b197eb3d63d1b..42d917747043e 100644
--- a/images/breeze/output-exec.svg
+++ b/images/breeze/output-exec.svg
@@ -19,44 +19,44 @@
font-weight: 700;
}
- .terminal-3580858145-matrix {
+ .terminal-4239167265-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-3580858145-title {
+ .terminal-4239167265-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-3580858145-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-3580858145-r2 { fill: #c5c8c6 }
-.terminal-3580858145-r3 { fill: #d0b344;font-weight: bold }
-.terminal-3580858145-r4 { fill: #868887 }
-.terminal-3580858145-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-3580858145-r6 { fill: #98a84b;font-weight: bold }
+ .terminal-4239167265-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-4239167265-r2 { fill: #c5c8c6 }
+.terminal-4239167265-r3 { fill: #d0b344;font-weight: bold }
+.terminal-4239167265-r4 { fill: #868887 }
+.terminal-4239167265-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-4239167265-r6 { fill: #98a84b;font-weight: bold }
- Command: exec
+ Command: exec
-
- Usage: breeze exec [OPTIONS] [EXEC_ARGS]...
-
- Joins the interactive shell of running airflow container
-
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze exec [OPTIONS] [EXEC_ARGS]...
+
+ Joins the interactive shell of running airflow container
+
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-find-newer-dependencies.svg b/images/breeze/output-find-newer-dependencies.svg
index ea60bb6ab5cc9..4f250427c6dc1 100644
--- a/images/breeze/output-find-newer-dependencies.svg
+++ b/images/breeze/output-find-newer-dependencies.svg
@@ -19,53 +19,53 @@
font-weight: 700;
}
- .terminal-1328349300-matrix {
+ .terminal-3402563700-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-1328349300-title {
+ .terminal-3402563700-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-1328349300-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1328349300-r2 { fill: #c5c8c6 }
-.terminal-1328349300-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1328349300-r4 { fill: #868887 }
-.terminal-1328349300-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1328349300-r6 { fill: #98a84b;font-weight: bold }
-.terminal-1328349300-r7 { fill: #8d7b39 }
+ .terminal-3402563700-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-3402563700-r2 { fill: #c5c8c6 }
+.terminal-3402563700-r3 { fill: #d0b344;font-weight: bold }
+.terminal-3402563700-r4 { fill: #868887 }
+.terminal-3402563700-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-3402563700-r6 { fill: #98a84b;font-weight: bold }
+.terminal-3402563700-r7 { fill: #8d7b39 }
- Command: find-newer-dependencies
+ Command: find-newer-dependencies
-
- Usage: breeze find-newer-dependencies [OPTIONS]
-
- Finds which dependencies are being upgraded.
-
-╭─ Find newer dependencies flags ──────────────────────────────────────────────────────────────────────────────────────╮
-│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --timezone Timezone to use during the check (TEXT) │
-│ --updated-on-or-after Date when the release was updated after (TEXT) │
-│ --max-age Max age of the last release (used if no updated-on-or-after if specified) (INTEGER) │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --airflow-constraints-reference Constraint reference to use. Useful with --use-airflow-version parameter to │
-│ specify constraints for the installed version and to find newer dependencies │
-│ (TEXT) │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze find-newer-dependencies [OPTIONS]
+
+ Finds which dependencies are being upgraded.
+
+╭─ Find newer dependencies flags ──────────────────────────────────────────────────────────────────────────────────────╮
+│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --timezone Timezone to use during the check (TEXT) │
+│ --updated-on-or-after Date when the release was updated after (TEXT) │
+│ --max-age Max age of the last release (used if no updated-on-or-after if specified) (INTEGER) │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --airflow-constraints-reference Constraint reference to use. Useful with --use-airflow-version parameter to │
+│ specify constraints for the installed version and to find newer dependencies │
+│ (TEXT) │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-fix-ownership.svg b/images/breeze/output-fix-ownership.svg
index a35f337ea38e0..2c160e6575f2f 100644
--- a/images/breeze/output-fix-ownership.svg
+++ b/images/breeze/output-fix-ownership.svg
@@ -19,46 +19,46 @@
font-weight: 700;
}
- .terminal-1748745364-matrix {
+ .terminal-2814491796-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-1748745364-title {
+ .terminal-2814491796-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-1748745364-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1748745364-r2 { fill: #c5c8c6 }
-.terminal-1748745364-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1748745364-r4 { fill: #868887 }
-.terminal-1748745364-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1748745364-r6 { fill: #98a84b;font-weight: bold }
-.terminal-1748745364-r7 { fill: #8d7b39 }
+ .terminal-2814491796-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-2814491796-r2 { fill: #c5c8c6 }
+.terminal-2814491796-r3 { fill: #d0b344;font-weight: bold }
+.terminal-2814491796-r4 { fill: #868887 }
+.terminal-2814491796-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-2814491796-r6 { fill: #98a84b;font-weight: bold }
+.terminal-2814491796-r7 { fill: #8d7b39 }
- Command: fix-ownership
+ Command: fix-ownership
-
- Usage: breeze fix-ownership [OPTIONS]
-
- Fix ownership of source files to be same as host user.
-
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze fix-ownership [OPTIONS]
+
+ Fix ownership of source files to be same as host user.
+
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-free-space.svg b/images/breeze/output-free-space.svg
index d0dd2934fe5fe..3f07573795c25 100644
--- a/images/breeze/output-free-space.svg
+++ b/images/breeze/output-free-space.svg
@@ -19,46 +19,46 @@
font-weight: 700;
}
- .terminal-1041213903-matrix {
+ .terminal-1942333903-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-1041213903-title {
+ .terminal-1942333903-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-1041213903-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1041213903-r2 { fill: #c5c8c6 }
-.terminal-1041213903-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1041213903-r4 { fill: #868887 }
-.terminal-1041213903-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1041213903-r6 { fill: #98a84b;font-weight: bold }
-.terminal-1041213903-r7 { fill: #8d7b39 }
+ .terminal-1942333903-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-1942333903-r2 { fill: #c5c8c6 }
+.terminal-1942333903-r3 { fill: #d0b344;font-weight: bold }
+.terminal-1942333903-r4 { fill: #868887 }
+.terminal-1942333903-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-1942333903-r6 { fill: #98a84b;font-weight: bold }
+.terminal-1942333903-r7 { fill: #8d7b39 }
- Command: free-space
+ Command: free-space
-
- Usage: breeze free-space [OPTIONS]
-
- Free space for jobs run in CI.
-
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze free-space [OPTIONS]
+
+ Free space for jobs run in CI.
+
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-generate-constraints.svg b/images/breeze/output-generate-constraints.svg
index ac2afa05a16bb..42dd0ddd0b699 100644
--- a/images/breeze/output-generate-constraints.svg
+++ b/images/breeze/output-generate-constraints.svg
@@ -19,64 +19,64 @@
font-weight: 700;
}
- .terminal-2001992685-matrix {
+ .terminal-2130901997-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-2001992685-title {
+ .terminal-2130901997-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-2001992685-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-2001992685-r2 { fill: #c5c8c6 }
-.terminal-2001992685-r3 { fill: #d0b344;font-weight: bold }
-.terminal-2001992685-r4 { fill: #868887 }
-.terminal-2001992685-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-2001992685-r6 { fill: #98a84b;font-weight: bold }
-.terminal-2001992685-r7 { fill: #8d7b39 }
+ .terminal-2130901997-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-2130901997-r2 { fill: #c5c8c6 }
+.terminal-2130901997-r3 { fill: #d0b344;font-weight: bold }
+.terminal-2130901997-r4 { fill: #868887 }
+.terminal-2130901997-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-2130901997-r6 { fill: #98a84b;font-weight: bold }
+.terminal-2130901997-r7 { fill: #8d7b39 }
- Command: generate-constraints
+ Command: generate-constraints
-
- Usage: breeze generate-constraints [OPTIONS]
-
- Generates pinned constraint files with all extras from setup.py in parallel.
-
-╭─ Generate constraints flags ─────────────────────────────────────────────────────────────────────────────────────────╮
-│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
-│ --python -p Python major/minor version used in Airflow image for images. │
-│ (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --airflow-constraints-mode Mode of constraints for CI image building │
-│ (constraints-source-providers | constraints | constraints-no-providers) │
-│ [default: constraints-source-providers] │
-│ --debug Drop user in shell instead of running the command. Useful for debugging. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --run-in-parallel Run the operation in parallel on all or selected subset of Python versions. │
-│ --parallelism Maximum number of processes to use while running the operation in parallel. (INTEGER RANGE) │
-│ [default: 4; 1<=x<=8] │
-│ --python-versions Space separated list of python versions used for build with multiple versions. (TEXT) │
-│ [default: 3.7 3.8 3.9 3.10] │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze generate-constraints [OPTIONS]
+
+ Generates pinned constraint files with all extras from setup.py in parallel.
+
+╭─ Generate constraints flags ─────────────────────────────────────────────────────────────────────────────────────────╮
+│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
+│ --python -p Python major/minor version used in Airflow image for images. │
+│ (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --airflow-constraints-mode Mode of constraints for CI image building │
+│ (constraints-source-providers | constraints | constraints-no-providers) │
+│ [default: constraints-source-providers] │
+│ --debug Drop user in shell instead of running the command. Useful for debugging. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --run-in-parallel Run the operation in parallel on all or selected subset of Python versions. │
+│ --parallelism Maximum number of processes to use while running the operation in parallel. (INTEGER RANGE) │
+│ [default: 4; 1<=x<=8] │
+│ --python-versions Space separated list of python versions used for build with multiple versions. (TEXT) │
+│ [default: 3.7 3.8 3.9 3.10] │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-prepare-airflow-package.svg b/images/breeze/output-prepare-airflow-package.svg
index 2a3c147b511fb..91122a0ec2375 100644
--- a/images/breeze/output-prepare-airflow-package.svg
+++ b/images/breeze/output-prepare-airflow-package.svg
@@ -19,51 +19,51 @@
font-weight: 700;
}
- .terminal-747892512-matrix {
+ .terminal-2766663456-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-747892512-title {
+ .terminal-2766663456-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-747892512-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-747892512-r2 { fill: #c5c8c6 }
-.terminal-747892512-r3 { fill: #d0b344;font-weight: bold }
-.terminal-747892512-r4 { fill: #868887 }
-.terminal-747892512-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-747892512-r6 { fill: #8d7b39 }
-.terminal-747892512-r7 { fill: #98a84b;font-weight: bold }
+ .terminal-2766663456-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-2766663456-r2 { fill: #c5c8c6 }
+.terminal-2766663456-r3 { fill: #d0b344;font-weight: bold }
+.terminal-2766663456-r4 { fill: #868887 }
+.terminal-2766663456-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-2766663456-r6 { fill: #8d7b39 }
+.terminal-2766663456-r7 { fill: #98a84b;font-weight: bold }
- Command: prepare-airflow-package
+ Command: prepare-airflow-package
-
- Usage: breeze prepare-airflow-package [OPTIONS]
-
- Prepare sdist/whl package of Airflow.
-
-╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --package-format Format of packages. (wheel | sdist | both) [default: wheel] │
-│ --version-suffix-for-pypi Version suffix used for PyPI packages (alpha, beta, rc1, etc.). (TEXT) │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --debug Drop user in shell instead of running the command. Useful for debugging. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze prepare-airflow-package [OPTIONS]
+
+ Prepare sdist/whl package of Airflow.
+
+╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --package-format Format of packages. (wheel | sdist | both) [default: wheel] │
+│ --version-suffix-for-pypi Version suffix used for PyPI packages (alpha, beta, rc1, etc.). (TEXT) │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --debug Drop user in shell instead of running the command. Useful for debugging. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-prepare-provider-documentation.svg b/images/breeze/output-prepare-provider-documentation.svg
index 2e5369e5cece8..7682e1753735c 100644
--- a/images/breeze/output-prepare-provider-documentation.svg
+++ b/images/breeze/output-prepare-provider-documentation.svg
@@ -19,61 +19,61 @@
font-weight: 700;
}
- .terminal-1009257927-matrix {
+ .terminal-2632715719-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-1009257927-title {
+ .terminal-2632715719-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-1009257927-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1009257927-r2 { fill: #c5c8c6 }
-.terminal-1009257927-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1009257927-r4 { fill: #868887 }
-.terminal-1009257927-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1009257927-r6 { fill: #98a84b;font-weight: bold }
-.terminal-1009257927-r7 { fill: #8d7b39 }
+ .terminal-2632715719-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-2632715719-r2 { fill: #c5c8c6 }
+.terminal-2632715719-r3 { fill: #d0b344;font-weight: bold }
+.terminal-2632715719-r4 { fill: #868887 }
+.terminal-2632715719-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-2632715719-r6 { fill: #98a84b;font-weight: bold }
+.terminal-2632715719-r7 { fill: #8d7b39 }
- Command: prepare-provider-documentation
+ Command: prepare-provider-documentation
-
- Usage: breeze prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra |
- apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin |
- apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop |
- arangodb | asana | celery | cloudant | cncf.kubernetes | databricks |
- datadog | dbt.cloud | dingding | discord | docker | elasticsearch |
- exasol | facebook | ftp | github | google | grpc | hashicorp | http |
- imap | influxdb | jdbc | jenkins | jira | microsoft.azure |
- microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql |
- neo4j | odbc | openfaas | opsgenie | oracle | pagerduty | papermill |
- plexus | postgres | presto | qubole | redis | salesforce | samba |
- segment | sendgrid | sftp | singularity | slack | snowflake | sqlite |
- ssh | tableau | telegram | trino | vertica | yandex | zendesk]...
-
- Prepare CHANGELOG, README and COMMITS information for providers.
-
-╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮
-│ --debug Drop user in shell instead of running the command. Useful for debugging. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra |
+ apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin |
+ apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop |
+ arangodb | asana | celery | cloudant | cncf.kubernetes | databricks |
+ datadog | dbt.cloud | dingding | discord | docker | elasticsearch |
+ exasol | facebook | ftp | github | google | grpc | hashicorp | http |
+ imap | influxdb | jdbc | jenkins | jira | microsoft.azure |
+ microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql |
+ neo4j | odbc | openfaas | opsgenie | oracle | pagerduty | papermill |
+ plexus | postgres | presto | qubole | redis | salesforce | samba |
+ segment | sendgrid | sftp | singularity | slack | snowflake | sqlite |
+ ssh | tableau | telegram | trino | vertica | yandex | zendesk]...
+
+ Prepare CHANGELOG, README and COMMITS information for providers.
+
+╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮
+│ --debug Drop user in shell instead of running the command. Useful for debugging. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-prepare-provider-packages.svg b/images/breeze/output-prepare-provider-packages.svg
index ec2a04598f2c7..339185f1092f7 100644
--- a/images/breeze/output-prepare-provider-packages.svg
+++ b/images/breeze/output-prepare-provider-packages.svg
@@ -19,63 +19,63 @@
font-weight: 700;
}
- .terminal-1292600021-matrix {
+ .terminal-3636495061-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-1292600021-title {
+ .terminal-3636495061-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-1292600021-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1292600021-r2 { fill: #c5c8c6 }
-.terminal-1292600021-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1292600021-r4 { fill: #868887 }
-.terminal-1292600021-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1292600021-r6 { fill: #8d7b39 }
-.terminal-1292600021-r7 { fill: #98a84b;font-weight: bold }
+ .terminal-3636495061-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-3636495061-r2 { fill: #c5c8c6 }
+.terminal-3636495061-r3 { fill: #d0b344;font-weight: bold }
+.terminal-3636495061-r4 { fill: #868887 }
+.terminal-3636495061-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-3636495061-r6 { fill: #8d7b39 }
+.terminal-3636495061-r7 { fill: #98a84b;font-weight: bold }
- Command: prepare-provider-packages
+ Command: prepare-provider-packages
-
- Usage: breeze prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra |
- apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin |
- apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop |
- arangodb | asana | celery | cloudant | cncf.kubernetes | databricks | datadog
- | dbt.cloud | dingding | discord | docker | elasticsearch | exasol | facebook
- | ftp | github | google | grpc | hashicorp | http | imap | influxdb | jdbc |
- jenkins | jira | microsoft.azure | microsoft.mssql | microsoft.psrp |
- microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas | opsgenie | oracle
- | pagerduty | papermill | plexus | postgres | presto | qubole | redis |
- salesforce | samba | segment | sendgrid | sftp | singularity | slack |
- snowflake | sqlite | ssh | tableau | telegram | trino | vertica | yandex |
- zendesk]...
-
- Prepare sdist/whl packages of Airflow Providers.
-
-╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --package-format Format of packages. (wheel | sdist | both) [default: wheel] │
-│ --version-suffix-for-pypi Version suffix used for PyPI packages (alpha, beta, rc1, etc.). (TEXT) │
-│ --package-list-file Read list of packages from text file (one package per line) (FILENAME) │
-│ --debug Drop user in shell instead of running the command. Useful for debugging. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra |
+ apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin |
+ apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop |
+ arangodb | asana | celery | cloudant | cncf.kubernetes | databricks | datadog
+ | dbt.cloud | dingding | discord | docker | elasticsearch | exasol | facebook
+ | ftp | github | google | grpc | hashicorp | http | imap | influxdb | jdbc |
+ jenkins | jira | microsoft.azure | microsoft.mssql | microsoft.psrp |
+ microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas | opsgenie | oracle
+ | pagerduty | papermill | plexus | postgres | presto | qubole | redis |
+ salesforce | samba | segment | sendgrid | sftp | singularity | slack |
+ snowflake | sqlite | ssh | tableau | telegram | trino | vertica | yandex |
+ zendesk]...
+
+ Prepare sdist/whl packages of Airflow Providers.
+
+╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --package-format Format of packages. (wheel | sdist | both) [default: wheel] │
+│ --version-suffix-for-pypi Version suffix used for PyPI packages (alpha, beta, rc1, etc.). (TEXT) │
+│ --package-list-file Read list of packages from text file (one package per line) (FILENAME) │
+│ --debug Drop user in shell instead of running the command. Useful for debugging. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-pull-image.svg b/images/breeze/output-pull-image.svg
index 42cf6b6689f44..d713be1d5aa9a 100644
--- a/images/breeze/output-pull-image.svg
+++ b/images/breeze/output-pull-image.svg
@@ -19,63 +19,63 @@
font-weight: 700;
}
- .terminal-2196825906-matrix {
+ .terminal-3638880050-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-2196825906-title {
+ .terminal-3638880050-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-2196825906-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-2196825906-r2 { fill: #c5c8c6 }
-.terminal-2196825906-r3 { fill: #d0b344;font-weight: bold }
-.terminal-2196825906-r4 { fill: #868887 }
-.terminal-2196825906-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-2196825906-r6 { fill: #98a84b;font-weight: bold }
-.terminal-2196825906-r7 { fill: #8d7b39 }
+ .terminal-3638880050-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-3638880050-r2 { fill: #c5c8c6 }
+.terminal-3638880050-r3 { fill: #d0b344;font-weight: bold }
+.terminal-3638880050-r4 { fill: #868887 }
+.terminal-3638880050-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-3638880050-r6 { fill: #98a84b;font-weight: bold }
+.terminal-3638880050-r7 { fill: #8d7b39 }
- Command: pull-image
+ Command: pull-image
-
- Usage: breeze pull-image [OPTIONS] [EXTRA_PYTEST_ARGS]...
-
- Pull and optionally verify CI images - possibly in parallel for all Python versions.
-
-╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
-│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --github-token The token used to authenticate to GitHub. (TEXT) │
-│ --verify-image Verify image. │
-│ --wait-for-image Wait until image is available. │
-│ --tag-as-latest Tags the image as latest and update checksum of all files after pulling. Useful when you │
-│ build or pull image with --image-tag. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --run-in-parallel Run the operation in parallel on all or selected subset of Python versions. │
-│ --parallelism Maximum number of processes to use while running the operation in parallel. (INTEGER RANGE) │
-│ [default: 4; 1<=x<=8] │
-│ --python-versions Space separated list of python versions used for build with multiple versions. (TEXT) │
-│ [default: 3.7 3.8 3.9 3.10] │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze pull-image [OPTIONS] [EXTRA_PYTEST_ARGS]...
+
+ Pull and optionally verify CI images - possibly in parallel for all Python versions.
+
+╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
+│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --github-token The token used to authenticate to GitHub. (TEXT) │
+│ --verify-image Verify image. │
+│ --wait-for-image Wait until image is available. │
+│ --tag-as-latest Tags the image as latest and update checksum of all files after pulling. Useful when you build │
+│ or pull image with --image-tag. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --run-in-parallel Run the operation in parallel on all or selected subset of Python versions. │
+│ --parallelism Maximum number of processes to use while running the operation in parallel. (INTEGER RANGE) │
+│ [default: 4; 1<=x<=8] │
+│ --python-versions Space separated list of python versions used for build with multiple versions. (TEXT) │
+│ [default: 3.7 3.8 3.9 3.10] │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-pull-prod-image.svg b/images/breeze/output-pull-prod-image.svg
index 9a12580b3bea1..01a30c28721ff 100644
--- a/images/breeze/output-pull-prod-image.svg
+++ b/images/breeze/output-pull-prod-image.svg
@@ -19,63 +19,63 @@
font-weight: 700;
}
- .terminal-2898717937-matrix {
+ .terminal-46787825-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-2898717937-title {
+ .terminal-46787825-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-2898717937-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-2898717937-r2 { fill: #c5c8c6 }
-.terminal-2898717937-r3 { fill: #d0b344;font-weight: bold }
-.terminal-2898717937-r4 { fill: #868887 }
-.terminal-2898717937-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-2898717937-r6 { fill: #98a84b;font-weight: bold }
-.terminal-2898717937-r7 { fill: #8d7b39 }
+ .terminal-46787825-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-46787825-r2 { fill: #c5c8c6 }
+.terminal-46787825-r3 { fill: #d0b344;font-weight: bold }
+.terminal-46787825-r4 { fill: #868887 }
+.terminal-46787825-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-46787825-r6 { fill: #98a84b;font-weight: bold }
+.terminal-46787825-r7 { fill: #8d7b39 }
- Command: pull-prod-image
+ Command: pull-prod-image
-
- Usage: breeze pull-prod-image [OPTIONS] [EXTRA_PYTEST_ARGS]...
-
- Pull and optionally verify Production images - possibly in parallel for all Python versions.
-
-╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
-│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --github-token The token used to authenticate to GitHub. (TEXT) │
-│ --verify-image Verify image. │
-│ --wait-for-image Wait until image is available. │
-│ --tag-as-latest Tags the image as latest and update checksum of all files after pulling. Useful when you │
-│ build or pull image with --image-tag. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --run-in-parallel Run the operation in parallel on all or selected subset of Python versions. │
-│ --parallelism Maximum number of processes to use while running the operation in parallel. (INTEGER RANGE) │
-│ [default: 4; 1<=x<=8] │
-│ --python-versions Space separated list of python versions used for build with multiple versions. (TEXT) │
-│ [default: 3.7 3.8 3.9 3.10] │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze pull-prod-image [OPTIONS] [EXTRA_PYTEST_ARGS]...
+
+ Pull and optionally verify Production images - possibly in parallel for all Python versions.
+
+╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
+│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --github-token The token used to authenticate to GitHub. (TEXT) │
+│ --verify-image Verify image. │
+│ --wait-for-image Wait until image is available. │
+│ --tag-as-latest Tags the image as latest and update checksum of all files after pulling. Useful when you build │
+│ or pull image with --image-tag. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --run-in-parallel Run the operation in parallel on all or selected subset of Python versions. │
+│ --parallelism Maximum number of processes to use while running the operation in parallel. (INTEGER RANGE) │
+│ [default: 4; 1<=x<=8] │
+│ --python-versions Space separated list of python versions used for build with multiple versions. (TEXT) │
+│ [default: 3.7 3.8 3.9 3.10] │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-release-prod-images.svg b/images/breeze/output-release-prod-images.svg
index 31f4da7b4d847..79d78779f454b 100644
--- a/images/breeze/output-release-prod-images.svg
+++ b/images/breeze/output-release-prod-images.svg
@@ -1,4 +1,4 @@
-
+
- Command: release-prod-images
+ Command: release-prod-images
-
- Usage: breeze release-prod-images [OPTIONS]
-
- Release production images to DockerHub (needs DockerHub permissions).
-
-╭─ Release PROD IMAGE flags ───────────────────────────────────────────────────────────────────────────────────────────╮
-│ * --airflow-version Airflow version to release (2.3.0, 2.3.0rc1 etc.) (TEXT) [required] │
-│ --dockerhub-repo DockerHub repository for the images (TEXT) [default: apache/airflow] │
-│ --slim-images Whether to prepare slim images instead of the regular ones. │
-│ --limit-python Specific python to build slim images for (if not specified - the images are built for all │
-│ available python versions) │
-│ (3.7 | 3.8 | 3.9 | 3.10) │
-│ --limit-platform Specific platform to build images for (if not specified, multiplatform images will be │
-│ built. │
-│ (linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) │
-│ [default: linux/amd64,linux/arm64] │
-│ --skip-latest Whether to skip publishing the latest images (so that 'latest' images are not updated). │
-│ This should only be used if you release image for previous branches. Automatically set │
-│ when rc/alpha/beta images are built. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze release-prod-images [OPTIONS]
+
+ Release production images to DockerHub (needs DockerHub permissions).
+
+╭─ Release PROD IMAGE flags ───────────────────────────────────────────────────────────────────────────────────────────╮
+│ * --airflow-version Airflow version to release (2.3.0, 2.3.0rc1 etc.) (TEXT) [required] │
+│ --dockerhub-repo DockerHub repository for the images (TEXT) [default: apache/airflow] │
+│ --slim-images Whether to prepare slim images instead of the regular ones. │
+│ --limit-python Specific python to build slim images for (if not specified - the images are built for all │
+│ available python versions) │
+│ (3.7 | 3.8 | 3.9 | 3.10) │
+│ --limit-platform Specific platform to build images for (if not specified, multiplatform images will be built. │
+│ (linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) │
+│ [default: linux/amd64,linux/arm64] │
+│ --skip-latest Whether to skip publishing the latest images (so that 'latest' images are not updated). This │
+│ should only be used if you release image for previous branches. Automatically set when │
+│ rc/alpha/beta images are built. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-resource-check.svg b/images/breeze/output-resource-check.svg
index 310eaf9a1e707..e10fd1169b879 100644
--- a/images/breeze/output-resource-check.svg
+++ b/images/breeze/output-resource-check.svg
@@ -19,44 +19,44 @@
font-weight: 700;
}
- .terminal-1721143440-matrix {
+ .terminal-2379452560-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-1721143440-title {
+ .terminal-2379452560-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-1721143440-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1721143440-r2 { fill: #c5c8c6 }
-.terminal-1721143440-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1721143440-r4 { fill: #868887 }
-.terminal-1721143440-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1721143440-r6 { fill: #98a84b;font-weight: bold }
+ .terminal-2379452560-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-2379452560-r2 { fill: #c5c8c6 }
+.terminal-2379452560-r3 { fill: #d0b344;font-weight: bold }
+.terminal-2379452560-r4 { fill: #868887 }
+.terminal-2379452560-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-2379452560-r6 { fill: #98a84b;font-weight: bold }
- Command: resource-check
+ Command: resource-check
-
- Usage: breeze resource-check [OPTIONS]
-
- Check if available docker resources are enough.
-
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze resource-check [OPTIONS]
+
+ Check if available docker resources are enough.
+
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-self-upgrade.svg b/images/breeze/output-self-upgrade.svg
index 147e6581b2598..7fd44e00f7216 100644
--- a/images/breeze/output-self-upgrade.svg
+++ b/images/breeze/output-self-upgrade.svg
@@ -19,46 +19,46 @@
font-weight: 700;
}
- .terminal-3684385563-matrix {
+ .terminal-103170843-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-3684385563-title {
+ .terminal-103170843-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-3684385563-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-3684385563-r2 { fill: #c5c8c6 }
-.terminal-3684385563-r3 { fill: #d0b344;font-weight: bold }
-.terminal-3684385563-r4 { fill: #868887 }
-.terminal-3684385563-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-3684385563-r6 { fill: #98a84b;font-weight: bold }
+ .terminal-103170843-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-103170843-r2 { fill: #c5c8c6 }
+.terminal-103170843-r3 { fill: #d0b344;font-weight: bold }
+.terminal-103170843-r4 { fill: #868887 }
+.terminal-103170843-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-103170843-r6 { fill: #98a84b;font-weight: bold }
- Command: self-upgrade
+ Command: self-upgrade
-
- Usage: breeze self-upgrade [OPTIONS]
-
- Self upgrade Breeze.
-
-╭─ Self-upgrade flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --use-current-airflow-sources -a Use current workdir Airflow sources for upgrade. │
-│ --force -f Force upgrade without asking question to the user. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze self-upgrade [OPTIONS]
+
+ Self upgrade Breeze.
+
+╭─ Self-upgrade flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --use-current-airflow-sources -a Use current workdir Airflow sources for upgrade. │
+│ --force -f Force upgrade without asking question to the user. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-setup-autocomplete.svg b/images/breeze/output-setup-autocomplete.svg
index b07eca2b16701..4caa5a2793f8b 100644
--- a/images/breeze/output-setup-autocomplete.svg
+++ b/images/breeze/output-setup-autocomplete.svg
@@ -19,49 +19,49 @@
font-weight: 700;
}
- .terminal-3033303522-matrix {
+ .terminal-4279732706-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-3033303522-title {
+ .terminal-4279732706-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-3033303522-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-3033303522-r2 { fill: #c5c8c6 }
-.terminal-3033303522-r3 { fill: #d0b344;font-weight: bold }
-.terminal-3033303522-r4 { fill: #868887 }
-.terminal-3033303522-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-3033303522-r6 { fill: #98a84b;font-weight: bold }
-.terminal-3033303522-r7 { fill: #8d7b39 }
+ .terminal-4279732706-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-4279732706-r2 { fill: #c5c8c6 }
+.terminal-4279732706-r3 { fill: #d0b344;font-weight: bold }
+.terminal-4279732706-r4 { fill: #868887 }
+.terminal-4279732706-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-4279732706-r6 { fill: #98a84b;font-weight: bold }
+.terminal-4279732706-r7 { fill: #8d7b39 }
- Command: setup-autocomplete
+ Command: setup-autocomplete
-
- Usage: breeze setup-autocomplete [OPTIONS]
-
- Enables autocompletion of breeze commands.
-
-╭─ Setup autocomplete flags ───────────────────────────────────────────────────────────────────────────────────────────╮
-│ --force -f Force autocomplete setup even if already setup before (overrides the setup). │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --verbose -v Print verbose information about performed steps. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze setup-autocomplete [OPTIONS]
+
+ Enables autocompletion of breeze commands.
+
+╭─ Setup autocomplete flags ───────────────────────────────────────────────────────────────────────────────────────────╮
+│ --force -f Force autocomplete setup even if already setup before (overrides the setup). │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --verbose -v Print verbose information about performed steps. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-shell.svg b/images/breeze/output-shell.svg
index c4a63835bc624..c4bce85bda61e 100644
--- a/images/breeze/output-shell.svg
+++ b/images/breeze/output-shell.svg
@@ -1,4 +1,4 @@
-
+
- Command: shell
+ Command: shell
-
- Usage: breeze shell [OPTIONS] [EXTRA_ARGS]...
-
- Enter breeze.py environment. this is the default command use when no other is selected.
-
-╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --backend -b Database backend to use. (>sqlite< | mysql | postgres | mssql) [default: sqlite] │
-│ --postgres-version -P Version of Postgres used. (>10< | 11 | 12 | 13 | 14) [default: 10] │
-│ --mysql-version -M Version of MySQL used. (>5.7< | 8) [default: 5.7] │
-│ --mssql-version -S Version of MsSQL used. (>2017-latest< | 2019-latest) [default: 2017-latest] │
-│ --integration Integration(s) to enable when running (can be more than one). │
-│ (cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | │
-│ all) │
-│ --forward-credentials -f Forward local credentials to container when running. │
-│ --db-reset -d Reset DB when entering the container. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮
-│ --use-airflow-version Use (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-extras Airflow extras to install when --use-airflow-version is used (TEXT) │
-│ --use-packages-from-dist Install all found packages (--package-format determines type) from 'dist' folder when │
-│ entering breeze. │
-│ --package-format Format of packages that should be installed from dist. (wheel | sdist) │
-│ [default: wheel] │
-│ --force-build Force image build no matter if it is determined as needed. │
-│ --mount-sources Choose scope of local sources should be mounted (default = selected). │
-│ (selected | all | none) │
-│ [default: selected] │
-│ --debian-version Debian version used for the image. (bullseye | buster) [default: bullseye] │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) │
-│ [default: apache/airflow] │
-│ --airflow-constraints-reference Constraint reference to use. Useful with --use-airflow-version parameter to │
-│ specify constraints for the installed version and to find newer dependencies │
-│ (TEXT) │
-│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze shell [OPTIONS] [EXTRA_ARGS]...
+
+ Enter breeze.py environment. this is the default command use when no other is selected.
+
+╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --backend -b Database backend to use. (>sqlite< | mysql | postgres | mssql) [default: sqlite] │
+│ --postgres-version -P Version of Postgres used. (>10< | 11 | 12 | 13 | 14) [default: 10] │
+│ --mysql-version -M Version of MySQL used. (>5.7< | 8) [default: 5.7] │
+│ --mssql-version -S Version of MsSQL used. (>2017-latest< | 2019-latest) [default: 2017-latest] │
+│ --integration Integration(s) to enable when running (can be more than one). │
+│ (cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | │
+│ all) │
+│ --forward-credentials -f Forward local credentials to container when running. │
+│ --db-reset -d Reset DB when entering the container. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮
+│ --use-airflow-version Use (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-extras Airflow extras to install when --use-airflow-version is used (TEXT) │
+│ --use-packages-from-dist Install all found packages (--package-format determines type) from 'dist' folder when │
+│ entering breeze. │
+│ --package-format Format of packages that should be installed from dist. (wheel | sdist) [default: wheel] │
+│ --force-build Force image build no matter if it is determined as needed. │
+│ --mount-sources Choose scope of local sources should be mounted (default = selected). │
+│ (selected | all | none) │
+│ [default: selected] │
+│ --debian-version Debian version used for the image. (bullseye | buster) [default: bullseye] │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) │
+│ [default: apache/airflow] │
+│ --airflow-constraints-reference Constraint reference to use. Useful with --use-airflow-version parameter to │
+│ specify constraints for the installed version and to find newer dependencies │
+│ (TEXT) │
+│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-start-airflow.svg b/images/breeze/output-start-airflow.svg
index 6f305e940e2fe..8487e5ebebc4f 100644
--- a/images/breeze/output-start-airflow.svg
+++ b/images/breeze/output-start-airflow.svg
@@ -1,4 +1,4 @@
-
+
- Command: start-airflow
+ Command: start-airflow
-
- Usage: breeze start-airflow [OPTIONS] [EXTRA_ARGS]...
-
- Enter breeze.py environment and starts all Airflow components in the tmux session.
-
-╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --python -p Python major/minor version used in Airflow image for images. │
-│ (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --load-example-dags -e Enable configuration to load example DAGs when starting Airflow. │
-│ --load-default-connections -c Enable configuration to load default connections when starting Airflow. │
-│ --backend -b Database backend to use. (>sqlite< | mysql | postgres | mssql) [default: sqlite] │
-│ --postgres-version -P Version of Postgres used. (>10< | 11 | 12 | 13 | 14) [default: 10] │
-│ --mysql-version -M Version of MySQL used. (>5.7< | 8) [default: 5.7] │
-│ --mssql-version -S Version of MsSQL used. (>2017-latest< | 2019-latest) [default: 2017-latest] │
-│ --integration Integration(s) to enable when running (can be more than one). │
-│ (cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | │
-│ trino | all) │
-│ --forward-credentials -f Forward local credentials to container when running. │
-│ --db-reset -d Reset DB when entering the container. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮
-│ --use-airflow-version Use (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-extras Airflow extras to install when --use-airflow-version is used (TEXT) │
-│ --use-packages-from-dist Install all found packages (--package-format determines type) from 'dist' folder when │
-│ entering breeze. │
-│ --package-format Format of packages that should be installed from dist. (wheel | sdist) │
-│ [default: wheel] │
-│ --force-build Force image build no matter if it is determined as needed. │
-│ --mount-sources Choose scope of local sources should be mounted (default = selected). │
-│ (selected | all | none) │
-│ [default: selected] │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) │
-│ [default: apache/airflow] │
-│ --airflow-constraints-reference Constraint reference to use. Useful with --use-airflow-version parameter to │
-│ specify constraints for the installed version and to find newer dependencies │
-│ (TEXT) │
-│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
-│ --verbose -v Print verbose information about performed steps. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze start-airflow [OPTIONS] [EXTRA_ARGS]...
+
+ Enter breeze.py environment and starts all Airflow components in the tmux session.
+
+╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --python -p Python major/minor version used in Airflow image for images. │
+│ (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --load-example-dags -e Enable configuration to load example DAGs when starting Airflow. │
+│ --load-default-connections -c Enable configuration to load default connections when starting Airflow. │
+│ --backend -b Database backend to use. (>sqlite< | mysql | postgres | mssql) [default: sqlite] │
+│ --postgres-version -P Version of Postgres used. (>10< | 11 | 12 | 13 | 14) [default: 10] │
+│ --mysql-version -M Version of MySQL used. (>5.7< | 8) [default: 5.7] │
+│ --mssql-version -S Version of MsSQL used. (>2017-latest< | 2019-latest) [default: 2017-latest] │
+│ --integration Integration(s) to enable when running (can be more than one). │
+│ (cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino │
+│ | all) │
+│ --forward-credentials -f Forward local credentials to container when running. │
+│ --db-reset -d Reset DB when entering the container. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮
+│ --use-airflow-version Use (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-extras Airflow extras to install when --use-airflow-version is used (TEXT) │
+│ --use-packages-from-dist Install all found packages (--package-format determines type) from 'dist' folder when │
+│ entering breeze. │
+│ --package-format Format of packages that should be installed from dist. (wheel | sdist) [default: wheel] │
+│ --force-build Force image build no matter if it is determined as needed. │
+│ --mount-sources Choose scope of local sources should be mounted (default = selected). │
+│ (selected | all | none) │
+│ [default: selected] │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) │
+│ [default: apache/airflow] │
+│ --airflow-constraints-reference Constraint reference to use. Useful with --use-airflow-version parameter to │
+│ specify constraints for the installed version and to find newer dependencies │
+│ (TEXT) │
+│ --answer -a Force answer to questions. (y | n | q | yes | no | quit) │
+│ --verbose -v Print verbose information about performed steps. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-static-checks.svg b/images/breeze/output-static-checks.svg
index e76f1a3f27ea3..73ad94e12df2d 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 -t Type(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 -f List of files to run the checks on. (PATH) │
-│ --all-files -a Run checks on all files. │
-│ --show-diff-on-failure -s Show diff for files modified by the checks. │
-│ --last-commit -c Run checks for all files in last commit. Mutually exclusive with --commit-ref. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --commit-ref -r Run checks for this commit reference only (can be any git commit-ish reference). Mutually │
-│ exclusive with --last-commit. │
-│ (TEXT) │
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]...
+
+ Run static checks.
+
+╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --type -t Type(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-partial-arguments | │
+│ 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 -f List of files to run the checks on. (PATH) │
+│ --all-files -a Run checks on all files. │
+│ --show-diff-on-failure -s Show diff for files modified by the checks. │
+│ --last-commit -c Run checks for all files in last commit. Mutually exclusive with --commit-ref. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --commit-ref -r Run checks for this commit reference only (can be any git commit-ish reference). Mutually │
+│ exclusive with --last-commit. │
+│ (TEXT) │
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-stop.svg b/images/breeze/output-stop.svg
index 6f5a9ca2d86d7..40f24b393dd3d 100644
--- a/images/breeze/output-stop.svg
+++ b/images/breeze/output-stop.svg
@@ -19,47 +19,47 @@
font-weight: 700;
}
- .terminal-640511514-matrix {
+ .terminal-1568632346-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-640511514-title {
+ .terminal-1568632346-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-640511514-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-640511514-r2 { fill: #c5c8c6 }
-.terminal-640511514-r3 { fill: #d0b344;font-weight: bold }
-.terminal-640511514-r4 { fill: #868887 }
-.terminal-640511514-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-640511514-r6 { fill: #98a84b;font-weight: bold }
+ .terminal-1568632346-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-1568632346-r2 { fill: #c5c8c6 }
+.terminal-1568632346-r3 { fill: #d0b344;font-weight: bold }
+.terminal-1568632346-r4 { fill: #868887 }
+.terminal-1568632346-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-1568632346-r6 { fill: #98a84b;font-weight: bold }
- Command: stop
+ Command: stop
-
- Usage: breeze stop [OPTIONS]
-
- Stop running breeze environment.
-
-╭─ Stop flags ─────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --preserve-volumes -p Skip removing volumes when stopping Breeze. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze stop [OPTIONS]
+
+ Stop running breeze environment.
+
+╭─ Stop flags ─────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --preserve-volumes -p Skip removing volumes when stopping Breeze. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-tests.svg b/images/breeze/output-tests.svg
index 0b344da888747..3a3d3ca450a75 100644
--- a/images/breeze/output-tests.svg
+++ b/images/breeze/output-tests.svg
@@ -19,53 +19,53 @@
font-weight: 700;
}
- .terminal-2156331930-matrix {
+ .terminal-4193911706-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-2156331930-title {
+ .terminal-4193911706-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-2156331930-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-2156331930-r2 { fill: #c5c8c6 }
-.terminal-2156331930-r3 { fill: #d0b344;font-weight: bold }
-.terminal-2156331930-r4 { fill: #868887 }
-.terminal-2156331930-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-2156331930-r6 { fill: #8d7b39 }
-.terminal-2156331930-r7 { fill: #98a84b;font-weight: bold }
+ .terminal-4193911706-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-4193911706-r2 { fill: #c5c8c6 }
+.terminal-4193911706-r3 { fill: #d0b344;font-weight: bold }
+.terminal-4193911706-r4 { fill: #868887 }
+.terminal-4193911706-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-4193911706-r6 { fill: #8d7b39 }
+.terminal-4193911706-r7 { fill: #98a84b;font-weight: bold }
- 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 ───────────────────────────────────────────────────────────────────────────────────────╮
-│ --integration Integration(s) to enable when running (can be more than one). │
-│ (cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | all) │
-│ --test-type -tt Type of test to run. │
-│ (All | Always | Core | Providers | API | CLI | Integration | Other | WWW | Postgres | MySQL | │
-│ Helm | Quarantined) │
-│ --db-reset -d Reset DB when entering the container. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --verbose -v Print verbose information about performed steps. │
-│ --help -h Show 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 ───────────────────────────────────────────────────────────────────────────────────────╮
+│ --integration Integration(s) to enable when running (can be more than one). │
+│ (cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | all) │
+│ --test-type -tt Type of test to run. │
+│ (All | Always | Core | Providers | API | CLI | Integration | Other | WWW | Postgres | MySQL | │
+│ Helm | Quarantined) │
+│ --db-reset -d Reset DB when entering the container. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --verbose -v Print verbose information about performed steps. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-verify-image.svg b/images/breeze/output-verify-image.svg
index e35e11819f8ed..ef652a60eeb7d 100644
--- a/images/breeze/output-verify-image.svg
+++ b/images/breeze/output-verify-image.svg
@@ -19,52 +19,52 @@
font-weight: 700;
}
- .terminal-2093741156-matrix {
+ .terminal-4166710372-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-2093741156-title {
+ .terminal-4166710372-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-2093741156-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-2093741156-r2 { fill: #c5c8c6 }
-.terminal-2093741156-r3 { fill: #d0b344;font-weight: bold }
-.terminal-2093741156-r4 { fill: #868887 }
-.terminal-2093741156-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-2093741156-r6 { fill: #98a84b;font-weight: bold }
-.terminal-2093741156-r7 { fill: #8d7b39 }
+ .terminal-4166710372-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-4166710372-r2 { fill: #c5c8c6 }
+.terminal-4166710372-r3 { fill: #d0b344;font-weight: bold }
+.terminal-4166710372-r4 { fill: #868887 }
+.terminal-4166710372-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-4166710372-r6 { fill: #98a84b;font-weight: bold }
+.terminal-4166710372-r7 { fill: #8d7b39 }
- Command: verify-image
+ Command: verify-image
-
- Usage: breeze verify-image [OPTIONS] [EXTRA_PYTEST_ARGS]...
-
- Verify CI image.
-
-╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --image-name -n Name of the image to verify (overrides --python and --image-tag). (TEXT) │
-│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze verify-image [OPTIONS] [EXTRA_PYTEST_ARGS]...
+
+ Verify CI image.
+
+╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --image-name -n Name of the image to verify (overrides --python and --image-tag). (TEXT) │
+│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-verify-prod-image.svg b/images/breeze/output-verify-prod-image.svg
index 3dab7852ead6a..46432f50968ce 100644
--- a/images/breeze/output-verify-prod-image.svg
+++ b/images/breeze/output-verify-prod-image.svg
@@ -19,52 +19,52 @@
font-weight: 700;
}
- .terminal-439155235-matrix {
+ .terminal-2512124451-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-439155235-title {
+ .terminal-2512124451-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-439155235-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-439155235-r2 { fill: #c5c8c6 }
-.terminal-439155235-r3 { fill: #d0b344;font-weight: bold }
-.terminal-439155235-r4 { fill: #868887 }
-.terminal-439155235-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-439155235-r6 { fill: #98a84b;font-weight: bold }
-.terminal-439155235-r7 { fill: #8d7b39 }
+ .terminal-2512124451-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-2512124451-r2 { fill: #c5c8c6 }
+.terminal-2512124451-r3 { fill: #d0b344;font-weight: bold }
+.terminal-2512124451-r4 { fill: #868887 }
+.terminal-2512124451-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-2512124451-r6 { fill: #98a84b;font-weight: bold }
+.terminal-2512124451-r7 { fill: #8d7b39 }
- Command: verify-prod-image
+ Command: verify-prod-image
-
- Usage: breeze verify-prod-image [OPTIONS] [EXTRA_PYTEST_ARGS]...
-
- Verify Production image.
-
-╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --image-name -n Name of the image to verify (overrides --python and --image-tag). (TEXT) │
-│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
-│ [default: 3.7] │
-│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze verify-prod-image [OPTIONS] [EXTRA_PYTEST_ARGS]...
+
+ Verify Production image.
+
+╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --image-name -n Name of the image to verify (overrides --python and --image-tag). (TEXT) │
+│ --python -p Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10) │
+│ [default: 3.7] │
+│ --image-tag -t Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT) │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-verify-provider-packages.svg b/images/breeze/output-verify-provider-packages.svg
index 061622d35349f..59a2dd397e321 100644
--- a/images/breeze/output-verify-provider-packages.svg
+++ b/images/breeze/output-verify-provider-packages.svg
@@ -19,62 +19,62 @@
font-weight: 700;
}
- .terminal-3092598158-matrix {
+ .terminal-1322667406-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-3092598158-title {
+ .terminal-1322667406-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-3092598158-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-3092598158-r2 { fill: #c5c8c6 }
-.terminal-3092598158-r3 { fill: #d0b344;font-weight: bold }
-.terminal-3092598158-r4 { fill: #868887 }
-.terminal-3092598158-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-3092598158-r6 { fill: #8d7b39 }
-.terminal-3092598158-r7 { fill: #98a84b;font-weight: bold }
+ .terminal-1322667406-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-1322667406-r2 { fill: #c5c8c6 }
+.terminal-1322667406-r3 { fill: #d0b344;font-weight: bold }
+.terminal-1322667406-r4 { fill: #868887 }
+.terminal-1322667406-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-1322667406-r6 { fill: #8d7b39 }
+.terminal-1322667406-r7 { fill: #98a84b;font-weight: bold }
- Command: verify-provider-packages
+ Command: verify-provider-packages
-
- Usage: breeze verify-provider-packages [OPTIONS]
-
- Verifies if all provider code is following expectations for providers.
-
-╭─ Provider verification flags ────────────────────────────────────────────────────────────────────────────────────────╮
-│ --use-airflow-version Use (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-constraints-reference Constraint reference to use. Useful with --use-airflow-version parameter to │
-│ specify constraints for the installed version and to find newer dependencies │
-│ (TEXT) │
-│ --airflow-extras Airflow extras to install when --use-airflow-version is used (TEXT) │
-│ --use-packages-from-dist Install all found packages (--package-format determines type) from 'dist' │
-│ folder when entering breeze. │
-│ --package-format Format of packages that should be installed from dist. (wheel | sdist) │
-│ [default: wheel] │
-│ --debug Drop user in shell instead of running the command. Useful for debugging. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
-│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze verify-provider-packages [OPTIONS]
+
+ Verifies if all provider code is following expectations for providers.
+
+╭─ Provider verification flags ────────────────────────────────────────────────────────────────────────────────────────╮
+│ --use-airflow-version Use (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-constraints-reference Constraint reference to use. Useful with --use-airflow-version parameter to │
+│ specify constraints for the installed version and to find newer dependencies │
+│ (TEXT) │
+│ --airflow-extras Airflow extras to install when --use-airflow-version is used (TEXT) │
+│ --use-packages-from-dist Install all found packages (--package-format determines type) from 'dist' folder │
+│ when entering breeze. │
+│ --package-format Format of packages that should be installed from dist. (wheel | sdist) │
+│ [default: wheel] │
+│ --debug Drop user in shell instead of running the command. Useful for debugging. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --dry-run -D If dry-run is set, commands are only printed, not executed. │
+│ --github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow] │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/images/breeze/output-version.svg b/images/breeze/output-version.svg
index d136f095946c2..6636182300ccc 100644
--- a/images/breeze/output-version.svg
+++ b/images/breeze/output-version.svg
@@ -19,43 +19,43 @@
font-weight: 700;
}
- .terminal-589851807-matrix {
+ .terminal-978414751-matrix {
font-family: Fira Code, monospace;
font-size: 20px;
line-height: 26.400000000000002px;
font-variant-east-asian: full-width;
}
- .terminal-589851807-title {
+ .terminal-978414751-title {
font-size: 18px;
font-weight: bold;
font-family: arial;
}
- .terminal-589851807-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-589851807-r2 { fill: #c5c8c6 }
-.terminal-589851807-r3 { fill: #d0b344;font-weight: bold }
-.terminal-589851807-r4 { fill: #868887 }
-.terminal-589851807-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-589851807-r6 { fill: #98a84b;font-weight: bold }
+ .terminal-978414751-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-978414751-r2 { fill: #c5c8c6 }
+.terminal-978414751-r3 { fill: #d0b344;font-weight: bold }
+.terminal-978414751-r4 { fill: #868887 }
+.terminal-978414751-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-978414751-r6 { fill: #98a84b;font-weight: bold }
- Command: version
+ Command: version
-
- Usage: breeze version [OPTIONS]
-
- Print information about version of apache-airflow-breeze.
-
-╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-│ --verbose -v Print verbose information about performed steps. │
-│ --help -h Show this message and exit. │
-╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+ Usage: breeze version [OPTIONS]
+
+ Print information about version of apache-airflow-breeze.
+
+╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+│ --verbose -v Print verbose information about performed steps. │
+│ --help -h Show this message and exit. │
+╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
diff --git a/scripts/ci/pre_commit/pre_commit_base_operator_partial_arguments.py b/scripts/ci/pre_commit/pre_commit_base_operator_partial_arguments.py
new file mode 100755
index 0000000000000..8af31ce17c17b
--- /dev/null
+++ b/scripts/ci/pre_commit/pre_commit_base_operator_partial_arguments.py
@@ -0,0 +1,164 @@
+#!/usr/bin/env python3
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+import ast
+import itertools
+import pathlib
+import sys
+import typing
+
+ROOT_DIR = pathlib.Path(__file__).resolve().parents[3]
+
+BASEOPERATOR_PY = ROOT_DIR.joinpath("airflow", "models", "baseoperator.py")
+MAPPEDOPERATOR_PY = ROOT_DIR.joinpath("airflow", "models", "mappedoperator.py")
+
+IGNORED = {
+ # These are only used in the worker and thus mappable.
+ "do_xcom_push",
+ "email_on_failure",
+ "email_on_retry",
+ "post_execute",
+ "pre_execute",
+ # Doesn't matter, not used anywhere.
+ "default_args",
+ # Deprecated and is aliased to max_active_tis_per_dag.
+ "task_concurrency",
+ # attrs internals.
+ "HIDE_ATTRS_FROM_UI",
+ # Only on BaseOperator.
+ "_dag",
+ "mapped_arguments_validated_by_init",
+ "output",
+ "partial",
+ "shallow_copy_attrs",
+ # Only on MappedOperator.
+ "mapped_kwargs",
+ "partial_kwargs",
+}
+
+
+BO_MOD = ast.parse(BASEOPERATOR_PY.read_text("utf-8"), str(BASEOPERATOR_PY))
+MO_MOD = ast.parse(MAPPEDOPERATOR_PY.read_text("utf-8"), str(MAPPEDOPERATOR_PY))
+
+BO_CLS = next(
+ node
+ for node in ast.iter_child_nodes(BO_MOD)
+ if isinstance(node, ast.ClassDef) and node.name == "BaseOperator"
+)
+BO_INIT = next(
+ node
+ for node in ast.iter_child_nodes(BO_CLS)
+ if isinstance(node, ast.FunctionDef) and node.name == "__init__"
+)
+BO_PARTIAL = next(
+ node
+ for node in ast.iter_child_nodes(BO_MOD)
+ if isinstance(node, ast.FunctionDef) and node.name == "partial"
+)
+MO_CLS = next(
+ node
+ for node in ast.iter_child_nodes(MO_MOD)
+ if isinstance(node, ast.ClassDef) and node.name == "MappedOperator"
+)
+
+
+def _compare(a: set[str], b: set[str], *, excludes: set[str]) -> tuple[set[str], set[str]]:
+ only_in_a = {n for n in a if n not in b and n not in excludes and n[0] != "_"}
+ only_in_b = {n for n in b if n not in a and n not in excludes and n[0] != "_"}
+ return only_in_a, only_in_b
+
+
+def _iter_arg_names(func: ast.FunctionDef) -> typing.Iterator[str]:
+ func_args = func.args
+ for arg in itertools.chain(func_args.args, getattr(func_args, "posonlyargs", ()), func_args.kwonlyargs):
+ yield arg.arg
+
+
+def check_baseoperator_partial_arguments() -> bool:
+ only_in_init, only_in_partial = _compare(
+ set(itertools.islice(_iter_arg_names(BO_INIT), 1, None)),
+ set(itertools.islice(_iter_arg_names(BO_PARTIAL), 1, None)),
+ excludes=IGNORED,
+ )
+ if only_in_init:
+ print("Arguments in BaseOperator missing from partial():", ", ".join(sorted(only_in_init)))
+ if only_in_partial:
+ print("Arguments in partial() missing from BaseOperator:", ", ".join(sorted(only_in_partial)))
+ if only_in_init or only_in_partial:
+ return False
+ return True
+
+
+def _iter_assignment_to_self_attributes(targets: typing.Iterable[ast.expr]) -> typing.Iterator[str]:
+ for t in targets:
+ if isinstance(t, ast.Attribute) and isinstance(t.value, ast.Name) and t.value.id == "self":
+ yield t.attr # Something like "self.foo = ...".
+ continue
+ # Recursively visit nodes in unpacking assignments like "a, b = ...".
+ yield from _iter_assignment_to_self_attributes(getattr(t, "elts", ()))
+
+
+def _iter_assignment_targets(func: ast.FunctionDef) -> typing.Iterator[str]:
+ for stmt in func.body:
+ if isinstance(stmt, ast.AnnAssign):
+ yield from _iter_assignment_to_self_attributes([stmt.target])
+ elif isinstance(stmt, ast.Assign):
+ yield from _iter_assignment_to_self_attributes(stmt.targets)
+
+
+def _is_property(f: ast.FunctionDef) -> bool:
+ if len(f.decorator_list) != 1:
+ return False
+ decorator = f.decorator_list[0]
+ if not isinstance(decorator, ast.Name):
+ return False
+ return decorator.id == "property"
+
+
+def _iter_member_names(klass: ast.ClassDef) -> typing.Iterator[str]:
+ for node in ast.iter_child_nodes(klass):
+ if isinstance(node, ast.AnnAssign) and isinstance(node.target, ast.Name):
+ yield node.target.id
+ elif isinstance(node, ast.FunctionDef) and _is_property(node):
+ yield node.name
+
+
+def check_operator_member_parity() -> bool:
+ only_in_base, only_in_mapped = _compare(
+ set(itertools.chain(_iter_assignment_targets(BO_INIT), _iter_member_names(BO_CLS))),
+ set(_iter_member_names(MO_CLS)),
+ excludes=IGNORED,
+ )
+ if only_in_base:
+ print("Members on BaseOperator missing from MappedOperator:", ", ".join(sorted(only_in_base)))
+ if only_in_mapped:
+ print("Members on MappedOperator missing from BaseOperator:", ", ".join(sorted(only_in_mapped)))
+ if only_in_base or only_in_mapped:
+ return False
+ return True
+
+
+if __name__ == "__main__":
+ results = [
+ check_baseoperator_partial_arguments(),
+ check_operator_member_parity(),
+ ]
+ sys.exit(not all(results))