diff --git a/airflow/providers/google/cloud/hooks/compute.py b/airflow/providers/google/cloud/hooks/compute.py index 4a32a54ba9f90c..7bc9d33e7c30e2 100644 --- a/airflow/providers/google/cloud/hooks/compute.py +++ b/airflow/providers/google/cloud/hooks/compute.py @@ -21,6 +21,11 @@ import time from typing import Any, Sequence +from google.api_core.retry import Retry +from google.cloud.compute_v1.services.instance_group_managers import InstanceGroupManagersClient +from google.cloud.compute_v1.services.instance_templates import InstanceTemplatesClient +from google.cloud.compute_v1.services.instances import InstancesClient +from google.cloud.compute_v1.types import Instance, InstanceGroupManager, InstanceTemplate from googleapiclient.discovery import build from airflow.exceptions import AirflowException @@ -66,12 +71,356 @@ def get_conn(self): """ Retrieves connection to Google Compute Engine. :return: Google Compute Engine services object + :rtype: dict """ if not self._conn: http_authorized = self._authorize() self._conn = build("compute", self.api_version, http=http_authorized, cache_discovery=False) return self._conn + def get_compute_instance_template_client(self): + """Returns Compute Engine Instance Template Client.""" + return InstanceTemplatesClient(credentials=self._get_credentials(), client_info=self.client_info) + + def get_compute_instance_client(self): + """Returns Compute Engine Instance Client.""" + return InstancesClient(credentials=self._get_credentials(), client_info=self.client_info) + + def get_compute_instance_group_managers_client(self): + """Returns Compute Engine Instance Group Managers Client.""" + return InstanceGroupManagersClient(credentials=self._get_credentials(), client_info=self.client_info) + + @GoogleBaseHook.fallback_to_default_project_id + def insert_instance_template( + self, + body: dict, + request_id: str | None = None, + project_id: str = PROVIDE_PROJECT_ID, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + ) -> None: + """ + Creates Instance Template using body specified. + Must be called with keyword arguments rather than positional. + + :param body: Instance Template representation as an object. + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :param project_id: Google Cloud project ID where the Compute Engine Instance Template exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + client = self.get_compute_instance_template_client() + client.insert( + # Calling method insert() on client to create Instance Template. + # This method accepts request object as an argument and should be of type + # Union[google.cloud.compute_v1.types.InsertInstanceTemplateRequest, dict] to construct a request + # message. + # The request object should be represented using arguments: + # instance_template_resource (google.cloud.compute_v1.types.InstanceTemplate): + # The body resource for this request. + # request_id (str): + # An optional request ID to identify requests. + # project (str): + # Project ID for this request. + request={ + "instance_template_resource": body, + "request_id": request_id, + "project": project_id, + }, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleBaseHook.fallback_to_default_project_id + def delete_instance_template( + self, + resource_id: str, + request_id: str | None = None, + project_id: str = PROVIDE_PROJECT_ID, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + ) -> None: + """ + Deletes Instance Template. + Deleting an Instance Template is permanent and cannot be undone. It + is not possible to delete templates that are already in use by a managed instance group. + Must be called with keyword arguments rather than positional. + + :param resource_id: Name of the Compute Engine Instance Template resource. + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :param project_id: Google Cloud project ID where the Compute Engine Instance Template exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + client = self.get_compute_instance_template_client() + client.delete( + # Calling method delete() on client to delete Instance Template. + # This method accepts request object as an argument and should be of type + # Union[google.cloud.compute_v1.types.DeleteInstanceTemplateRequest, dict] to + # construct a request message. + # The request object should be represented using arguments: + # instance_template (str): + # The name of the Instance Template to delete. + # project (str): + # Project ID for this request. + # request_id (str): + # An optional request ID to identify requests. + request={ + "instance_template": resource_id, + "project": project_id, + "request_id": request_id, + }, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleBaseHook.fallback_to_default_project_id + def get_instance_template( + self, + resource_id: str, + project_id: str = PROVIDE_PROJECT_ID, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + ) -> InstanceTemplate: + """ + Retrieves Instance Template by project_id and resource_id. + Must be called with keyword arguments rather than positional. + + :param resource_id: Name of the Instance Template. + :param project_id: Google Cloud project ID where the Compute Engine Instance Template exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + :return: Instance Template representation as object according to + https://cloud.google.com/compute/docs/reference/rest/v1/instanceTemplates + :rtype: object + """ + client = self.get_compute_instance_template_client() + instance_template_obj = client.get( + # Calling method get() on client to get the specified Instance Template. + # This method accepts request object as an argument and should be of type + # Union[google.cloud.compute_v1.types.GetInstanceTemplateRequest, dict] to construct a request + # message. + # The request object should be represented using arguments: + # instance_template (str): + # The name of the Instance Template. + # project (str): + # Project ID for this request. + request={ + "instance_template": resource_id, + "project": project_id, + }, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + return instance_template_obj + + @GoogleBaseHook.fallback_to_default_project_id + def insert_instance( + self, + body: dict, + zone: str, + project_id: str = PROVIDE_PROJECT_ID, + request_id: str | None = None, + source_instance_template: str | None = None, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + ) -> None: + """ + Creates Instance using body specified. + Must be called with keyword arguments rather than positional. + + :param body: Instance representation as an object. Should at least include 'name', 'machine_type', + 'disks' and 'network_interfaces' fields but doesn't include 'zone' field, as it will be specified + in 'zone' parameter. + Full or partial URL and can be represented as examples below: + 1. "machine_type": "projects/your-project-name/zones/your-zone/machineTypes/your-machine-type" + 2. "source_image": "projects/your-project-name/zones/your-zone/diskTypes/your-disk-type" + 3. "subnetwork": "projects/your-project-name/regions/your-region/subnetworks/your-subnetwork" + :param zone: Google Cloud zone where the Instance exists + :param project_id: Google Cloud project ID where the Compute Engine Instance Template exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param source_instance_template: Existing Instance Template that will be used as a base while + creating new Instance. + When specified, only name of new Instance should be provided as input arguments in 'body' + parameter when creating new Instance. All other parameters, such as machine_type, disks + and network_interfaces and etc will be passed to Instance as they are specified + in the Instance Template. + Full or partial URL and can be represented as examples below: + 1. "https://www.googleapis.com/compute/v1/projects/your-project/global/instanceTemplates/temp" + 2. "projects/your-project/global/instanceTemplates/temp" + 3. "global/instanceTemplates/temp" + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + client = self.get_compute_instance_client() + client.insert( + # Calling method insert() on client to create Instance. + # This method accepts request object as an argument and should be of type + # Union[google.cloud.compute_v1.types.InsertInstanceRequest, dict] to construct a request + # message. + # The request object should be represented using arguments: + # instance_resource (google.cloud.compute_v1.types.Instance): + # The body resource for this request. + # request_id (str): + # Optional, request ID to identify requests. + # project (str): + # Project ID for this request. + # zone (str): + # The name of the zone for this request. + # source_instance_template (str): + # Optional, link to Instance Template, that can be used to create new Instance. + request={ + "instance_resource": body, + "request_id": request_id, + "project": project_id, + "zone": zone, + "source_instance_template": source_instance_template, + }, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleBaseHook.fallback_to_default_project_id + def get_instance( + self, + resource_id: str, + zone: str, + project_id: str = PROVIDE_PROJECT_ID, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + ) -> Instance: + """ + Retrieves Instance by project_id and resource_id. + Must be called with keyword arguments rather than positional. + + :param resource_id: Name of the Instance + :param zone: Google Cloud zone where the Instance exists + :param project_id: Google Cloud project ID where the Compute Engine Instance exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + :return: Instance representation as object according to + https://cloud.google.com/compute/docs/reference/rest/v1/instances + :rtype: object + """ + client = self.get_compute_instance_client() + instance_obj = client.get( + # Calling method get() on client to get the specified Instance. + # This method accepts request object as an argument and should be of type + # Union[google.cloud.compute_v1.types.GetInstanceRequest, dict] to construct a request + # message. + # The request object should be represented using arguments: + # instance (str): + # The name of the Instance. + # project (str): + # Project ID for this request. + # zone (str): + # The name of the zone for this request. + request={ + "instance": resource_id, + "project": project_id, + "zone": zone, + }, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + return instance_obj + + @GoogleBaseHook.fallback_to_default_project_id + def delete_instance( + self, + resource_id: str, + zone: str, + project_id: str = PROVIDE_PROJECT_ID, + request_id: str | None = None, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + ) -> None: + """ + Deletes Instance. + Deleting an Instance is permanent and cannot be undone. + It is not possible to delete Instances that are already in use by a managed instance group. + Must be called with keyword arguments rather than positional. + + :param resource_id: Name of the Compute Engine Instance Template resource. + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :param project_id: Google Cloud project ID where the Compute Engine Instance Template exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param zone: Google Cloud zone where the Instance exists + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + client = self.get_compute_instance_client() + client.delete( + # Calling method delete() on client to delete Instance. + # This method accepts request object as an argument and should be of type + # Union[google.cloud.compute_v1.types.DeleteInstanceRequest, dict] to construct a request + # message. + # The request object should be represented using arguments: + # instance (str): + # Name of the Instance resource to delete. + # project (str): + # Project ID for this request. + # request_id (str): + # An optional request ID to identify requests. + # zone (str): + # The name of the zone for this request. + request={ + "instance": resource_id, + "project": project_id, + "request_id": request_id, + "zone": zone, + }, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + @GoogleBaseHook.fallback_to_default_project_id def start_instance(self, zone: str, resource_id: str, project_id: str) -> None: """ @@ -155,86 +504,167 @@ def _execute_set_machine_type(self, zone: str, resource_id: str, body: dict, pro ) @GoogleBaseHook.fallback_to_default_project_id - def get_instance_template(self, resource_id: str, project_id: str) -> dict: - """ - Retrieves instance template by project_id and resource_id. - Must be called with keyword arguments rather than positional. - - :param resource_id: Name of the instance template - :param project_id: Optional, Google Cloud project ID where the - Compute Engine Instance exists. If set to None or missing, - the default project_id from the Google Cloud connection is used. - :return: Instance template representation as object according to - https://cloud.google.com/compute/docs/reference/rest/v1/instanceTemplates - """ - response = ( - self.get_conn() - .instanceTemplates() - .get(project=project_id, instanceTemplate=resource_id) - .execute(num_retries=self.num_retries) - ) - return response - - @GoogleBaseHook.fallback_to_default_project_id - def insert_instance_template( + def insert_instance_group_manager( self, body: dict, + zone: str, project_id: str = PROVIDE_PROJECT_ID, request_id: str | None = None, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), ) -> None: """ - Inserts instance template using body specified + Creates an Instance Group Managers using the body specified. + After the group is created, instances in the group are created using the specified Instance Template. Must be called with keyword arguments rather than positional. - :param body: Instance template representation as object according to - https://cloud.google.com/compute/docs/reference/rest/v1/instanceTemplates - :param request_id: Optional, unique request_id that you might add to achieve + :param body: Instance Group Manager representation as an object. + :param request_id: Unique request_id that you might add to achieve full idempotence (for example when client call times out repeating the request - with the same request id will not create a new instance template again) + with the same request id will not create a new Instance Group Managers again) It should be in UUID format as defined in RFC 4122 - :param project_id: Optional, Google Cloud project ID where the - Compute Engine Instance exists. If set to None or missing, - the default project_id from the Google Cloud connection is used. - :return: None + :param project_id: Google Cloud project ID where the Compute Engine Instance Group Managers exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param zone: Google Cloud zone where the Instance exists + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. """ - response = ( - self.get_conn() - .instanceTemplates() - .insert(project=project_id, body=body, requestId=request_id) - .execute(num_retries=self.num_retries) + client = self.get_compute_instance_group_managers_client() + client.insert( + # Calling method insert() on client to create the specified Instance Group Managers. + # This method accepts request object as an argument and should be of type + # Union[google.cloud.compute_v1.types.InsertInstanceGroupManagerRequest, dict] to construct + # a request message. + # The request object should be represented using arguments: + # instance_group_manager_resource (google.cloud.compute_v1.types.InstanceGroupManager): + # The body resource for this request. + # project (str): + # Project ID for this request. + # zone (str): + # The name of the zone where you want to create the managed instance group. + # request_id (str): + # An optional request ID to identify requests. + request={ + "instance_group_manager_resource": body, + "project": project_id, + "zone": zone, + "request_id": request_id, + }, + retry=retry, + timeout=timeout, + metadata=metadata, ) - try: - operation_name = response["name"] - except KeyError: - raise AirflowException(f"Wrong response '{response}' returned - it should contain 'name' field") - self._wait_for_operation_to_complete(project_id=project_id, operation_name=operation_name) @GoogleBaseHook.fallback_to_default_project_id def get_instance_group_manager( self, - zone: str, resource_id: str, + zone: str, project_id: str = PROVIDE_PROJECT_ID, - ) -> dict: + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + ) -> InstanceGroupManager: """ Retrieves Instance Group Manager by project_id, zone and resource_id. Must be called with keyword arguments rather than positional. - :param zone: Google Cloud zone where the Instance Group Manager exists - :param resource_id: Name of the Instance Group Manager - :param project_id: Optional, Google Cloud project ID where the - Compute Engine Instance exists. If set to None or missing, - the default project_id from the Google Cloud connection is used. - :return: Instance group manager representation as object according to - https://cloud.google.com/compute/docs/reference/rest/beta/instanceGroupManagers + :param resource_id: The name of the Managed Instance Group + :param zone: Google Cloud zone where the Instance Group Managers exists + :param project_id: Google Cloud project ID where the Compute Engine Instance Group Managers exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + :return: Instance Group Managers representation as object according to + https://cloud.google.com/compute/docs/reference/rest/v1/instanceGroupManagers + :rtype: object """ - response = ( - self.get_conn() - .instanceGroupManagers() - .get(project=project_id, zone=zone, instanceGroupManager=resource_id) - .execute(num_retries=self.num_retries) + client = self.get_compute_instance_group_managers_client() + instance_group_manager_obj = client.get( + # Calling method get() on client to get the specified Instance Group Manager. + # This method accepts request object as an argument and should be of type + # Union[google.cloud.compute_v1.types.GetInstanceGroupManagerRequest, dict] to construct a + # request message. + # The request object should be represented using arguments: + # instance_group_manager (str): + # The name of the Managed Instance Group. + # project (str): + # Project ID for this request. + # zone (str): + # The name of the zone for this request. + request={ + "instance_group_manager": resource_id, + "project": project_id, + "zone": zone, + }, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + return instance_group_manager_obj + + @GoogleBaseHook.fallback_to_default_project_id + def delete_instance_group_manager( + self, + resource_id: str, + zone: str, + project_id: str = PROVIDE_PROJECT_ID, + request_id: str | None = None, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + ) -> None: + """ + Deletes Instance Group Managers. + Deleting an Instance Group Manager is permanent and cannot be undone. + Must be called with keyword arguments rather than positional. + + :param resource_id: Name of the Compute Engine Instance Group Managers resource. + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :param project_id: Google Cloud project ID where the Compute Engine Instance Group Managers exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param zone: Google Cloud zone where the Instance Group Managers exists + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + client = self.get_compute_instance_group_managers_client() + client.delete( + # Calling method delete() on client to delete Instance Group Managers. + # This method accepts request object as an argument and should be of type + # Union[google.cloud.compute_v1.types.DeleteInstanceGroupManagerRequest, dict] to construct a + # request message. + # The request object should be represented using arguments: + # instance_group_manager (str): + # Name of the Instance resource to delete. + # project (str): + # Project ID for this request. + # request_id (str): + # An optional request ID to identify requests. + # zone (str): + # The name of the zone for this request. + request={ + "instance_group_manager": resource_id, + "project": project_id, + "request_id": request_id, + "zone": zone, + }, + retry=retry, + timeout=timeout, + metadata=metadata, ) - return response @GoogleBaseHook.fallback_to_default_project_id def patch_instance_group_manager( diff --git a/airflow/providers/google/cloud/hooks/compute_ssh.py b/airflow/providers/google/cloud/hooks/compute_ssh.py index 74cc4b9c13d0fa..5e8d343196ba45 100644 --- a/airflow/providers/google/cloud/hooks/compute_ssh.py +++ b/airflow/providers/google/cloud/hooks/compute_ssh.py @@ -257,7 +257,7 @@ def _connect_to_instance(self, user, hostname, pkey, proxy_command) -> paramiko. raise self.log.info("Failed to connect. Waiting %ds to retry", time_to_wait) time.sleep(time_to_wait) - raise AirflowException("Caa not connect to instance") + raise AirflowException("Can not connect to instance") def _authorize_compute_engine_instance_metadata(self, pubkey): self.log.info("Appending SSH public key to instance metadata") diff --git a/airflow/providers/google/cloud/links/compute.py b/airflow/providers/google/cloud/links/compute.py new file mode 100644 index 00000000000000..c2f15b273004bd --- /dev/null +++ b/airflow/providers/google/cloud/links/compute.py @@ -0,0 +1,112 @@ +# +# 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. +"""This module contains Google Compute Engine links.""" +from __future__ import annotations + +from typing import TYPE_CHECKING + +from airflow.models import BaseOperator +from airflow.providers.google.cloud.links.base import BaseGoogleLink + +if TYPE_CHECKING: + from airflow.utils.context import Context + +COMPUTE_BASE_LINK = "https://console.cloud.google.com/compute" +COMPUTE_LINK = ( + COMPUTE_BASE_LINK + "/instancesDetail/zones/{location_id}/instances/{resource_id}?project={project_id}" +) +COMPUTE_TEMPLATE_LINK = COMPUTE_BASE_LINK + "/instanceTemplates/details/{resource_id}?project={project_id}" +COMPUTE_GROUP_MANAGER_LINK = ( + COMPUTE_BASE_LINK + "/instanceGroups/details/{location_id}/{resource_id}?project={project_id}" +) + + +class ComputeInstanceDetailsLink(BaseGoogleLink): + """Helper class for constructing Compute Instance details Link""" + + name = "Compute Instance details" + key = "compute_instance_details" + format_str = COMPUTE_LINK + + @staticmethod + def persist( + context: Context, + task_instance: BaseOperator, + location_id: str, + resource_id: str, + project_id: str | None, + ): + task_instance.xcom_push( + context, + key=ComputeInstanceDetailsLink.key, + value={ + "location_id": location_id, + "resource_id": resource_id, + "project_id": project_id, + }, + ) + + +class ComputeInstanceTemplateDetailsLink(BaseGoogleLink): + """Helper class for constructing Compute Instance Template details Link""" + + name = "Compute Instance Template details" + key = "compute_instance_template_details" + format_str = COMPUTE_TEMPLATE_LINK + + @staticmethod + def persist( + context: Context, + task_instance: BaseOperator, + resource_id: str, + project_id: str | None, + ): + task_instance.xcom_push( + context, + key=ComputeInstanceTemplateDetailsLink.key, + value={ + "resource_id": resource_id, + "project_id": project_id, + }, + ) + + +class ComputeInstanceGroupManagerDetailsLink(BaseGoogleLink): + """Helper class for constructing Compute Instance Group Manager details Link""" + + name = "Compute Instance Group Manager" + key = "compute_instance_group_manager_details" + format_str = COMPUTE_GROUP_MANAGER_LINK + + @staticmethod + def persist( + context: Context, + task_instance: BaseOperator, + location_id: str, + resource_id: str, + project_id: str | None, + ): + task_instance.xcom_push( + context, + key=ComputeInstanceGroupManagerDetailsLink.key, + value={ + "location_id": location_id, + "resource_id": resource_id, + "project_id": project_id, + }, + ) diff --git a/airflow/providers/google/cloud/operators/compute.py b/airflow/providers/google/cloud/operators/compute.py index fa23e7abda6f43..37a6761b0b7918 100644 --- a/airflow/providers/google/cloud/operators/compute.py +++ b/airflow/providers/google/cloud/operators/compute.py @@ -21,12 +21,19 @@ from copy import deepcopy from typing import TYPE_CHECKING, Any, Sequence -from googleapiclient.errors import HttpError +from google.api_core import exceptions +from google.api_core.retry import Retry +from google.cloud.compute_v1.types import Instance, InstanceGroupManager, InstanceTemplate from json_merge_patch import merge from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.providers.google.cloud.hooks.compute import ComputeEngineHook +from airflow.providers.google.cloud.links.compute import ( + ComputeInstanceDetailsLink, + ComputeInstanceGroupManagerDetailsLink, + ComputeInstanceTemplateDetailsLink, +) from airflow.providers.google.cloud.utils.field_sanitizer import GcpBodyFieldSanitizer from airflow.providers.google.cloud.utils.field_validator import GcpBodyFieldValidator @@ -62,13 +69,475 @@ def _validate_inputs(self) -> None: raise AirflowException("The required parameter 'project_id' is missing") if not self.zone: raise AirflowException("The required parameter 'zone' is missing") - if not self.resource_id: - raise AirflowException("The required parameter 'resource_id' is missing") def execute(self, context: Context): pass +class ComputeEngineInsertInstanceOperator(ComputeEngineBaseOperator): + """ + Creates an Instance in Google Compute Engine based on specified parameters. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:ComputeEngineInsertInstanceOperator` + + :param body: Instance representation as an object. Should at least include 'name', 'machine_type', + 'disks' and 'network_interfaces' fields but doesn't include 'zone' field, as it will be specified + in 'zone' parameter. + Full or partial URL and can be represented as examples below: + 1. "machine_type": "projects/your-project-name/zones/your-zone/machineTypes/your-machine-type" + 2. "disk_type": "projects/your-project-name/zones/your-zone/diskTypes/your-disk-type" + 3. "subnetwork": "projects/your-project-name/regions/your-region/subnetworks/your-subnetwork" + :param zone: Google Cloud zone where the Instance exists + :param project_id: Google Cloud project ID where the Compute Engine Instance exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param resource_id: Name of the Instance. If the name of Instance is not specified in body['name'], + the name will be taken from 'resource_id' parameter + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :param gcp_conn_id: The connection ID used to connect to Google Cloud. Defaults to 'google_cloud_default'. + :param api_version: API version used (for example v1 - or beta). Defaults to v1. + :param impersonation_chain: Service account to impersonate using short-term + credentials, or chained list of accounts required to get the access_token + of the last account in the list, which will be impersonated in the request. + If set as a string, the account must grant the originating account + the Service Account Token Creator IAM role. + If set as a sequence, the identities from the list must grant + Service Account Token Creator IAM role to the directly preceding identity, with first + account from the list granting this role to the originating account (templated). + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + + operator_extra_links = (ComputeInstanceDetailsLink(),) + + # [START gce_instance_insert_fields] + template_fields: Sequence[str] = ( + "body", + "project_id", + "zone", + "request_id", + "gcp_conn_id", + "api_version", + "impersonation_chain", + ) + # [END gce_instance_insert_fields] + + def __init__( + self, + *, + body: dict, + zone: str, + resource_id: str | None = None, + project_id: str | None = None, + request_id: str | None = None, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + gcp_conn_id: str = "google_cloud_default", + api_version: str = "v1", + validate_body: bool = True, + impersonation_chain: str | Sequence[str] | None = None, + **kwargs, + ) -> None: + self.body = body + self.zone = zone + self.request_id = request_id + self.resource_id = self.body["name"] if "name" in body else resource_id + self._field_validator = None # Optional[GcpBodyFieldValidator] + self.retry = retry + self.timeout = timeout + self.metadata = metadata + + if validate_body: + self._field_validator = GcpBodyFieldValidator( + GCE_INSTANCE_TEMPLATE_VALIDATION_PATCH_SPECIFICATION, api_version=api_version + ) + self._field_sanitizer = GcpBodyFieldSanitizer(GCE_INSTANCE_FIELDS_TO_SANITIZE) + super().__init__( + resource_id=self.resource_id, + zone=zone, + project_id=project_id, + gcp_conn_id=gcp_conn_id, + api_version=api_version, + impersonation_chain=impersonation_chain, + **kwargs, + ) + + def check_body_fields(self) -> None: + required_params = ["machine_type", "disks", "network_interfaces"] + for param in required_params: + if param in self.body: + continue + readable_param = param.replace("_", " ") + raise AirflowException( + f"The body '{self.body}' should contain at least {readable_param} for the new operator " + f"in the '{param}' field. Check (google.cloud.compute_v1.types.Instance) " + f"for more details about body fields description." + ) + + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id and "name" not in self.body: + raise AirflowException( + "The required parameters 'resource_id' and body['name'] are missing. " + "Please, provide at least one of them." + ) + + def _validate_all_body_fields(self) -> None: + if self._field_validator: + self._field_validator.validate(self.body) + + def execute(self, context: Context) -> dict: + hook = ComputeEngineHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + impersonation_chain=self.impersonation_chain, + ) + self._validate_all_body_fields() + self.check_body_fields() + try: + # Idempotence check (sort of) - we want to check if the new Instance + # is already created and if is, then we assume it was created previously - we do + # not check if content of the Instance is as expected. + # We assume success if the Instance is simply present. + existing_instance = hook.get_instance( + resource_id=self.resource_id, + project_id=self.project_id, + zone=self.zone, + ) + except exceptions.NotFound as e: + # We actually expect to get 404 / Not Found here as the should not yet exist + if not e.code == 404: + raise e + else: + self.log.info("The %s Instance already exists", self.resource_id) + ComputeInstanceDetailsLink.persist( + context=context, + task_instance=self, + location_id=self.zone, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) + return Instance.to_dict(existing_instance) + self._field_sanitizer.sanitize(self.body) + self.log.info("Creating Instance with specified body: %s", self.body) + hook.insert_instance( + body=self.body, + request_id=self.request_id, + project_id=self.project_id, + zone=self.zone, + ) + self.log.info("The specified Instance has been created SUCCESSFULLY") + new_instance = hook.get_instance( + resource_id=self.resource_id, + project_id=self.project_id, + zone=self.zone, + ) + ComputeInstanceDetailsLink.persist( + context=context, + task_instance=self, + location_id=self.zone, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) + return Instance.to_dict(new_instance) + + +class ComputeEngineInsertInstanceFromTemplateOperator(ComputeEngineBaseOperator): + """ + Creates an Instance in Google Compute Engine based on specified parameters from existing Template. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:ComputeEngineInsertInstanceFromTemplateOperator` + + :param body: Instance representation as object. For this Operator only 'name' parameter is required for + creating new Instance since all other parameters will be passed through the Template. + :param source_instance_template: Existing Instance Template that will be used as a base while creating + new Instance. When specified, only name of new Instance should be provided as input arguments in + 'body' parameter when creating new Instance. All other parameters, such as 'machine_type', 'disks' + and 'network_interfaces' will be passed to Instance as they are specified in the Instance Template. + Full or partial URL and can be represented as examples below: + 1. "https://www.googleapis.com/compute/v1/projects/your-project-name/global/instanceTemplates/temp" + 2. "projects/your-project-name/global/instanceTemplates/temp" + 3. "global/instanceTemplates/temp" + :param zone: Google Cloud zone where the instance exists. + :param project_id: Google Cloud project ID where the Compute Engine Instance exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param resource_id: Name of the Instance. If the name of Instance is not specified in body['name'], + the name will be taken from 'resource_id' parameter + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :param gcp_conn_id: The connection ID used to connect to Google Cloud. Defaults to 'google_cloud_default'. + :param api_version: API version used (for example v1 - or beta). Defaults to v1. + :param impersonation_chain: Service account to impersonate using short-term + credentials, or chained list of accounts required to get the access_token + of the last account in the list, which will be impersonated in the request. + If set as a string, the account must grant the originating account + the Service Account Token Creator IAM role. + If set as a sequence, the identities from the list must grant + Service Account Token Creator IAM role to the directly preceding identity, with first + account from the list granting this role to the originating account (templated). + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + + operator_extra_links = (ComputeInstanceDetailsLink(),) + + # [START gce_instance_insert_from_template_fields] + template_fields: Sequence[str] = ( + "body", + "source_instance_template", + "project_id", + "zone", + "request_id", + "gcp_conn_id", + "api_version", + "impersonation_chain", + ) + # [END gce_instance_insert_from_template_fields] + + def __init__( + self, + *, + source_instance_template: str, + body: dict, + zone: str, + resource_id: str | None = None, + project_id: str | None = None, + request_id: str | None = None, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + gcp_conn_id: str = "google_cloud_default", + api_version: str = "v1", + validate_body: bool = True, + impersonation_chain: str | Sequence[str] | None = None, + **kwargs, + ) -> None: + self.source_instance_template = source_instance_template + self.body = body + self.zone = zone + self.resource_id = self.body["name"] if "name" in body else resource_id + self.request_id = request_id + self._field_validator = None # Optional[GcpBodyFieldValidator] + self.retry = retry + self.timeout = timeout + self.metadata = metadata + + if validate_body: + self._field_validator = GcpBodyFieldValidator( + GCE_INSTANCE_TEMPLATE_VALIDATION_PATCH_SPECIFICATION, api_version=api_version + ) + self._field_sanitizer = GcpBodyFieldSanitizer(GCE_INSTANCE_FIELDS_TO_SANITIZE) + super().__init__( + resource_id=self.resource_id, + zone=zone, + project_id=project_id, + gcp_conn_id=gcp_conn_id, + api_version=api_version, + impersonation_chain=impersonation_chain, + **kwargs, + ) + + def _validate_all_body_fields(self) -> None: + if self._field_validator: + self._field_validator.validate(self.body) + + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id and "name" not in self.body: + raise AirflowException( + "The required parameters 'resource_id' and body['name'] are missing. " + "Please, provide at least one of them." + ) + + def execute(self, context: Context) -> dict: + hook = ComputeEngineHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + impersonation_chain=self.impersonation_chain, + ) + self._validate_all_body_fields() + try: + # Idempotence check (sort of) - we want to check if the new Instance + # is already created and if is, then we assume it was created - we do + # not check if content of the Instance is as expected. + # We assume success if the Instance is simply present + existing_instance = hook.get_instance( + resource_id=self.resource_id, + project_id=self.project_id, + zone=self.zone, + ) + except exceptions.NotFound as e: + # We actually expect to get 404 / Not Found here as the template should + # not yet exist + if not e.code == 404: + raise e + else: + self.log.info("The %s Instance already exists", self.resource_id) + ComputeInstanceDetailsLink.persist( + context=context, + task_instance=self, + location_id=self.zone, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) + return Instance.to_dict(existing_instance) + self._field_sanitizer.sanitize(self.body) + self.log.info("Creating Instance with specified body: %s", self.body) + hook.insert_instance( + body=self.body, + request_id=self.request_id, + project_id=self.project_id, + zone=self.zone, + source_instance_template=self.source_instance_template, + ) + self.log.info("The specified Instance has been created SUCCESSFULLY") + new_instance_from_template = hook.get_instance( + resource_id=self.resource_id, + project_id=self.project_id, + zone=self.zone, + ) + ComputeInstanceDetailsLink.persist( + context=context, + task_instance=self, + location_id=self.zone, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) + return Instance.to_dict(new_instance_from_template) + + +class ComputeEngineDeleteInstanceOperator(ComputeEngineBaseOperator): + """ + Deletes an Instance in Google Compute Engine. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:ComputeEngineDeleteInstanceOperator` + + :param project_id: Google Cloud project ID where the Compute Engine Instance exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param zone: Google Cloud zone where the instance exists. + :param resource_id: Name of the Instance. + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :param gcp_conn_id: The connection ID used to connect to Google Cloud. Defaults to 'google_cloud_default'. + :param api_version: API version used (for example v1 - or beta). Defaults to v1. + :param impersonation_chain: Service account to impersonate using short-term + credentials, or chained list of accounts required to get the access_token + of the last account in the list, which will be impersonated in the request. + If set as a string, the account must grant the originating account + the Service Account Token Creator IAM role. + If set as a sequence, the identities from the list must grant + Service Account Token Creator IAM role to the directly preceding identity, with first + account from the list granting this role to the originating account (templated). + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + + # [START gce_instance_delete_template_fields] + template_fields: Sequence[str] = ( + "zone", + "resource_id", + "request_id", + "project_id", + "gcp_conn_id", + "api_version", + "impersonation_chain", + ) + # [END gce_instance_delete_template_fields] + + def __init__( + self, + *, + resource_id: str, + zone: str, + request_id: str | None = None, + project_id: str | None = None, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + gcp_conn_id: str = "google_cloud_default", + api_version: str = "v1", + validate_body: bool = True, + impersonation_chain: str | Sequence[str] | None = None, + **kwargs, + ) -> None: + self.zone = zone + self.request_id = request_id + self.resource_id = resource_id + self._field_validator = None # Optional[GcpBodyFieldValidator] + self.retry = retry + self.timeout = timeout + self.metadata = metadata + + if validate_body: + self._field_validator = GcpBodyFieldValidator( + GCE_INSTANCE_TEMPLATE_VALIDATION_PATCH_SPECIFICATION, api_version=api_version + ) + self._field_sanitizer = GcpBodyFieldSanitizer(GCE_INSTANCE_FIELDS_TO_SANITIZE) + super().__init__( + project_id=project_id, + zone=zone, + resource_id=resource_id, + gcp_conn_id=gcp_conn_id, + api_version=api_version, + impersonation_chain=impersonation_chain, + **kwargs, + ) + + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id: + raise AirflowException("The required parameter 'resource_id' is missing. ") + + def execute(self, context: Context) -> None: + hook = ComputeEngineHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + impersonation_chain=self.impersonation_chain, + ) + try: + # Checking if specified Instance exists and if it does, delete it + hook.get_instance( + resource_id=self.resource_id, + project_id=self.project_id, + zone=self.zone, + ) + self.log.info("Successfully found Instance %s", self.resource_id) + hook.delete_instance( + resource_id=self.resource_id, + project_id=self.project_id, + request_id=self.request_id, + zone=self.zone, + ) + self.log.info("Successfully deleted Instance %s", self.resource_id) + except exceptions.NotFound as e: + # Expecting 404 Error in case if Instance doesn't exist. + if e.code == 404: + self.log.error("Instance %s doesn't exist", self.resource_id) + raise e + + class ComputeEngineStartInstanceOperator(ComputeEngineBaseOperator): """ Starts an instance in Google Compute Engine. @@ -96,6 +565,8 @@ class ComputeEngineStartInstanceOperator(ComputeEngineBaseOperator): account from the list granting this role to the originating account (templated). """ + operator_extra_links = (ComputeInstanceDetailsLink(),) + # [START gce_instance_start_template_fields] template_fields: Sequence[str] = ( "project_id", @@ -107,13 +578,25 @@ class ComputeEngineStartInstanceOperator(ComputeEngineBaseOperator): ) # [END gce_instance_start_template_fields] + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id: + raise AirflowException("The required parameter 'resource_id' is missing. ") + def execute(self, context: Context) -> None: hook = ComputeEngineHook( gcp_conn_id=self.gcp_conn_id, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) - return hook.start_instance(zone=self.zone, resource_id=self.resource_id, project_id=self.project_id) + ComputeInstanceDetailsLink.persist( + context=context, + task_instance=self, + location_id=self.zone, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) + hook.start_instance(zone=self.zone, resource_id=self.resource_id, project_id=self.project_id) class ComputeEngineStopInstanceOperator(ComputeEngineBaseOperator): @@ -143,6 +626,8 @@ class ComputeEngineStopInstanceOperator(ComputeEngineBaseOperator): account from the list granting this role to the originating account (templated). """ + operator_extra_links = (ComputeInstanceDetailsLink(),) + # [START gce_instance_stop_template_fields] template_fields: Sequence[str] = ( "project_id", @@ -154,12 +639,24 @@ class ComputeEngineStopInstanceOperator(ComputeEngineBaseOperator): ) # [END gce_instance_stop_template_fields] + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id: + raise AirflowException("The required parameter 'resource_id' is missing. ") + def execute(self, context: Context) -> None: hook = ComputeEngineHook( gcp_conn_id=self.gcp_conn_id, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) + ComputeInstanceDetailsLink.persist( + context=context, + task_instance=self, + location_id=self.zone, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) hook.stop_instance(zone=self.zone, resource_id=self.resource_id, project_id=self.project_id) @@ -200,6 +697,8 @@ class ComputeEngineSetMachineTypeOperator(ComputeEngineBaseOperator): account from the list granting this role to the originating account (templated). """ + operator_extra_links = (ComputeInstanceDetailsLink(),) + # [START gce_instance_set_machine_type_template_fields] template_fields: Sequence[str] = ( "project_id", @@ -245,6 +744,11 @@ def _validate_all_body_fields(self) -> None: if self._field_validator: self._field_validator.validate(self.body) + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id: + raise AirflowException("The required parameter 'resource_id' is missing. ") + def execute(self, context: Context) -> None: hook = ComputeEngineHook( gcp_conn_id=self.gcp_conn_id, @@ -252,7 +756,14 @@ def execute(self, context: Context) -> None: impersonation_chain=self.impersonation_chain, ) self._validate_all_body_fields() - return hook.set_machine_type( + ComputeInstanceDetailsLink.persist( + context=context, + task_instance=self, + location_id=self.zone, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) + hook.set_machine_type( zone=self.zone, resource_id=self.resource_id, body=self.body, project_id=self.project_id ) @@ -298,22 +809,296 @@ def execute(self, context: Context) -> None: ), ] -GCE_INSTANCE_TEMPLATE_FIELDS_TO_SANITIZE = [ - "kind", - "id", - "name", - "creationTimestamp", - "properties.disks.sha256", - "properties.disks.kind", - "properties.disks.sourceImageEncryptionKey.sha256", - "properties.disks.index", - "properties.disks.licenses", - "properties.networkInterfaces.kind", - "properties.networkInterfaces.accessConfigs.kind", - "properties.networkInterfaces.name", - "properties.metadata.kind", - "selfLink", -] +GCE_INSTANCE_FIELDS_TO_SANITIZE = [ + "kind", + "id", + "creationTimestamp", + "properties.disks.sha256", + "properties.disks.kind", + "properties.disks.sourceImageEncryptionKey.sha256", + "properties.disks.index", + "properties.disks.licenses", + "properties.networkInterfaces.kind", + "properties.networkInterfaces.accessConfigs.kind", + "properties.networkInterfaces.name", + "properties.metadata.kind", + "selfLink", +] + + +class ComputeEngineInsertInstanceTemplateOperator(ComputeEngineBaseOperator): + """ + Creates an Instance Template using specified fields. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:ComputeEngineInsertInstanceTemplateOperator` + + :param body: Instance template representation as object. + :param project_id: Google Cloud project ID where the Compute Engine Instance exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :param resource_id: Name of the Instance Template. If the name of Instance Template is not specified in + body['name'], the name will be taken from 'resource_id' parameter + :param gcp_conn_id: The connection ID used to connect to Google Cloud. Defaults to 'google_cloud_default'. + :param api_version: API version used (for example v1 - or beta). Defaults to v1. + :param impersonation_chain: Service account to impersonate using short-term + credentials, or chained list of accounts required to get the access_token + of the last account in the list, which will be impersonated in the request. + If set as a string, the account must grant the originating account + the Service Account Token Creator IAM role. + If set as a sequence, the identities from the list must grant + Service Account Token Creator IAM role to the directly preceding identity, with first + account from the list granting this role to the originating account (templated). + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + + operator_extra_links = (ComputeInstanceTemplateDetailsLink(),) + + # [START gce_instance_template_insert_fields] + template_fields: Sequence[str] = ( + "body", + "project_id", + "request_id", + "gcp_conn_id", + "api_version", + "impersonation_chain", + ) + # [END gce_instance_template_insert_fields] + + def __init__( + self, + *, + body: dict, + project_id: str | None = None, + resource_id: str | None = None, + request_id: str | None = None, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + gcp_conn_id: str = "google_cloud_default", + api_version: str = "v1", + validate_body: bool = True, + impersonation_chain: str | Sequence[str] | None = None, + **kwargs, + ) -> None: + self.body = body + self.request_id = request_id + self.resource_id = self.body["name"] if "name" in body else resource_id + self._field_validator = None # Optional[GcpBodyFieldValidator] + self.retry = retry + self.timeout = timeout + self.metadata = metadata + + if validate_body: + self._field_validator = GcpBodyFieldValidator( + GCE_INSTANCE_TEMPLATE_VALIDATION_PATCH_SPECIFICATION, api_version=api_version + ) + self._field_sanitizer = GcpBodyFieldSanitizer(GCE_INSTANCE_FIELDS_TO_SANITIZE) + super().__init__( + project_id=project_id, + zone="global", + resource_id=self.resource_id, + gcp_conn_id=gcp_conn_id, + api_version=api_version, + impersonation_chain=impersonation_chain, + **kwargs, + ) + + def check_body_fields(self) -> None: + required_params = ["machine_type", "disks", "network_interfaces"] + for param in required_params: + if param in self.body["properties"]: + continue + readable_param = param.replace("_", " ") + raise AirflowException( + f"The body '{self.body}' should contain at least {readable_param} for the new operator " + f"in the '{param}' field. Check (google.cloud.compute_v1.types.Instance) " + f"for more details about body fields description." + ) + + def _validate_all_body_fields(self) -> None: + if self._field_validator: + self._field_validator.validate(self.body) + + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id and "name" not in self.body: + raise AirflowException( + "The required parameters 'resource_id' and body['name'] are missing. " + "Please, provide at least one of them." + ) + + def execute(self, context: Context) -> dict: + hook = ComputeEngineHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + impersonation_chain=self.impersonation_chain, + ) + self._validate_all_body_fields() + self.check_body_fields() + try: + # Idempotence check (sort of) - we want to check if the new Template + # is already created and if is, then we assume it was created by previous run + # of operator - we do not check if content of the Template + # is as expected. Templates are immutable, so we cannot update it anyway + # and deleting/recreating is not worth the hassle especially + # that we cannot delete template if it is already used in some Instance + # Group Manager. We assume success if the template is simply present + existing_template = hook.get_instance_template( + resource_id=self.resource_id, project_id=self.project_id + ) + except exceptions.NotFound as e: + # We actually expect to get 404 / Not Found here as the template should + # not yet exist + if not e.code == 404: + raise e + else: + self.log.info("The %s Template already exists.", existing_template) + ComputeInstanceTemplateDetailsLink.persist( + context=context, + task_instance=self, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) + return InstanceTemplate.to_dict(existing_template) + self._field_sanitizer.sanitize(self.body) + self.log.info("Creating Instance Template with specified body: %s", self.body) + hook.insert_instance_template( + body=self.body, + request_id=self.request_id, + project_id=self.project_id, + ) + self.log.info("The specified Instance Template has been created SUCCESSFULLY", self.body) + new_template = hook.get_instance_template( + resource_id=self.resource_id, + project_id=self.project_id, + ) + ComputeInstanceTemplateDetailsLink.persist( + context=context, + task_instance=self, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) + return InstanceTemplate.to_dict(new_template) + + +class ComputeEngineDeleteInstanceTemplateOperator(ComputeEngineBaseOperator): + """ + Deletes an Instance Template in Google Compute Engine. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:ComputeEngineDeleteInstanceTemplateOperator` + + :param resource_id: Name of the Instance Template. + :param project_id: Google Cloud project ID where the Compute Engine Instance exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :param gcp_conn_id: The connection ID used to connect to Google Cloud. Defaults to 'google_cloud_default'. + :param api_version: API version used (for example v1 - or beta). Defaults to v1. + :param impersonation_chain: Service account to impersonate using short-term + credentials, or chained list of accounts required to get the access_token + of the last account in the list, which will be impersonated in the request. + If set as a string, the account must grant the originating account + the Service Account Token Creator IAM role. + If set as a sequence, the identities from the list must grant + Service Account Token Creator IAM role to the directly preceding identity, with first + account from the list granting this role to the originating account (templated). + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + + # [START gce_instance_template_delete_fields] + template_fields: Sequence[str] = ( + "resource_id", + "request_id", + "project_id", + "gcp_conn_id", + "api_version", + "impersonation_chain", + ) + # [END gce_instance_template_delete_fields] + + def __init__( + self, + *, + resource_id: str, + request_id: str | None = None, + project_id: str | None = None, + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + gcp_conn_id: str = "google_cloud_default", + api_version: str = "v1", + validate_body: bool = True, + impersonation_chain: str | Sequence[str] | None = None, + **kwargs, + ) -> None: + self.request_id = request_id + self.resource_id = resource_id + self._field_validator = None # Optional[GcpBodyFieldValidator] + self.retry = retry + self.timeout = timeout + self.metadata = metadata + + if validate_body: + self._field_validator = GcpBodyFieldValidator( + GCE_INSTANCE_TEMPLATE_VALIDATION_PATCH_SPECIFICATION, api_version=api_version + ) + self._field_sanitizer = GcpBodyFieldSanitizer(GCE_INSTANCE_FIELDS_TO_SANITIZE) + super().__init__( + project_id=project_id, + zone="global", + resource_id=resource_id, + gcp_conn_id=gcp_conn_id, + api_version=api_version, + impersonation_chain=impersonation_chain, + **kwargs, + ) + + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id: + raise AirflowException("The required parameter 'resource_id' is missing.") + + def execute(self, context: Context) -> None: + hook = ComputeEngineHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + impersonation_chain=self.impersonation_chain, + ) + try: + # Checking if specified Instance Template exists and if it does, delete it + hook.get_instance_template( + resource_id=self.resource_id, + project_id=self.project_id, + ) + self.log.info("Successfully found Instance Template %s", self.resource_id) + hook.delete_instance_template( + resource_id=self.resource_id, + project_id=self.project_id, + request_id=self.request_id, + ) + self.log.info("Successfully deleted Instance template") + except exceptions.NotFound as e: + # Expecting 404 Error in case if Instance template doesn't exist. + if e.code == 404: + self.log.error("Instance template %s doesn't exist", self.resource_id) + raise e class ComputeEngineCopyInstanceTemplateOperator(ComputeEngineBaseOperator): @@ -355,6 +1140,8 @@ class ComputeEngineCopyInstanceTemplateOperator(ComputeEngineBaseOperator): account from the list granting this role to the originating account (templated). """ + operator_extra_links = (ComputeInstanceTemplateDetailsLink(),) + # [START gce_instance_template_copy_operator_template_fields] template_fields: Sequence[str] = ( "project_id", @@ -391,7 +1178,7 @@ def __init__( self._field_validator = GcpBodyFieldValidator( GCE_INSTANCE_TEMPLATE_VALIDATION_PATCH_SPECIFICATION, api_version=api_version ) - self._field_sanitizer = GcpBodyFieldSanitizer(GCE_INSTANCE_TEMPLATE_FIELDS_TO_SANITIZE) + self._field_sanitizer = GcpBodyFieldSanitizer(GCE_INSTANCE_FIELDS_TO_SANITIZE) super().__init__( project_id=project_id, zone="global", @@ -406,6 +1193,11 @@ def _validate_all_body_fields(self) -> None: if self._field_validator: self._field_validator.validate(self.body_patch) + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id: + raise AirflowException("The required parameter 'resource_id' is missing.") + def execute(self, context: Context) -> dict: hook = ComputeEngineHook( gcp_conn_id=self.gcp_conn_id, @@ -417,31 +1209,53 @@ def execute(self, context: Context) -> dict: # Idempotence check (sort of) - we want to check if the new template # is already created and if is, then we assume it was created by previous run # of CopyTemplate operator - we do not check if content of the template - # is as expected. Templates are immutable so we cannot update it anyway + # is as expected. Templates are immutable, so we cannot update it anyway # and deleting/recreating is not worth the hassle especially # that we cannot delete template if it is already used in some Instance # Group Manager. We assume success if the template is simply present existing_template = hook.get_instance_template( - resource_id=self.body_patch["name"], project_id=self.project_id + resource_id=self.body_patch["name"], + project_id=self.project_id, ) + except exceptions.NotFound as e: + # We actually expect to get 404 / Not Found here as the template should + # not yet exist + if not e.code == 404: + raise e + else: self.log.info( - "The %s template already existed. It was likely created by previous run of the operator. " + "The %s template already exists. It was likely created by previous run of the operator. " "Assuming success.", existing_template, ) - return existing_template - except HttpError as e: - # We actually expect to get 404 / Not Found here as the template should - # not yet exist - if not e.resp.status == 404: - raise e - old_body = hook.get_instance_template(resource_id=self.resource_id, project_id=self.project_id) + ComputeInstanceTemplateDetailsLink.persist( + context=context, + task_instance=self, + resource_id=self.body_patch["name"], + project_id=self.project_id or hook.project_id, + ) + return InstanceTemplate.to_dict(existing_template) + old_body = InstanceTemplate.to_dict( + hook.get_instance_template( + resource_id=self.resource_id, + project_id=self.project_id, + ) + ) new_body = deepcopy(old_body) self._field_sanitizer.sanitize(new_body) new_body = merge(new_body, self.body_patch) self.log.info("Calling insert instance template with updated body: %s", new_body) hook.insert_instance_template(body=new_body, request_id=self.request_id, project_id=self.project_id) - return hook.get_instance_template(resource_id=self.body_patch["name"], project_id=self.project_id) + instance_template = hook.get_instance_template( + resource_id=self.body_patch["name"], project_id=self.project_id + ) + ComputeInstanceTemplateDetailsLink.persist( + context=context, + task_instance=self, + resource_id=self.body_patch["name"], + project_id=self.project_id or hook.project_id, + ) + return InstanceTemplate.to_dict(instance_template) class ComputeEngineInstanceGroupUpdateManagerTemplateOperator(ComputeEngineBaseOperator): @@ -479,6 +1293,8 @@ class ComputeEngineInstanceGroupUpdateManagerTemplateOperator(ComputeEngineBaseO account from the list granting this role to the originating account (templated). """ + operator_extra_links = (ComputeInstanceGroupManagerDetailsLink(),) + # [START gce_igm_update_template_operator_template_fields] template_fields: Sequence[str] = ( "project_id", @@ -530,6 +1346,11 @@ def __init__( **kwargs, ) + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id: + raise AirflowException("The required parameter 'resource_id' is missing. ") + def _possibly_replace_template(self, dictionary: dict) -> None: if dictionary.get("instanceTemplate") == self.source_template: dictionary["instanceTemplate"] = self.destination_template @@ -545,10 +1366,11 @@ def execute(self, context: Context) -> bool | None: zone=self.zone, resource_id=self.resource_id, project_id=self.project_id ) patch_body = {} - if "versions" in old_instance_group_manager: - patch_body["versions"] = old_instance_group_manager["versions"] - if "instanceTemplate" in old_instance_group_manager: - patch_body["instanceTemplate"] = old_instance_group_manager["instanceTemplate"] + igm_dict = InstanceGroupManager.to_dict(old_instance_group_manager) + if "versions" in igm_dict: + patch_body["versions"] = igm_dict["versions"] + if "instanceTemplate" in igm_dict: + patch_body["instanceTemplate"] = igm_dict["instanceTemplate"] if self.update_policy: patch_body["updatePolicy"] = self.update_policy self._possibly_replace_template(patch_body) @@ -557,6 +1379,13 @@ def execute(self, context: Context) -> bool | None: self._possibly_replace_template(version) if self._change_performed or self.update_policy: self.log.info("Calling patch instance template with updated body: %s", patch_body) + ComputeInstanceGroupManagerDetailsLink.persist( + context=context, + task_instance=self, + location_id=self.zone, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) return hook.patch_instance_group_manager( zone=self.zone, resource_id=self.resource_id, @@ -566,4 +1395,295 @@ def execute(self, context: Context) -> bool | None: ) else: # Idempotence achieved + ComputeInstanceGroupManagerDetailsLink.persist( + context=context, + task_instance=self, + location_id=self.zone, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) return True + + +class ComputeEngineInsertInstanceGroupManagerOperator(ComputeEngineBaseOperator): + """ + Creates an Instance Group Managers using the body specified. + After the group is created, instances in the group are created using the specified Instance Template. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:ComputeEngineInsertInstanceGroupManagerOperator` + + :param body: Instance Group Managers representation as object. + :param project_id: Google Cloud project ID where the Compute Engine Instance Group Managers exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new Instance Group Managers again) + It should be in UUID format as defined in RFC 4122 + :param resource_id: Name of the Instance Group Managers. If the name of Instance Group Managers is + not specified in body['name'], the name will be taken from 'resource_id' parameter. + :param gcp_conn_id: The connection ID used to connect to Google Cloud. Defaults to 'google_cloud_default'. + :param api_version: API version used (for example v1 - or beta). Defaults to v1. + :param impersonation_chain: Service account to impersonate using short-term + credentials, or chained list of accounts required to get the access_token + of the last account in the list, which will be impersonated in the request. + If set as a string, the account must grant the originating account + the Service Account Token Creator IAM role. + If set as a sequence, the identities from the list must grant + Service Account Token Creator IAM role to the directly preceding identity, with first + account from the list granting this role to the originating account (templated). + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + + operator_extra_links = (ComputeInstanceGroupManagerDetailsLink(),) + + # [START gce_igm_insert_fields] + template_fields: Sequence[str] = ( + "project_id", + "body", + "zone", + "request_id", + "gcp_conn_id", + "api_version", + "impersonation_chain", + ) + # [END gce_igm_insert_fields] + + def __init__( + self, + *, + body: dict, + zone: str, + project_id: str | None = None, + resource_id: str | None = None, + request_id: str | None = None, + gcp_conn_id: str = "google_cloud_default", + api_version="v1", + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + impersonation_chain: str | Sequence[str] | None = None, + validate_body: bool = True, + **kwargs, + ) -> None: + self.body = body + self.zone = zone + self.request_id = request_id + self.resource_id = self.body["name"] if "name" in body else resource_id + self._field_validator = None # Optional[GcpBodyFieldValidator] + self.retry = retry + self.timeout = timeout + self.metadata = metadata + if validate_body: + self._field_validator = GcpBodyFieldValidator( + GCE_INSTANCE_TEMPLATE_VALIDATION_PATCH_SPECIFICATION, api_version=api_version + ) + self._field_sanitizer = GcpBodyFieldSanitizer(GCE_INSTANCE_FIELDS_TO_SANITIZE) + super().__init__( + project_id=project_id, + zone=zone, + resource_id=self.resource_id, + gcp_conn_id=gcp_conn_id, + api_version=api_version, + impersonation_chain=impersonation_chain, + **kwargs, + ) + + def check_body_fields(self) -> None: + required_params = ["base_instance_name", "target_size", "instance_template"] + for param in required_params: + if param in self.body: + continue + readable_param = param.replace("_", " ") + raise AirflowException( + f"The body '{self.body}' should contain at least {readable_param} for the new operator " + f"in the '{param}' field. Check (google.cloud.compute_v1.types.Instance) " + f"for more details about body fields description." + ) + + def _validate_all_body_fields(self) -> None: + if self._field_validator: + self._field_validator.validate(self.body) + + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id and "name" not in self.body: + raise AirflowException( + "The required parameters 'resource_id' and body['name'] are missing. " + "Please, provide at least one of them." + ) + + def execute(self, context: Context) -> dict: + hook = ComputeEngineHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + impersonation_chain=self.impersonation_chain, + ) + self._validate_all_body_fields() + self.check_body_fields() + try: + # Idempotence check (sort of) - we want to check if the new Instance Group Manager + # is already created and if isn't, we create new one + existing_instance_group_manager = hook.get_instance_group_manager( + resource_id=self.resource_id, + project_id=self.project_id, + zone=self.zone, + ) + except exceptions.NotFound as e: + # We actually expect to get 404 / Not Found here as the Instance Group Manager should + # not yet exist + if not e.code == 404: + raise e + else: + self.log.info("The %s Instance Group Manager already exists", existing_instance_group_manager) + ComputeInstanceGroupManagerDetailsLink.persist( + context=context, + task_instance=self, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + location_id=self.zone, + ) + return InstanceGroupManager.to_dict(existing_instance_group_manager) + self._field_sanitizer.sanitize(self.body) + self.log.info("Creating Instance Group Manager with specified body: %s", self.body) + hook.insert_instance_group_manager( + body=self.body, + request_id=self.request_id, + project_id=self.project_id, + zone=self.zone, + ) + self.log.info("The specified Instance Group Manager has been created SUCCESSFULLY", self.body) + new_instance_group_manager = hook.get_instance_group_manager( + resource_id=self.resource_id, + project_id=self.project_id, + zone=self.zone, + ) + ComputeInstanceGroupManagerDetailsLink.persist( + context=context, + task_instance=self, + location_id=self.zone, + resource_id=self.resource_id, + project_id=self.project_id or hook.project_id, + ) + return InstanceGroupManager.to_dict(new_instance_group_manager) + + +class ComputeEngineDeleteInstanceGroupManagerOperator(ComputeEngineBaseOperator): + """ + Deletes an Instance Group Managers. + Deleting an Instance Group Manager is permanent and cannot be undone. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:ComputeEngineDeleteInstanceGroupManagerOperator` + + :param resource_id: Name of the Instance Group Managers. + :param project_id: Google Cloud project ID where the Compute Engine Instance Group Managers exists. + If set to None or missing, the default project_id from the Google Cloud connection is used. + :param request_id: Unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new Instance Group Managers again) + It should be in UUID format as defined in RFC 4122 + :param gcp_conn_id: The connection ID used to connect to Google Cloud. Defaults to 'google_cloud_default'. + :param api_version: API version used (for example v1 - or beta). Defaults to v1. + :param impersonation_chain: Service account to impersonate using short-term + credentials, or chained list of accounts required to get the access_token + of the last account in the list, which will be impersonated in the request. + If set as a string, the account must grant the originating account + the Service Account Token Creator IAM role. + If set as a sequence, the identities from the list must grant + Service Account Token Creator IAM role to the directly preceding identity, with first + account from the list granting this role to the originating account (templated). + :param retry: A retry object used to retry requests. If `None` is specified, requests + will not be retried. + :param timeout: The amount of time, in seconds, to wait for the request to complete. + Note that if `retry` is specified, the timeout applies to each individual attempt. + :param metadata: Additional metadata that is provided to the method. + """ + + # [START gce_igm_delete_fields] + template_fields: Sequence[str] = ( + "project_id", + "resource_id", + "zone", + "request_id", + "gcp_conn_id", + "api_version", + "impersonation_chain", + ) + # [END gce_igm_delete_fields] + + def __init__( + self, + *, + resource_id: str, + zone: str, + project_id: str | None = None, + request_id: str | None = None, + gcp_conn_id: str = "google_cloud_default", + api_version="v1", + retry: Retry | None = None, + timeout: float | None = None, + metadata: Sequence[tuple[str, str]] = (), + impersonation_chain: str | Sequence[str] | None = None, + validate_body: bool = True, + **kwargs, + ) -> None: + self.zone = zone + self.request_id = request_id + self.resource_id = resource_id + self._field_validator = None # Optional[GcpBodyFieldValidator] + self.retry = retry + self.timeout = timeout + self.metadata = metadata + if validate_body: + self._field_validator = GcpBodyFieldValidator( + GCE_INSTANCE_TEMPLATE_VALIDATION_PATCH_SPECIFICATION, api_version=api_version + ) + self._field_sanitizer = GcpBodyFieldSanitizer(GCE_INSTANCE_FIELDS_TO_SANITIZE) + super().__init__( + project_id=project_id, + zone=zone, + resource_id=resource_id, + gcp_conn_id=gcp_conn_id, + api_version=api_version, + impersonation_chain=impersonation_chain, + **kwargs, + ) + + def _validate_inputs(self) -> None: + super()._validate_inputs() + if not self.resource_id: + raise AirflowException("The required parameter 'resource_id' is missing. ") + + def execute(self, context: Context): + hook = ComputeEngineHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + impersonation_chain=self.impersonation_chain, + ) + try: + # Checking if specified Instance Group Managers exists and if it does, delete it + hook.get_instance_group_manager( + resource_id=self.resource_id, + project_id=self.project_id, + zone=self.zone, + ) + self.log.info("Successfully found Group Manager %s", self.resource_id) + hook.delete_instance_group_manager( + resource_id=self.resource_id, + project_id=self.project_id, + request_id=self.request_id, + zone=self.zone, + ) + self.log.info("Successfully deleted Instance Group Managers") + except exceptions.NotFound as e: + # Expecting 404 Error in case if Instance Group Managers doesn't exist. + if e.code == 404: + self.log.error("Instance Group Managers %s doesn't exist", self.resource_id) + raise e diff --git a/airflow/providers/google/provider.yaml b/airflow/providers/google/provider.yaml index dad775d8b35b39..3533fba5f6b0a1 100644 --- a/airflow/providers/google/provider.yaml +++ b/airflow/providers/google/provider.yaml @@ -79,6 +79,7 @@ dependencies: - google-cloud-bigquery-datatransfer>=3.0.0 - google-cloud-bigtable>=1.0.0,<2.0.0 - google-cloud-build>=3.0.0 + - google-cloud-compute>=0.1.0,<2.0.0 - google-cloud-container>=2.2.0,<3.0.0 - google-cloud-dataform>=0.2.0 - google-cloud-datacatalog>=3.0.0 @@ -972,6 +973,9 @@ extra-links: - airflow.providers.google.cloud.links.bigquery.BigQueryDatasetLink - airflow.providers.google.cloud.links.bigquery.BigQueryTableLink - airflow.providers.google.cloud.links.bigquery_dts.BigQueryDataTransferConfigLink + - airflow.providers.google.cloud.links.compute.ComputeInstanceDetailsLink + - airflow.providers.google.cloud.links.compute.ComputeInstanceTemplateDetailsLink + - airflow.providers.google.cloud.links.compute.ComputeInstanceGroupManagerDetailsLink - airflow.providers.google.cloud.links.cloud_tasks.CloudTasksQueueLink - airflow.providers.google.cloud.links.cloud_tasks.CloudTasksLink - airflow.providers.google.cloud.links.datacatalog.DataCatalogEntryGroupLink diff --git a/docs/apache-airflow-providers-google/index.rst b/docs/apache-airflow-providers-google/index.rst index 2d870105c2a83d..400b3a45465e6f 100644 --- a/docs/apache-airflow-providers-google/index.rst +++ b/docs/apache-airflow-providers-google/index.rst @@ -113,6 +113,7 @@ PIP package Version required ``google-cloud-bigquery-datatransfer`` ``>=3.0.0`` ``google-cloud-bigtable`` ``>=1.0.0,<2.0.0`` ``google-cloud-build`` ``>=3.0.0`` +``google-cloud-compute`` ``>=1.4.0,<2.0.0`` ``google-cloud-container`` ``>=2.2.0,<3.0.0`` ``google-cloud-dataform`` ``>=0.2.0`` ``google-cloud-datacatalog`` ``>=3.0.0`` diff --git a/docs/apache-airflow-providers-google/operators/cloud/compute.rst b/docs/apache-airflow-providers-google/operators/cloud/compute.rst index c5163c535ed30d..a5a9433778a802 100644 --- a/docs/apache-airflow-providers-google/operators/cloud/compute.rst +++ b/docs/apache-airflow-providers-google/operators/cloud/compute.rst @@ -25,6 +25,133 @@ Prerequisite Tasks .. include::/operators/_partials/prerequisite_tasks.rst +.. _howto/operator:ComputeEngineInsertInstanceOperator: + +ComputeEngineInsertInstanceOperator +----------------------------------- + +Use the +:class:`~airflow.providers.google.cloud.operators.compute.ComputeEngineInsertInstanceOperator` +to create new Google Compute Engine instance. + +Using the operator +"""""""""""""""""" + +The code to create the operator: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_insert] + :end-before: [END howto_operator_gce_insert] + +You can also create the operator without project id - project id will be retrieved +from the Google Cloud connection id used: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_insert_no_project_id] + :end-before: [END howto_operator_gce_insert_no_project_id] + + +Templating +"""""""""" + +.. literalinclude:: /../../airflow/providers/google/cloud/operators/compute.py + :language: python + :dedent: 4 + :start-after: [START gce_instance_insert_fields] + :end-before: [END gce_instance_insert_fields] + +More information +"""""""""""""""" + +See Google Compute Engine API documentation to `insert an instance +`_. + +.. _howto/operator:ComputeEngineInsertInstanceFromTemplateOperator: + +ComputeEngineInsertInstanceFromTemplateOperator +----------------------------------------------- + +Use the +:class:`~airflow.providers.google.cloud.operators.compute.ComputeEngineInsertInstanceFromTemplateOperator` +to create new Google Compute Engine instance based on specified instance template. + +Using the operator +"""""""""""""""""" + +The code to create the operator: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_insert_from_template] + :end-before: [END howto_operator_gce_insert_from_template] + +You can also create the operator without project id - project id will be retrieved +from the Google Cloud connection id used: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_insert_from_template_no_project_id] + :end-before: [END howto_operator_gce_insert_from_template_no_project_id] + + +Templating +"""""""""" + +.. literalinclude:: /../../airflow/providers/google/cloud/operators/compute.py + :language: python + :dedent: 4 + :start-after: [START gce_instance_insert_from_template_fields] + :end-before: [END gce_instance_insert_from_template_fields] + +More information +"""""""""""""""" + +See Google Compute Engine API documentation to `insert an instance from template +`_. + +.. _howto/operator:ComputeEngineDeleteInstanceOperator: + +ComputeEngineDeleteInstanceOperator +----------------------------------- + +Use the +:class:`~airflow.providers.google.cloud.operators.compute.ComputeEngineDeleteInstanceOperator` +to delete an existing Google Compute Engine instance. + +Using the operator +"""""""""""""""""" + +You can create the operator without project id - project id will be retrieved +from the Google Cloud connection id used. The code to create the operator: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_delete_no_project_id] + :end-before: [END howto_operator_gce_delete_no_project_id] + + +Templating +"""""""""" + +.. literalinclude:: /../../airflow/providers/google/cloud/operators/compute.py + :language: python + :dedent: 4 + :start-after: [START gce_instance_delete_template_fields] + :end-before: [END gce_instance_delete_template_fields] + +More information +"""""""""""""""" + +See Google Compute Engine API documentation to `delete an instance +`_. + .. _howto/operator:ComputeEngineStartInstanceOperator: ComputeEngineStartInstanceOperator @@ -39,7 +166,7 @@ Using the operator The code to create the operator: -.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_compute.py +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py :language: python :dedent: 4 :start-after: [START howto_operator_gce_start] @@ -48,7 +175,7 @@ The code to create the operator: You can also create the operator without project id - project id will be retrieved from the Google Cloud connection id used: -.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_compute.py +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py :language: python :dedent: 4 :start-after: [START howto_operator_gce_start_no_project_id] @@ -85,7 +212,7 @@ Using the operator The code to create the operator: -.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_compute.py +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py :language: python :dedent: 4 :start-after: [START howto_operator_gce_stop] @@ -94,7 +221,7 @@ The code to create the operator: You can also create the operator without project id - project id will be retrieved from the Google Cloud connection used: -.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_compute.py +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py :language: python :dedent: 4 :start-after: [START howto_operator_gce_stop_no_project_id] @@ -135,7 +262,7 @@ Using the operator The code to create the operator: -.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_compute.py +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py :language: python :dedent: 4 :start-after: [START howto_operator_gce_set_machine_type] @@ -144,7 +271,7 @@ The code to create the operator: You can also create the operator without project id - project id will be retrieved from the Google Cloud connection used: -.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_compute.py +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute.py :language: python :dedent: 4 :start-after: [START howto_operator_gce_set_machine_type_no_project_id] @@ -165,6 +292,90 @@ More information See Google Compute Engine API documentation to `set the machine type `_. +.. _howto/operator:ComputeEngineDeleteInstanceTemplateOperator: + +ComputeEngineDeleteInstanceTemplateOperator +------------------------------------------- + +Use the operator to delete Google Compute Engine instance template. + +For parameter definition, take a look at +:class:`~airflow.providers.google.cloud.operators.compute.ComputeEngineDeleteInstanceTemplateOperator`. + +Using the operator +"""""""""""""""""" + +The code to create the operator: + +You can create the operator without project id - project id will be retrieved +from the Google Cloud connection used. The code to create the operator: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute_igm.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_delete_old_template_no_project_id] + :end-before: [END howto_operator_gce_delete_old_template_no_project_id] + +Templating +"""""""""" + +.. literalinclude:: /../../airflow/providers/google/cloud/operators/compute.py + :language: python + :dedent: 4 + :start-after: [START gce_instance_template_delete_fields] + :end-before: [END gce_instance_template_delete_fields] + +More information +"""""""""""""""" + +See Google Compute Engine API documentation to `delete a template +`_. + +.. _howto/operator:ComputeEngineInsertInstanceTemplateOperator: + +ComputeEngineInsertInstanceTemplateOperator +------------------------------------------- + +Use the operator to create Google Compute Engine instance template. + +For parameter definition, take a look at +:class:`~airflow.providers.google.cloud.operators.compute.ComputeEngineInsertInstanceTemplateOperator`. + +Using the operator +"""""""""""""""""" + +The code to create the operator: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute_igm.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_igm_insert_template] + :end-before: [END howto_operator_gce_igm_insert_template] + +You can also create the operator without project id - project id will be retrieved +from the Google Cloud connection used: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute_igm.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_igm_insert_template_no_project_id] + :end-before: [END howto_operator_gce_igm_insert_template_no_project_id] + +Templating +"""""""""" + +.. literalinclude:: /../../airflow/providers/google/cloud/operators/compute.py + :language: python + :dedent: 4 + :start-after: [START gce_instance_template_insert_fields] + :end-before: [END gce_instance_template_insert_fields] + +More information +"""""""""""""""" + +See Google Compute Engine API documentation to `create a new template +`_. + .. _howto/operator:ComputeEngineCopyInstanceTemplateOperator: ComputeEngineCopyInstanceTemplateOperator @@ -181,12 +392,7 @@ Using the operator The code to create the operator: -.. exampleinclude:: /../../tests/system/providers/google/cloud/compute_igm/example_compute_igm.py - :language: python - :start-after: [START howto_operator_compute_template_copy_args] - :end-before: [END howto_operator_compute_template_copy_args] - -.. exampleinclude:: /../../tests/system/providers/google/cloud/compute_igm/example_compute_igm.py +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute_igm.py :language: python :dedent: 4 :start-after: [START howto_operator_gce_igm_copy_template] @@ -195,7 +401,7 @@ The code to create the operator: You can also create the operator without project id - project id will be retrieved from the Google Cloud connection used: -.. exampleinclude:: /../../tests/system/providers/google/cloud/compute_igm/example_compute_igm.py +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute_igm.py :language: python :dedent: 4 :start-after: [START howto_operator_gce_igm_copy_template_no_project_id] @@ -216,6 +422,97 @@ More information See Google Compute Engine API documentation to `create a new instance with an existing template `_. +.. _howto/operator:ComputeEngineInsertInstanceGroupManagerOperator: + +ComputeEngineInsertInstanceGroupManagerOperator +----------------------------------------------- + +Use the operator to create a Compute Engine Instance Group Manager. + +For parameter definition, take a look at +:class:`~airflow.providers.google.cloud.operators.compute.ComputeEngineInsertInstanceGroupManagerOperator`. + +Arguments +""""""""" + + +Using the operator +"""""""""""""""""" + +The code to create the operator: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute_igm.py + :language: python + :start-after: [START howto_operator_gce_insert_igm] + :end-before: [END howto_operator_gce_insert_igm] + +You can also create the operator without project id - project id will be retrieved +from the Google Cloud connection used: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute_igm.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_insert_igm_no_project_id] + :end-before: [END howto_operator_gce_insert_igm_no_project_id] + + +Templating +"""""""""" + +.. literalinclude:: /../../airflow/providers/google/cloud/operators/compute.py + :language: python + :dedent: 4 + :start-after: [START gce_igm_insert_fields] + :end-before: [END gce_igm_insert_fields] + +More information +"""""""""""""""" + +See Google Compute Engine API documentation to `create a group instance +`_. + +.. _howto/operator:ComputeEngineDeleteInstanceGroupManagerOperator: + +ComputeEngineDeleteInstanceGroupManagerOperator +----------------------------------------------- + +Use the operator to delete a Compute Engine Instance Group Manager. + +For parameter definition, take a look at +:class:`~airflow.providers.google.cloud.operators.compute.ComputeEngineDeleteInstanceGroupManagerOperator`. + +Arguments +""""""""" + + +Using the operator +"""""""""""""""""" + +You can create the operator without project id - project id will be retrieved +from the Google Cloud connection used. The code to create the operator: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute_igm.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_delete_igm_no_project_id] + :end-before: [END howto_operator_gce_delete_igm_no_project_id] + + +Templating +"""""""""" + +.. literalinclude:: /../../airflow/providers/google/cloud/operators/compute.py + :language: python + :dedent: 4 + :start-after: [START gce_igm_delete_fields] + :end-before: [END gce_igm_delete_fields] + +More information +"""""""""""""""" + +See Google Compute Engine API documentation to `delete a group instance +`_. + .. _howto/operator:ComputeEngineInstanceGroupUpdateManagerTemplateOperator: ComputeEngineInstanceGroupUpdateManagerTemplateOperator @@ -235,12 +532,7 @@ Using the operator The code to create the operator: -.. exampleinclude:: /../../tests/system/providers/google/cloud/compute_igm/example_compute_igm.py - :language: python - :start-after: [START howto_operator_compute_igm_update_template_args] - :end-before: [END howto_operator_compute_igm_update_template_args] - -.. exampleinclude:: /../../tests/system/providers/google/cloud/compute_igm/example_compute_igm.py +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute_igm.py :language: python :dedent: 4 :start-after: [START howto_operator_gce_igm_update_template] @@ -249,7 +541,7 @@ The code to create the operator: You can also create the operator without project id - project id will be retrieved from the Google Cloud connection used: -.. exampleinclude:: /../../tests/system/providers/google/cloud/compute_igm/example_compute_igm.py +.. exampleinclude:: /../../tests/system/providers/google/cloud/compute/example_compute_igm.py :language: python :dedent: 4 :start-after: [START howto_operator_gce_igm_update_template_no_project_id] diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index 74b527b5ec7698..c07f4d795bec22 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -257,6 +257,8 @@ colours CommandType comparator compat +Compute +compute Computenodes ComputeNodeState concat @@ -425,6 +427,7 @@ DirectRunner dirs discoverability discoverable +diskTypes displayName distcp distro @@ -863,6 +866,7 @@ Lowin lshift lxml Lyft +machineTypes macOS macosx mailto @@ -1377,6 +1381,7 @@ submodules subnet subnets subnetwork +subnetworks subpackage Subpackages subpackages diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json index afa144de0b76c7..e408ef7e37a354 100644 --- a/generated/provider_dependencies.json +++ b/generated/provider_dependencies.json @@ -316,6 +316,7 @@ "google-cloud-bigquery-datatransfer>=3.0.0", "google-cloud-bigtable>=1.0.0,<2.0.0", "google-cloud-build>=3.0.0", + "google-cloud-compute>=0.1.0,<2.0.0", "google-cloud-container>=2.2.0,<3.0.0", "google-cloud-datacatalog>=3.0.0", "google-cloud-dataform>=0.2.0", diff --git a/tests/providers/google/cloud/hooks/test_compute.py b/tests/providers/google/cloud/hooks/test_compute.py index b4a9a6c9224aae..c8e22d51a1d9cf 100644 --- a/tests/providers/google/cloud/hooks/test_compute.py +++ b/tests/providers/google/cloud/hooks/test_compute.py @@ -22,6 +22,7 @@ from unittest.mock import PropertyMock import pytest +from google.api_core.retry import Retry from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.compute import ComputeEngineHook, GceOperationStatus @@ -37,6 +38,471 @@ GCE_REQUEST_ID = "request_id" GCE_INSTANCE_GROUP_MANAGER = "instance_group_manager" +PROJECT_ID = "project-id" +RESOURCE_ID = "resource-id" +BODY = {"body": "test"} +ZONE = "zone" +SOURCE_INSTANCE_TEMPLATE = "source-instance-template" +TIMEOUT = 120 +RETRY = mock.MagicMock(Retry) +METADATA = [("key", "value")] + +GCP_CONN_ID = "test-conn" +IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3@google.com"] +API_VERSION = "v1" + +BASE_STRING = "airflow.providers.google.common.hooks.base_google.{}" +COMPUTE_ENGINE_HOOK_PATH = "airflow.providers.google.cloud.hooks.compute.{}" + + +class TestGcpComputeHookApiCall(unittest.TestCase): + def setUp(self): + with mock.patch( + BASE_STRING.format("GoogleBaseHook.__init__"), + new=mock_base_gcp_hook_default_project_id, + ): + self.hook = ComputeEngineHook( + gcp_conn_id=GCP_CONN_ID, + api_version=API_VERSION, + impersonation_chain=IMPERSONATION_CHAIN, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_template_client")) + def test_insert_template_should_execute_successfully(self, mock_client): + self.hook.insert_instance_template( + project_id=PROJECT_ID, + body=BODY, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.insert.assert_called_once_with( + request=dict( + instance_template_resource=BODY, + project=PROJECT_ID, + request_id=None, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", + new_callable=PropertyMock, + return_value="mocked-google", + ) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_template_client")) + def test_insert_template_should_not_throw_ex_when_project_id_none(self, mock_client, mocked_project_id): + self.hook.insert_instance_template( + body=BODY, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.insert.assert_called_once_with( + request=dict( + instance_template_resource=BODY, + request_id=None, + project="mocked-google", + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_template_client")) + def test_delete_template_should_execute_successfully(self, mock_client): + self.hook.delete_instance_template( + project_id=PROJECT_ID, + resource_id=RESOURCE_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.delete.assert_called_once_with( + request=dict( + instance_template=RESOURCE_ID, + project=PROJECT_ID, + request_id=None, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", + new_callable=PropertyMock, + return_value="mocked-google", + ) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_template_client")) + def test_delete_template_should_not_throw_ex_when_project_id_none(self, mock_client, mocked_project_id): + self.hook.delete_instance_template( + resource_id=RESOURCE_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.delete.assert_called_once_with( + request=dict( + instance_template=RESOURCE_ID, + project="mocked-google", + request_id=None, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_template_client")) + def test_get_template_should_execute_successfully(self, mock_client): + self.hook.get_instance_template( + project_id=PROJECT_ID, + resource_id=RESOURCE_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.get.assert_called_once_with( + request=dict( + instance_template=RESOURCE_ID, + project=PROJECT_ID, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", + new_callable=PropertyMock, + return_value="mocked-google", + ) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_template_client")) + def test_get_template_should_not_throw_ex_when_project_id_none(self, mock_client, mocked_project_id): + self.hook.get_instance_template( + resource_id=RESOURCE_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.get.assert_called_once_with( + request=dict( + instance_template=RESOURCE_ID, + project="mocked-google", + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_client")) + def test_insert_instance_should_execute_successfully(self, mock_client): + self.hook.insert_instance( + project_id=PROJECT_ID, + body=BODY, + zone=ZONE, + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.insert.assert_called_once_with( + request=dict( + instance_resource=BODY, + project=PROJECT_ID, + request_id=None, + zone=ZONE, + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", + new_callable=PropertyMock, + return_value="mocked-google", + ) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_client")) + def test_insert_instance_should_not_throw_ex_when_project_id_none(self, mock_client, mocked_project_id): + self.hook.insert_instance( + body=BODY, + zone=ZONE, + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.insert.assert_called_once_with( + request=dict( + instance_resource=BODY, + project="mocked-google", + request_id=None, + zone=ZONE, + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_client")) + def test_get_instance_should_execute_successfully(self, mock_client): + self.hook.get_instance( + resource_id=RESOURCE_ID, + zone=ZONE, + project_id=PROJECT_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.get.assert_called_once_with( + request=dict( + instance=RESOURCE_ID, + project=PROJECT_ID, + zone=ZONE, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", + new_callable=PropertyMock, + return_value="mocked-google", + ) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_client")) + def test_get_instance_should_not_throw_ex_when_project_id_none(self, mock_client, mocked_project_id): + self.hook.get_instance( + resource_id=RESOURCE_ID, + zone=ZONE, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.get.assert_called_once_with( + request=dict( + instance=RESOURCE_ID, + project="mocked-google", + zone=ZONE, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_client")) + def test_delete_instance_should_execute_successfully(self, mock_client): + self.hook.delete_instance( + resource_id=RESOURCE_ID, + zone=ZONE, + project_id=PROJECT_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.delete.assert_called_once_with( + request=dict( + instance=RESOURCE_ID, + project=PROJECT_ID, + request_id=None, + zone=ZONE, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", + new_callable=PropertyMock, + return_value="mocked-google", + ) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_client")) + def test_delete_instance_should_not_throw_ex_when_project_id_none(self, mock_client, mocked_project_id): + self.hook.delete_instance( + resource_id=RESOURCE_ID, + zone=ZONE, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.delete.assert_called_once_with( + request=dict( + instance=RESOURCE_ID, + project="mocked-google", + request_id=None, + zone=ZONE, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_group_managers_client") + ) + def test_insert_instance_group_manager_should_execute_successfully(self, mock_client): + self.hook.insert_instance_group_manager( + body=BODY, + zone=ZONE, + project_id=PROJECT_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.insert.assert_called_once_with( + request=dict( + instance_group_manager_resource=BODY, + project=PROJECT_ID, + request_id=None, + zone=ZONE, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", + new_callable=PropertyMock, + return_value="mocked-google", + ) + @mock.patch( + COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_group_managers_client") + ) + def test_insert_instance_group_manager_should_not_throw_ex_when_project_id_none( + self, mock_client, mocked_project_id + ): + self.hook.insert_instance_group_manager( + body=BODY, + zone=ZONE, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.insert.assert_called_once_with( + request=dict( + instance_group_manager_resource=BODY, + project="mocked-google", + request_id=None, + zone=ZONE, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_group_managers_client") + ) + def test_get_instance_group_manager_should_execute_successfully(self, mock_client): + self.hook.get_instance_group_manager( + resource_id=RESOURCE_ID, + zone=ZONE, + project_id=PROJECT_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.get.assert_called_once_with( + request=dict( + instance_group_manager=RESOURCE_ID, + project=PROJECT_ID, + zone=ZONE, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", + new_callable=PropertyMock, + return_value="mocked-google", + ) + @mock.patch( + COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_group_managers_client") + ) + def test_get_instance_group_manager_should_not_throw_ex_when_project_id_none( + self, mock_client, mocked_project_id + ): + self.hook.get_instance_group_manager( + resource_id=RESOURCE_ID, + zone=ZONE, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.get.assert_called_once_with( + request=dict( + instance_group_manager=RESOURCE_ID, + project="mocked-google", + zone=ZONE, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_group_managers_client") + ) + def test_delete_instance_group_manager_should_execute_successfully(self, mock_client): + self.hook.delete_instance_group_manager( + resource_id=RESOURCE_ID, + zone=ZONE, + project_id=PROJECT_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.delete.assert_called_once_with( + request=dict( + instance_group_manager=RESOURCE_ID, + project=PROJECT_ID, + zone=ZONE, + request_id=None, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + + @mock.patch( + "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", + new_callable=PropertyMock, + return_value="mocked-google", + ) + @mock.patch( + COMPUTE_ENGINE_HOOK_PATH.format("ComputeEngineHook.get_compute_instance_group_managers_client") + ) + def test_delete_instance_group_manager_should_not_throw_ex_when_project_id_none( + self, mock_client, mocked_project_id + ): + self.hook.delete_instance_group_manager( + resource_id=RESOURCE_ID, + zone=ZONE, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + mock_client.return_value.delete.assert_called_once_with( + request=dict( + instance_group_manager=RESOURCE_ID, + project="mocked-google", + zone=ZONE, + request_id=None, + ), + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + ) + class TestGcpComputeHookNoDefaultProjectId(unittest.TestCase): def setUp(self): @@ -114,61 +580,6 @@ def test_set_machine_type_overridden_project_id(self, wait_for_operation_to_comp project_id="example-project", operation_name="operation_id", zone="zone" ) - @mock.patch("airflow.providers.google.cloud.hooks.compute.ComputeEngineHook.get_conn") - @mock.patch( - "airflow.providers.google.cloud.hooks.compute.ComputeEngineHook._wait_for_operation_to_complete" - ) - def test_get_instance_template_overridden_project_id(self, wait_for_operation_to_complete, get_conn): - get_method = get_conn.return_value.instanceTemplates.return_value.get - execute_method = get_method.return_value.execute - execute_method.return_value = {"name": "operation_id"} - wait_for_operation_to_complete.return_value = None - res = self.gce_hook_no_project_id.get_instance_template( - resource_id=GCE_INSTANCE_TEMPLATE, project_id="example-project" - ) - assert res is not None - get_method.assert_called_once_with(instanceTemplate="instance-template", project="example-project") - execute_method.assert_called_once_with(num_retries=5) - wait_for_operation_to_complete.assert_not_called() - - @mock.patch("airflow.providers.google.cloud.hooks.compute.ComputeEngineHook.get_conn") - @mock.patch( - "airflow.providers.google.cloud.hooks.compute.ComputeEngineHook._wait_for_operation_to_complete" - ) - def test_insert_instance_template_overridden_project_id(self, wait_for_operation_to_complete, get_conn): - insert_method = get_conn.return_value.instanceTemplates.return_value.insert - execute_method = insert_method.return_value.execute - execute_method.return_value = {"name": "operation_id"} - wait_for_operation_to_complete.return_value = None - res = self.gce_hook_no_project_id.insert_instance_template( - project_id=GCP_PROJECT_ID_HOOK_UNIT_TEST, body={}, request_id=GCE_REQUEST_ID - ) - assert res is None - insert_method.assert_called_once_with(body={}, project="example-project", requestId="request_id") - execute_method.assert_called_once_with(num_retries=5) - wait_for_operation_to_complete.assert_called_once_with( - project_id="example-project", operation_name="operation_id" - ) - - @mock.patch("airflow.providers.google.cloud.hooks.compute.ComputeEngineHook.get_conn") - @mock.patch( - "airflow.providers.google.cloud.hooks.compute.ComputeEngineHook._wait_for_operation_to_complete" - ) - def test_get_instance_group_manager_overridden_project_id(self, wait_for_operation_to_complete, get_conn): - get_method = get_conn.return_value.instanceGroupManagers.return_value.get - execute_method = get_method.return_value.execute - execute_method.return_value = {"name": "operation_id"} - wait_for_operation_to_complete.return_value = None - res = self.gce_hook_no_project_id.get_instance_group_manager( - project_id=GCP_PROJECT_ID_HOOK_UNIT_TEST, zone=GCE_ZONE, resource_id=GCE_INSTANCE_GROUP_MANAGER - ) - assert res is not None - get_method.assert_called_once_with( - instanceGroupManager="instance_group_manager", project="example-project", zone="zone" - ) - execute_method.assert_called_once_with(num_retries=5) - wait_for_operation_to_complete.assert_not_called() - @mock.patch("airflow.providers.google.cloud.hooks.compute.ComputeEngineHook.get_conn") @mock.patch( "airflow.providers.google.cloud.hooks.compute.ComputeEngineHook._wait_for_operation_to_complete" @@ -337,133 +748,6 @@ def test_set_machine_type_instance_overridden_project_id(self, wait_for_operatio project_id="new-project", operation_name="operation_id", zone="zone" ) - @mock.patch( - "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", - new_callable=PropertyMock, - return_value=GCP_PROJECT_ID_HOOK_UNIT_TEST, - ) - @mock.patch("airflow.providers.google.cloud.hooks.compute.ComputeEngineHook.get_conn") - @mock.patch( - "airflow.providers.google.cloud.hooks.compute.ComputeEngineHook._wait_for_operation_to_complete" - ) - def test_get_instance_template(self, wait_for_operation_to_complete, get_conn, mock_project_id): - get_method = get_conn.return_value.instanceTemplates.return_value.get - execute_method = get_method.return_value.execute - execute_method.return_value = {"name": "operation_id"} - wait_for_operation_to_complete.return_value = None - res = self.gce_hook.get_instance_template( - resource_id=GCE_INSTANCE_TEMPLATE, project_id=GCP_PROJECT_ID_HOOK_UNIT_TEST - ) - assert res is not None - get_method.assert_called_once_with(instanceTemplate="instance-template", project="example-project") - execute_method.assert_called_once_with(num_retries=5) - wait_for_operation_to_complete.assert_not_called() - - @mock.patch("airflow.providers.google.cloud.hooks.compute.ComputeEngineHook.get_conn") - @mock.patch( - "airflow.providers.google.cloud.hooks.compute.ComputeEngineHook._wait_for_operation_to_complete" - ) - def test_get_instance_template_overridden_project_id(self, wait_for_operation_to_complete, get_conn): - get_method = get_conn.return_value.instanceTemplates.return_value.get - execute_method = get_method.return_value.execute - execute_method.return_value = {"name": "operation_id"} - wait_for_operation_to_complete.return_value = None - res = self.gce_hook.get_instance_template(project_id="new-project", resource_id=GCE_INSTANCE_TEMPLATE) - assert res is not None - get_method.assert_called_once_with(instanceTemplate="instance-template", project="new-project") - execute_method.assert_called_once_with(num_retries=5) - wait_for_operation_to_complete.assert_not_called() - - @mock.patch( - "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", - new_callable=PropertyMock, - return_value=GCP_PROJECT_ID_HOOK_UNIT_TEST, - ) - @mock.patch("airflow.providers.google.cloud.hooks.compute.ComputeEngineHook.get_conn") - @mock.patch( - "airflow.providers.google.cloud.hooks.compute.ComputeEngineHook._wait_for_operation_to_complete" - ) - def test_insert_instance_template(self, wait_for_operation_to_complete, get_conn, mock_project_id): - insert_method = get_conn.return_value.instanceTemplates.return_value.insert - execute_method = insert_method.return_value.execute - execute_method.return_value = {"name": "operation_id"} - wait_for_operation_to_complete.return_value = None - res = self.gce_hook.insert_instance_template( - body={}, - request_id=GCE_REQUEST_ID, - project_id=GCP_PROJECT_ID_HOOK_UNIT_TEST, - ) - assert res is None - insert_method.assert_called_once_with(body={}, project="example-project", requestId="request_id") - execute_method.assert_called_once_with(num_retries=5) - wait_for_operation_to_complete.assert_called_once_with( - project_id="example-project", operation_name="operation_id" - ) - - @mock.patch("airflow.providers.google.cloud.hooks.compute.ComputeEngineHook.get_conn") - @mock.patch( - "airflow.providers.google.cloud.hooks.compute.ComputeEngineHook._wait_for_operation_to_complete" - ) - def test_insert_instance_template_overridden_project_id(self, wait_for_operation_to_complete, get_conn): - insert_method = get_conn.return_value.instanceTemplates.return_value.insert - execute_method = insert_method.return_value.execute - execute_method.return_value = {"name": "operation_id"} - wait_for_operation_to_complete.return_value = None - res = self.gce_hook.insert_instance_template( - project_id="new-project", body={}, request_id=GCE_REQUEST_ID - ) - assert res is None - insert_method.assert_called_once_with(body={}, project="new-project", requestId="request_id") - execute_method.assert_called_once_with(num_retries=5) - wait_for_operation_to_complete.assert_called_once_with( - project_id="new-project", operation_name="operation_id" - ) - - @mock.patch( - "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", - new_callable=PropertyMock, - return_value=GCP_PROJECT_ID_HOOK_UNIT_TEST, - ) - @mock.patch("airflow.providers.google.cloud.hooks.compute.ComputeEngineHook.get_conn") - @mock.patch( - "airflow.providers.google.cloud.hooks.compute.ComputeEngineHook._wait_for_operation_to_complete" - ) - def test_get_instance_group_manager(self, wait_for_operation_to_complete, get_conn, mock_project_id): - get_method = get_conn.return_value.instanceGroupManagers.return_value.get - execute_method = get_method.return_value.execute - execute_method.return_value = {"name": "operation_id"} - wait_for_operation_to_complete.return_value = None - res = self.gce_hook.get_instance_group_manager( - zone=GCE_ZONE, - resource_id=GCE_INSTANCE_GROUP_MANAGER, - project_id=GCP_PROJECT_ID_HOOK_UNIT_TEST, - ) - assert res is not None - get_method.assert_called_once_with( - instanceGroupManager="instance_group_manager", project="example-project", zone="zone" - ) - execute_method.assert_called_once_with(num_retries=5) - wait_for_operation_to_complete.assert_not_called() - - @mock.patch("airflow.providers.google.cloud.hooks.compute.ComputeEngineHook.get_conn") - @mock.patch( - "airflow.providers.google.cloud.hooks.compute.ComputeEngineHook._wait_for_operation_to_complete" - ) - def test_get_instance_group_manager_overridden_project_id(self, wait_for_operation_to_complete, get_conn): - get_method = get_conn.return_value.instanceGroupManagers.return_value.get - execute_method = get_method.return_value.execute - execute_method.return_value = {"name": "operation_id"} - wait_for_operation_to_complete.return_value = None - res = self.gce_hook.get_instance_group_manager( - project_id="new-project", zone=GCE_ZONE, resource_id=GCE_INSTANCE_GROUP_MANAGER - ) - assert res is not None - get_method.assert_called_once_with( - instanceGroupManager="instance_group_manager", project="new-project", zone="zone" - ) - execute_method.assert_called_once_with(num_retries=5) - wait_for_operation_to_complete.assert_not_called() - @mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.project_id", new_callable=PropertyMock, diff --git a/tests/providers/google/cloud/hooks/test_compute_ssh_system.py b/tests/providers/google/cloud/hooks/test_compute_ssh_system.py deleted file mode 100644 index a94dda8df796fd..00000000000000 --- a/tests/providers/google/cloud/hooks/test_compute_ssh_system.py +++ /dev/null @@ -1,79 +0,0 @@ -# -# 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 pytest - -from airflow.providers.google.cloud.example_dags.example_compute_ssh import ( - GCE_INSTANCE, - GCE_ZONE, - GCP_PROJECT_ID, -) -from tests.providers.google.cloud.utils.gcp_authenticator import GCP_COMPUTE_SSH_KEY -from tests.test_utils.gcp_system_helpers import CLOUD_DAG_FOLDER, GoogleSystemTest, provide_gcp_context - - -@pytest.mark.backend("mysql", "postgres") -@pytest.mark.credential_file(GCP_COMPUTE_SSH_KEY) -class GcpComputeSSHExampleDagsSystemTest(GoogleSystemTest): - @provide_gcp_context(GCP_COMPUTE_SSH_KEY) - def setUp(self) -> None: - super().setUp() - self.create_target_instance() - - @provide_gcp_context(GCP_COMPUTE_SSH_KEY) - def tearDown(self) -> None: - self.delete_target_instance() - super().tearDown() - - @provide_gcp_context(GCP_COMPUTE_SSH_KEY) - def test_run_example_dag_compute_engine_ssh(self): - self.run_dag("example_compute_ssh", CLOUD_DAG_FOLDER) - - def delete_target_instance(self): - self.execute_cmd( - [ - "gcloud", - "compute", - "instances", - "delete", - GCE_INSTANCE, - "--project", - GCP_PROJECT_ID, - "--quiet", - "--verbosity=none", - "--zone", - GCE_ZONE, - ] - ) - - def create_target_instance(self): - self.execute_cmd( - [ - "gcloud", - "compute", - "instances", - "create", - GCE_INSTANCE, - "--project", - GCP_PROJECT_ID, - "--zone", - GCE_ZONE, - "--metadata=enable-oslogin=TRUE", - ] - ) diff --git a/tests/providers/google/cloud/operators/test_compute.py b/tests/providers/google/cloud/operators/test_compute.py index be33371e4aad1f..44b57e5eb4beee 100644 --- a/tests/providers/google/cloud/operators/test_compute.py +++ b/tests/providers/google/cloud/operators/test_compute.py @@ -21,13 +21,21 @@ from copy import deepcopy from unittest import mock -import httplib2 import pytest -from googleapiclient.errors import HttpError +from google.api_core.exceptions import NotFound +from google.api_core.retry import Retry +from google.cloud.compute_v1.types import Instance, InstanceGroupManager, InstanceTemplate from airflow.exceptions import AirflowException from airflow.providers.google.cloud.operators.compute import ( ComputeEngineCopyInstanceTemplateOperator, + ComputeEngineDeleteInstanceGroupManagerOperator, + ComputeEngineDeleteInstanceOperator, + ComputeEngineDeleteInstanceTemplateOperator, + ComputeEngineInsertInstanceFromTemplateOperator, + ComputeEngineInsertInstanceGroupManagerOperator, + ComputeEngineInsertInstanceOperator, + ComputeEngineInsertInstanceTemplateOperator, ComputeEngineInstanceGroupUpdateManagerTemplateOperator, ComputeEngineSetMachineTypeOperator, ComputeEngineStartInstanceOperator, @@ -36,38 +44,450 @@ from airflow.utils import timezone EMPTY_CONTENT = b"" - GCP_PROJECT_ID = "project-id" GCE_ZONE = "zone" -RESOURCE_ID = "resource-id" +GCE_RESOURCE_ID = "resource-id" GCE_SHORT_MACHINE_TYPE_NAME = "n1-machine-type" -SET_MACHINE_TYPE_BODY = {"machineType": f"zones/{GCE_ZONE}/machineTypes/{GCE_SHORT_MACHINE_TYPE_NAME}"} +GCE_SOURCE_IMAGE = "projects/debian-cloud/global/images/debian-11-bullseye-v20220621" +GCP_INSTANCE_TEMPLATE_NAME = "instance-template-name" +TASK_ID = "task-id" +SET_MACHINE_TYPE_BODY = {"machineType": f"zones/{GCE_ZONE}/machineTypes/{GCE_SHORT_MACHINE_TYPE_NAME}"} +SOURCE_INSTANCE_TEMPLATE = f"global/InstanceTemplates/{GCP_INSTANCE_TEMPLATE_NAME}" DEFAULT_DATE = timezone.datetime(2017, 1, 1) +GCP_CONN_ID = "test-conn" +IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] +TIMEOUT = 120 +RETRY = mock.MagicMock(Retry) +METADATA = [("key", "value")] +API_VERSION = "v1" + +COMPUTE_ENGINE_HOOK_PATH = "airflow.providers.google.cloud.operators.compute.ComputeEngineHook" +IGM_PATH = "airflow.providers.google.cloud.operators.compute.InstanceGroupManager" + +GCP_INSTANCE_BODY_FROM_TEMPLATE = {"name": "a-instance-from-template"} +GCE_INSTANCE_BODY_API_CALL = { + "name": "instance-name", + "machine_type": f"zones/{GCE_ZONE}/machineTypes/{GCE_SHORT_MACHINE_TYPE_NAME}", + "disks": [ + { + "boot": True, + "device_name": "instance-name", + "initialize_params": { + "disk_size_gb": "10", + "disk_type": f"zones/{GCE_ZONE}/diskTypes/pd-balanced", + "source_image": GCE_SOURCE_IMAGE, + }, + } + ], + "network_interfaces": [ + { + "access_configs": [{"name": "External NAT", "network_tier": "PREMIUM"}], + "stack_type": "IPV4_ONLY", + "subnetwork": "regions/us-central1/subnetworks/default", + } + ], +} +GCE_INSTANCE_BODY_WITHOUT_NAME_API_CALL = { + "machine_type": f"zones/{GCE_ZONE}/machineTypes/{GCE_SHORT_MACHINE_TYPE_NAME}", + "disks": [ + { + "boot": True, + "device_name": "instance-name", + "initialize_params": { + "disk_size_gb": "10", + "disk_type": f"zones/{GCE_ZONE}/diskTypes/pd-balanced", + "source_image": GCE_SOURCE_IMAGE, + }, + } + ], + "network_interfaces": [ + { + "access_configs": [{"name": "External NAT", "network_tier": "PREMIUM"}], + "stack_type": "IPV4_ONLY", + "subnetwork": "regions/us-central1/subnetworks/default", + } + ], +} + + +class TestGceInstanceInsert: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_should_execute_successfully(self, mock_hook): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + op = ComputeEngineInsertInstanceOperator( + project_id=GCP_PROJECT_ID, + resource_id=GCE_RESOURCE_ID, + body=GCE_INSTANCE_BODY_API_CALL, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + api_version=API_VERSION, + ) + mock_hook.return_value.insert_instance.assert_called_once_with( + project_id=GCP_PROJECT_ID, + body=GCE_INSTANCE_BODY_API_CALL, + zone=GCE_ZONE, + request_id=None, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_should_throw_ex_when_missing_project_id(self, mock_hook): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + with pytest.raises(AirflowException, match=r"The required parameter 'project_id' is missing"): + op = ComputeEngineInsertInstanceOperator( + project_id="", + body=GCE_INSTANCE_BODY_API_CALL, + zone=GCE_ZONE, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_should_not_throw_ex_when_project_id_none(self, mock_hook): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + op = ComputeEngineInsertInstanceOperator( + resource_id=GCE_RESOURCE_ID, + body=GCE_INSTANCE_BODY_API_CALL, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.insert_instance.assert_called_once_with( + body=GCE_INSTANCE_BODY_API_CALL, + zone=GCE_ZONE, + request_id=None, + project_id=None, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_should_throw_ex_when_missing_zone(self, mock_hook): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + with pytest.raises(AirflowException, match=r"The required parameter 'zone' is missing"): + op = ComputeEngineInsertInstanceOperator( + resource_id=GCE_RESOURCE_ID, + body=GCE_INSTANCE_BODY_API_CALL, + zone="", + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_should_throw_ex_when_missing_resource_id(self, mock_hook): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + with pytest.raises( + AirflowException, + match=r"The required parameters 'resource_id' and " + r"body\['name'\] are missing\. Please, provide " + r"at least one of them", + ): + op = ComputeEngineInsertInstanceOperator( + project_id=GCP_PROJECT_ID, + zone=GCE_ZONE, + body=GCE_INSTANCE_BODY_WITHOUT_NAME_API_CALL, + task_id=TASK_ID, + resource_id="", + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + +class TestGceInstanceInsertFromTemplate: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_from_template_should_execute_successfully(self, mock_hook): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + op = ComputeEngineInsertInstanceFromTemplateOperator( + project_id=GCP_PROJECT_ID, + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + body=GCP_INSTANCE_BODY_FROM_TEMPLATE, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.insert_instance.assert_called_once_with( + project_id=GCP_PROJECT_ID, + zone=GCE_ZONE, + body=GCP_INSTANCE_BODY_FROM_TEMPLATE, + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + request_id=None, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_from_template_should_throw_ex_when_missing_project_id(self, mock_hook): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + with pytest.raises(AirflowException, match=r"The required parameter 'project_id' is missing"): + op = ComputeEngineInsertInstanceFromTemplateOperator( + project_id="", + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + body=GCP_INSTANCE_BODY_FROM_TEMPLATE, + zone=GCE_ZONE, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_from_template_should_not_throw_ex_when_project_id_none(self, mock_hook): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + op = ComputeEngineInsertInstanceFromTemplateOperator( + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + body=GCP_INSTANCE_BODY_FROM_TEMPLATE, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.insert_instance.assert_called_once_with( + project_id=None, + zone=GCE_ZONE, + body=GCP_INSTANCE_BODY_FROM_TEMPLATE, + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + request_id=None, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_from_template_should_throw_ex_when_missing_zone(self, mock_hook): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + with pytest.raises(AirflowException, match=r"The required parameter 'zone' is missing"): + op = ComputeEngineInsertInstanceFromTemplateOperator( + project_id=GCP_PROJECT_ID, + zone="", + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + body=GCP_INSTANCE_BODY_FROM_TEMPLATE, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_from_template_should_throw_ex_when_missing_source_instance_template( + self, mock_hook + ): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + with pytest.raises(AirflowException, match=r"missing keyword argument 'source_instance_template'"): + op = ComputeEngineInsertInstanceFromTemplateOperator( + project_id=GCP_PROJECT_ID, + body=GCP_INSTANCE_BODY_FROM_TEMPLATE, + zone=GCE_ZONE, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_instance_from_template_should_throw_ex_when_missing_body(self, mock_hook): + get_instance_obj_mock = mock.MagicMock() + get_instance_obj_mock.__class__ = Instance + mock_hook.return_value.get_instance.side_effect = [ + NotFound("Error message"), + get_instance_obj_mock, + ] + with pytest.raises(AirflowException, match=r"missing keyword argument 'body'"): + op = ComputeEngineInsertInstanceFromTemplateOperator( + project_id=GCP_PROJECT_ID, + source_instance_template=SOURCE_INSTANCE_TEMPLATE, + zone=GCE_ZONE, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + +class TestGceInstanceDelete: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_instance_should_execute_successfully(self, mock_hook): + op = ComputeEngineDeleteInstanceOperator( + resource_id=GCE_RESOURCE_ID, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.delete_instance.assert_called_once_with( + project_id=None, + request_id=None, + resource_id=GCE_RESOURCE_ID, + zone=GCE_ZONE, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_instance_should_throw_ex_when_missing_zone(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'zone' is missing"): + op = ComputeEngineDeleteInstanceOperator( + resource_id=GCE_RESOURCE_ID, + zone="", + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_instance_should_throw_ex_when_missing_resource_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'resource_id' is missing"): + op = ComputeEngineDeleteInstanceOperator( + resource_id="", + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + class TestGceInstanceStart: - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_instance_start(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_start_instance_should_execute_successfully(self, mock_hook): mock_hook.return_value.start_instance.return_value = True op = ComputeEngineStartInstanceOperator( - project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id=RESOURCE_ID, task_id="id" + project_id=GCP_PROJECT_ID, + zone=GCE_ZONE, + resource_id=GCE_RESOURCE_ID, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + context = mock.MagicMock() + op.execute(context=context) + mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.start_instance.assert_called_once_with( - zone=GCE_ZONE, resource_id=RESOURCE_ID, project_id=GCP_PROJECT_ID + zone=GCE_ZONE, resource_id=GCE_RESOURCE_ID, project_id=GCP_PROJECT_ID ) - assert result # Setting all the operator's input parameters as template dag_ids # (could be anything else) just to test if the templating works for all fields - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_instance_start_with_templates(self, _, create_task_instance_of_operator): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_start_instance_with_templates(self, _, create_task_instance_of_operator): dag_id = "test_instance_start_with_templates" ti = create_task_instance_of_operator( ComputeEngineStartInstanceOperator, @@ -86,64 +506,89 @@ def test_instance_start_with_templates(self, _, create_task_instance_of_operator assert dag_id == ti.task.gcp_conn_id assert dag_id == ti.task.api_version - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_start_should_throw_ex_when_missing_project_id(self, mock_hook): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_instance_start_should_throw_ex_when_missing_project_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'project_id' is missing"): op = ComputeEngineStartInstanceOperator( - project_id="", zone=GCE_ZONE, resource_id=RESOURCE_ID, task_id="id" + project_id="", + zone=GCE_ZONE, + resource_id=GCE_RESOURCE_ID, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - op.execute(None) - err = ctx.value - assert "The required parameter 'project_id' is missing" in str(err) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_not_called() - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_start_should_not_throw_ex_when_project_id_none(self, _): - op = ComputeEngineStartInstanceOperator(zone=GCE_ZONE, resource_id=RESOURCE_ID, task_id="id") - op.execute(None) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_instance_start_should_not_throw_ex_when_project_id_none(self, _): + op = ComputeEngineStartInstanceOperator( + zone=GCE_ZONE, + resource_id=GCE_RESOURCE_ID, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + context = mock.MagicMock() + op.execute(context=context) - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_start_should_throw_ex_when_missing_zone(self, mock_hook): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_instance_start_should_throw_ex_when_missing_zone(self, mock_hook): + with pytest.raises(AirflowException, match=r"he required parameter 'zone' is missing"): op = ComputeEngineStartInstanceOperator( - project_id=GCP_PROJECT_ID, zone="", resource_id=RESOURCE_ID, task_id="id" + project_id=GCP_PROJECT_ID, + zone="", + resource_id=GCE_RESOURCE_ID, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - op.execute(None) - err = ctx.value - assert "The required parameter 'zone' is missing" in str(err) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_not_called() - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_start_should_throw_ex_when_missing_resource_id(self, mock_hook): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_instance_start_should_throw_ex_when_missing_resource_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'resource_id' is missing"): op = ComputeEngineStartInstanceOperator( - project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id="", task_id="id" + project_id=GCP_PROJECT_ID, + zone=GCE_ZONE, + resource_id="", + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - op.execute(None) - err = ctx.value - assert "The required parameter 'resource_id' is missing" in str(err) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_not_called() class TestGceInstanceStop: - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_instance_stop(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_instance_stop_should_execute_successfully(self, mock_hook): op = ComputeEngineStopInstanceOperator( - project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id=RESOURCE_ID, task_id="id" + project_id=GCP_PROJECT_ID, + zone=GCE_ZONE, + resource_id=GCE_RESOURCE_ID, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.stop_instance.assert_called_once_with( - zone=GCE_ZONE, resource_id=RESOURCE_ID, project_id=GCP_PROJECT_ID + zone=GCE_ZONE, resource_id=GCE_RESOURCE_ID, project_id=GCP_PROJECT_ID ) # Setting all the operator's input parameters as templated dag_ids # (could be anything else) just to test if the templating works for all fields - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) def test_instance_stop_with_templates(self, _, create_task_instance_of_operator): dag_id = "test_instance_stop_with_templates" ti = create_task_instance_of_operator( @@ -163,78 +608,79 @@ def test_instance_stop_with_templates(self, _, create_task_instance_of_operator) assert dag_id == ti.task.gcp_conn_id assert dag_id == ti.task.api_version - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_stop_should_throw_ex_when_missing_project_id(self, mock_hook): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_instance_stop_should_throw_ex_when_missing_project_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'project_id' is missing"): op = ComputeEngineStopInstanceOperator( - project_id="", zone=GCE_ZONE, resource_id=RESOURCE_ID, task_id="id" + project_id="", zone=GCE_ZONE, resource_id=GCE_RESOURCE_ID, task_id="id" ) - op.execute(None) - err = ctx.value - assert "The required parameter 'project_id' is missing" in str(err) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_not_called() - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_stop_should_not_throw_ex_when_project_id_none(self, mock_hook): - op = ComputeEngineStopInstanceOperator(zone=GCE_ZONE, resource_id=RESOURCE_ID, task_id="id") - op.execute(None) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_instance_stop_should_not_throw_ex_when_project_id_none(self, mock_hook): + op = ComputeEngineStopInstanceOperator(zone=GCE_ZONE, resource_id=GCE_RESOURCE_ID, task_id="id") + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", gcp_conn_id="google_cloud_default", impersonation_chain=None, ) mock_hook.return_value.stop_instance.assert_called_once_with( - zone=GCE_ZONE, resource_id=RESOURCE_ID, project_id=None + zone=GCE_ZONE, resource_id=GCE_RESOURCE_ID, project_id=None ) - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_stop_should_throw_ex_when_missing_zone(self, mock_hook): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_instance_stop_should_throw_ex_when_missing_zone(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'zone' is missing"): op = ComputeEngineStopInstanceOperator( - project_id=GCP_PROJECT_ID, zone="", resource_id=RESOURCE_ID, task_id="id" + project_id=GCP_PROJECT_ID, zone="", resource_id=GCE_RESOURCE_ID, task_id="id" ) - op.execute(None) - err = ctx.value - assert "The required parameter 'zone' is missing" in str(err) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_not_called() - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_stop_should_throw_ex_when_missing_resource_id(self, mock_hook): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_instance_stop_should_throw_ex_when_missing_resource_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'resource_id' is missing"): op = ComputeEngineStopInstanceOperator( project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id="", task_id="id" ) - op.execute(None) - err = ctx.value - assert "The required parameter 'resource_id' is missing" in str(err) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_not_called() class TestGceInstanceSetMachineType: - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_set_machine_type(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_machine_type_set_should_execute_successfully(self, mock_hook): mock_hook.return_value.set_machine_type.return_value = True op = ComputeEngineSetMachineTypeOperator( project_id=GCP_PROJECT_ID, zone=GCE_ZONE, - resource_id=RESOURCE_ID, + resource_id=GCE_RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, - task_id="id", + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.set_machine_type.assert_called_once_with( - zone=GCE_ZONE, resource_id=RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, project_id=GCP_PROJECT_ID + zone=GCE_ZONE, resource_id=GCE_RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, project_id=GCP_PROJECT_ID ) # Setting all the operator's input parameters as templated dag_ids # (could be anything else) just to test if the templating works for all fields - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_set_machine_type_with_templates(self, _, create_task_instance_of_operator): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_machine_type_set_with_templates(self, _, create_task_instance_of_operator): dag_id = "test_set_machine_type_with_templates" ti = create_task_instance_of_operator( ComputeEngineSetMachineTypeOperator, @@ -254,79 +700,89 @@ def test_set_machine_type_with_templates(self, _, create_task_instance_of_operat assert dag_id == ti.task.gcp_conn_id assert dag_id == ti.task.api_version - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_set_machine_type_should_throw_ex_when_missing_project_id(self, mock_hook): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_machine_type_set_should_throw_ex_when_missing_project_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'project_id' is missing"): op = ComputeEngineSetMachineTypeOperator( project_id="", zone=GCE_ZONE, - resource_id=RESOURCE_ID, + resource_id=GCE_RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, - task_id="id", + task_id=TASK_ID, ) - op.execute(None) - err = ctx.value - assert "The required parameter 'project_id' is missing" in str(err) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_not_called() - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_set_machine_type_should_not_throw_ex_when_project_id_none(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_machine_type_set_should_not_throw_ex_when_project_id_none(self, mock_hook): op = ComputeEngineSetMachineTypeOperator( - zone=GCE_ZONE, resource_id=RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, task_id="id" + zone=GCE_ZONE, + resource_id=GCE_RESOURCE_ID, + body=SET_MACHINE_TYPE_BODY, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.set_machine_type.assert_called_once_with( - zone=GCE_ZONE, resource_id=RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, project_id=None + zone=GCE_ZONE, resource_id=GCE_RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, project_id=None ) - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_set_machine_type_should_throw_ex_when_missing_zone(self, mock_hook): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_machine_type_set_should_throw_ex_when_missing_zone(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'zone' is missing"): op = ComputeEngineSetMachineTypeOperator( project_id=GCP_PROJECT_ID, zone="", - resource_id=RESOURCE_ID, + resource_id=GCE_RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, - task_id="id", + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - op.execute(None) - err = ctx.value - assert "The required parameter 'zone' is missing" in str(err) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_not_called() - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_set_machine_type_should_throw_ex_when_missing_resource_id(self, mock_hook): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_machine_type_set_should_throw_ex_when_missing_resource_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'resource_id' is missing"): op = ComputeEngineSetMachineTypeOperator( project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id="", body=SET_MACHINE_TYPE_BODY, - task_id="id", + task_id=TASK_ID, ) - op.execute(None) - err = ctx.value - assert "The required parameter 'resource_id' is missing" in str(err) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_not_called() - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_set_machine_type_should_throw_ex_when_missing_machine_type(self, mock_hook): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_machine_type_set_should_throw_ex_when_missing_machine_type(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required body field 'machineType' is missing"): op = ComputeEngineSetMachineTypeOperator( - project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id=RESOURCE_ID, body={}, task_id="id" + project_id=GCP_PROJECT_ID, + zone=GCE_ZONE, + resource_id=GCE_RESOURCE_ID, + body={}, + task_id=TASK_ID, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - op.execute(None) - err = ctx.value - assert "The required body field 'machineType' is missing. Please add it." in str(err) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) MOCK_OP_RESPONSE = ( @@ -359,52 +815,252 @@ def test_set_machine_type_should_throw_ex_when_missing_machine_type(self, mock_h "airflow.providers.google.cloud.operators.compute.ComputeEngineHook._execute_set_machine_type" ) @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook.get_conn") - def test_set_machine_type_should_handle_and_trim_gce_error( + def test_machine_type_set_should_handle_and_trim_gce_error( self, get_conn, _execute_set_machine_type, _check_zone_operation_status ): get_conn.return_value = {} _execute_set_machine_type.return_value = {"name": "test-operation"} _check_zone_operation_status.return_value = ast.literal_eval(self.MOCK_OP_RESPONSE) - with pytest.raises(AirflowException) as ctx: + with pytest.raises(AirflowException, match=r"400 BAD REQUEST: {.+UNSUPPORTED_OPERATION"): op = ComputeEngineSetMachineTypeOperator( project_id=GCP_PROJECT_ID, zone=GCE_ZONE, - resource_id=RESOURCE_ID, + resource_id=GCE_RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, - task_id="id", + task_id=TASK_ID, ) - op.execute(None) - err = ctx.value + context = mock.MagicMock() + op.execute(context=context) _check_zone_operation_status.assert_called_once_with( {}, "test-operation", GCP_PROJECT_ID, GCE_ZONE, mock.ANY ) _execute_set_machine_type.assert_called_once_with( - GCE_ZONE, RESOURCE_ID, SET_MACHINE_TYPE_BODY, GCP_PROJECT_ID + GCE_ZONE, GCE_RESOURCE_ID, SET_MACHINE_TYPE_BODY, GCP_PROJECT_ID ) - # Checking the full message was sometimes failing due to different order - # of keys in the serialized JSON - assert "400 BAD REQUEST: {" in str(err) # checking the square bracket trim - assert "UNSUPPORTED_OPERATION" in str(err) + + +GCE_INSTANCE_TEMPLATE_NAME_API_CALL = "instance-template-test-api" +GCE_INSTANCE_TEMPLATE_BODY_API_CALL = { + "name": GCE_INSTANCE_TEMPLATE_NAME_API_CALL, + "properties": { + "machine_type": GCE_SHORT_MACHINE_TYPE_NAME, + "disks": [ + { + "auto_delete": True, + "boot": True, + "device_name": GCE_INSTANCE_TEMPLATE_NAME_API_CALL, + "initialize_params": { + "disk_size_gb": "10", + "disk_type": "pd-balanced", + "source_image": GCE_SOURCE_IMAGE, + }, + } + ], + "network_interfaces": [{"network": "/global/networks/default"}], + }, +} + + +class TestGceTemplateInsert: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_template_should_execute_successfully(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate + mock_hook.return_value.get_instance_template.side_effect = [ + NotFound("Error message"), + get_template_obj_mock, + ] + op = ComputeEngineInsertInstanceTemplateOperator( + project_id=GCP_PROJECT_ID, + body=GCE_INSTANCE_TEMPLATE_BODY_API_CALL, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.insert_instance_template.assert_called_once_with( + project_id=GCP_PROJECT_ID, + body=GCE_INSTANCE_TEMPLATE_BODY_API_CALL, + request_id=None, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_template_should_throw_ex_when_missing_project_id(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate + mock_hook.return_value.get_instance_template.side_effect = [ + NotFound("Error message"), + get_template_obj_mock, + ] + with pytest.raises(AirflowException, match=r"The required parameter 'project_id' is missing"): + op = ComputeEngineInsertInstanceTemplateOperator( + project_id="", + body=GCE_INSTANCE_TEMPLATE_BODY_API_CALL, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_template_should_not_throw_ex_when_project_id_none(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate + mock_hook.return_value.get_instance_template.side_effect = [ + NotFound("Error message"), + get_template_obj_mock, + ] + op = ComputeEngineInsertInstanceTemplateOperator( + body=GCE_INSTANCE_TEMPLATE_BODY_API_CALL, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.insert_instance_template.assert_called_once_with( + project_id=None, + body=GCE_INSTANCE_TEMPLATE_BODY_API_CALL, + request_id=None, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_template_should_throw_ex_when_missing_body(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate + mock_hook.return_value.get_instance_template.side_effect = [ + NotFound("Error message"), + get_template_obj_mock, + ] + with pytest.raises(AirflowException, match=r"missing keyword argument 'body'"): + op = ComputeEngineInsertInstanceTemplateOperator( + task_id=TASK_ID, + project_id=GCP_PROJECT_ID, + retry=RETRY, + timeout=TIMEOUT, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + +class TestGceTemplateDelete: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_template_should_execute_successfully(self, mock_hook): + op = ComputeEngineDeleteInstanceTemplateOperator( + resource_id=GCE_RESOURCE_ID, + project_id=GCP_PROJECT_ID, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.delete_instance_template.assert_called_once_with( + project_id=GCP_PROJECT_ID, + request_id=None, + resource_id=GCE_RESOURCE_ID, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_template_should_throw_ex_when_missing_project_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'project_id' is missing"): + op = ComputeEngineDeleteInstanceTemplateOperator( + project_id="", + resource_id=GCE_RESOURCE_ID, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_template_should_not_throw_ex_when_project_id_none(self, mock_hook): + op = ComputeEngineDeleteInstanceTemplateOperator( + resource_id=GCE_RESOURCE_ID, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.delete_instance_template.assert_called_once_with( + resource_id=GCE_RESOURCE_ID, + project_id=None, + request_id=None, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_template_should_throw_ex_when_missing_resource_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'resource_id' is missing"): + op = ComputeEngineDeleteInstanceTemplateOperator( + resource_id="", + project_id=GCP_PROJECT_ID, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() GCE_INSTANCE_TEMPLATE_NAME = "instance-template-test" GCE_INSTANCE_TEMPLATE_NEW_NAME = "instance-template-test-new" GCE_INSTANCE_TEMPLATE_REQUEST_ID = "e12d5b48-4826-4ba9-ada6-0cff1e0b36a6" - GCE_INSTANCE_TEMPLATE_BODY_GET = { "kind": "compute#instanceTemplate", "id": "6950321349997439715", - "creationTimestamp": "2018-10-15T06:20:12.777-07:00", + "creation_timestamp": "2018-10-15T06:20:12.777-07:00", "name": GCE_INSTANCE_TEMPLATE_NAME, "description": "", "properties": { - "machineType": "n1-standard-1", - "networkInterfaces": [ + "machine_type": "n1-standard-1", + "network_interfaces": [ { "kind": "compute#networkInterface", "network": "https://www.googleapis.com/compute/v1/" "projects/project/global/networks/default", - "accessConfigs": [ + "access_configs": [ { "kind": "compute#accessConfig", "type": "ONE_TO_ONE_NAT", @@ -414,7 +1070,7 @@ def test_set_machine_type_should_handle_and_trim_gce_error( { "network": "https://www.googleapis.com/compute/v1/" "projects/project/global/networks/default", - "accessConfigs": [{"kind": "compute#accessConfig", "networkTier": "PREMIUM"}], + "access_configs": [{"kind": "compute#accessConfig", "networkTier": "PREMIUM"}], }, ], "disks": [ @@ -426,158 +1082,154 @@ def test_set_machine_type_should_handle_and_trim_gce_error( ], } ], - "metadata": {"kind": "compute#metadata", "fingerprint": "GDPUYxlwHe4="}, }, - "selfLink": "https://www.googleapis.com/compute/v1/projects/project" + "self_link": "https://www.googleapis.com/compute/v1/projects/project" "/global/instanceTemplates/instance-template-test", } - GCE_INSTANCE_TEMPLATE_BODY_INSERT = { "name": GCE_INSTANCE_TEMPLATE_NEW_NAME, - "description": "", - "properties": { - "machineType": "n1-standard-1", - "networkInterfaces": [ - { - "network": "https://www.googleapis.com/compute/v1/" - "projects/project/global/networks/default", - "accessConfigs": [ - { - "type": "ONE_TO_ONE_NAT", - } - ], - }, - { - "network": "https://www.googleapis.com/compute/v1/" - "projects/project/global/networks/default", - "accessConfigs": [{"networkTier": "PREMIUM"}], - }, - ], - "disks": [ - { - "type": "PERSISTENT", - } - ], - "metadata": {"fingerprint": "GDPUYxlwHe4="}, - }, } - GCE_INSTANCE_TEMPLATE_BODY_GET_NEW = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_GET) GCE_INSTANCE_TEMPLATE_BODY_GET_NEW["name"] = GCE_INSTANCE_TEMPLATE_NEW_NAME class TestGceInstanceTemplateCopy: - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_copy_template(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_copy_template_should_execute_successfully(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate mock_hook.return_value.get_instance_template.side_effect = [ - HttpError(resp=httplib2.Response({"status": 404}), content=EMPTY_CONTENT), - GCE_INSTANCE_TEMPLATE_BODY_GET, - GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, + NotFound("Error message"), + get_template_obj_mock, + get_template_obj_mock, ] op = ComputeEngineCopyInstanceTemplateOperator( project_id=GCP_PROJECT_ID, resource_id=GCE_INSTANCE_TEMPLATE_NAME, - task_id="id", - body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME}, + task_id=TASK_ID, + body_patch=GCE_INSTANCE_TEMPLATE_BODY_INSERT, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.insert_instance_template.assert_called_once_with( project_id=GCP_PROJECT_ID, body=GCE_INSTANCE_TEMPLATE_BODY_INSERT, request_id=None ) - assert GCE_INSTANCE_TEMPLATE_BODY_GET_NEW == result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_copy_template_missing_project_id(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_copy_template_should_not_throw_ex_when_missing_project_id(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate mock_hook.return_value.get_instance_template.side_effect = [ - HttpError(resp=httplib2.Response({"status": 404}), content=EMPTY_CONTENT), - GCE_INSTANCE_TEMPLATE_BODY_GET, - GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, + NotFound("Error message"), + get_template_obj_mock, + get_template_obj_mock, ] op = ComputeEngineCopyInstanceTemplateOperator( resource_id=GCE_INSTANCE_TEMPLATE_NAME, - task_id="id", - body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME}, + task_id=TASK_ID, + body_patch=GCE_INSTANCE_TEMPLATE_BODY_INSERT, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.insert_instance_template.assert_called_once_with( project_id=None, body=GCE_INSTANCE_TEMPLATE_BODY_INSERT, request_id=None ) - assert GCE_INSTANCE_TEMPLATE_BODY_GET_NEW == result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_idempotent_copy_template_when_already_copied(self, mock_hook): - mock_hook.return_value.get_instance_template.side_effect = [GCE_INSTANCE_TEMPLATE_BODY_GET_NEW] + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_idempotent_copy_template_when_already_copied_should_execute_successfully(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate + mock_hook.return_value.get_instance_template.side_effect = [ + get_template_obj_mock, + get_template_obj_mock, + get_template_obj_mock, + ] op = ComputeEngineCopyInstanceTemplateOperator( project_id=GCP_PROJECT_ID, resource_id=GCE_INSTANCE_TEMPLATE_NAME, - task_id="id", + task_id=TASK_ID, body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME}, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.insert_instance_template.assert_not_called() - assert GCE_INSTANCE_TEMPLATE_BODY_GET_NEW == result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_copy_template_with_request_id(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_copy_template_with_request_id_should_execute_successfully(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate mock_hook.return_value.get_instance_template.side_effect = [ - HttpError(resp=httplib2.Response({"status": 404}), content=EMPTY_CONTENT), - GCE_INSTANCE_TEMPLATE_BODY_GET, - GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, + NotFound("Error message"), + get_template_obj_mock, + get_template_obj_mock, ] op = ComputeEngineCopyInstanceTemplateOperator( project_id=GCP_PROJECT_ID, resource_id=GCE_INSTANCE_TEMPLATE_NAME, request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID, - task_id="id", - body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME}, + task_id=TASK_ID, + body_patch=GCE_INSTANCE_TEMPLATE_BODY_INSERT, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.insert_instance_template.assert_called_once_with( project_id=GCP_PROJECT_ID, body=GCE_INSTANCE_TEMPLATE_BODY_INSERT, request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID, ) - assert GCE_INSTANCE_TEMPLATE_BODY_GET_NEW == result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_copy_template_with_description_fields(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_copy_template_with_description_fields_should_execute_successfully(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate mock_hook.return_value.get_instance_template.side_effect = [ - HttpError(resp=httplib2.Response({"status": 404}), content=EMPTY_CONTENT), - GCE_INSTANCE_TEMPLATE_BODY_GET, - GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, + NotFound("Error message"), + get_template_obj_mock, + get_template_obj_mock, ] op = ComputeEngineCopyInstanceTemplateOperator( project_id=GCP_PROJECT_ID, resource_id=GCE_INSTANCE_TEMPLATE_NAME, request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID, - task_id="id", + task_id=TASK_ID, body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME, "description": "New description"}, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT) @@ -587,127 +1239,94 @@ def test_successful_copy_template_with_description_fields(self, mock_hook): body=body_insert, request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID, ) - assert GCE_INSTANCE_TEMPLATE_BODY_GET_NEW == result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_copy_with_some_validation_warnings(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_copy_template_with_some_validation_warnings_should_execute_successfully(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate mock_hook.return_value.get_instance_template.side_effect = [ - HttpError(resp=httplib2.Response({"status": 404}), content=EMPTY_CONTENT), - GCE_INSTANCE_TEMPLATE_BODY_GET, - GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, + NotFound("Error message"), + get_template_obj_mock, + get_template_obj_mock, ] op = ComputeEngineCopyInstanceTemplateOperator( project_id=GCP_PROJECT_ID, resource_id=GCE_INSTANCE_TEMPLATE_NAME, - task_id="id", + task_id=TASK_ID, body_patch={ "name": GCE_INSTANCE_TEMPLATE_NEW_NAME, "some_wrong_field": "test", "properties": {"some_other_wrong_field": "test"}, }, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT) body_insert["some_wrong_field"] = "test" + body_insert["properties"] = {} body_insert["properties"]["some_other_wrong_field"] = "test" mock_hook.return_value.insert_instance_template.assert_called_once_with( project_id=GCP_PROJECT_ID, body=body_insert, request_id=None, ) - assert GCE_INSTANCE_TEMPLATE_BODY_GET_NEW == result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_copy_template_with_updated_nested_fields(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_copy_template_with_updated_nested_fields_should_execute_successfully(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate mock_hook.return_value.get_instance_template.side_effect = [ - HttpError(resp=httplib2.Response({"status": 404}), content=EMPTY_CONTENT), - GCE_INSTANCE_TEMPLATE_BODY_GET, - GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, + NotFound("Error message"), + get_template_obj_mock, + get_template_obj_mock, ] op = ComputeEngineCopyInstanceTemplateOperator( project_id=GCP_PROJECT_ID, resource_id=GCE_INSTANCE_TEMPLATE_NAME, - task_id="id", + task_id=TASK_ID, body_patch={ "name": GCE_INSTANCE_TEMPLATE_NEW_NAME, "properties": { - "machineType": "n1-standard-2", + "machine_type": "n1-standard-2", }, }, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT) - body_insert["properties"]["machineType"] = "n1-standard-2" + body_insert["properties"] = {} + body_insert["properties"]["machine_type"] = "n1-standard-2" mock_hook.return_value.insert_instance_template.assert_called_once_with( project_id=GCP_PROJECT_ID, body=body_insert, request_id=None ) - assert GCE_INSTANCE_TEMPLATE_BODY_GET_NEW == result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_copy_template_with_smaller_array_fields(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_copy_template_with_bigger_array_fields_should_execute_successfully(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate mock_hook.return_value.get_instance_template.side_effect = [ - HttpError(resp=httplib2.Response({"status": 404}), content=EMPTY_CONTENT), - GCE_INSTANCE_TEMPLATE_BODY_GET, - GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, + NotFound("Error message"), + get_template_obj_mock, + get_template_obj_mock, ] op = ComputeEngineCopyInstanceTemplateOperator( project_id=GCP_PROJECT_ID, resource_id=GCE_INSTANCE_TEMPLATE_NAME, - task_id="id", - body_patch={ - "name": GCE_INSTANCE_TEMPLATE_NEW_NAME, - "properties": { - "machineType": "n1-standard-1", - "networkInterfaces": [ - { - "network": "https://www.googleapis.com/compute/v1/" - "projects/project/global/networks/default", - "accessConfigs": [{"type": "ONE_TO_ONE_NAT", "natIP": "8.8.8.8"}], - } - ], - }, - }, - ) - result = op.execute(None) - mock_hook.assert_called_once_with( - api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, - ) - body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT) - body_insert["properties"]["networkInterfaces"] = [ - { - "network": "https://www.googleapis.com/compute/v1/" - "projects/project/global/networks/default", - "accessConfigs": [{"type": "ONE_TO_ONE_NAT", "natIP": "8.8.8.8"}], - } - ] - mock_hook.return_value.insert_instance_template.assert_called_once_with( - project_id=GCP_PROJECT_ID, body=body_insert, request_id=None - ) - assert GCE_INSTANCE_TEMPLATE_BODY_GET_NEW == result - - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_copy_template_with_bigger_array_fields(self, mock_hook): - mock_hook.return_value.get_instance_template.side_effect = [ - HttpError(resp=httplib2.Response({"status": 404}), content=EMPTY_CONTENT), - GCE_INSTANCE_TEMPLATE_BODY_GET, - GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, - ] - op = ComputeEngineCopyInstanceTemplateOperator( - project_id=GCP_PROJECT_ID, - resource_id=GCE_INSTANCE_TEMPLATE_NAME, - task_id="id", + task_id=TASK_ID, body_patch={ "name": GCE_INSTANCE_TEMPLATE_NEW_NAME, "properties": { @@ -729,15 +1348,18 @@ def test_successful_copy_template_with_bigger_array_fields(self, mock_hook): ], }, }, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + context = mock.MagicMock() + op.execute(context=context) mock_hook.assert_called_once_with( api_version="v1", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT) + body_insert["properties"] = {} body_insert["properties"]["disks"] = [ { "kind": "compute#attachedDisk", @@ -759,26 +1381,30 @@ def test_successful_copy_template_with_bigger_array_fields(self, mock_hook): body=body_insert, request_id=None, ) - assert GCE_INSTANCE_TEMPLATE_BODY_GET_NEW == result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_missing_name(self, mock_hook): + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_copy_template_with_missing_name_should_execute_successfully(self, mock_hook): + get_template_obj_mock = mock.MagicMock() + get_template_obj_mock.__class__ = InstanceTemplate mock_hook.return_value.get_instance_template.side_effect = [ - HttpError(resp=httplib2.Response({"status": 404}), content=EMPTY_CONTENT), - GCE_INSTANCE_TEMPLATE_BODY_GET, - GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, + NotFound("Error message"), + get_template_obj_mock, + get_template_obj_mock, ] - with pytest.raises(AirflowException) as ctx: + with pytest.raises( + AirflowException, + match=r"should contain at least name for the new operator in the 'name' field", + ): op = ComputeEngineCopyInstanceTemplateOperator( project_id=GCP_PROJECT_ID, resource_id=GCE_INSTANCE_TEMPLATE_NAME, request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID, - task_id="id", + task_id=TASK_ID, body_patch={"description": "New description"}, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) op.execute(None) - err = ctx.value - assert "should contain at least name for the new operator in the 'name' field" in str(err) mock_hook.assert_not_called() @@ -839,6 +1465,14 @@ def test_missing_name(self, mock_hook): "serviceAccount": "198907790164@cloudservices.gserviceaccount.com", } +GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY = { + "type": "OPPORTUNISTIC", + "minimalAction": "RESTART", + "maxSurge": {"fixed": 1}, + "maxUnavailable": {"percent": 10}, + "minReadySec": 1800, +} + GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH = { "instanceTemplate": GCE_INSTANCE_TEMPLATE_DESTINATION_URL, "versions": [ @@ -852,38 +1486,232 @@ def test_missing_name(self, mock_hook): "instanceTemplate": GCE_INSTANCE_TEMPLATE_OTHER_URL, }, ], + "updatePolicy": GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY, } GCE_INSTANCE_GROUP_MANAGER_REQUEST_ID = "e12d5b48-4826-4ba9-ada6-0cff1e0b36a6" -GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY = { - "type": "OPPORTUNISTIC", - "minimalAction": "RESTART", - "maxSurge": {"fixed": 1}, - "maxUnavailable": {"percent": 10}, - "minReadySec": 1800, +GCE_INSTANCE_GROUP_MANAGER_BODY_API_CALL = { + "name": GCE_INSTANCE_GROUP_MANAGER_NAME, + "base_instance_name": GCE_INSTANCE_GROUP_MANAGER_NAME, + "instance_template": GCE_INSTANCE_TEMPLATE_SOURCE_URL, + "target_size": 1, } -class TestGceInstanceGroupManagerUpdate: - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_instance_group_update(self, mock_hook): - mock_hook.return_value.get_instance_group_manager.return_value = deepcopy( - GCE_INSTANCE_GROUP_MANAGER_GET +class TestGceInstanceGroupManagerInsert: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_igm_should_execute_successfully(self, mock_hook): + get_instance_group_manager_obj_mock = mock.MagicMock() + get_instance_group_manager_obj_mock.__class__ = InstanceGroupManager + mock_hook.return_value.get_instance_group_manager.side_effect = [ + NotFound("Error message"), + get_instance_group_manager_obj_mock, + ] + op = ComputeEngineInsertInstanceGroupManagerOperator( + project_id=GCP_PROJECT_ID, + body=GCE_INSTANCE_GROUP_MANAGER_BODY_API_CALL, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.insert_instance_group_manager.assert_called_once_with( + zone=GCE_ZONE, + project_id=GCP_PROJECT_ID, + body=GCE_INSTANCE_GROUP_MANAGER_BODY_API_CALL, + request_id=None, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_igm_should_throw_ex_when_missing_project_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'project_id' is missing"): + op = ComputeEngineInsertInstanceGroupManagerOperator( + project_id="", + body=GCE_INSTANCE_GROUP_MANAGER_BODY_API_CALL, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + get_instance_group_manager_obj_mock = mock.MagicMock() + get_instance_group_manager_obj_mock.__class__ = InstanceGroupManager + op.execute(context=get_instance_group_manager_obj_mock) + mock_hook.assert_not_called() + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_igm_should_not_throw_ex_when_project_id_none(self, mock_hook): + get_instance_group_manager_obj_mock = mock.MagicMock() + get_instance_group_manager_obj_mock.__class__ = InstanceGroupManager + mock_hook.return_value.get_instance_group_manager.side_effect = [ + NotFound("Error message"), + get_instance_group_manager_obj_mock, + ] + op = ComputeEngineInsertInstanceGroupManagerOperator( + body=GCE_INSTANCE_GROUP_MANAGER_BODY_API_CALL, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.insert_instance_group_manager.assert_called_once_with( + zone=GCE_ZONE, + project_id=None, + body=GCE_INSTANCE_GROUP_MANAGER_BODY_API_CALL, + request_id=None, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_insert_igm_should_throw_ex_when_missing_body(self, mock_hook): + with pytest.raises(AirflowException, match=r"missing keyword argument 'body'"): + op = ComputeEngineInsertInstanceGroupManagerOperator( + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + get_instance_group_manager_obj_mock = mock.MagicMock() + get_instance_group_manager_obj_mock.__class__ = InstanceGroupManager + op.execute(context=get_instance_group_manager_obj_mock) + mock_hook.assert_not_called() + + +class TestGceInstanceGroupManagerDelete: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_igm_should_execute_successfully(self, mock_hook): + op = ComputeEngineDeleteInstanceGroupManagerOperator( + project_id=GCP_PROJECT_ID, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.delete_instance_group_manager.assert_called_once_with( + zone=GCE_ZONE, + request_id=None, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + project_id=GCP_PROJECT_ID, + ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_igm_should_throw_ex_when_missing_project_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'project_id' is missing"): + op = ComputeEngineDeleteInstanceGroupManagerOperator( + project_id="", + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_igm_should_not_throw_ex_when_project_id_none(self, mock_hook): + op = ComputeEngineDeleteInstanceGroupManagerOperator( + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_called_once_with( + api_version=API_VERSION, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + mock_hook.return_value.delete_instance_group_manager.assert_called_once_with( + zone=GCE_ZONE, + request_id=None, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + project_id=None, ) + + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_delete_igm_should_throw_ex_when_missing_resource_id(self, mock_hook): + with pytest.raises(AirflowException, match=r"The required parameter 'resource_id' is missing"): + op = ComputeEngineDeleteInstanceGroupManagerOperator( + resource_id="", + zone=GCE_ZONE, + task_id=TASK_ID, + retry=RETRY, + timeout=TIMEOUT, + metadata=METADATA, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, + ) + op.execute(context=mock.MagicMock()) + mock_hook.assert_not_called() + + +class TestGceInstanceGroupManagerUpdate: + @mock.patch(IGM_PATH) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_update_igm_should_execute_successfully(self, mock_hook, igm): + get_instance_group_manager_obj_mock = mock.MagicMock() + get_instance_group_manager_obj_mock.__class__ = InstanceGroupManager + mock_hook.return_value.get_instance_group_manager.side_effect = [ + get_instance_group_manager_obj_mock, + ] + igm.to_dict.return_value = GCE_INSTANCE_GROUP_MANAGER_GET + op = ComputeEngineInstanceGroupUpdateManagerTemplateOperator( project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, - task_id="id", + task_id=TASK_ID, source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL, + update_policy=GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + result = op.execute(context=mock.MagicMock()) mock_hook.assert_called_once_with( api_version="beta", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.patch_instance_group_manager.assert_called_once_with( project_id=GCP_PROJECT_ID, @@ -894,23 +1722,30 @@ def test_successful_instance_group_update(self, mock_hook): ) assert result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_instance_group_update_missing_project_id(self, mock_hook): - mock_hook.return_value.get_instance_group_manager.return_value = deepcopy( - GCE_INSTANCE_GROUP_MANAGER_GET - ) + @mock.patch(IGM_PATH) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_update_instance_group_missing_project_id_should_execute_successfully(self, mock_hook, igm): + get_instance_group_manager_obj_mock = mock.MagicMock() + get_instance_group_manager_obj_mock.__class__ = InstanceGroupManager + mock_hook.return_value.get_instance_group_manager.side_effect = [ + get_instance_group_manager_obj_mock, + ] + igm.to_dict.return_value = GCE_INSTANCE_GROUP_MANAGER_GET op = ComputeEngineInstanceGroupUpdateManagerTemplateOperator( zone=GCE_ZONE, resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, - task_id="id", + task_id=TASK_ID, source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL, + update_policy=GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + result = op.execute(context=mock.MagicMock()) mock_hook.assert_called_once_with( api_version="beta", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.patch_instance_group_manager.assert_called_once_with( project_id=None, @@ -921,24 +1756,35 @@ def test_successful_instance_group_update_missing_project_id(self, mock_hook): ) assert result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_instance_group_update_no_instance_template_field(self, mock_hook): + @mock.patch(IGM_PATH) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_update_instance_group_no_instance_template_field_should_execute_successfully( + self, mock_hook, igm + ): instance_group_manager_no_template = deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET) del instance_group_manager_no_template["instanceTemplate"] - mock_hook.return_value.get_instance_group_manager.return_value = instance_group_manager_no_template + get_instance_group_manager_obj_mock = mock.MagicMock() + get_instance_group_manager_obj_mock.__class__ = InstanceGroupManager + mock_hook.return_value.get_instance_group_manager.side_effect = [ + get_instance_group_manager_obj_mock, + ] + igm.to_dict.return_value = instance_group_manager_no_template op = ComputeEngineInstanceGroupUpdateManagerTemplateOperator( project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, - task_id="id", + task_id=TASK_ID, source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL, + update_policy=GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + result = op.execute(context=mock.MagicMock()) mock_hook.assert_called_once_with( api_version="beta", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) expected_patch_no_instance_template = deepcopy(GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH) del expected_patch_no_instance_template["instanceTemplate"] @@ -951,24 +1797,33 @@ def test_successful_instance_group_update_no_instance_template_field(self, mock_ ) assert result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_instance_group_update_no_versions_field(self, mock_hook): + @mock.patch(IGM_PATH) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_update_instance_group_no_versions_field_should_execute_successfully(self, mock_hook, igm): instance_group_manager_no_versions = deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET) del instance_group_manager_no_versions["versions"] - mock_hook.return_value.get_instance_group_manager.return_value = instance_group_manager_no_versions + get_instance_group_manager_obj_mock = mock.MagicMock() + get_instance_group_manager_obj_mock.__class__ = InstanceGroupManager + mock_hook.return_value.get_instance_group_manager.side_effect = [ + get_instance_group_manager_obj_mock, + ] + igm.to_dict.return_value = instance_group_manager_no_versions op = ComputeEngineInstanceGroupUpdateManagerTemplateOperator( project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, - task_id="id", + task_id=TASK_ID, source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL, + update_policy=GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + result = op.execute(context=mock.MagicMock()) mock_hook.assert_called_once_with( api_version="beta", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) expected_patch_no_versions = deepcopy(GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH) del expected_patch_no_versions["versions"] @@ -981,56 +1836,32 @@ def test_successful_instance_group_update_no_versions_field(self, mock_hook): ) assert result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_instance_group_update_with_update_policy(self, mock_hook): - mock_hook.return_value.get_instance_group_manager.return_value = deepcopy( - GCE_INSTANCE_GROUP_MANAGER_GET - ) - op = ComputeEngineInstanceGroupUpdateManagerTemplateOperator( - project_id=GCP_PROJECT_ID, - zone=GCE_ZONE, - resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, - task_id="id", - update_policy=GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY, - source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, - destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL, - ) - result = op.execute(None) - mock_hook.assert_called_once_with( - api_version="beta", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, - ) - expected_patch_with_update_policy = deepcopy(GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH) - expected_patch_with_update_policy["updatePolicy"] = GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY - mock_hook.return_value.patch_instance_group_manager.assert_called_once_with( - project_id=GCP_PROJECT_ID, - zone=GCE_ZONE, - resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, - body=expected_patch_with_update_policy, - request_id=None, - ) - assert result - - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_successful_instance_group_update_with_request_id(self, mock_hook): - mock_hook.return_value.get_instance_group_manager.return_value = deepcopy( - GCE_INSTANCE_GROUP_MANAGER_GET - ) + @mock.patch(IGM_PATH) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_update_instance_group_with_request_id_should_execute_successfully(self, mock_hook, igm): + get_instance_group_manager_obj_mock = mock.MagicMock() + get_instance_group_manager_obj_mock.__class__ = InstanceGroupManager + mock_hook.return_value.get_instance_group_manager.side_effect = [ + get_instance_group_manager_obj_mock, + ] + igm.to_dict.return_value = GCE_INSTANCE_GROUP_MANAGER_GET op = ComputeEngineInstanceGroupUpdateManagerTemplateOperator( project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, - task_id="id", + task_id=TASK_ID, source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, request_id=GCE_INSTANCE_GROUP_MANAGER_REQUEST_ID, destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL, + update_policy=GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + result = op.execute(context=mock.MagicMock()) mock_hook.assert_called_once_with( api_version="beta", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.patch_instance_group_manager.assert_called_once_with( project_id=GCP_PROJECT_ID, @@ -1041,39 +1872,45 @@ def test_successful_instance_group_update_with_request_id(self, mock_hook): ) assert result - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_try_to_use_api_v1(self, _): - with pytest.raises(AirflowException) as ctx: + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_update_instance_group_try_to_use_api_v1_should_throw_ex(self, _): + with pytest.raises(AirflowException, match=r"Use beta api version or above"): ComputeEngineInstanceGroupUpdateManagerTemplateOperator( project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, - task_id="id", + task_id=TASK_ID, api_version="v1", source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL, ) - err = ctx.value - assert "Use beta api version or above" in str(err) - @mock.patch("airflow.providers.google.cloud.operators.compute.ComputeEngineHook") - def test_try_to_use_non_existing_template(self, mock_hook): - mock_hook.return_value.get_instance_group_manager.return_value = deepcopy( - GCE_INSTANCE_GROUP_MANAGER_GET - ) + @mock.patch(IGM_PATH) + @mock.patch(COMPUTE_ENGINE_HOOK_PATH) + def test_update_instance_group_try_to_use_non_existing_template_should_not_call_patch( + self, mock_hook, igm + ): + get_instance_group_manager_obj_mock = mock.MagicMock() + get_instance_group_manager_obj_mock.__class__ = InstanceGroupManager + mock_hook.return_value.get_instance_group_manager.side_effect = [ + get_instance_group_manager_obj_mock, + ] + igm.to_dict.return_value = GCE_INSTANCE_GROUP_MANAGER_GET op = ComputeEngineInstanceGroupUpdateManagerTemplateOperator( project_id=GCP_PROJECT_ID, zone=GCE_ZONE, resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, - task_id="id", + task_id=TASK_ID, source_template=GCE_INSTANCE_TEMPLATE_NON_EXISTING_URL, destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) - result = op.execute(None) + result = op.execute(context=mock.MagicMock()) mock_hook.assert_called_once_with( api_version="beta", - gcp_conn_id="google_cloud_default", - impersonation_chain=None, + gcp_conn_id=GCP_CONN_ID, + impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.patch_instance_group_manager.assert_not_called() assert result diff --git a/tests/providers/google/cloud/operators/test_compute_system.py b/tests/providers/google/cloud/operators/test_compute_system.py deleted file mode 100644 index 74b81e6032ab2f..00000000000000 --- a/tests/providers/google/cloud/operators/test_compute_system.py +++ /dev/null @@ -1,45 +0,0 @@ -# -# 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 pytest - -from tests.providers.google.cloud.operators.test_compute_system_helper import GCPComputeTestHelper -from tests.providers.google.cloud.utils.gcp_authenticator import GCP_COMPUTE_KEY -from tests.test_utils.gcp_system_helpers import CLOUD_DAG_FOLDER, GoogleSystemTest, provide_gcp_context - - -@pytest.mark.backend("mysql", "postgres") -@pytest.mark.credential_file(GCP_COMPUTE_KEY) -class GcpComputeExampleDagsSystemTest(GoogleSystemTest): - helper = GCPComputeTestHelper() - - @provide_gcp_context(GCP_COMPUTE_KEY) - def setUp(self): - super().setUp() - self.helper.delete_instance() - self.helper.create_instance() - - @provide_gcp_context(GCP_COMPUTE_KEY) - def tearDown(self): - self.helper.delete_instance() - super().tearDown() - - @provide_gcp_context(GCP_COMPUTE_KEY) - def test_run_example_dag_compute(self): - self.run_dag("example_gcp_compute", CLOUD_DAG_FOLDER) diff --git a/tests/providers/google/cloud/operators/test_compute_system_helper.py b/tests/providers/google/cloud/operators/test_compute_system_helper.py deleted file mode 100755 index f86c1e5902e5bb..00000000000000 --- a/tests/providers/google/cloud/operators/test_compute_system_helper.py +++ /dev/null @@ -1,215 +0,0 @@ -#!/usr/bin/env python -# -# 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 argparse -import os - -from tests.providers.google.cloud.utils.gcp_authenticator import GCP_COMPUTE_KEY, GcpAuthenticator -from tests.test_utils.logging_command_executor import CommandExecutor - -GCE_INSTANCE = os.environ.get("GCE_INSTANCE", "testinstance") -GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project") -GCE_INSTANCE_GROUP_MANAGER_NAME = os.environ.get("GCE_INSTANCE_GROUP_MANAGER_NAME", "instance-group-test") -GCE_ZONE = os.environ.get("GCE_ZONE", "europe-west1-b") -GCE_TEMPLATE_NAME = os.environ.get("GCE_TEMPLATE_NAME", "instance-template-test") -GCE_NEW_TEMPLATE_NAME = os.environ.get("GCE_NEW_TEMPLATE_NAME", "instance-template-test-new") - - -class GCPComputeTestHelper(CommandExecutor): - def delete_instance(self): - self.execute_cmd( - [ - "gcloud", - "beta", - "compute", - "--project", - GCP_PROJECT_ID, - "--quiet", - "--verbosity=none", - "instances", - "delete", - GCE_INSTANCE, - "--zone", - GCE_ZONE, - ] - ) - - def create_instance(self): - self.execute_cmd( - [ - "gcloud", - "beta", - "compute", - "--project", - GCP_PROJECT_ID, - "--quiet", - "instances", - "create", - GCE_INSTANCE, - "--zone", - GCE_ZONE, - ] - ) - - def delete_instance_group_and_template(self, silent=False): - self.execute_cmd( - [ - "gcloud", - "beta", - "compute", - "--project", - GCP_PROJECT_ID, - "--quiet", - "--verbosity=none", - "instance-groups", - "managed", - "delete", - GCE_INSTANCE_GROUP_MANAGER_NAME, - "--zone", - GCE_ZONE, - ], - silent=silent, - ) - self.execute_cmd( - [ - "gcloud", - "beta", - "compute", - "--project", - GCP_PROJECT_ID, - "--quiet", - "--verbosity=none", - "instance-templates", - "delete", - GCE_NEW_TEMPLATE_NAME, - ], - silent=silent, - ) - self.execute_cmd( - [ - "gcloud", - "beta", - "compute", - "--project", - GCP_PROJECT_ID, - "--quiet", - "--verbosity=none", - "instance-templates", - "delete", - GCE_TEMPLATE_NAME, - ], - silent=silent, - ) - - def create_instance_group_and_template(self): - self.execute_cmd( - [ - "gcloud", - "beta", - "compute", - "--project", - GCP_PROJECT_ID, - "--quiet", - "instance-templates", - "create", - GCE_TEMPLATE_NAME, - ] - ) - self.execute_cmd( - [ - "gcloud", - "beta", - "compute", - "--project", - GCP_PROJECT_ID, - "--quiet", - "instance-groups", - "managed", - "create", - GCE_INSTANCE_GROUP_MANAGER_NAME, - "--template", - GCE_TEMPLATE_NAME, - "--zone", - GCE_ZONE, - "--size=1", - ] - ) - self.execute_cmd( - [ - "gcloud", - "beta", - "compute", - "--project", - GCP_PROJECT_ID, - "--quiet", - "instance-groups", - "managed", - "wait-until-stable", - GCE_INSTANCE_GROUP_MANAGER_NAME, - "--zone", - GCE_ZONE, - ] - ) - - -if __name__ == "__main__": - parser = argparse.ArgumentParser( - description="Create or delete GCE instances/instance groups for system tests." - ) - parser.add_argument( - "--action", - dest="action", - required=True, - choices=( - "create-instance", - "delete-instance", - "create-instance-group", - "delete-instance-group", - "before-tests", - "after-tests", - ), - ) - action = parser.parse_args().action - - helper = GCPComputeTestHelper() - gcp_authenticator = GcpAuthenticator(GCP_COMPUTE_KEY) - helper.log.info("Starting action: %s", action) - - gcp_authenticator.gcp_store_authentication() - try: - gcp_authenticator.gcp_authenticate() - if action == "before-tests": - pass - elif action == "after-tests": - pass - elif action == "create-instance": - helper.create_instance() - elif action == "delete-instance": - helper.delete_instance() - elif action == "create-instance-group": - helper.create_instance_group_and_template() - elif action == "delete-instance-group": - helper.delete_instance_group_and_template() - else: - raise Exception(f"Unknown action: {action}") - finally: - gcp_authenticator.gcp_restore_authentication() - - helper.log.info("Finishing action: %s", action) diff --git a/tests/system/providers/google/cloud/compute/__init__.py b/tests/system/providers/google/cloud/compute/__init__.py new file mode 100644 index 00000000000000..13a83393a9124b --- /dev/null +++ b/tests/system/providers/google/cloud/compute/__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/system/providers/google/cloud/compute/example_compute.py b/tests/system/providers/google/cloud/compute/example_compute.py new file mode 100644 index 00000000000000..59c7e63a2f90d4 --- /dev/null +++ b/tests/system/providers/google/cloud/compute/example_compute.py @@ -0,0 +1,286 @@ +# +# 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. + +""" +Example Airflow DAG that starts, stops and sets the machine type of a Google Compute +Engine instance. + +""" + +from __future__ import annotations + +import os +from datetime import datetime + +from airflow import models +from airflow.models.baseoperator import chain +from airflow.operators.bash import BashOperator +from airflow.providers.google.cloud.operators.compute import ( + ComputeEngineDeleteInstanceOperator, + ComputeEngineDeleteInstanceTemplateOperator, + ComputeEngineInsertInstanceFromTemplateOperator, + ComputeEngineInsertInstanceOperator, + ComputeEngineInsertInstanceTemplateOperator, + ComputeEngineSetMachineTypeOperator, + ComputeEngineStartInstanceOperator, + ComputeEngineStopInstanceOperator, +) +from airflow.utils.trigger_rule import TriggerRule + +# [START howto_operator_gce_args_common] +ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID") +PROJECT_ID = os.environ.get("SYSTEM_TESTS_GCP_PROJECT") +DAG_ID = "cloud_compute" + +LOCATION = "europe-west1-b" +REGION = "europe-west1" +GCE_INSTANCE_NAME = "instance-1" +SHORT_MACHINE_TYPE_NAME = "n1-standard-1" +TEMPLATE_NAME = "instance-template" + +INSTANCE_TEMPLATE_BODY = { + "name": TEMPLATE_NAME, + "properties": { + "machine_type": SHORT_MACHINE_TYPE_NAME, + "disks": [ + { + "auto_delete": True, + "boot": True, + "device_name": TEMPLATE_NAME, + "initialize_params": { + "disk_size_gb": "10", + "disk_type": "pd-balanced", + "source_image": "projects/debian-cloud/global/images/debian-11-bullseye-v20220621", + }, + } + ], + "network_interfaces": [{"network": "global/networks/default"}], + }, +} +GCE_INSTANCE_BODY = { + "name": GCE_INSTANCE_NAME, + "machine_type": f"zones/{LOCATION}/machineTypes/{SHORT_MACHINE_TYPE_NAME}", + "disks": [ + { + "boot": True, + "device_name": GCE_INSTANCE_NAME, + "initialize_params": { + "disk_size_gb": "10", + "disk_type": f"zones/{LOCATION}/diskTypes/pd-balanced", + "source_image": "projects/debian-cloud/global/images/debian-11-bullseye-v20220621", + }, + } + ], + "network_interfaces": [ + { + "access_configs": [{"name": "External NAT", "network_tier": "PREMIUM"}], + "stack_type": "IPV4_ONLY", + "subnetwork": f"regions/{REGION}/subnetworks/default", + } + ], +} +GCE_INSTANCE_FROM_TEMPLATE_BODY = { + "name": GCE_INSTANCE_NAME, +} +# [END howto_operator_gce_args_common] + +with models.DAG( + DAG_ID, + schedule_interval="@once", + start_date=datetime(2021, 1, 1), + catchup=False, + tags=["example"], +) as dag: + # [START howto_operator_gce_insert] + gce_instance_insert = ComputeEngineInsertInstanceOperator( + task_id="gcp_compute_create_instance_task", + project_id=PROJECT_ID, + zone=LOCATION, + body=GCE_INSTANCE_BODY, + ) + # [END howto_operator_gce_insert] + + # Duplicate start for idempotence testing + # [START howto_operator_gce_insert_no_project_id] + gce_instance_insert2 = ComputeEngineInsertInstanceOperator( + task_id="gcp_compute_create_instance_task_2", + zone=LOCATION, + body=GCE_INSTANCE_BODY, + ) + # [END howto_operator_gce_insert_no_project_id] + + # [START howto_operator_gce_igm_insert_template] + gce_instance_template_insert = ComputeEngineInsertInstanceTemplateOperator( + task_id="gcp_compute_create_template_task", + project_id=PROJECT_ID, + body=INSTANCE_TEMPLATE_BODY, + ) + # [END howto_operator_gce_igm_insert_template] + + # Added to check for idempotence + # [START howto_operator_gce_igm_insert_template_no_project_id] + gce_instance_template_insert2 = ComputeEngineInsertInstanceTemplateOperator( + task_id="gcp_compute_create_template_task_2", + body=INSTANCE_TEMPLATE_BODY, + ) + # [END howto_operator_gce_igm_insert_template_no_project_id] + + # [START howto_operator_gce_insert_from_template] + gce_instance_insert_from_template = ComputeEngineInsertInstanceFromTemplateOperator( + task_id="gcp_compute_create_instance_from_template_task", + project_id=PROJECT_ID, + zone=LOCATION, + body=GCE_INSTANCE_FROM_TEMPLATE_BODY, + source_instance_template=f"global/instanceTemplates/{TEMPLATE_NAME}", + ) + # [END howto_operator_gce_insert_from_template] + + # Duplicate start for idempotence testing + # [START howto_operator_gce_insert_from_template_no_project_id] + gce_instance_insert_from_template2 = ComputeEngineInsertInstanceFromTemplateOperator( + task_id="gcp_compute_create_instance_from_template_task_2", + zone=LOCATION, + body=GCE_INSTANCE_FROM_TEMPLATE_BODY, + source_instance_template=f"global/instanceTemplates/{TEMPLATE_NAME}", + ) + # [END howto_operator_gce_insert_from_template_no_project_id] + + # [START howto_operator_gce_start] + gce_instance_start = ComputeEngineStartInstanceOperator( + task_id="gcp_compute_start_task", + project_id=PROJECT_ID, + zone=LOCATION, + resource_id=GCE_INSTANCE_NAME, + ) + # [END howto_operator_gce_start] + + # Duplicate start for idempotence testing + # [START howto_operator_gce_start_no_project_id] + gce_instance_start2 = ComputeEngineStartInstanceOperator( + task_id="gcp_compute_start_task_2", + zone=LOCATION, + resource_id=GCE_INSTANCE_NAME, + ) + # [END howto_operator_gce_start_no_project_id] + + # [START howto_operator_gce_stop] + gce_instance_stop = ComputeEngineStopInstanceOperator( + task_id="gcp_compute_stop_task", + project_id=PROJECT_ID, + zone=LOCATION, + resource_id=GCE_INSTANCE_NAME, + ) + # [END howto_operator_gce_stop] + gce_instance_stop.trigger_rule = TriggerRule.ALL_DONE + + # Duplicate stop for idempotence testing + # [START howto_operator_gce_stop_no_project_id] + gce_instance_stop2 = ComputeEngineStopInstanceOperator( + task_id="gcp_compute_stop_task_2", + zone=LOCATION, + resource_id=GCE_INSTANCE_NAME, + ) + # [END howto_operator_gce_stop_no_project_id] + gce_instance_stop2.trigger_rule = TriggerRule.ALL_DONE + + # [START howto_operator_gce_set_machine_type] + gce_set_machine_type = ComputeEngineSetMachineTypeOperator( + task_id="gcp_compute_set_machine_type", + project_id=PROJECT_ID, + zone=LOCATION, + resource_id=GCE_INSTANCE_NAME, + body={"machineType": f"zones/{LOCATION}/machineTypes/{SHORT_MACHINE_TYPE_NAME}"}, + ) + # [END howto_operator_gce_set_machine_type] + + # Duplicate set machine type for idempotence testing + # [START howto_operator_gce_set_machine_type_no_project_id] + gce_set_machine_type2 = ComputeEngineSetMachineTypeOperator( + zone=LOCATION, + resource_id=GCE_INSTANCE_NAME, + body={"machineType": f"zones/{LOCATION}/machineTypes/{SHORT_MACHINE_TYPE_NAME}"}, + task_id="gcp_compute_set_machine_type_2", + ) + # [END howto_operator_gce_set_machine_type_no_project_id] + + # [START howto_operator_gce_delete_no_project_id] + gce_instance_delete = ComputeEngineDeleteInstanceOperator( + task_id="gcp_compute_delete_instance_task", + zone=LOCATION, + resource_id=GCE_INSTANCE_NAME, + ) + # [END howto_operator_gce_delete_no_project_id] + gce_instance_delete.trigger_rule = TriggerRule.ALL_DONE + + # [START howto_operator_gce_delete_no_project_id] + gce_instance_delete2 = ComputeEngineDeleteInstanceOperator( + task_id="gcp_compute_delete_instance_task_2", + zone=LOCATION, + resource_id=GCE_INSTANCE_NAME, + ) + # [END howto_operator_gce_delete_no_project_id] + gce_instance_delete.trigger_rule = TriggerRule.ALL_DONE + + # [START howto_operator_gce_delete_new_template_no_project_id] + gce_instance_template_delete = ComputeEngineDeleteInstanceTemplateOperator( + task_id="gcp_compute_delete_template_task", + resource_id=TEMPLATE_NAME, + ) + # [END howto_operator_gce_delete_new_template_no_project_id] + gce_instance_template_delete.trigger_rule = TriggerRule.ALL_DONE + + bash_wait_operator = BashOperator(task_id="delay_bash_task", bash_command="sleep 3m") + + bash_wait_operator2 = BashOperator(task_id="delay_bash_task2", bash_command="sleep 3m") + + bash_wait_operator3 = BashOperator(task_id="delay_bash_task3", bash_command="sleep 3m") + + chain( + gce_instance_insert, + gce_instance_insert2, + bash_wait_operator, + gce_instance_delete, + gce_instance_template_insert, + gce_instance_template_insert2, + bash_wait_operator2, + gce_instance_insert_from_template, + gce_instance_insert_from_template2, + gce_instance_start, + gce_instance_start2, + gce_instance_stop, + gce_instance_stop2, + gce_set_machine_type, + gce_set_machine_type2, + gce_instance_delete2, + gce_instance_template_delete, + bash_wait_operator3, + ) + + # ### Everything below this line is not part of example ### + # ### Just for system tests purpose ### + from tests.system.utils.watcher import watcher + + # This test needs watcher in order to properly mark success/failure + # when "tearDown" task with trigger rule is part of the DAG + list(dag.tasks) >> watcher() + + +from tests.system.utils import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/tests/system/providers/google/cloud/compute/example_compute_igm.py b/tests/system/providers/google/cloud/compute/example_compute_igm.py new file mode 100644 index 00000000000000..54dc4e37118840 --- /dev/null +++ b/tests/system/providers/google/cloud/compute/example_compute_igm.py @@ -0,0 +1,254 @@ +# +# 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. + +""" +Example Airflow DAG that: +* creates a copy of existing Instance Template +* updates existing template in Instance Group Manager + +""" +from __future__ import annotations + +import os +from datetime import datetime + +from airflow import models +from airflow.models.baseoperator import chain +from airflow.operators.bash import BashOperator +from airflow.providers.google.cloud.operators.compute import ( + ComputeEngineCopyInstanceTemplateOperator, + ComputeEngineDeleteInstanceGroupManagerOperator, + ComputeEngineDeleteInstanceTemplateOperator, + ComputeEngineInsertInstanceGroupManagerOperator, + ComputeEngineInsertInstanceTemplateOperator, + ComputeEngineInstanceGroupUpdateManagerTemplateOperator, +) +from airflow.utils.trigger_rule import TriggerRule + +ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID") +PROJECT_ID = os.environ.get("SYSTEM_TESTS_GCP_PROJECT") + +LOCATION = "europe-west1-b" +REGION = "europe-west1" +SHORT_MACHINE_TYPE_NAME = "n1-standard-1" +DAG_ID = "cloud_compute_igm" + +# [START howto_operator_compute_template_copy_args] +TEMPLATE_NAME = "instance-template-igm-test" +NEW_TEMPLATE_NAME = "instance-template-test-new" + +INSTANCE_TEMPLATE_BODY = { + "name": TEMPLATE_NAME, + "properties": { + "machine_type": SHORT_MACHINE_TYPE_NAME, + "disks": [ + { + "auto_delete": True, + "boot": True, + "device_name": TEMPLATE_NAME, + "initialize_params": { + "disk_size_gb": "10", + "disk_type": "pd-balanced", + "source_image": "projects/debian-cloud/global/images/debian-11-bullseye-v20220621", + }, + } + ], + "network_interfaces": [{"network": "global/networks/default"}], + }, +} + +NEW_DESCRIPTION = "Test new description" +INSTANCE_TEMPLATE_BODY_UPDATE = { + "name": NEW_TEMPLATE_NAME, + "description": NEW_DESCRIPTION, + "properties": {"machine_type": "n1-standard-2"}, +} +# [END howto_operator_compute_template_copy_args] + +# [START howto_operator_compute_igm_update_template_args] +INSTANCE_GROUP_MANAGER_NAME = "instance-group-test" +INSTANCE_GROUP_MANAGER_BODY = { + "name": INSTANCE_GROUP_MANAGER_NAME, + "base_instance_name": INSTANCE_GROUP_MANAGER_NAME, + "instance_template": f"global/instanceTemplates/{TEMPLATE_NAME}", + "target_size": 1, +} + +SOURCE_TEMPLATE_URL = ( + f"https://www.googleapis.com/compute/beta/projects/{PROJECT_ID}/" + f"global/instanceTemplates/{TEMPLATE_NAME}" +) + + +DESTINATION_TEMPLATE_URL = ( + f"https://www.googleapis.com/compute/beta/projects/{PROJECT_ID}/" + f"global/instanceTemplates/{NEW_TEMPLATE_NAME}" +) + +UPDATE_POLICY = { + "type": "OPPORTUNISTIC", + "minimalAction": "RESTART", + "maxSurge": {"fixed": 1}, + "minReadySec": 1800, +} +# [END howto_operator_compute_igm_update_template_args] + + +with models.DAG( + DAG_ID, + schedule_interval="@once", + start_date=datetime(2021, 1, 1), + catchup=False, + tags=["example"], +) as dag: + # [START howto_operator_gce_igm_insert_template] + gce_instance_template_insert = ComputeEngineInsertInstanceTemplateOperator( + task_id="gcp_compute_create_template_task", + project_id=PROJECT_ID, + body=INSTANCE_TEMPLATE_BODY, + ) + # [END howto_operator_gce_igm_insert_template] + + # Added to check for idempotence + # [START howto_operator_gce_igm_insert_template_no_project_id] + gce_instance_template_insert2 = ComputeEngineInsertInstanceTemplateOperator( + task_id="gcp_compute_create_template_task_2", + body=INSTANCE_TEMPLATE_BODY, + ) + # [END howto_operator_gce_igm_insert_template_no_project_id] + + # [START howto_operator_gce_igm_copy_template] + gce_instance_template_copy = ComputeEngineCopyInstanceTemplateOperator( + task_id="gcp_compute_igm_copy_template_task", + project_id=PROJECT_ID, + resource_id=TEMPLATE_NAME, + body_patch=INSTANCE_TEMPLATE_BODY_UPDATE, + ) + # [END howto_operator_gce_igm_copy_template] + + # Added to check for idempotence + # [START howto_operator_gce_igm_copy_template_no_project_id] + gce_instance_template_copy2 = ComputeEngineCopyInstanceTemplateOperator( + task_id="gcp_compute_igm_copy_template_task_2", + resource_id=TEMPLATE_NAME, + body_patch=INSTANCE_TEMPLATE_BODY_UPDATE, + ) + # [END howto_operator_gce_igm_copy_template_no_project_id] + + # [START howto_operator_gce_insert_igm] + gce_igm_insert = ComputeEngineInsertInstanceGroupManagerOperator( + task_id="gcp_compute_create_group_task", + zone=LOCATION, + body=INSTANCE_GROUP_MANAGER_BODY, + project_id=PROJECT_ID, + ) + # [END howto_operator_gce_insert_igm] + + # Added to check for idempotence + # [START howto_operator_gce_insert_igm_no_project_id] + gce_igm_insert2 = ComputeEngineInsertInstanceGroupManagerOperator( + task_id="gcp_compute_create_group_task_2", + zone=LOCATION, + body=INSTANCE_GROUP_MANAGER_BODY, + ) + # [END howto_operator_gce_insert_igm_no_project_id] + + bash_wait_operator = BashOperator(task_id="delay_bash_task", bash_command="sleep 3m") + + # [START howto_operator_gce_igm_update_template] + gce_instance_group_manager_update_template = ComputeEngineInstanceGroupUpdateManagerTemplateOperator( + task_id="gcp_compute_igm_group_manager_update_template", + project_id=PROJECT_ID, + resource_id=INSTANCE_GROUP_MANAGER_NAME, + zone=LOCATION, + source_template=SOURCE_TEMPLATE_URL, + destination_template=DESTINATION_TEMPLATE_URL, + update_policy=UPDATE_POLICY, + ) + # [END howto_operator_gce_igm_update_template] + + # Added to check for idempotence (and without UPDATE_POLICY) + # [START howto_operator_gce_igm_update_template_no_project_id] + gce_instance_group_manager_update_template2 = ComputeEngineInstanceGroupUpdateManagerTemplateOperator( + task_id="gcp_compute_igm_group_manager_update_template_2", + resource_id=INSTANCE_GROUP_MANAGER_NAME, + zone=LOCATION, + source_template=SOURCE_TEMPLATE_URL, + destination_template=DESTINATION_TEMPLATE_URL, + ) + # [END howto_operator_gce_igm_update_template_no_project_id] + + bash_wait_operator1 = BashOperator(task_id="delay_bash_task_1", bash_command="sleep 3m") + + # [START howto_operator_gce_delete_old_template_no_project_id] + gce_instance_template_old_delete = ComputeEngineDeleteInstanceTemplateOperator( + task_id="gcp_compute_delete_old_template_task", + resource_id=TEMPLATE_NAME, + ) + # [END howto_operator_gce_delete_old_template_no_project_id] + gce_instance_template_old_delete.trigger_rule = TriggerRule.ALL_DONE + + # [START howto_operator_gce_delete_new_template_no_project_id] + gce_instance_template_new_delete = ComputeEngineDeleteInstanceTemplateOperator( + task_id="gcp_compute_delete_new_template_task", + resource_id=NEW_TEMPLATE_NAME, + ) + # [END howto_operator_gce_delete_new_template_no_project_id] + gce_instance_template_new_delete.trigger_rule = TriggerRule.ALL_DONE + + # [START howto_operator_gce_delete_igm_no_project_id] + gce_igm_delete = ComputeEngineDeleteInstanceGroupManagerOperator( + task_id="gcp_compute_delete_group_task", + resource_id=INSTANCE_GROUP_MANAGER_NAME, + zone=LOCATION, + ) + # [END howto_operator_gce_delete_igm_no_project_id] + gce_igm_delete.trigger_rule = TriggerRule.ALL_DONE + + bash_wait_operator2 = BashOperator(task_id="delay_bash_task_2", bash_command="sleep 3m") + + chain( + gce_instance_template_insert, + gce_instance_template_insert2, + gce_instance_template_copy, + gce_instance_template_copy2, + gce_igm_insert, + gce_igm_insert2, + bash_wait_operator, + gce_instance_group_manager_update_template, + gce_instance_group_manager_update_template2, + bash_wait_operator1, + gce_igm_delete, + gce_instance_template_old_delete, + gce_instance_template_new_delete, + bash_wait_operator2, + ) + + # ### Everything below this line is not part of example ### + # ### Just for system tests purpose ### + from tests.system.utils.watcher import watcher + + # This test needs watcher in order to properly mark success/failure + # when "tearDown" task with trigger rule is part of the DAG + list(dag.tasks) >> watcher() + + +from tests.system.utils import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/tests/system/providers/google/cloud/compute/example_compute_ssh.py b/tests/system/providers/google/cloud/compute/example_compute_ssh.py new file mode 100644 index 00000000000000..48107d8ae0f423 --- /dev/null +++ b/tests/system/providers/google/cloud/compute/example_compute_ssh.py @@ -0,0 +1,129 @@ +# 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. + +""" +Example Airflow DAG that starts, stops and sets the machine type of a Google Compute +Engine instance. + +""" + +from __future__ import annotations + +import os +from datetime import datetime + +from airflow import models +from airflow.models.baseoperator import chain +from airflow.providers.google.cloud.hooks.compute_ssh import ComputeEngineSSHHook +from airflow.providers.google.cloud.operators.compute import ( + ComputeEngineDeleteInstanceOperator, + ComputeEngineInsertInstanceOperator, +) +from airflow.providers.ssh.operators.ssh import SSHOperator +from airflow.utils.trigger_rule import TriggerRule + +# [START howto_operator_gce_args_common] +ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID") +PROJECT_ID = os.environ.get("SYSTEM_TESTS_GCP_PROJECT") + +DAG_ID = "cloud_compute_ssh" +LOCATION = "europe-west1-b" +REGION = "europe-west1" +GCE_INSTANCE_NAME = "instance-1" +SHORT_MACHINE_TYPE_NAME = "n1-standard-1" +GCE_INSTANCE_BODY = { + "name": GCE_INSTANCE_NAME, + "machine_type": f"zones/{LOCATION}/machineTypes/{SHORT_MACHINE_TYPE_NAME}", + "disks": [ + { + "boot": True, + "device_name": GCE_INSTANCE_NAME, + "initialize_params": { + "disk_size_gb": "10", + "disk_type": f"zones/{LOCATION}/diskTypes/pd-balanced", + "source_image": "projects/debian-cloud/global/images/debian-11-bullseye-v20220621", + }, + } + ], + "network_interfaces": [ + { + "access_configs": [{"name": "External NAT", "network_tier": "PREMIUM"}], + "stack_type": "IPV4_ONLY", + "subnetwork": f"regions/{REGION}/subnetworks/default", + } + ], +} +# [END howto_operator_gce_args_common] + +with models.DAG( + DAG_ID, + schedule_interval="@once", + start_date=datetime(2021, 1, 1), + catchup=False, + tags=["example"], +) as dag: + # [START howto_operator_gce_insert] + gce_instance_insert = ComputeEngineInsertInstanceOperator( + task_id="gcp_compute_create_instance_task", + project_id=PROJECT_ID, + zone=LOCATION, + body=GCE_INSTANCE_BODY, + ) + # [END howto_operator_gce_insert] + + # [START howto_execute_command_on_remote] + metadata_without_iap_tunnel = SSHOperator( + task_id="metadata_without_iap_tunnel", + ssh_hook=ComputeEngineSSHHook( + user="username", + instance_name=GCE_INSTANCE_NAME, + zone=LOCATION, + use_oslogin=False, + use_iap_tunnel=False, + ), + command="echo metadata_without_iap_tunnel", + ) + # [END howto_execute_command_on_remote] + + # [START howto_operator_gce_delete_no_project_id] + gce_instance_delete = ComputeEngineDeleteInstanceOperator( + task_id="gcp_compute_delete_instance_task", + zone=LOCATION, + resource_id=GCE_INSTANCE_NAME, + ) + # [END howto_operator_gce_delete_no_project_id] + gce_instance_delete.trigger_rule = TriggerRule.ALL_DONE + + chain( + gce_instance_insert, + metadata_without_iap_tunnel, + gce_instance_delete, + ) + + # ### Everything below this line is not part of example ### + # ### Just for system tests purpose ### + from tests.system.utils.watcher import watcher + + # This test needs watcher in order to properly mark success/failure + # when "tearDown" task with trigger rule is part of the DAG + list(dag.tasks) >> watcher() + + +from tests.system.utils import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag)