Skip to content

Commit

Permalink
Add SalesforceToGcsOperator (#10760)
Browse files Browse the repository at this point in the history
Adds SalesforceToGcsOperator that allows users to transfer data from
Salesforce to GCS bucket.

Co-authored-by: Tomek Urbaszek <[email protected]>
  • Loading branch information
chipmyersjr and turbaszek committed Oct 30, 2020
1 parent 9687b3b commit 2f703df
Show file tree
Hide file tree
Showing 9 changed files with 493 additions and 1 deletion.
2 changes: 1 addition & 1 deletion CONTRIBUTING.rst
Original file line number Diff line number Diff line change
Expand Up @@ -584,7 +584,7 @@ apache.hive amazon,microsoft.mssql,mysql,presto,samba,vertica
apache.livy http
dingding http
discord http
google amazon,apache.cassandra,cncf.kubernetes,facebook,microsoft.azure,microsoft.mssql,mysql,postgres,presto,sftp
google amazon,apache.cassandra,cncf.kubernetes,facebook,microsoft.azure,microsoft.mssql,mysql,postgres,presto,salesforce,sftp
hashicorp google
microsoft.azure google,oracle
microsoft.mssql odbc
Expand Down
1 change: 1 addition & 0 deletions airflow/providers/dependencies.json
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
"mysql",
"postgres",
"presto",
"salesforce",
"sftp"
],
"hashicorp": [
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
# 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 SalesforceToGcsOperator.
"""
import os

from airflow import models
from airflow.providers.google.cloud.operators.bigquery import (
BigQueryCreateEmptyDatasetOperator,
BigQueryCreateEmptyTableOperator,
BigQueryDeleteDatasetOperator,
BigQueryExecuteQueryOperator,
)
from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
from airflow.providers.google.cloud.transfers.gcs_to_bigquery import GCSToBigQueryOperator
from airflow.providers.google.cloud.transfers.salesforce_to_gcs import SalesforceToGcsOperator
from airflow.utils.dates import days_ago

GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project")
GCS_BUCKET = os.environ.get("GCS_BUCKET", "airflow-salesforce-bucket")
DATASET_NAME = os.environ.get("SALESFORCE_DATASET_NAME", "salesforce_test_dataset")
TABLE_NAME = os.environ.get("SALESFORCE_TABLE_NAME", "salesforce_test_datatable")
GCS_OBJ_PATH = os.environ.get("GCS_OBJ_PATH", "results.csv")
QUERY = "SELECT Id, Name, Company, Phone, Email, CreatedDate, LastModifiedDate, IsDeleted FROM Lead"
GCS_CONN_ID = os.environ.get("GCS_CONN_ID", "google_cloud_default")
SALESFORCE_CONN_ID = os.environ.get("SALESFORCE_CONN_ID", "salesforce_default")


with models.DAG(
"example_salesforce_to_gcs",
schedule_interval=None, # Override to match your needs
start_date=days_ago(1),
) as dag:
create_bucket = GCSCreateBucketOperator(
task_id="create_bucket",
bucket_name=GCS_BUCKET,
project_id=GCP_PROJECT_ID,
gcp_conn_id=GCS_CONN_ID,
)

# [START howto_operator_salesforce_to_gcs]
gcs_upload_task = SalesforceToGcsOperator(
query=QUERY,
include_deleted=True,
bucket_name=GCS_BUCKET,
object_name=GCS_OBJ_PATH,
salesforce_conn_id=SALESFORCE_CONN_ID,
export_format='csv',
coerce_to_timestamp=False,
record_time_added=False,
gcp_conn_id=GCS_CONN_ID,
task_id="upload_to_gcs",
dag=dag,
)
# [END howto_operator_salesforce_to_gcs]

create_dataset = BigQueryCreateEmptyDatasetOperator(
task_id="create_dataset", dataset_id=DATASET_NAME, project_id=GCP_PROJECT_ID, gcp_conn_id=GCS_CONN_ID
)

create_table = BigQueryCreateEmptyTableOperator(
task_id="create_table",
dataset_id=DATASET_NAME,
table_id=TABLE_NAME,
schema_fields=[
{'name': 'id', 'type': 'STRING', 'mode': 'NULLABLE'},
{'name': 'name', 'type': 'STRING', 'mode': 'NULLABLE'},
{'name': 'company', 'type': 'STRING', 'mode': 'NULLABLE'},
{'name': 'phone', 'type': 'STRING', 'mode': 'NULLABLE'},
{'name': 'email', 'type': 'STRING', 'mode': 'NULLABLE'},
{'name': 'createddate', 'type': 'STRING', 'mode': 'NULLABLE'},
{'name': 'lastmodifieddate', 'type': 'STRING', 'mode': 'NULLABLE'},
{'name': 'isdeleted', 'type': 'BOOL', 'mode': 'NULLABLE'},
],
)

load_csv = GCSToBigQueryOperator(
task_id='gcs_to_bq',
bucket=GCS_BUCKET,
source_objects=[GCS_OBJ_PATH],
destination_project_dataset_table=f"{DATASET_NAME}.{TABLE_NAME}",
write_disposition='WRITE_TRUNCATE',
)

read_data_from_gcs = BigQueryExecuteQueryOperator(
task_id="read_data_from_gcs",
sql=f"SELECT COUNT(*) FROM `{GCP_PROJECT_ID}.{DATASET_NAME}.{TABLE_NAME}`",
use_legacy_sql=False,
)

delete_bucket = GCSDeleteBucketOperator(
task_id="delete_bucket",
bucket_name=GCS_BUCKET,
)

delete_dataset = BigQueryDeleteDatasetOperator(
task_id="delete_dataset",
project_id=GCP_PROJECT_ID,
dataset_id=DATASET_NAME,
delete_contents=True,
)

create_bucket >> gcs_upload_task >> load_csv
create_dataset >> create_table >> load_csv
load_csv >> read_data_from_gcs
read_data_from_gcs >> delete_bucket
read_data_from_gcs >> delete_dataset
125 changes: 125 additions & 0 deletions airflow/providers/google/cloud/transfers/salesforce_to_gcs.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,125 @@
# 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.

import os
import tempfile
from typing import Dict, Optional

from airflow.models import BaseOperator
from airflow.providers.google.cloud.hooks.gcs import GCSHook
from airflow.providers.salesforce.hooks.salesforce import SalesforceHook


class SalesforceToGcsOperator(BaseOperator):
"""
Submits Salesforce query and uploads results to Google Cloud Storage
.. seealso::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:SalesforceToGcsOperator`
:param query: The query to make to Salesforce.
:type query: str
:param bucket_name: The bucket to upload to.
:type bucket_name: str
:param object_name: The object name to set when uploading the file.
:type object_name: str
:param salesforce_conn_id: the name of the connection that has the parameters
we need to connect to Salesforce.
:type conn_id: str
:param include_deleted: True if the query should include deleted records.
:type include_deleted: bool
:param query_params: Additional optional arguments
:type query_params: dict
:param export_format: Desired format of files to be exported.
:type export_format: str
:param coerce_to_timestamp: True if you want all datetime fields to be converted into Unix timestamps.
False if you want them to be left in the same format as they were in Salesforce.
Leaving the value as False will result in datetimes being strings. Default: False
:type coerce_to_timestamp: bool
:param record_time_added: True if you want to add a Unix timestamp field
to the resulting data that marks when the data was fetched from Salesforce. Default: False
:type record_time_added: bool
:param gzip: Option to compress local file or file data for upload
:type gzip: bool
:param gcp_conn_id: the name of the connection that has the parameters we need to connect to GCS.
:type conn_id: str
"""

template_fields = (
'query',
'bucket_name',
'object_name',
)
template_ext = ('.sql',)

def __init__(
self,
*,
query: str,
bucket_name: str,
object_name: str,
salesforce_conn_id: str,
include_deleted: bool = False,
query_params: Optional[dict] = None,
export_format: str = "csv",
coerce_to_timestamp: bool = False,
record_time_added: bool = False,
gzip: bool = False,
gcp_conn_id: str = "google_cloud_default",
**kwargs,
):
super().__init__(**kwargs)
self.query = query
self.bucket_name = bucket_name
self.object_name = object_name
self.salesforce_conn_id = salesforce_conn_id
self.export_format = export_format
self.coerce_to_timestamp = coerce_to_timestamp
self.record_time_added = record_time_added
self.gzip = gzip
self.gcp_conn_id = gcp_conn_id
self.include_deleted = include_deleted
self.query_params = query_params

def execute(self, context: Dict):
salesforce = SalesforceHook(conn_id=self.salesforce_conn_id)
response = salesforce.make_query(
query=self.query, include_deleted=self.include_deleted, query_params=self.query_params
)

with tempfile.TemporaryDirectory() as tmp:
path = os.path.join(tmp, "salesforce_temp_file")
salesforce.write_object_to_file(
query_results=response["records"],
filename=path,
fmt=self.export_format,
coerce_to_timestamp=self.coerce_to_timestamp,
record_time_added=self.record_time_added,
)

hook = GCSHook(gcp_conn_id=self.gcp_conn_id)
hook.upload(
bucket_name=self.bucket_name,
object_name=self.object_name,
filename=path,
gzip=self.gzip,
)

gcs_uri = "gs://{}/{}".format(self.bucket_name, self.object_name)
self.log.info("%s uploaded to GCS", gcs_uri)
return gcs_uri
49 changes: 49 additions & 0 deletions docs/howto/operator/google/transfer/salesforce_to_gcs.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
.. 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.
Salesforce To GCS Operators
==============================

.. contents::
:depth: 1
:local:

Prerequisite Tasks
^^^^^^^^^^^^^^^^^^

.. include:: /howto/operator/google/_partials/prerequisite_tasks.rst

.. _howto/operator:SalesforceToGcsOperator:

SalesforceToGcsOperator
------------------------------

Use the
:class:`~airflow.providers.google.cloud.transfers.salesforce_to_gcs.SalesforceToGcsOperator`
to execute a Salesforce query to fetch data and load it to GCS.

.. exampleinclude:: /../airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py
:language: python
:start-after: [START howto_operator_salesforce_to_gcs]
:end-before: [END howto_operator_salesforce_to_gcs]

Reference
^^^^^^^^^

For further information, look at:

* `Simple Salesforce Documentation <https://simple-salesforce.readthedocs.io/en/latest/>`__
5 changes: 5 additions & 0 deletions docs/operators-and-hooks-ref.rst
Original file line number Diff line number Diff line change
Expand Up @@ -992,6 +992,11 @@ These integrations allow you to copy data from/to Google Cloud.
- :doc:`How to use <howto/operator/google/transfer/facebook_ads_to_gcs>`
- :mod:`airflow.providers.google.cloud.transfers.facebook_ads_to_gcs`

* - `Salesforce <https://www.salesforce.com/>`__
- `Google Cloud Storage (GCS) <https://cloud.google.com/gcs/>`__
- :doc:`How to use <howto/operator/google/transfer/salesforce_to_gcs>`
- :mod:`airflow.providers.google.cloud.transfers.salesforce_to_gcs`


* - `Google Ads <https://ads.google.com/>`__
- `Google Cloud Storage (GCS) <https://cloud.google.com/gcs/>`__
Expand Down

0 comments on commit 2f703df

Please sign in to comment.