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 @@ - + - Breeze commands + Breeze commands -                                                                                                                          - Usage: breeze [OPTIONS] COMMAND [ARGS]...                                                                               -                                                                                                                         -╭─ Basic flags for the default (shell) command ────────────────────────────────────────────────────────────────────────╮ -  --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 flags for the default (shell) command ─────────────────────────────────────────────────────────────────────╮ -  --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]           -  --answer             -a  Force answer to questions. (y | n | q | yes | no | quit)                                    -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Developer tools ────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  shell          Enter breeze.py environment. this is the default command use when no other is selected.               -  start-airflow  Enter breeze.py environment and starts all Airflow components in the tmux session.                    -  exec           Joins the interactive shell of running airflow container                                              -  stop           Stop running breeze environment.                                                                      -  build-docs     Build documentation in the container.                                                                 -  static-checks  Run static checks.                                                                                    -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Testing ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  docker-compose-tests  Run docker-compose tests.                                                                      -  tests                 Run the specified unit test targets. Multiple targets may be specified separated by spaces.    -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Configuration & maintenance ────────────────────────────────────────────────────────────────────────────────────────╮ -  cleanup              Cleans the cache of parameters, docker cache and optionally - currently downloaded images.      -  self-upgrade         Self upgrade Breeze.                                                                            -  setup-autocomplete   Enables autocompletion of breeze commands.                                                      -  config               Show/update configuration (Python, Backend, Cheatsheet, ASCIIART).                              -  resource-check       Check if available docker resources are enough.                                                 -  free-space           Free space for jobs run in CI.                                                                  -  fix-ownership        Fix ownership of source files to be same as host user.                                          -  command-hash-export  Outputs hash of all click commands to file or stdout if `-` is used (useful to see if images    -                       should be regenerated).                                                                         -  version              Print information about version of apache-airflow-breeze.                                       -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ CI Image tools ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  build-image   Build CI image. Include building multiple images for all python versions (sequentially).               -  pull-image    Pull and optionally verify CI images - possibly in parallel for all Python versions.                   -  verify-image  Verify CI image.                                                                                       -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Production Image tools ─────────────────────────────────────────────────────────────────────────────────────────────╮ -  build-prod-image   Build Production image. Include building multiple images for all or selected Python versions      -                     sequentially.                                                                                     -  pull-prod-image    Pull and optionally verify Production images - possibly in parallel for all Python versions.      -  verify-prod-image  Verify Production image.                                                                          -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Release management ─────────────────────────────────────────────────────────────────────────────────────────────────╮ -  verify-provider-packages        Verifies if all provider code is following expectations for providers.               -  prepare-provider-documentation  Prepare CHANGELOG, README and COMMITS information for providers.                     -  prepare-provider-packages       Prepare sdist/whl packages of Airflow Providers.                                     -  prepare-airflow-package         Prepare sdist/whl package of Airflow.                                                -  release-prod-images             Release production images to DockerHub (needs DockerHub permissions).                -  generate-constraints            Generates pinned constraint files with all extras from setup.py in parallel.         -  find-newer-dependencies         Finds which dependencies are being upgraded.                                         -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +                                                                                                                          + Usage: breeze [OPTIONS] COMMAND [ARGS]...                                                                               +                                                                                                                         +╭─ Basic flags for the default (shell) command ────────────────────────────────────────────────────────────────────────╮ + --python               -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 flags for the default (shell) command ─────────────────────────────────────────────────────────────────────╮ + --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]            + --answer             -a  Force answer to questions. (y | n | q | yes | no | quit)                                     + --help               -h  Show this message and exit.                                                                  +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Developer tools ────────────────────────────────────────────────────────────────────────────────────────────────────╮ + shell            Enter breeze.py environment. this is the default command use when no other is selected.              + start-airflow    Enter breeze.py environment and starts all Airflow components in the tmux session.                   + exec             Joins the interactive shell of running airflow container                                             + stop             Stop running breeze environment.                                                                     + build-docs       Build documentation in the container.                                                                + static-checks    Run static checks.                                                                                   +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Testing ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ + docker-compose-tests   Run docker-compose tests.                                                                      + tests                  Run the specified unit test targets. Multiple targets may be specified separated by spaces.    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Configuration & maintenance ────────────────────────────────────────────────────────────────────────────────────────╮ + cleanup              Cleans the cache of parameters, docker cache and optionally - currently downloaded images.       + self-upgrade         Self upgrade Breeze.                                                                             + setup-autocomplete   Enables autocompletion of breeze commands.                                                       + config               Show/update configuration (Python, Backend, Cheatsheet, ASCIIART).                               + resource-check       Check if available docker resources are enough.                                                  + free-space           Free space for jobs run in CI.                                                                   + fix-ownership        Fix ownership of source files to be same as host user.                                           + command-hash-export  Outputs hash of all click commands to file or stdout if `-` is used (useful to see if images     +                      should be regenerated).                                                                          + version              Print information about version of apache-airflow-breeze.                                        +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ CI Image tools ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ + build-image     Build CI image. Include building multiple images for all python versions (sequentially).              + pull-image      Pull and optionally verify CI images - possibly in parallel for all Python versions.                  + verify-image    Verify CI image.                                                                                      +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Production Image tools ─────────────────────────────────────────────────────────────────────────────────────────────╮ + build-prod-image   Build Production image. Include building multiple images for all or selected Python versions       +                    sequentially.                                                                                      + pull-prod-image    Pull and optionally verify Production images - possibly in parallel for all Python versions.       + verify-prod-image  Verify Production image.                                                                           +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Release management ─────────────────────────────────────────────────────────────────────────────────────────────────╮ + verify-provider-packages           Verifies if all provider code is following expectations for providers.             + prepare-provider-documentation     Prepare CHANGELOG, README and COMMITS information for providers.                   + prepare-provider-packages          Prepare sdist/whl packages of Airflow Providers.                                   + prepare-airflow-package            Prepare sdist/whl package of Airflow.                                              + release-prod-images                Release production images to DockerHub (needs DockerHub permissions).              + generate-constraints               Generates pinned constraint files with all extras from setup.py in parallel.       + find-newer-dependencies            Finds which dependencies are being upgraded.                                       +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output-config.svg b/images/breeze/output-config.svg index f3c2e8c23e987..fbbaa5fcc6b03 100644 --- a/images/breeze/output-config.svg +++ b/images/breeze/output-config.svg @@ -19,56 +19,56 @@ font-weight: 700; } - .terminal-1927308537-matrix { + .terminal-119432441-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 26.400000000000002px; font-variant-east-asian: full-width; } - .terminal-1927308537-title { + .terminal-119432441-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-1927308537-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-1927308537-r2 { fill: #c5c8c6 } -.terminal-1927308537-r3 { fill: #d0b344;font-weight: bold } -.terminal-1927308537-r4 { fill: #868887 } -.terminal-1927308537-r5 { fill: #68a0b3;font-weight: bold } -.terminal-1927308537-r6 { fill: #98a84b;font-weight: bold } -.terminal-1927308537-r7 { fill: #8d7b39 } + .terminal-119432441-r1 { fill: #c5c8c6;font-weight: bold } +.terminal-119432441-r2 { fill: #c5c8c6 } +.terminal-119432441-r3 { fill: #d0b344;font-weight: bold } +.terminal-119432441-r4 { fill: #868887 } +.terminal-119432441-r5 { fill: #68a0b3;font-weight: bold } +.terminal-119432441-r6 { fill: #98a84b;font-weight: bold } +.terminal-119432441-r7 { fill: #8d7b39 } - Command: config + Command: config -                                                                                                                          - Usage: breeze config [OPTIONS]                                                                                          -                                                                                                                         - Show/update configuration (Python, Backend, Cheatsheet, ASCIIART).                                                      -                                                                                                                         -╭─ Config 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]                                                               -  --cheatsheet/--no-cheatsheet  -C/-c  Enable/disable cheatsheet.                                                      -  --asciiart/--no-asciiart      -A/-a  Enable/disable ASCIIart.                                                        -  --colour/--no-colour          -B/-b  Enable/disable Colour mode (useful for colour blind-friendly communication).    -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --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]                  -  --help              -h  Show this message and exit.                                                                  -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +                                                                                                                          + Usage: breeze config [OPTIONS]                                                                                          +                                                                                                                         + Show/update configuration (Python, Backend, Cheatsheet, ASCIIART).                                                      +                                                                                                                         +╭─ Config 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]                                                                + --cheatsheet/--no-cheatsheet  -C/-c  Enable/disable cheatsheet.                                                       + --asciiart/--no-asciiart      -A/-a  Enable/disable ASCIIart.                                                         + --colour/--no-colour          -B/-b  Enable/disable Colour mode (useful for colour blind-friendly communication).     +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ + --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]                   + --help              -h  Show this message and exit.                                                                   +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ 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))