Skip to content

Commit

Permalink
Create CLI commands for AWS auth manager to create AWS Identity Cente…
Browse files Browse the repository at this point in the history
…r related resources (#37407)
  • Loading branch information
vincbeck committed Feb 14, 2024
1 parent 0640e6d commit 56c27f8
Show file tree
Hide file tree
Showing 5 changed files with 299 additions and 3 deletions.
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/auth_manager/cli/avp_commands.py
Expand Up @@ -55,7 +55,7 @@ def init_avp(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. "
"the CLI made no changes to this policy store for security reasons. "
"Any modification to this policy store must be done manually.",
)
else:
Expand Down Expand Up @@ -115,7 +115,7 @@ def _create_policy_store(client: BaseClient, args) -> tuple[str | None, bool]:
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}'."
f"Dry run, not creating the policy store with description '{args.policy_store_description}'."
)
return None, True

Expand Down
14 changes: 14 additions & 0 deletions airflow/providers/amazon/aws/auth_manager/cli/definition.py
Expand Up @@ -35,6 +35,14 @@
action="store_true",
)

# AWS IAM Identity Center
ARG_INSTANCE_NAME = Arg(("--instance-name",), help="Instance name in Identity Center", default="Airflow")

ARG_APPLICATION_NAME = Arg(
("--application-name",), help="Application name in Identity Center", default="Airflow"
)


# Amazon Verified Permissions
ARG_POLICY_STORE_DESCRIPTION = Arg(
("--policy-store-description",), help="Policy store description", default="Airflow"
Expand All @@ -47,6 +55,12 @@
################

AWS_AUTH_MANAGER_COMMANDS = (
ActionCommand(
name="init-identity-center",
help="Initialize AWS IAM identity Center resources to be used by AWS manager",
func=lazy_load_command("airflow.providers.amazon.aws.auth_manager.cli.idc_commands.init_idc"),
args=(ARG_INSTANCE_NAME, ARG_APPLICATION_NAME, ARG_DRY_RUN, ARG_VERBOSE),
),
ActionCommand(
name="init-avp",
help="Initialize Amazon Verified resources to be used by AWS manager",
Expand Down
148 changes: 148 additions & 0 deletions airflow/providers/amazon/aws/auth_manager/cli/idc_commands.py
@@ -0,0 +1,148 @@
# 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 logging
from typing import TYPE_CHECKING

import boto3
from botocore.exceptions import ClientError

from airflow.configuration import conf
from airflow.exceptions import AirflowOptionalProviderFeatureException
from airflow.providers.amazon.aws.auth_manager.constants import CONF_REGION_NAME_KEY, CONF_SECTION_NAME
from airflow.utils import cli as cli_utils

try:
from airflow.utils.providers_configuration_loader import providers_configuration_loaded
except ImportError:
raise AirflowOptionalProviderFeatureException(
"Failed to import avp_commands. This feature is only available in Airflow "
"version >= 2.8.0 where Auth Managers are introduced."
)

if TYPE_CHECKING:
from botocore.client import BaseClient

log = logging.getLogger(__name__)


@cli_utils.action_cli
@providers_configuration_loaded
def init_idc(args):
"""Initialize AWS IAM Identity Center resources."""
client = _get_client()

# Create the instance if needed
instance_arn = _create_instance(client, args)

# Create the application if needed
_create_application(client, instance_arn, args)

if not args.dry_run:
print("AWS IAM Identity Center resources created successfully.")


def _get_client():
"""Return AWS IAM Identity Center client."""
region_name = conf.get(CONF_SECTION_NAME, CONF_REGION_NAME_KEY)
return boto3.client("sso-admin", region_name=region_name)


def _create_instance(client: BaseClient, args) -> str | None:
"""Create if needed AWS IAM Identity Center instance."""
instances = client.list_instances()

if args.verbose:
log.debug("Instances found: %s", instances)

if len(instances["Instances"]) > 0:
print(
f"There is already an instance configured in AWS IAM Identity Center: '{instances['Instances'][0]['InstanceArn']}'. "
"No need to create a new one."
)
return instances["Instances"][0]["InstanceArn"]
else:
print("No instance configured in AWS IAM Identity Center, creating one.")
if args.dry_run:
print("Dry run, not creating the instance.")
return None

response = client.create_instance(Name=args.instance_name)
if args.verbose:
log.debug("Response from create_instance: %s", response)

print(f"Instance created: '{response['InstanceArn']}'")

return response["InstanceArn"]


def _create_application(client: BaseClient, instance_arn: str | None, args) -> str | None:
"""Create if needed AWS IAM identity Center application."""
paginator = client.get_paginator("list_applications")
pages = paginator.paginate(InstanceArn=instance_arn or "")
applications = [application for page in pages for application in page["Applications"]]
existing_applications = [
application for application in applications if application["Name"] == args.application_name
]

if args.verbose:
log.debug("Applications found: %s", applications)
log.debug("Existing applications found: %s", existing_applications)

if len(existing_applications) > 0:
print(
f"There is already an application named '{args.application_name}' in AWS IAM Identity Center: '{existing_applications[0]['ApplicationArn']}'. "
"Using this application."
)
return existing_applications[0]["ApplicationArn"]
else:
print(f"No application named {args.application_name} found, creating one.")
if args.dry_run:
print("Dry run, not creating the application.")
return None

try:
response = client.create_application(
ApplicationProviderArn="arn:aws:sso::aws:applicationProvider/custom-saml",
Description="Application automatically created through the Airflow CLI. This application is used to access Airflow environment.",
InstanceArn=instance_arn,
Name=args.application_name,
PortalOptions={
"SignInOptions": {
"Origin": "IDENTITY_CENTER",
},
"Visibility": "ENABLED",
},
Status="ENABLED",
)
if args.verbose:
log.debug("Response from create_application: %s", response)
except ClientError as e:
# This is needed because as of today, the create_application in AWS Identity Center does not support SAML application
# Remove this part when it is supported
if "is not supported for this action" in e.response["Error"]["Message"]:
print(
"Creation of SAML applications is only supported in AWS console today. "
"Please create the application through the console."
)
raise

print(f"Application created: '{response['ApplicationArn']}'")

return response["ApplicationArn"]
Expand Up @@ -21,4 +21,4 @@

class TestAwsCliDefinition:
def test_aws_auth_manager_cli_commands(self):
assert len(AWS_AUTH_MANAGER_COMMANDS) == 2
assert len(AWS_AUTH_MANAGER_COMMANDS) == 3
134 changes: 134 additions & 0 deletions tests/providers/amazon/aws/auth_manager/cli/test_idc_commands.py
@@ -0,0 +1,134 @@
# 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 Mock, patch

import pytest

from airflow.cli import cli_parser
from airflow.providers.amazon.aws.auth_manager.cli.idc_commands import init_idc
from tests.test_utils.config import conf_vars

mock_boto3 = Mock()


@pytest.mark.db_test
class TestIdcCommands:
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.idc_commands._get_client")
def test_init_idc_with_no_existing_resources(self, mock_get_client, dry_run, verbose):
mock_get_client.return_value = mock_boto3

instance_name = "test-instance"
instance_arn = "test-instance-arn"
application_name = "test-application"
application_arn = "test-application-arn"

paginator = Mock()
paginator.paginate.return_value = []

mock_boto3.list_instances.return_value = {"Instances": []}
mock_boto3.create_instance.return_value = {"InstanceArn": instance_arn}
mock_boto3.get_paginator.return_value = paginator
mock_boto3.create_application.return_value = {"ApplicationArn": application_arn}

with conf_vars({("database", "check_migrations"): "False"}):
params = [
"aws-auth-manager",
"init-identity-center",
"--instance-name",
instance_name,
"--application-name",
application_name,
]
if dry_run:
params.append("--dry-run")
if verbose:
params.append("--verbose")
init_idc(self.arg_parser.parse_args(params))

mock_boto3.list_instances.assert_called_once_with()
if not dry_run:
mock_boto3.create_instance.assert_called_once_with(Name=instance_name)
mock_boto3.create_application.assert_called_once()

@pytest.mark.parametrize(
"dry_run, verbose",
[
(False, False),
(True, True),
],
)
@patch("airflow.providers.amazon.aws.auth_manager.cli.idc_commands._get_client")
def test_init_idc_with_existing_resources(self, mock_get_client, dry_run, verbose):
mock_get_client.return_value = mock_boto3

instance_name = "test-instance"
instance_arn = "test-instance-arn"
application_name = "test-application"
application_arn = "test-application-arn"

paginator = Mock()
paginator.paginate.return_value = [
{"Applications": [{"Name": application_name, "ApplicationArn": application_arn}]}
]

mock_boto3.list_instances.return_value = {"Instances": [{"InstanceArn": instance_arn}]}
mock_boto3.get_paginator.return_value = paginator

with conf_vars({("database", "check_migrations"): "False"}):
params = [
"aws-auth-manager",
"init-identity-center",
"--instance-name",
instance_name,
"--application-name",
application_name,
]
if dry_run:
params.append("--dry-run")
if verbose:
params.append("--verbose")
init_idc(self.arg_parser.parse_args(params))

mock_boto3.list_instances.assert_called_once_with()
mock_boto3.create_instance.assert_not_called()
mock_boto3.create_application.assert_not_called()

0 comments on commit 56c27f8

Please sign in to comment.