diff --git a/airflow/providers/amazon/aws/auth_manager/aws_auth_manager.py b/airflow/providers/amazon/aws/auth_manager/aws_auth_manager.py index 82cca57abb751..905a7e76be397 100644 --- a/airflow/providers/amazon/aws/auth_manager/aws_auth_manager.py +++ b/airflow/providers/amazon/aws/auth_manager/aws_auth_manager.py @@ -16,15 +16,20 @@ # under the License. from __future__ import annotations +import argparse from functools import cached_property from typing import TYPE_CHECKING from flask import session, url_for +from airflow.cli.cli_config import CLICommand, DefaultHelpParser, GroupCommand from airflow.configuration import conf from airflow.exceptions import AirflowOptionalProviderFeatureException from airflow.providers.amazon.aws.auth_manager.avp.entities import AvpEntities from airflow.providers.amazon.aws.auth_manager.avp.facade import AwsAuthManagerAmazonVerifiedPermissionsFacade +from airflow.providers.amazon.aws.auth_manager.cli.definition import ( + AWS_AUTH_MANAGER_COMMANDS, +) from airflow.providers.amazon.aws.auth_manager.constants import ( CONF_ENABLE_KEY, CONF_SECTION_NAME, @@ -195,3 +200,25 @@ def get_url_logout(self) -> str: @cached_property def security_manager(self) -> AwsSecurityManagerOverride: return AwsSecurityManagerOverride(self.appbuilder) + + @staticmethod + def get_cli_commands() -> list[CLICommand]: + """Vends CLI commands to be included in Airflow CLI.""" + return [ + GroupCommand( + name="aws-auth-manager", + help="Manage resources used by AWS auth manager", + subcommands=AWS_AUTH_MANAGER_COMMANDS, + ), + ] + + +def get_parser() -> argparse.ArgumentParser: + """Generate documentation; used by Sphinx argparse.""" + from airflow.cli.cli_parser import AirflowHelpFormatter, _add_command + + parser = DefaultHelpParser(prog="airflow", formatter_class=AirflowHelpFormatter) + subparsers = parser.add_subparsers(dest="subcommand", metavar="GROUP_OR_COMMAND") + for group_command in AwsAuthManager.get_cli_commands(): + _add_command(subparsers, group_command) + return parser diff --git a/airflow/providers/amazon/aws/auth_manager/cli/__init__.py b/airflow/providers/amazon/aws/auth_manager/cli/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/airflow/providers/amazon/aws/auth_manager/cli/__init__.py @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/providers/amazon/aws/auth_manager/cli/avp_commands.py b/airflow/providers/amazon/aws/auth_manager/cli/avp_commands.py new file mode 100644 index 0000000000000..0430f9e3b4585 --- /dev/null +++ b/airflow/providers/amazon/aws/auth_manager/cli/avp_commands.py @@ -0,0 +1,170 @@ +# 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. +"""User sub-commands.""" +from __future__ import annotations + +import json +import logging +from pathlib import Path +from typing import TYPE_CHECKING + +import boto3 + +from airflow.configuration import conf +from airflow.providers.amazon.aws.auth_manager.constants import CONF_REGION_NAME_KEY, CONF_SECTION_NAME +from airflow.utils import cli as cli_utils +from airflow.utils.providers_configuration_loader import providers_configuration_loaded + +if TYPE_CHECKING: + from botocore.client import BaseClient + +log = logging.getLogger(__name__) + + +@cli_utils.action_cli +@providers_configuration_loaded +def init_avp(args): + """Initialize Amazon Verified Permissions resources.""" + client = _get_client() + + # Create the policy store if needed + policy_store_id, is_new_policy_store = _create_policy_store(client, args) + + if not is_new_policy_store: + print( + f"Since an existing policy store with description '{args.policy_store_description}' has been found in Amazon Verified Permissions, " + "the CLI nade no changes to this policy store for security reasons. " + "Any modification to this policy store must be done manually.", + ) + else: + # Set the schema + _set_schema(client, policy_store_id, args) + + if not args.dry_run: + print("Amazon Verified Permissions resources created successfully.") + print("Please set them in Airflow configuration under AIRFLOW__AWS_AUTH_MANAGER__.") + print(json.dumps({"avp_policy_store_id": policy_store_id}, indent=4)) + + +@cli_utils.action_cli +@providers_configuration_loaded +def update_schema(args): + """Update Amazon Verified Permissions policy store schema.""" + client = _get_client() + _set_schema(client, args.policy_store_id, args) + + if not args.dry_run: + print("Amazon Verified Permissions policy store schema updated successfully.") + + +def _get_client(): + """Returns Amazon Verified Permissions client.""" + region_name = conf.get(CONF_SECTION_NAME, CONF_REGION_NAME_KEY) + return boto3.client("verifiedpermissions", region_name=region_name) + + +def _create_policy_store(client: BaseClient, args) -> tuple[str | None, bool]: + """ + Create if needed the policy store. + + This function returns two elements: + - the policy store ID + - whether the policy store ID returned refers to a newly created policy store. + """ + paginator = client.get_paginator("list_policy_stores") + pages = paginator.paginate() + policy_stores = [application for page in pages for application in page["policyStores"]] + existing_policy_stores = [ + policy_store + for policy_store in policy_stores + if policy_store.get("description") == args.policy_store_description + ] + + if args.verbose: + log.debug("Policy stores found: %s", policy_stores) + log.debug("Existing policy stores found: %s", existing_policy_stores) + + if len(existing_policy_stores) > 0: + print( + f"There is already a policy store with description '{args.policy_store_description}' in Amazon Verified Permissions: '{existing_policy_stores[0]['policyStoreId']}'." + ) + return existing_policy_stores[0]["policyStoreId"], False + else: + print(f"No policy store with description '{args.policy_store_description}' found, creating one.") + if args.dry_run: + print( + "Dry run, not creating the policy store with description '{args.policy_store_description}'." + ) + return None, True + + response = client.create_policy_store( + validationSettings={ + "mode": "OFF", + }, + description=args.policy_store_description, + ) + if args.verbose: + log.debug("Response from create_policy_store: %s", response) + + print(f"Policy store created: '{response['policyStoreId']}'") + + return response["policyStoreId"], True + + +def _set_schema(client: BaseClient, policy_store_id: str, args) -> None: + """Set the policy store schema.""" + if args.dry_run: + print(f"Dry run, not updating the schema of the policy store with ID '{policy_store_id}'.") + return + + if args.verbose: + log.debug("Disabling schema validation before updating schema") + + response = client.update_policy_store( + policyStoreId=policy_store_id, + validationSettings={ + "mode": "OFF", + }, + ) + + if args.verbose: + log.debug("Response from update_policy_store: %s", response) + + schema_path = Path(__file__).parents[0].joinpath("schema.json").resolve() + with open(schema_path) as schema_file: + response = client.put_schema( + policyStoreId=policy_store_id, + definition={ + "cedarJson": schema_file.read(), + }, + ) + + if args.verbose: + log.debug("Response from put_schema: %s", response) + + if args.verbose: + log.debug("Enabling schema validation after updating schema") + + response = client.update_policy_store( + policyStoreId=policy_store_id, + validationSettings={ + "mode": "STRICT", + }, + ) + + if args.verbose: + log.debug("Response from update_policy_store: %s", response) diff --git a/airflow/providers/amazon/aws/auth_manager/cli/definition.py b/airflow/providers/amazon/aws/auth_manager/cli/definition.py new file mode 100644 index 0000000000000..4355dcce9bbd3 --- /dev/null +++ b/airflow/providers/amazon/aws/auth_manager/cli/definition.py @@ -0,0 +1,62 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +from airflow.cli.cli_config import ( + ActionCommand, + Arg, + lazy_load_command, +) + +############ +# # ARGS # # +############ + +ARG_VERBOSE = Arg(("-v", "--verbose"), help="Make logging output more verbose", action="store_true") + +ARG_DRY_RUN = Arg( + ("--dry-run",), + help="Perform a dry run", + action="store_true", +) + +# Amazon Verified Permissions +ARG_POLICY_STORE_DESCRIPTION = Arg( + ("--policy-store-description",), help="Policy store description", default="Airflow" +) +ARG_POLICY_STORE_ID = Arg(("--policy-store-id",), help="Policy store ID") + + +################ +# # COMMANDS # # +################ + +AWS_AUTH_MANAGER_COMMANDS = ( + ActionCommand( + name="init-avp", + help="Initialize Amazon Verified resources to be used by AWS manager", + func=lazy_load_command("airflow.providers.amazon.aws.auth_manager.cli.avp_commands.init_avp"), + args=(ARG_POLICY_STORE_DESCRIPTION, ARG_DRY_RUN, ARG_VERBOSE), + ), + ActionCommand( + name="update-avp-schema", + help="Update Amazon Verified permissions policy store schema to the latest version in 'airflow/providers/amazon/aws/auth_manager/cli/schema.json'", + func=lazy_load_command("airflow.providers.amazon.aws.auth_manager.cli.avp_commands.update_schema"), + args=(ARG_POLICY_STORE_ID, ARG_DRY_RUN, ARG_VERBOSE), + ), +) diff --git a/airflow/providers/amazon/aws/auth_manager/cli/schema.json b/airflow/providers/amazon/aws/auth_manager/cli/schema.json new file mode 100644 index 0000000000000..e7d9b81d16446 --- /dev/null +++ b/airflow/providers/amazon/aws/auth_manager/cli/schema.json @@ -0,0 +1,171 @@ +{ + "Airflow": { + "actions": { + "Connection::DELETE": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Connection"] + } + }, + "Connection::GET": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Connection"] + } + }, + "Connection::POST": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Connection"] + } + }, + "Connection::PUT": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Connection"] + } + }, + "Configuration::GET": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Configuration"] + } + }, + "Dag::DELETE": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Dag"], + "context": { + "attributes": { + "dag_entity": { + "type": "String", + "required": false + } + }, + "type": "Record" + } + } + }, + "Dag::GET": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Dag"], + "context": { + "attributes": { + "dag_entity": { + "required": false, + "type": "String" + } + }, + "type": "Record" + } + } + }, + "Dag::POST": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Dag"], + "context": { + "attributes": { + "dag_entity": { + "required": false, + "type": "String" + } + }, + "type": "Record" + } + } + }, + "Dag::PUT": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Dag"], + "context": { + "attributes": { + "dag_entity": { + "required": false, + "type": "String" + } + }, + "type": "Record" + } + } + }, + "Dataset::GET": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Dataset"] + } + }, + "Pool::DELETE": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Pool"] + } + }, + "Pool::GET": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Pool"] + } + }, + "Pool::POST": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Pool"] + } + }, + "Pool::PUT": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Pool"] + } + }, + "Variable::DELETE": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Variable"] + } + }, + "Variable::GET": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Variable"] + } + }, + "Variable::POST": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Variable"] + } + }, + "Variable::PUT": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["Variable"] + } + }, + "View::GET": { + "appliesTo": { + "principalTypes": ["User"], + "resourceTypes": ["View"] + } + } + }, + "entityTypes": { + "Configuration": {}, + "Connection": {}, + "Dag": {}, + "Dataset": {}, + "Pool": {}, + "Role": {}, + "User": { + "memberOfTypes": [ + "Role" + ] + }, + "Variable": {}, + "View": {} + } + } +} diff --git a/docs/apache-airflow-providers-amazon/cli-ref.rst b/docs/apache-airflow-providers-amazon/cli-ref.rst new file mode 100644 index 0000000000000..0faf35d47a176 --- /dev/null +++ b/docs/apache-airflow-providers-amazon/cli-ref.rst @@ -0,0 +1,24 @@ + .. 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. + +Amazon CLI Commands +=================== + +.. argparse:: + :module: airflow.providers.amazon.aws.auth_manager.aws_auth_manager + :func: get_parser + :prog: airflow diff --git a/docs/apache-airflow-providers-amazon/index.rst b/docs/apache-airflow-providers-amazon/index.rst index 953357e1cc344..5034816ce81c6 100644 --- a/docs/apache-airflow-providers-amazon/index.rst +++ b/docs/apache-airflow-providers-amazon/index.rst @@ -42,6 +42,7 @@ Logging for Tasks Configuration Executors + CLI .. toctree:: :hidden: diff --git a/docs/apache-airflow/cli-and-env-variables-ref.rst b/docs/apache-airflow/cli-and-env-variables-ref.rst index d6c607a0c6889..b71379f4da463 100644 --- a/docs/apache-airflow/cli-and-env-variables-ref.rst +++ b/docs/apache-airflow/cli-and-env-variables-ref.rst @@ -39,6 +39,7 @@ contributed by the community providers: * 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` +* AWS and related CLI commands: :doc:`apache-airflow-providers-amazon:cli-ref` .. argparse:: :module: airflow.cli.cli_parser diff --git a/tests/providers/amazon/aws/auth_manager/cli/__init__.py b/tests/providers/amazon/aws/auth_manager/cli/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/providers/amazon/aws/auth_manager/cli/__init__.py @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/tests/providers/amazon/aws/auth_manager/cli/test_avp_commands.py b/tests/providers/amazon/aws/auth_manager/cli/test_avp_commands.py new file mode 100644 index 0000000000000..c65dae87e68fe --- /dev/null +++ b/tests/providers/amazon/aws/auth_manager/cli/test_avp_commands.py @@ -0,0 +1,175 @@ +# 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 importlib +from unittest.mock import ANY, Mock, patch + +import pytest + +from airflow.cli import cli_parser +from airflow.providers.amazon.aws.auth_manager.cli.avp_commands import init_avp, update_schema +from tests.test_utils.config import conf_vars + +mock_boto3 = Mock() + + +@pytest.mark.db_test +class TestAvpCommands: + def setup_method(self): + mock_boto3.reset_mock() + + @classmethod + def setup_class(cls): + with conf_vars( + { + ( + "core", + "auth_manager", + ): "airflow.providers.amazon.aws.auth_manager.aws_auth_manager.AwsAuthManager" + } + ): + importlib.reload(cli_parser) + cls.arg_parser = cli_parser.get_parser() + + @pytest.mark.parametrize( + "dry_run, verbose", + [ + (False, False), + (True, True), + ], + ) + @patch("airflow.providers.amazon.aws.auth_manager.cli.avp_commands._get_client") + def test_init_avp_with_no_existing_resources(self, mock_get_client, dry_run, verbose): + mock_get_client.return_value = mock_boto3 + + policy_store_description = "test-policy-store" + policy_store_id = "test-policy-store-id" + + paginator = Mock() + paginator.paginate.return_value = [] + + mock_boto3.get_paginator.return_value = paginator + mock_boto3.create_policy_store.return_value = {"policyStoreId": policy_store_id} + + with conf_vars({("database", "check_migrations"): "False"}): + params = [ + "aws-auth-manager", + "init-avp", + "--policy-store-description", + policy_store_description, + ] + if dry_run: + params.append("--dry-run") + if verbose: + params.append("--verbose") + init_avp(self.arg_parser.parse_args(params)) + + if dry_run: + mock_boto3.create_policy_store.assert_not_called() + mock_boto3.update_policy_store.assert_not_called() + mock_boto3.put_schema.assert_not_called() + else: + mock_boto3.create_policy_store.assert_called_once_with( + validationSettings={ + "mode": "OFF", + }, + description=policy_store_description, + ) + assert mock_boto3.update_policy_store.call_count == 2 + mock_boto3.put_schema.assert_called_once_with( + policyStoreId=policy_store_id, + definition={ + "cedarJson": ANY, + }, + ) + + @pytest.mark.parametrize( + "dry_run, verbose", + [ + (False, False), + (True, True), + ], + ) + @patch("airflow.providers.amazon.aws.auth_manager.cli.avp_commands._get_client") + def test_init_avp_with_existing_resources(self, mock_get_client, dry_run, verbose): + mock_get_client.return_value = mock_boto3 + + policy_store_description = "test-policy-store" + policy_store_id = "test-policy-store-id" + + paginator = Mock() + paginator.paginate.return_value = [ + {"policyStores": [{"description": policy_store_description, "policyStoreId": policy_store_id}]} + ] + + mock_boto3.get_paginator.return_value = paginator + + with conf_vars({("database", "check_migrations"): "False"}): + params = [ + "aws-auth-manager", + "init-avp", + "--policy-store-description", + policy_store_description, + ] + if dry_run: + params.append("--dry-run") + if verbose: + params.append("--verbose") + init_avp(self.arg_parser.parse_args(params)) + + mock_boto3.create_policy_store.assert_not_called() + mock_boto3.update_policy_store.assert_not_called() + mock_boto3.put_schema.assert_not_called() + + @pytest.mark.parametrize( + "dry_run, verbose", + [ + (False, False), + (True, True), + ], + ) + @patch("airflow.providers.amazon.aws.auth_manager.cli.avp_commands._get_client") + def test_update_schema(self, mock_get_client, dry_run, verbose): + mock_get_client.return_value = mock_boto3 + + policy_store_id = "test-policy-store-id" + + with conf_vars({("database", "check_migrations"): "False"}): + params = [ + "aws-auth-manager", + "update-avp-schema", + "--policy-store-id", + policy_store_id, + ] + if dry_run: + params.append("--dry-run") + if verbose: + params.append("--verbose") + update_schema(self.arg_parser.parse_args(params)) + + if dry_run: + mock_boto3.update_policy_store.assert_not_called() + mock_boto3.put_schema.assert_not_called() + else: + assert mock_boto3.update_policy_store.call_count == 2 + mock_boto3.put_schema.assert_called_once_with( + policyStoreId=policy_store_id, + definition={ + "cedarJson": ANY, + }, + ) diff --git a/tests/providers/amazon/aws/auth_manager/cli/test_definition.py b/tests/providers/amazon/aws/auth_manager/cli/test_definition.py new file mode 100644 index 0000000000000..079df886f6039 --- /dev/null +++ b/tests/providers/amazon/aws/auth_manager/cli/test_definition.py @@ -0,0 +1,24 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +from airflow.providers.amazon.aws.auth_manager.cli.definition import AWS_AUTH_MANAGER_COMMANDS + + +class TestAwsCliDefinition: + def test_aws_auth_manager_cli_commands(self): + assert len(AWS_AUTH_MANAGER_COMMANDS) == 2 diff --git a/tests/providers/amazon/aws/auth_manager/test_aws_auth_manager.py b/tests/providers/amazon/aws/auth_manager/test_aws_auth_manager.py index 051bdf4cfb762..4f31703d1c822 100644 --- a/tests/providers/amazon/aws/auth_manager/test_aws_auth_manager.py +++ b/tests/providers/amazon/aws/auth_manager/test_aws_auth_manager.py @@ -374,3 +374,6 @@ def test_get_url_logout(self, mock_url_for, auth_manager): @pytest.mark.db_test def test_security_manager_return_default_security_manager(self, auth_manager_with_appbuilder): assert isinstance(auth_manager_with_appbuilder.security_manager, AwsSecurityManagerOverride) + + def test_get_cli_commands_return_cli_commands(self, auth_manager): + assert len(auth_manager.get_cli_commands()) > 0