Skip to content

Commit

Permalink
[AIRFLOW-6732] Add GoogleAdsHook and GoogleAdsToGcsOperator (#7692)
Browse files Browse the repository at this point in the history
* [AIRFLOW-6732] Add GoogleAdsHook and GoogleAdsToGcsOperator

Provide ability to connect to Google Ads API using service account and execute arbitrary search (API call) using Google Ads Query language.
Convert the API results to CSV and store in designated GCS bucket.

Add google-ads to setup.py as a gcp package.

* Update operators-and-hooks-ref for Google Ads hook and operator

* Make updates for minor  PR comments

- Add api_version as an attribute of GoogleAdsHook
- Update CLIENT_IDS in example_google_ads DAG to make it clear that they are dummy values
- Add Google Ads API link in hook and operator docstrings

* Add tests

* Add flush to tempfile

Required to ensure the csvfile within the context manager is correctly flushed before being uploaded to GCS. Without this flush, the written data is not correctly saved before upload

* Fix docs

* Move code to google.ads directory

* fixup! Move code to google.ads directory

* fixup! fixup! Move code to google.ads directory

* fixup! fixup! fixup! Move code to google.ads directory

* fixup! fixup! fixup! fixup! Move code to google.ads directory

Co-authored-by: Todd de Quincey <[email protected]>
Co-authored-by: Todd de Quincey <[email protected]>
Co-authored-by: Tomek Urbaszek <[email protected]>
  • Loading branch information
4 people committed Mar 20, 2020
1 parent ff3700d commit 6b9b214
Show file tree
Hide file tree
Showing 15 changed files with 643 additions and 0 deletions.
16 changes: 16 additions & 0 deletions airflow/providers/google/ads/__init__.py
Original file line number Diff line number Diff line change
@@ -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.
16 changes: 16 additions & 0 deletions airflow/providers/google/ads/example_dags/__init__.py
Original file line number Diff line number Diff line change
@@ -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.
81 changes: 81 additions & 0 deletions airflow/providers/google/ads/example_dags/example_ads.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
#
# 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 shows how to use GoogleAdsToGcsOperator.
"""
import os

from airflow import models
from airflow.providers.google.ads.operators.ads import GoogleAdsToGcsOperator
from airflow.utils import dates

# [START howto_google_ads_env_variables]
CLIENT_IDS = ["1111111111", "2222222222"]
BUCKET = os.environ.get("GOOGLE_ADS_BUCKET", "gs://test-google-ads-bucket")
GCS_OBJ_PATH = "folder_name/google-ads-api-results.csv"
QUERY = """
SELECT
segments.date,
customer.id,
campaign.id,
ad_group.id,
ad_group_ad.ad.id,
metrics.impressions,
metrics.clicks,
metrics.conversions,
metrics.all_conversions,
metrics.cost_micros
FROM
ad_group_ad
WHERE
segments.date >= '2020-02-01'
AND segments.date <= '2020-02-29'
"""

FIELDS_TO_EXTRACT = [
"segments.date.value",
"customer.id.value",
"campaign.id.value",
"ad_group.id.value",
"ad_group_ad.ad.id.value",
"metrics.impressions.value",
"metrics.clicks.value",
"metrics.conversions.value",
"metrics.all_conversions.value",
"metrics.cost_micros.value",
]

# [END howto_google_ads_env_variables]

default_args = {"start_date": dates.days_ago(1)}

with models.DAG(
"example_google_ads",
default_args=default_args,
schedule_interval=None, # Override to match your needs
) as dag:
# [START howto_google_ads_to_gcs_operator]
run_operator = GoogleAdsToGcsOperator(
client_ids=CLIENT_IDS,
query=QUERY,
attributes=FIELDS_TO_EXTRACT,
obj=GCS_OBJ_PATH,
bucket=BUCKET,
task_id="run_operator",
)
# [END howto_google_ads_to_gcs_operator]
16 changes: 16 additions & 0 deletions airflow/providers/google/ads/hooks/__init__.py
Original file line number Diff line number Diff line change
@@ -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.
151 changes: 151 additions & 0 deletions airflow/providers/google/ads/hooks/ads.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
#
# 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 Ad hook.
"""
from tempfile import NamedTemporaryFile
from typing import IO, Any, Dict, Generator, List

from google.ads.google_ads.client import GoogleAdsClient
from google.ads.google_ads.errors import GoogleAdsException
from google.ads.google_ads.v2.types import GoogleAdsRow
from google.api_core.page_iterator import GRPCIterator
from google.auth.exceptions import GoogleAuthError

from airflow import AirflowException
from airflow.hooks.base_hook import BaseHook


class GoogleAdsHook(BaseHook):
"""
Hook for the Google Ads API
.. seealso::
For more information on the Google Ads API, take a look at the API docs:
https://developers.google.com/google-ads/api/docs/start
:param gcp_conn_id: The connection ID with the service account details.
:type gcp_conn_id: str
:param google_ads_conn_id: The connection ID with the details of Google Ads config.yaml file.
:type google_ads_conn_id: str
:return: list of Google Ads Row object(s)
:rtype: list[GoogleAdsRow]
"""

def __init__(
self,
gcp_conn_id: str = "google_cloud_default",
google_ads_conn_id: str = "google_ads_default",
api_version: str = "v2",
) -> None:
self.gcp_conn_id = gcp_conn_id
self.google_ads_conn_id = google_ads_conn_id
self.gcp_conn_id = gcp_conn_id
self.api_version = api_version
self.google_ads_config: Dict[str, Any] = {}

def _get_service(self) -> GoogleAdsClient:
"""Connects and authenticates with the Google Ads API using a service account"""
with NamedTemporaryFile("w", suffix=".json") as secrets_temp:
self._get_config()
self._update_config_with_secret(secrets_temp)
try:
client = GoogleAdsClient.load_from_dict(self.google_ads_config)
return client.get_service("GoogleAdsService", version=self.api_version)
except GoogleAuthError as e:
self.log.error("Google Auth Error: %s", e)
raise

def _get_config(self) -> None:
"""
Gets google ads connection from meta db and sets google_ads_config attribute with returned config file
"""
conn = self.get_connection(self.google_ads_conn_id)
if "google_ads_client" not in conn.extra_dejson:
raise AirflowException("google_ads_client not found in extra field")

self.google_ads_config = conn.extra_dejson["google_ads_client"]

def _update_config_with_secret(self, secrets_temp: IO[str]) -> None:
"""
Gets GCP secret from connection and saves the contents to the temp file
Updates google ads config with file path of the temp file containing the secret
Note, the secret must be passed as a file path for Google Ads API
"""
secret_conn = self.get_connection(self.gcp_conn_id)
secret = secret_conn.extra_dejson["extra__google_cloud_platform__keyfile_dict"]
secrets_temp.write(secret)
secrets_temp.flush()

self.google_ads_config["path_to_private_key_file"] = secrets_temp.name

def search(
self, client_ids: List[str], query: str, page_size: int = 10000, **kwargs
) -> List[GoogleAdsRow]:
"""
Pulls data from the Google Ads API
:param client_ids: Google Ads client ID(s) to query the API for.
:type client_ids: List[str]
:param query: Google Ads Query Language query.
:type query: str
:param page_size: Number of results to return per page. Max 10000.
:type page_size: int
:return: Google Ads API response, converted to Google Ads Row objects
:rtype: list[GoogleAdsRow]
"""
service = self._get_service()
iterators = (
service.search(client_id, query=query, page_size=page_size, **kwargs)
for client_id in client_ids
)
self.log.info("Fetched Google Ads Iterators")

return self._extract_rows(iterators)

def _extract_rows(
self, iterators: Generator[GRPCIterator, None, None]
) -> List[GoogleAdsRow]:
"""
Convert Google Page Iterator (GRPCIterator) objects to Google Ads Rows
:param iterators: List of Google Page Iterator (GRPCIterator) objects
:type iterators: generator[GRPCIterator, None, None]
:return: API response for all clients in the form of Google Ads Row object(s)
:rtype: list[GoogleAdsRow]
"""
try:
self.log.info("Extracting data from returned Google Ads Iterators")
return [row for iterator in iterators for row in iterator]
except GoogleAdsException as e:
self.log.error(
"Request ID %s failed with status %s and includes the following errors:",
e.request_id,
e.error.code().name,
)
for error in e.failure.errors:
self.log.error("\tError with message: %s.", error.message)
if error.location:
for field_path_element in error.location.field_path_elements:
self.log.error(
"\t\tOn field: %s", field_path_element.field_name
)
raise
16 changes: 16 additions & 0 deletions airflow/providers/google/ads/operators/__init__.py
Original file line number Diff line number Diff line change
@@ -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.

0 comments on commit 6b9b214

Please sign in to comment.