Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
31 commits
Select commit Hold shift + click to select a range
fe3f8fc
Initial plan
Copilot Dec 25, 2025
1060a30
Add CLI section to provider system - schema and core implementation
Copilot Dec 25, 2025
3da06a3
Add CLI functions for Keycloak and Amazon auth managers
Copilot Dec 25, 2025
31ce808
Simplify CLI schema and move functions to definition.py
Copilot Dec 25, 2025
7ac980c
Add CLI latency benchmark script
jason810496 Dec 25, 2025
2c89d47
Final refactor for ProviderManager
jason810496 Dec 25, 2025
0df9207
Add CLI command definitions to provider info for Amazon, Fab, and Key…
jason810496 Dec 25, 2025
f59437c
Refactor CLI commands for Celery executor and add provider info section
jason810496 Dec 25, 2025
1b79602
Refactor Kubernetes CLI integration and add provider info section
jason810496 Dec 25, 2025
2244be2
Add CLI commands for Edge Worker and integrate provider info retrieval
jason810496 Dec 25, 2025
4cc6614
Fix CLI command definitions for Celery and Kubernetes providers
jason810496 Dec 25, 2025
f885027
Fix static check
jason810496 Dec 26, 2025
5ebab54
Refactor CLI tests with ProviderManger instead of AuthManger +
jason810496 Dec 27, 2025
c69aa2e
Add unit tests for Celery, Kubernetes, and Edge CLI command definitions
jason810496 Dec 27, 2025
85fd59e
Fix k8s, edge compat test
jason810496 Dec 28, 2025
94d61cf
Move cli_commands.definition to cli.definition for FAB
jason810496 Dec 28, 2025
b2f7509
Add prek check to avoid import heavy module in cli.definition
jason810496 Dec 28, 2025
63f13cc
Remove auth_manager prefix for CLI definition for AWS, FAB, and Keycloak
jason810496 Dec 29, 2025
863fc65
Doc: Add CLI commands directive and template for provider-level CLI c…
jason810496 Dec 29, 2025
97fa538
Doc: Move generate doc get_parser to .cli.definition for each provider
jason810496 Dec 29, 2025
82a9ea1
Doc: mention provider-level CLI in airflow-core doc
jason810496 Dec 29, 2025
e6c0c3f
Doc: add CLI section to provider documentation and clarify CLI comman…
jason810496 Dec 29, 2025
c8fd960
Improve cli_parser speed by skipping _correctness_check for AuthManager
jason810496 Dec 29, 2025
dc780de
Fix mypy error and import for test
jason810496 Jan 2, 2026
5a8f9fd
Enhance CLI warnings for missing 'cli' sections in provider info for …
jason810496 Jan 2, 2026
a32fa3f
Fix tests
jason810496 Jan 3, 2026
fb2df07
Fix list has no add attribute error
jason810496 Jan 4, 2026
6ee5ef7
Refactor skip_cli_test function to simplify logic and remove unnecess…
jason810496 Jan 3, 2026
f2e28da
Finialize compatibility test
jason810496 Jan 5, 2026
44b88ae
Add test for ProviderManager change
jason810496 Jan 5, 2026
b6eb8d4
fixup! Remove unused __future__.annotations
jason810496 Jan 5, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion .pre-commit-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -615,7 +615,7 @@ repos:
^providers/google/src/airflow/providers/google/cloud/operators/cloud_build\.py$|
^providers/google/src/airflow/providers/google/cloud/operators/dataproc\.py$|
^providers/google/src/airflow/providers/google/cloud/operators/mlengine\.py$|
^providers/keycloak/src/airflow/providers/keycloak/auth_manager/cli/definition.py|
^providers/keycloak/src/airflow/providers/keycloak/cli/definition.py|
^providers/microsoft/azure/docs/connections/azure_cosmos\.rst$|
^providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/cosmos\.py$|
^providers/microsoft/winrm/src/airflow/providers/microsoft/winrm/hooks/winrm\.py$|
Expand Down
5 changes: 5 additions & 0 deletions airflow-core/docs/cli-and-env-variables-ref.rst
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,11 @@ development and testing.
Providers that implement executors might contribute additional commands to the CLI. Here are the commands
contributed by the community providers:


.. important::
Starting in Airflow ``3.2.0``, provider-level CLI commands are available to manage core extensions such as auth managers and executors. Implementing provider-level CLI commands can reduce CLI startup time by avoiding heavy imports when they are not required.
See :doc:`provider-level CLI <apache-airflow-providers:core-extensions/cli-commands>` for implementation guidance.

* Celery Executor and related CLI commands: :doc:`apache-airflow-providers-celery:cli-ref`
* Kubernetes Executor and related CLI commands: :doc:`apache-airflow-providers-cncf-kubernetes:cli-ref`
* Edge Executor and related CLI commands: :doc:`apache-airflow-providers-edge3:cli-ref`
Expand Down
4 changes: 4 additions & 0 deletions airflow-core/docs/core-concepts/auth-manager/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -209,6 +209,10 @@ The following methods aren't required to override to have a functional Airflow a
CLI
^^^

.. important::
Starting in Airflow ``3.2.0``, provider-level CLI commands are available to manage core extensions such as auth managers and executors. Implementing provider-level CLI commands can reduce CLI startup time by avoiding heavy imports when they are not required.
See :doc:`provider-level CLI <apache-airflow-providers:core-extensions/cli-commands>` for implementation guidance.

Auth managers may vend CLI commands which will be included in the ``airflow`` command line tool by implementing the ``get_cli_commands`` method. The commands can be used to setup required resources. Commands are only vended for the currently configured auth manager. A pseudo-code example of implementing CLI command vending from an auth manager can be seen below:

.. code-block:: python
Expand Down
4 changes: 4 additions & 0 deletions airflow-core/docs/core-concepts/executor/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -317,6 +317,10 @@ The ``BaseExecutor`` class interface contains a set of attributes that Airflow c
CLI
^^^

.. important::
Starting in Airflow ``3.2.0``, provider-level CLI commands are available to manage core extensions such as auth managers and executors. Implementing provider-level CLI commands can reduce CLI startup time by avoiding heavy imports when they are not required.
See :doc:`provider-level CLI <apache-airflow-providers:core-extensions/cli-commands>` for implementation guidance.

Executors may vend CLI commands which will be included in the ``airflow`` command line tool by implementing the ``get_cli_commands`` method. Executors such as ``CeleryExecutor`` and ``KubernetesExecutor`` for example, make use of this mechanism. The commands can be used to setup required workers, initialize environment or set other configuration. Commands are only vended for the currently configured executor. A pseudo-code example of implementing CLI command vending from an executor can be seen below:

.. code-block:: python
Expand Down
135 changes: 110 additions & 25 deletions airflow-core/src/airflow/cli/cli_parser.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@

import argparse
import logging
import sys
import os
from argparse import Action
from collections import Counter
from collections.abc import Iterable
Expand All @@ -36,7 +36,7 @@
import lazy_object_proxy
from rich_argparse import RawTextRichHelpFormatter, RichHelpFormatter

from airflow.api_fastapi.app import get_auth_manager_cls
from airflow._shared.module_loading import import_string
from airflow.cli.cli_config import (
DAG_CLI_DICT,
ActionCommand,
Expand All @@ -46,7 +46,7 @@
)
from airflow.cli.utils import CliConflictError
from airflow.exceptions import AirflowException
from airflow.executors.executor_loader import ExecutorLoader
from airflow.providers_manager import ProvidersManager
from airflow.utils.helpers import partition

if TYPE_CHECKING:
Expand All @@ -59,32 +59,117 @@

log = logging.getLogger(__name__)

# AIRFLOW_PACKAGE_NAME is set when generating docs and we don't want to load provider commands when generating airflow-core CLI docs
if not os.environ.get("AIRFLOW_PACKAGE_NAME", None):
providers_manager = ProvidersManager()
# Load CLI commands from providers
try:
for cli_function in providers_manager.cli_command_functions:
try:
airflow_commands.extend(cli_function())
except Exception:
log.exception("Failed to load CLI commands from provider function: %s", cli_function.__name__)
log.error("Ensure all dependencies are met and try again.")
# Do not re-raise the exception since we want the CLI to still function for
# other commands.
except Exception as e:
log.warning("Failed to load CLI commands from providers: %s", e)
# do not re-raise for the same reason as above

WARNING_TEMPLATE = """
Please define the 'cli' section in the 'get_provider_info' for custom {component} to avoid this warning.
For community providers, please update to the version that support 'cli' section.
For more details, see https://airflow.apache.org/docs/apache-airflow-providers/core-extensions/cli-commands.html

Providers with {component} missing 'cli' section in 'get_provider_info': {not_defined_cli_dict}
"""

for executor_name in ExecutorLoader.get_executor_names(validate_teams=False):
# compat loading for older providers that define get_cli_commands methods on Executors
try:
executor, _ = ExecutorLoader.import_executor_cls(executor_name)
airflow_commands.extend(executor.get_cli_commands())
except Exception:
log.exception("Failed to load CLI commands from executor: %s", executor_name)
log.error(
"Ensure all dependencies are met and try again. If using a Celery based executor install "
"a 3.3.0+ version of the Celery provider. If using a Kubernetes executor, install a "
"7.4.0+ version of the CNCF provider"
# if there is any executor_provider not in cli_provider, we have to do compat loading
# we use without check to avoid actual loading in this check
executors_not_defined_cli = {
executor_name: executor_provider
for executor_name, executor_provider in providers_manager.executor_without_check
if executor_provider not in providers_manager.cli_command_providers
}
if executors_not_defined_cli:
log.warning(
WARNING_TEMPLATE.format(
component="executors", not_defined_cli_dict=str(executors_not_defined_cli)
)
)
from airflow.executors.executor_loader import ExecutorLoader

for executor_name in ExecutorLoader.get_executor_names(validate_teams=False):
# Skip if the executor already has CLI commands defined via the 'cli' section in provider.yaml
if executor_name.module_path not in executors_not_defined_cli:
log.debug(
"Skipping loading for '%s' as it is defined in 'cli' section.",
executor_name.module_path,
)
continue

try:
executor, _ = ExecutorLoader.import_executor_cls(executor_name)
airflow_commands.extend(executor.get_cli_commands())
except Exception:
log.exception("Failed to load CLI commands from executor: %s", executor_name)
log.error(
"Ensure all dependencies are met and try again. If using a Celery based executor install "
"a 3.3.0+ version of the Celery provider. If using a Kubernetes executor, install a "
"7.4.0+ version of the CNCF provider"
)
# Do not re-raise the exception since we want the CLI to still function for
# other commands.

except Exception as e:
log.warning(
"Failed to load CLI commands from executors that didn't define `get_cli_commands` in `.cli.definition`: %s",
e,
)
# Do not re-raise the exception since we want the CLI to still function for
# other commands.

try:
auth_mgr = get_auth_manager_cls()
airflow_commands.extend(auth_mgr.get_cli_commands())
except Exception as e:
log.warning("cannot load CLI commands from auth manager: %s", e)
log.warning("Auth manager is not configured and api-server will not be able to start.")
# do not re-raise for the same reason as above
if len(sys.argv) > 1 and sys.argv[1] == "api-server":
log.exception(e)
sys.exit(1)
# compat loading for older providers that define get_cli_commands methods on AuthManagers
try:
# if there is any auth_manager not in cli_provider, we have to do compat loading
# we use without check to avoid actual loading in this check
auth_managers_not_defined_cli = {
auth_manager_name: auth_manager_provider
for auth_manager_name, auth_manager_provider in providers_manager.auth_manager_without_check
if auth_manager_provider not in providers_manager.cli_command_providers
}
if auth_managers_not_defined_cli:
log.warning(
WARNING_TEMPLATE.format(
component="auth manager", not_defined_cli_dict=str(auth_managers_not_defined_cli)
)
)

from airflow.configuration import conf
from airflow.exceptions import AirflowConfigException

auth_manager_cls_path = conf.get(section="core", key="auth_manager")

if not auth_manager_cls_path:
raise AirflowConfigException(
"No auth manager defined in the config. Please specify one using section/key [core/auth_manager]."
)

if auth_manager_cls_path in auth_managers_not_defined_cli:
try:
auth_manager_cls = import_string(auth_manager_cls_path)
auth_manager = auth_manager_cls()
airflow_commands.extend(auth_manager.get_cli_commands())
except Exception:
log.exception("Failed to load CLI commands from auth manager: %s", auth_manager_cls)
log.error("Ensure all dependencies are met and try again.")
# Do not re-raise the exception since we want the CLI to still function for
# other commands.
except Exception as e:
log.warning(
"Failed to load CLI commands from auth managers that didn't define `get_cli_commands` in `.cli.definition`: %s",
e,
)

ALL_COMMANDS_DICT: dict[str, CLICommand] = {sp.name: sp for sp in airflow_commands}

Expand All @@ -94,7 +179,7 @@
dup = {k for k, v in Counter([c.name for c in airflow_commands]).items() if v > 1}
raise CliConflictError(
f"The following CLI {len(dup)} command(s) are defined more than once: {sorted(dup)}\n"
f"This can be due to an Executor or Auth Manager redefining core airflow CLI commands."
f"This can be due to a Provider redefining core airflow CLI commands."
)


Expand Down
7 changes: 7 additions & 0 deletions airflow-core/src/airflow/provider.yaml.schema.json
Original file line number Diff line number Diff line change
Expand Up @@ -419,6 +419,13 @@
"type": "string"
}
},
"cli": {
"type": "array",
"description": "CLI command functions exposed by the provider",
"items": {
"type": "string"
}
},
"config": {
"type": "object",
"additionalProperties": {
Expand Down
7 changes: 7 additions & 0 deletions airflow-core/src/airflow/provider_info.schema.json
Original file line number Diff line number Diff line change
Expand Up @@ -355,6 +355,13 @@
"type": "string"
}
},
"cli": {
"type": "array",
"description": "CLI command functions exposed by the provider",
"items": {
"type": "string"
}
},
"config": {
"type": "object",
"additionalProperties": {
Expand Down
Loading