From c44c93ccea19c27b1a13b52b6af5e59a357258ec Mon Sep 17 00:00:00 2001 From: Serina Grill <42048900+serinamarie@users.noreply.github.com> Date: Thu, 14 Dec 2023 17:42:58 -0600 Subject: [PATCH] Allow Azure Container Instance resource name customization in interactive push pool CLI (#11412) Co-authored-by: Alexander Streed Co-authored-by: Alexander Streed --- .../provisioners/container_instance.py | 215 +++-- .../provisioners/test_container_instance.py | 741 +++++++++++++++++- 2 files changed, 887 insertions(+), 69 deletions(-) diff --git a/src/prefect/infrastructure/provisioners/container_instance.py b/src/prefect/infrastructure/provisioners/container_instance.py index 6aa3ea851146..6cc24b9306c3 100644 --- a/src/prefect/infrastructure/provisioners/container_instance.py +++ b/src/prefect/infrastructure/provisioners/container_instance.py @@ -28,7 +28,7 @@ from rich.prompt import Confirm from rich.syntax import Syntax -from prefect.cli._prompts import prompt_select_from_table +from prefect.cli._prompts import prompt, prompt_select_from_table from prefect.client.orchestration import PrefectClient from prefect.client.schemas.actions import BlockDocumentCreate from prefect.client.utilities import inject_client @@ -132,7 +132,6 @@ class ContainerInstancePushProvisioner: _subscription_name (str): Azure subscription name. _resource_group (str): Azure resource group name. _location (str): Azure resource location. - _container_image (str): Docker image for the container instance. azure_cli (AzureCLI): An instance of AzureCLI for running Azure commands. Methods: @@ -144,24 +143,21 @@ class ContainerInstancePushProvisioner: _generate_secret_for_app: Generates a secret for the app registration. _get_service_principal_object_id: Retrieves the object ID of the service principal associated with the app registration. _assign_contributor_role: Assigns the Contributor role to the service account. - _create_container_instance: Creates an Azure Container Instance. _create_aci_credentials_block: Creates an Azure Container Instance credentials block. provision: Orchestrates the provisioning of Azure resources and setup for the push work pool. """ - DEFAULT_LOCATION = "eastus" - RESOURCE_GROUP_NAME = "prefect-aci-push-pool-rg" - CONTAINER_IMAGE = "docker.io/prefecthq/prefect:2-latest" - APP_REGISTRATION_NAME = "prefect-aci-push-pool-app" - REGISTRY_NAME_PREFIX = "prefect" - def __init__(self): self._console = Console() self._subscription_id = None self._subscription_name = None - self._location = None + self._location = "eastus" self._identity_name = "prefect-acr-identity" self.azure_cli = AzureCLI(self.console) + self._credentials_block_name = None + self._resource_group_name = "prefect-aci-push-pool-rg" + self._app_registration_name = "prefect-aci-push-pool-app" + self._registry_name_prefix = "prefect" @property def console(self) -> Console: @@ -183,7 +179,8 @@ async def set_location(self): 'az account list-locations --query "[?isDefault].name" --output tsv' ) output = await self.azure_cli.run_command(command) - self._location = output if output else self.DEFAULT_LOCATION + if output: + self._location = output except subprocess.CalledProcessError as e: raise RuntimeError("Failed to get default location.") from e @@ -284,7 +281,7 @@ async def _create_resource_group(self): subprocess.CalledProcessError: If the Azure CLI command execution fails. """ check_exists_command = ( - f"az group exists --name {self.RESOURCE_GROUP_NAME} --subscription" + f"az group exists --name {self._resource_group_name} --subscription" f" {self._subscription_id}" ) exists_result = await self.azure_cli.run_command( @@ -293,7 +290,7 @@ async def _create_resource_group(self): if exists_result is True: self._console.print( ( - f"Resource group '{self.RESOURCE_GROUP_NAME}' already exists in" + f"Resource group '{self._resource_group_name}' already exists in" f" subscription {self._subscription_name}." ), style="yellow", @@ -301,16 +298,16 @@ async def _create_resource_group(self): return resource_group_command = ( - f"az group create --name '{self.RESOURCE_GROUP_NAME}' --location" + f"az group create --name '{self._resource_group_name}' --location" f" '{self._location}' --subscription '{self._subscription_id}'" ) await self.azure_cli.run_command( resource_group_command, success_message=( - f"Resource group '{self.RESOURCE_GROUP_NAME}' created successfully" + f"Resource group '{self._resource_group_name}' created successfully" ), failure_message=( - f"Failed to create resource group '{self.RESOURCE_GROUP_NAME}' in" + f"Failed to create resource group '{self._resource_group_name}' in" f" subscription '{self._subscription_name}'" ), ignore_if_exists=True, @@ -328,7 +325,7 @@ async def _create_app_registration(self) -> str: """ # Check if the app registration already exists check_exists_command = ( - f"az ad app list --display-name {self.APP_REGISTRATION_NAME} --output json" + f"az ad app list --display-name {self._app_registration_name} --output json" ) app_registrations = await self.azure_cli.run_command( check_exists_command, @@ -341,29 +338,29 @@ async def _create_app_registration(self) -> str: ( app for app in app_registrations - if app["displayName"] == self.APP_REGISTRATION_NAME + if app["displayName"] == self._app_registration_name ), None, ) if existing_app_registration: self._console.print( - f"App registration '{self.APP_REGISTRATION_NAME}' already exists.", + f"App registration '{self._app_registration_name}' already exists.", style="yellow", ) return existing_app_registration["appId"] app_registration_command = ( - f"az ad app create --display-name {self.APP_REGISTRATION_NAME} " + f"az ad app create --display-name {self._app_registration_name} " "--output json" ) app_registration = await self.azure_cli.run_command( app_registration_command, success_message=( - f"App registration '{self.APP_REGISTRATION_NAME}' created successfully" + f"App registration '{self._app_registration_name}' created successfully" ), failure_message=( "Failed to create app registration with name" - f" '{self.APP_REGISTRATION_NAME}'" + f" '{self._app_registration_name}'" ), ignore_if_exists=True, ) @@ -399,8 +396,8 @@ async def _generate_secret_for_app(self, app_id: str) -> tuple: failure_message=( "Failed to generate secret for app registration with client ID" f" '{app_id}'. If you have already generated 2 secrets for this app" - " registration, please delete one from the `prefect-aci-push-pool-app`" - " resource and try again." + " registration, please delete one from the" + f" `{self._app_registration_name}` resource and try again." ), ignore_if_exists=True, return_json=True, @@ -548,7 +545,7 @@ async def _get_or_create_registry( # check to see if there are any registries starting with 'prefect' command_get_registries = ( 'az acr list --query "[?starts_with(name,' - f" '{self.REGISTRY_NAME_PREFIX}')]\" --subscription" + f" '{self._registry_name_prefix}')]\" --subscription" f" {subscription_id} --output json" ) response = await self.azure_cli.run_command( @@ -560,8 +557,8 @@ async def _get_or_create_registry( if response: self._console.print( ( - f"Registry with prefix {self.REGISTRY_NAME_PREFIX!r} already exists" - f" in subscription '{subscription_id}'." + f"Registry with prefix {self._registry_name_prefix!r} already" + f" exists in subscription '{subscription_id}'." ), style="yellow", ) @@ -618,7 +615,7 @@ async def _assign_contributor_role(self, app_id: str, subscription_id: str) -> N if service_principal_id: role = "Contributor" - scope = f"/subscriptions/{self._subscription_id}/resourceGroups/{self.RESOURCE_GROUP_NAME}" + scope = f"/subscriptions/{self._subscription_id}/resourceGroups/{self._resource_group_name}" # Check if the role is already assigned check_role_command = ( @@ -629,7 +626,7 @@ async def _assign_contributor_role(self, app_id: str, subscription_id: str) -> N role_assignments = await self.azure_cli.run_command( check_role_command, return_json=True ) - if any( + if role_assignments and any( ra for ra in role_assignments if ra["roleDefinitionName"] == role and ra["scope"] == scope @@ -704,7 +701,7 @@ async def _create_aci_credentials_block( Raises: ObjectAlreadyExists: If a credentials block with the same name already exists. """ - credentials_block_name = f"{work_pool_name}-push-pool-credentials" + credentials_block_name = self._credentials_block_name credentials_block_type = await client.read_block_type_by_slug( "azure-container-instance-credentials" ) @@ -778,6 +775,84 @@ async def _aci_credentials_block_exists( except ObjectNotFound: return False + def _validate_user_input(self, name): + if 2 < len(name) < 40 and name.isalnum(): + return True + else: + return False + + async def _create_provision_table(self, work_pool_name: str, client: PrefectClient): + return Panel( + dedent( + f"""\ + Provisioning infrastructure for your work pool [blue]{work_pool_name}[/] will require: + + Updates in subscription: [blue]{self._subscription_name}[/] + + - Create a resource group in location: [blue]{self._location}[/] + - Create an app registration in Azure AD: [blue]{self._app_registration_name}[/] + - Create/use a service principal for app registration + - Generate a secret for app registration + - Create an Azure Container Registry with prefix [blue]{self._registry_name_prefix}[/] + - Create an identity [blue]{self._identity_name}[/] to allow access to the created registry + - Assign Contributor role to service account + - Create an ACR registry for image hosting + - Create an identity for Azure Container Instance to allow access to the registry + + Updates in Prefect workspace + + - Create Azure Container Instance credentials block: [blue]{self._credentials_block_name}[/] + """ + ), + expand=False, + ) + + async def _customize_resource_names( + self, work_pool_name: str, client: PrefectClient + ) -> bool: + self._resource_group_name = prompt( + "Please enter a name for the resource group", + default=self._resource_group_name, + ) + self._app_registration_name = prompt( + "Please enter a name for the app registration", + default=self._app_registration_name, + ) + while True: + self._registry_name_prefix = prompt( + "Please enter a prefix for the Azure Container Registry", + default=self._registry_name_prefix, + ) + if self._validate_user_input(self._registry_name_prefix): + break + else: + self._console.print( + "The prefix must be alphanumeric and between 3-50 characters.", + style="red", + ) + while True: + self._identity_name = prompt( + "Please enter a name for the identity (used for ACR access)", + default=self._identity_name, + ) + if self._validate_user_input(self._identity_name): + break + else: + self._console.print( + "The identity name must be alphanumeric and at least 3 characters.", + style="red", + ) + self._credentials_block_name = prompt( + "Please enter a name for the ACI credentials block", + default=self._credentials_block_name, + ) + table = await self._create_provision_table(work_pool_name, client) + self._console.print(table) + + return Confirm.ask( + "Proceed with infrastructure provisioning?", console=self._console + ) + @inject_client async def provision( self, @@ -809,40 +884,48 @@ async def provision( await self._verify_az_ready() await self._select_subscription() await self.set_location() + self._credentials_block_name = f"{work_pool_name}-push-pool-credentials" - table = Panel( - dedent( - f"""\ - Provisioning infrastructure for your work pool [blue]{work_pool_name}[/] will require: - - Updates in subscription [blue]{self._subscription_name}[/] - - - Create a resource group in location [blue]{self._location}[/] - - Create an app registration in Azure AD [blue]{self.APP_REGISTRATION_NAME}[/] - - Create/use a service principal for app registration - - Generate a secret for app registration - - Create an Azure Container Registry with prefix [blue]{self.REGISTRY_NAME_PREFIX}[/] - - Create an identity [blue]{self._identity_name}[/] to allow access to the created registry - - Assign Contributor role to service account - - Create an ACR registry for image hosting - - Create an identity for Azure Container Instance to allow access to the registry - - Updates in Prefect workspace - - - Create Azure Container Instance credentials block [blue]aci-push-pool-credentials[/] - """ - ), - expand=False, - ) + table = await self._create_provision_table(work_pool_name, client) self._console.print(table) if self._console.is_interactive: - if not Confirm.ask( - "Proceed with infrastructure provisioning?", console=self._console + chosen_option = prompt_select_from_table( + self._console, + "Proceed with infrastructure provisioning with default resource names?", + [ + {"header": "Options:", "key": "option"}, + ], + [ + { + "option": ( + "Yes, proceed with infrastructure provisioning with default" + " resource names" + ) + }, + {"option": "Customize resource names"}, + {"option": "Do not proceed with infrastructure provisioning"}, + ], + ) + if chosen_option["option"] == "Customize resource names": + if not await self._customize_resource_names(work_pool_name, client): + return base_job_template + + elif ( + chosen_option["option"] + == "Do not proceed with infrastructure provisioning" ): return base_job_template + elif ( + chosen_option["option"] + != "Yes, proceed with infrastructure provisioning with default" + " resource names" + ): + # basically, we should never hit this. i'm concerned that we might change + # the options in the future and forget to update this check + raise ValueError(f"Invalid option selected: {chosen_option['option']}") credentials_block_exists = await self._aci_credentials_block_exists( - block_name=f"{work_pool_name}-push-pool-credentials", client=client + block_name=self._credentials_block_name, client=client ) if not credentials_block_exists: @@ -862,7 +945,7 @@ async def provision( progress.advance(task) credentials_block_exists = await self._aci_credentials_block_exists( - block_name=f"{work_pool_name}-push-pool-credentials", client=client + block_name=self._credentials_block_name, client=client ) if not credentials_block_exists: @@ -882,7 +965,7 @@ async def provision( "ACI credentials block already exists.", style="yellow" ) block_doc = await client.read_block_document_by_name( - name=f"{work_pool_name}-push-pool-credentials", + name=self._credentials_block_name, block_type_slug="azure-container-instance-credentials", ) block_doc_id = block_doc.id @@ -894,10 +977,14 @@ async def provision( ) progress.advance(task) - progress.console.print("Creating Azure Container Registry") + progress.console.print( + "Creating Azure Container Registry (this make take a few minutes)" + ) + + registry_name = self._generate_acr_name(self._registry_name_prefix) registry = await self._get_or_create_registry( - registry_name=self._generate_acr_name(self.REGISTRY_NAME_PREFIX), - resource_group_name=self.RESOURCE_GROUP_NAME, + registry_name=registry_name, + resource_group_name=self._resource_group_name, location=self._location, subscription_id=self._subscription_id, ) @@ -913,7 +1000,7 @@ async def provision( progress.console.print("Creating identity") identity = await self._get_or_create_identity( identity_name=self._identity_name, - resource_group_name=self.RESOURCE_GROUP_NAME, + resource_group_name=self._resource_group_name, subscription_id=self._subscription_id, ) await self._assign_acr_pull_role( @@ -930,7 +1017,7 @@ async def provision( base_job_template_copy["variables"]["properties"]["resource_group_name"][ "default" - ] = self.RESOURCE_GROUP_NAME + ] = self._resource_group_name base_job_template_copy["variables"]["properties"]["subscription_id"][ "default" diff --git a/tests/infrastructure/provisioners/test_container_instance.py b/tests/infrastructure/provisioners/test_container_instance.py index f38bf42ca29b..953e8b44f379 100644 --- a/tests/infrastructure/provisioners/test_container_instance.py +++ b/tests/infrastructure/provisioners/test_container_instance.py @@ -534,7 +534,7 @@ async def test_aci_resource_group_creation_creates_new_group(provisioner): ), call( ( - "az group create --name 'prefect-aci-push-pool-rg' --location 'None'" + "az group create --name 'prefect-aci-push-pool-rg' --location 'eastus'" " --subscription 'None'" ), success_message=( @@ -583,7 +583,7 @@ async def test_aci_resource_group_creation_handles_errors(provisioner): ), call( ( - "az group create --name 'prefect-aci-push-pool-rg' --location 'None'" + "az group create --name 'prefect-aci-push-pool-rg' --location 'eastus'" " --subscription 'None'" ), success_message=( @@ -1140,7 +1140,7 @@ async def test_assign_acr_pull_role(provisioner): async def test_get_or_create_identity_existing_identity(provisioner): identity_name = "test-identity" - resource_group_name = "test-resource-group" + resource_group_name = "test-rg" provisioner.azure_cli.run_command.side_effect = [ [{"name": identity_name}], @@ -1169,7 +1169,7 @@ async def test_get_or_create_identity_existing_identity(provisioner): async def test_get_or_create_identity_new_identity(provisioner): identity_name = "test-identity" - resource_group_name = "test-resource-group" + resource_group_name = "test-rg" provisioner.azure_cli.run_command.side_effect = [ [], # Identity does not exist @@ -1209,7 +1209,7 @@ async def test_get_or_create_identity_new_identity(provisioner): async def test_get_or_create_identity_error(provisioner): identity_name = "test-identity" - resource_group_name = "test-resource-group" + resource_group_name = "test-rg" error = CalledProcessError(1, "cmd", output="output", stderr="error") provisioner.azure_cli.run_command.side_effect = [None, error] @@ -1865,3 +1865,734 @@ async def test_aci_provision_existing_credentials_block( new_base_job_template["variables"]["properties"]["identities"]["default"] = [ "/subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/prefect-aci-push-pool-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/prefect-aci-push-pool-identity" ] + + +async def test_aci_provision_interactive_default_provisioning( + prefect_client: PrefectClient, + monkeypatch, + default_base_job_template, + provisioner: ContainerInstancePushProvisioner, +): + mock_prompt_select_from_table = MagicMock( + side_effect=[ + {"id": "12345678-1234-1234-1234-123456789012", "name": "subscription_1"}, + { + "option": ( + "Yes, proceed with infrastructure provisioning with default" + " resource names" + ) + }, + ] + ) + + monkeypatch.setattr( + "prefect.infrastructure.provisioners.container_instance.prompt_select_from_table", + mock_prompt_select_from_table, + ) + + mock_confirm = MagicMock(return_value=True) + + monkeypatch.setattr( + "prefect.infrastructure.provisioners.container_instance.Confirm.ask", + mock_confirm, + ) + monkeypatch.setattr( + provisioner, + "_generate_acr_name", + lambda *args, **kwargs: "prefectacipushpoolregistry", + ) + subscription_list = [ + { + "cloudName": "AzureCloud", + "id": "12345678-1234-1234-1234-123456789012", + "isDefault": True, + "name": "subscription_1", + "state": "Enabled", + "tenantId": "12345678-1234-1234-1234-123456789012", + } + ] + + app_registration = { + "appId": "12345678-1234-1234-1234-123456789012", + "displayName": "prefect-aci-push-pool-app", + "identifierUris": ["https://prefect-aci-push-pool-app"], + } + + client_secret = { + "appId": "5407b48a-a28d-49ea-a740-54504847153f", + "password": "", + "tenant": "9ee4947a-f114-4939-a5ac-7f0ed786de36", + } + + new_service_principal = [ + { + "id": "abf1b3a0-1b1b-4c1c-9c9c-1c1c1c1c1c1c", + "accountEnabled": True, + "addIns": [], + "alternativeNames": [], + "appDescription": None, + "appDisplayName": "prefect-aci-push-pool-app", + "appId": "bcbeb824-fc3a-41f7-afc0-fc00297c1355", + } + ] + + role_assignments = { + "roleDefinitionName": "Contributor", + } + + new_registry = { + "id": "/subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/prefect-aci-push-pool-rg/providers/Microsoft.ContainerRegistry/registries/prefectacipushpoolregistry", + "loginServer": "prefectacipushpoolregistry.azurecr.io", + } + + new_identity = { + "id": "/subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/prefect-aci-push-pool-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/prefect-aci-push-pool-identity", + "principalId": "12345678-1234-1234-1234-123456789012", + } + + provisioner.azure_cli.run_command.side_effect = [ + "2.0.0", # Azure CLI is installed + subscription_list, # Azure login check + subscription_list, # Select subscription + "westus", # Set location + None, # Resource group does not exist + "New resource group created", # Successful creation + None, # App does not exist + app_registration, # Successful creation + client_secret, # Generate app secret + [], # Principal does not exist + None, # Successful creation + new_service_principal, # Successful retrieval + [], # Role does not exist + role_assignments, # Successful creation + [], # Registry does not exist + new_registry, # Successful creation + None, # Log in to registry + [], # Identity does not exist + new_identity, # Successful creation + None, # Assign identity to registry + ] + + monkeypatch.setattr(provisioner._console, "is_interactive", True) + + new_base_job_template = await provisioner.provision( + work_pool_name="test-work-pool", + base_job_template=default_base_job_template, + client=prefect_client, + ) + + assert new_base_job_template + + expected_calls = [ + # _verify_az_ready + call("az --version", ignore_if_exists=True), + call("az account list --output json", return_json=True), + # _select_subscription + call( + "az account list --output json", + failure_message=( + "No Azure subscriptions found. Please create an Azure subscription and" + " try again." + ), + ignore_if_exists=True, + return_json=True, + ), + # _set_location + call('az account list-locations --query "[?isDefault].name" --output tsv'), + # _create_resource_group + call( + ( + "az group exists --name prefect-aci-push-pool-rg --subscription" + " 12345678-1234-1234-1234-123456789012" + ), + return_json=True, + ), + call( + ( + "az group create --name 'prefect-aci-push-pool-rg' --location 'westus'" + " --subscription '12345678-1234-1234-1234-123456789012'" + ), + success_message=( + "Resource group 'prefect-aci-push-pool-rg' created successfully" + ), + failure_message=( + "Failed to create resource group 'prefect-aci-push-pool-rg' in" + " subscription 'subscription_1'" + ), + ignore_if_exists=True, + ), + # _create_app_registration + call("az ad app list --display-name prefect-aci-push-pool-app --output json"), + call( + "az ad app create --display-name prefect-aci-push-pool-app --output json", + success_message=( + "App registration 'prefect-aci-push-pool-app' created successfully" + ), + failure_message=( + "Failed to create app registration with name" + " 'prefect-aci-push-pool-app'" + ), + ignore_if_exists=True, + ), + # _create secret + call( + ( + "az ad app credential reset --id 12345678-1234-1234-1234-123456789012" + " --append --output json" + ), + success_message=( + "Secret generated for app registration with client ID" + " '12345678-1234-1234-1234-123456789012'" + ), + failure_message=( + "Failed to generate secret for app registration with client ID" + " '12345678-1234-1234-1234-123456789012'. If you have already generated" + " 2 secrets for this app registration, please delete one from the" + " `prefect-aci-push-pool-app` resource and try again." + ), + ignore_if_exists=True, + return_json=True, + ), + # _create_service_principal + call( + ( + "az ad sp list --all --query" + " \"[?appId=='12345678-1234-1234-1234-123456789012']\" --output json" + ), + return_json=True, + ), + call( + "az ad sp create --id 12345678-1234-1234-1234-123456789012", + success_message=( + "Service principal created for app ID" + " '12345678-1234-1234-1234-123456789012'" + ), + failure_message=( + "Failed to create service principal for app ID" + " '12345678-1234-1234-1234-123456789012'" + ), + ), + call( + ( + "az ad sp list --all --query" + " \"[?appId=='12345678-1234-1234-1234-123456789012']\" --output json" + ), + failure_message=( + "Failed to retrieve new service principal for app ID" + " 12345678-1234-1234-1234-123456789012" + ), + return_json=True, + ), + # _assign_contributor_role + call( + ( + "az role assignment list --assignee" + " abf1b3a0-1b1b-4c1c-9c9c-1c1c1c1c1c1c --role Contributor --scope" + " /subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/prefect-aci-push-pool-rg" + " --subscription 12345678-1234-1234-1234-123456789012 --output json" + ), + return_json=True, + ), + call( + ( + "az role assignment create --role Contributor --assignee-object-id" + " abf1b3a0-1b1b-4c1c-9c9c-1c1c1c1c1c1c --scope" + " /subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/prefect-aci-push-pool-rg" + " --subscription 12345678-1234-1234-1234-123456789012" + ), + success_message=( + "Contributor role assigned to service principal with object ID" + " 'abf1b3a0-1b1b-4c1c-9c9c-1c1c1c1c1c1c'" + ), + failure_message=( + "Failed to assign Contributor role to service principal with object ID" + " 'abf1b3a0-1b1b-4c1c-9c9c-1c1c1c1c1c1c'" + ), + ignore_if_exists=True, + ), + # _get_or_create_registry + call( + ( + "az acr list --query \"[?starts_with(name, 'prefect')]\" --subscription" + " 12345678-1234-1234-1234-123456789012 --output json" + ), + return_json=True, + ), + call( + ( + "az acr create --name prefectacipushpoolregistry --resource-group" + " prefect-aci-push-pool-rg --subscription" + " 12345678-1234-1234-1234-123456789012 --location westus --sku Basic" + ), + success_message="Registry created", + failure_message="Failed to create registry", + return_json=True, + ), + # _log_into_registry + call( + ( + "az acr login --name prefectacipushpoolregistry.azurecr.io" + " --subscription 12345678-1234-1234-1234-123456789012" + ), + success_message=( + "Logged into registry prefectacipushpoolregistry.azurecr.io" + ), + failure_message=( + "Failed to log into registry prefectacipushpoolregistry.azurecr.io" + ), + ), + # _get_or_create_identity + call( + ( + "az identity list --query \"[?name=='prefect-acr-identity']\"" + " --resource-group prefect-aci-push-pool-rg --subscription" + " 12345678-1234-1234-1234-123456789012 --output json" + ), + return_json=True, + ), + call( + ( + "az identity create --name prefect-acr-identity --resource-group" + " prefect-aci-push-pool-rg --subscription" + " 12345678-1234-1234-1234-123456789012 --output json" + ), + success_message="Identity 'prefect-acr-identity' created", + failure_message="Failed to create identity 'prefect-acr-identity'", + return_json=True, + ), + # _assign_acr_pull_role + call( + ( + "az role assignment create --assignee" + " 12345678-1234-1234-1234-123456789012 --scope" + " /subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/prefect-aci-push-pool-rg/providers/Microsoft.ContainerRegistry/registries/prefectacipushpoolregistry" + " --role AcrPull --subscription 12345678-1234-1234-1234-123456789012" + ), + ignore_if_exists=True, + ), + ] + + provisioner.azure_cli.run_command.assert_has_calls(expected_calls) + + new_block_doc_id = new_base_job_template["variables"]["properties"][ + "aci_credentials" + ]["default"]["$ref"]["block_document_id"] + + assert new_block_doc_id + + block_doc = await prefect_client.read_block_document(new_block_doc_id) + + assert block_doc.name == "test-work-pool-push-pool-credentials" + + assert block_doc.data == { + "client_id": "12345678-1234-1234-1234-123456789012", + "tenant_id": "9ee4947a-f114-4939-a5ac-7f0ed786de36", + "client_secret": "", + } + + new_base_job_template["variables"]["properties"]["subscription_id"][ + "default" + ] = "12345678-1234-1234-1234-123456789012" + + new_base_job_template["variables"]["properties"]["resource_group_name"][ + "default" + ] = "prefect-aci-push-pool-rg" + + new_base_job_template["variables"]["properties"]["image_registry"]["default"] = { + "registry_url": "prefectacipushpoolregistry.azurecr.io", + "identity": "/subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/prefect-aci-push-pool-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/prefect-aci-push-pool-identity", + } + + new_base_job_template["variables"]["properties"]["identities"]["default"] = [ + "/subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/prefect-aci-push-pool-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/prefect-aci-push-pool-identity" + ] + + +async def test_aci_provision_interactive_custom_resource_names( + prefect_client: PrefectClient, + monkeypatch, + default_base_job_template, + provisioner: ContainerInstancePushProvisioner, +): + def prompt_mocks(*args, **kwargs): + if "Please enter a name for the resource group" in args[0]: + return "custom-rg-name" + if "Please enter a name for the app registration" in args[0]: + return "custom-app-name" + if "Please enter a prefix for the Azure Container Registry" in args[0]: + return "customregistryname" + if "Please enter a name for the identity" in args[0]: + return "customidentityname" + if "Please enter a name for the ACI credentials block" in args[0]: + return "custom-credentials-name" + + mock_prompt = MagicMock(side_effect=prompt_mocks) + + monkeypatch.setattr( + "prefect.infrastructure.provisioners.container_instance.prompt", mock_prompt + ) + + mock_prompt_select_from_table = MagicMock( + side_effect=[ + {"id": "12345678-1234-1234-1234-123456789012", "name": "subscription_1"}, + {"option": "Customize resource names"}, + ] + ) + + monkeypatch.setattr( + "prefect.infrastructure.provisioners.container_instance.prompt_select_from_table", + mock_prompt_select_from_table, + ) + + mock_confirm = MagicMock(return_value=True) + + monkeypatch.setattr( + "prefect.infrastructure.provisioners.container_instance.Confirm.ask", + mock_confirm, + ) + + subscription_list = [ + { + "cloudName": "AzureCloud", + "id": "12345678-1234-1234-1234-123456789012", + "isDefault": True, + "name": "subscription_1", + "state": "Enabled", + "tenantId": "12345678-1234-1234-1234-123456789012", + } + ] + + app_registration = { + "appId": "12345678-1234-1234-1234-123456789012", + "displayName": "prefect-aci-push-pool-app", + "identifierUris": ["https://prefect-aci-push-pool-app"], + } + + client_secret = { + "appId": "5407b48a-a28d-49ea-a740-54504847153f", + "password": "", + "tenant": "9ee4947a-f114-4939-a5ac-7f0ed786de36", + } + + new_service_principal = [ + { + "id": "abf1b3a0-1b1b-4c1c-9c9c-1c1c1c1c1c1c", + "accountEnabled": True, + "addIns": [], + "alternativeNames": [], + "appDescription": None, + "appDisplayName": "prefect-aci-push-pool-app", + "appId": "bcbeb824-fc3a-41f7-afc0-fc00297c1355", + } + ] + + role_assignments = { + "roleDefinitionName": "Contributor", + } + + new_registry = { + "id": "/subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/custom-rg-name/providers/Microsoft.ContainerRegistry/registries/customregistryname", + "loginServer": "customregistryname.azurecr.io", + } + + new_identity = { + "id": "/subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/custom-rg-name/providers/Microsoft.ManagedIdentity/userAssignedIdentities/customidentityname", + "principalId": "12345678-1234-1234-1234-123456789012", + } + + provisioner.azure_cli.run_command.side_effect = [ + "2.0.0", # Azure CLI is installed + subscription_list, # Azure login check + subscription_list, # Select subscription + "westus", # Set location + None, # Resource group does not exist + "New resource group created", # Successful creation + None, # App does not exist + app_registration, # Successful creation + client_secret, # Generate app secret + [], # Principal does not exist + None, # Successful creation + new_service_principal, # Successful retrieval + [], # Role does not exist + role_assignments, # Successful creation + [], # Registry does not exist + new_registry, # Successful creation + None, # Log in to registry + [], # Identity does not exist + new_identity, # Successful creation + None, # Assign identity to registry + ] + + monkeypatch.setattr( + provisioner, + "_generate_acr_name", + lambda *args, **kwargs: "customregistryname", + ) + + monkeypatch.setattr(provisioner._console, "is_interactive", True) + + new_base_job_template = await provisioner.provision( + work_pool_name="test-work-pool", + base_job_template=default_base_job_template, + client=prefect_client, + ) + + assert new_base_job_template + + expected_calls = [ + # _verify_az_ready + call("az --version", ignore_if_exists=True), + call("az account list --output json", return_json=True), + # _select_subscription + call( + "az account list --output json", + failure_message=( + "No Azure subscriptions found. Please create an Azure subscription and" + " try again." + ), + ignore_if_exists=True, + return_json=True, + ), + # _set_location + call('az account list-locations --query "[?isDefault].name" --output tsv'), + # _create_resource_group + call( + ( + "az group exists --name custom-rg-name --subscription" + " 12345678-1234-1234-1234-123456789012" + ), + return_json=True, + ), + call( + ( + "az group create --name 'custom-rg-name' --location 'westus'" + " --subscription '12345678-1234-1234-1234-123456789012'" + ), + success_message="Resource group 'custom-rg-name' created successfully", + failure_message=( + "Failed to create resource group 'custom-rg-name' in" + " subscription 'subscription_1'" + ), + ignore_if_exists=True, + ), + # _create_app_registration + call("az ad app list --display-name custom-app-name --output json"), + call( + "az ad app create --display-name custom-app-name --output json", + success_message="App registration 'custom-app-name' created successfully", + failure_message=( + "Failed to create app registration with name 'custom-app-name'" + ), + ignore_if_exists=True, + ), + # _create secret + call( + ( + "az ad app credential reset --id 12345678-1234-1234-1234-123456789012" + " --append --output json" + ), + success_message=( + "Secret generated for app registration with client ID" + " '12345678-1234-1234-1234-123456789012'" + ), + failure_message=( + "Failed to generate secret for app registration with client ID" + " '12345678-1234-1234-1234-123456789012'. If you have already generated" + " 2 secrets for this app registration, please delete one from the" + " `custom-app-name` resource and try again." + ), + ignore_if_exists=True, + return_json=True, + ), + # _create_service_principal + call( + ( + "az ad sp list --all --query" + " \"[?appId=='12345678-1234-1234-1234-123456789012']\" --output json" + ), + return_json=True, + ), + call( + "az ad sp create --id 12345678-1234-1234-1234-123456789012", + success_message=( + "Service principal created for app ID" + " '12345678-1234-1234-1234-123456789012'" + ), + failure_message=( + "Failed to create service principal for app ID" + " '12345678-1234-1234-1234-123456789012'" + ), + ), + call( + ( + "az ad sp list --all --query" + " \"[?appId=='12345678-1234-1234-1234-123456789012']\" --output json" + ), + failure_message=( + "Failed to retrieve new service principal for app ID" + " 12345678-1234-1234-1234-123456789012" + ), + return_json=True, + ), + # _assign_contributor_role + call( + ( + "az role assignment list --assignee" + " abf1b3a0-1b1b-4c1c-9c9c-1c1c1c1c1c1c --role Contributor --scope" + " /subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/custom-rg-name" + " --subscription 12345678-1234-1234-1234-123456789012 --output json" + ), + return_json=True, + ), + call( + ( + "az role assignment create --role Contributor --assignee-object-id" + " abf1b3a0-1b1b-4c1c-9c9c-1c1c1c1c1c1c --scope" + " /subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/custom-rg-name" + " --subscription 12345678-1234-1234-1234-123456789012" + ), + success_message=( + "Contributor role assigned to service principal with object ID" + " 'abf1b3a0-1b1b-4c1c-9c9c-1c1c1c1c1c1c'" + ), + failure_message=( + "Failed to assign Contributor role to service principal with object ID" + " 'abf1b3a0-1b1b-4c1c-9c9c-1c1c1c1c1c1c'" + ), + ignore_if_exists=True, + ), + # _get_or_create_registry + call( + ( + "az acr list --query \"[?starts_with(name, 'customregistryname')]\"" + " --subscription 12345678-1234-1234-1234-123456789012 --output json" + ), + return_json=True, + ), + call( + ( + "az acr create --name customregistryname --resource-group" + " custom-rg-name --subscription 12345678-1234-1234-1234-123456789012" + " --location westus --sku Basic" + ), + success_message="Registry created", + failure_message="Failed to create registry", + return_json=True, + ), + # _log_into_registry + call( + ( + "az acr login --name customregistryname.azurecr.io --subscription" + " 12345678-1234-1234-1234-123456789012" + ), + success_message="Logged into registry customregistryname.azurecr.io", + failure_message="Failed to log into registry customregistryname.azurecr.io", + ), + # _get_or_create_identity + call( + ( + "az identity list --query \"[?name=='customidentityname']\"" + " --resource-group custom-rg-name --subscription" + " 12345678-1234-1234-1234-123456789012 --output json" + ), + return_json=True, + ), + call( + ( + "az identity create --name customidentityname --resource-group" + " custom-rg-name --subscription 12345678-1234-1234-1234-123456789012" + " --output json" + ), + success_message="Identity 'customidentityname' created", + failure_message="Failed to create identity 'customidentityname'", + return_json=True, + ), + # _assign_acr_pull_role + call( + ( + "az role assignment create --assignee" + " 12345678-1234-1234-1234-123456789012 --scope" + " /subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/custom-rg-name/providers/Microsoft.ContainerRegistry/registries/customregistryname" + " --role AcrPull --subscription 12345678-1234-1234-1234-123456789012" + ), + ignore_if_exists=True, + ), + ] + + provisioner.azure_cli.run_command.assert_has_calls(expected_calls) + + new_block_doc_id = new_base_job_template["variables"]["properties"][ + "aci_credentials" + ]["default"]["$ref"]["block_document_id"] + + assert new_block_doc_id + + block_doc = await prefect_client.read_block_document(new_block_doc_id) + + assert block_doc.name == "custom-credentials-name" + + assert block_doc.data == { + "client_id": "12345678-1234-1234-1234-123456789012", + "tenant_id": "9ee4947a-f114-4939-a5ac-7f0ed786de36", + "client_secret": "", + } + + new_base_job_template["variables"]["properties"]["subscription_id"][ + "default" + ] = "12345678-1234-1234-1234-123456789012" + + new_base_job_template["variables"]["properties"]["resource_group_name"][ + "default" + ] = "custom-rg-name" + + new_base_job_template["variables"]["properties"]["image_registry"]["default"] = { + "registry_url": "custom-app-name.azurecr.io", + "identity": "/subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/custom-rg-name/providers/Microsoft.ManagedIdentity/userAssignedIdentities/customidentityname", + } + + new_base_job_template["variables"]["properties"]["identities"]["default"] = [ + "/subscriptions/12345678-1234-1234-1234-123456789012/resourceGroups/custom-rg-name/providers/Microsoft.ManagedIdentity/userAssignedIdentities/customidentityname" + ] + + +async def test_aci_provision_interactive_reject_provisioning( + prefect_client: PrefectClient, + monkeypatch, + default_base_job_template, + provisioner: ContainerInstancePushProvisioner, +): + mock_prompt_select_from_table = MagicMock( + side_effect=[ + {"id": "12345678-1234-1234-1234-123456789012", "name": "subscription_1"}, + {"option": "Do not proceed with infrastructure provisioning"}, + ] + ) + + mock_confirm = MagicMock(return_value=False) + + monkeypatch.setattr( + "prefect.infrastructure.provisioners.container_instance.prompt_select_from_table", + mock_prompt_select_from_table, + ) + + monkeypatch.setattr( + "prefect.infrastructure.provisioners.container_instance.Confirm.ask", + mock_confirm, + ) + + provisioner.azure_cli.run_command.side_effect = [ + "2.0.0", # Azure CLI is installed + '{"account_a": "b"}', # Login check + '{"account_a": "b"}', # Select subscription + "westus", # Set location + ] + + monkeypatch.setattr(provisioner._console, "is_interactive", True) + + unchanged_base_job_template = await provisioner.provision( + work_pool_name="test-work-pool", + base_job_template=default_base_job_template, + client=prefect_client, + ) + + assert unchanged_base_job_template == default_base_job_template