Skip to content

Commit

Permalink
Add GCSToTrinoOperator (#21704)
Browse files Browse the repository at this point in the history
  • Loading branch information
rsg17 committed Feb 27, 2022
1 parent df31902 commit 942f8fd
Show file tree
Hide file tree
Showing 11 changed files with 435 additions and 0 deletions.
1 change: 1 addition & 0 deletions CONTRIBUTING.rst
Original file line number Diff line number Diff line change
Expand Up @@ -691,6 +691,7 @@ salesforce tableau
sftp ssh
slack http
snowflake slack
trino google
========================== ===========================

.. END PACKAGE DEPENDENCIES HERE
Expand Down
3 changes: 3 additions & 0 deletions airflow/providers/dependencies.json
Original file line number Diff line number Diff line change
Expand Up @@ -84,5 +84,8 @@
],
"snowflake": [
"slack"
],
"trino": [
"google"
]
}
17 changes: 17 additions & 0 deletions airflow/providers/trino/example_dags/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
#
# 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.
46 changes: 46 additions & 0 deletions airflow/providers/trino/example_dags/example_gcs_to_trino.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
#
# 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 DAG using GCSToTrinoOperator.
"""

import os
from datetime import datetime

from airflow import models
from airflow.providers.trino.transfers.gcs_to_trino import GCSToTrinoOperator

BUCKET = os.environ.get("GCP_GCS_BUCKET", "test28397yeo")
PATH_TO_FILE = os.environ.get("GCP_PATH", "path/to/file")
TRINO_TABLE = os.environ.get("TRINO_TABLE", "test_table")

with models.DAG(
dag_id="example_gcs_to_trino",
schedule_interval='@once', # Override to match your needs
start_date=datetime(2022, 1, 1),
catchup=False,
tags=["example"],
) as dag:
# [START gcs_csv_to_trino_table]
gcs_csv_to_trino_table = GCSToTrinoOperator(
task_id="gcs_csv_to_trino_table",
source_bucket=BUCKET,
source_object=PATH_TO_FILE,
trino_table=TRINO_TABLE,
)
# [END gcs_csv_to_trino_table]
6 changes: 6 additions & 0 deletions airflow/providers/trino/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,12 @@ hooks:
python-modules:
- airflow.providers.trino.hooks.trino

transfers:
- source-integration-name: Google Cloud Storage (GCS)
target-integration-name: Trino
how-to-guide: /docs/apache-airflow-providers-trino/operators/transfer/gcs_to_trino.rst
python-module: airflow.providers.trino.transfers.gcs_to_trino

hook-class-names: # deprecated - to be removed after providers add dependency on Airflow 2.2.0+
- airflow.providers.trino.hooks.trino.TrinoHook

Expand Down
17 changes: 17 additions & 0 deletions airflow/providers/trino/transfers/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
#
# 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.
124 changes: 124 additions & 0 deletions airflow/providers/trino/transfers/gcs_to_trino.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
#
# 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 Cloud Storage to Trino operator."""

import csv
import json
from tempfile import NamedTemporaryFile
from typing import TYPE_CHECKING, Iterable, Optional, Sequence, Union

from airflow.models import BaseOperator
from airflow.providers.google.cloud.hooks.gcs import GCSHook
from airflow.providers.trino.hooks.trino import TrinoHook

if TYPE_CHECKING:
from airflow.utils.context import Context


class GCSToTrinoOperator(BaseOperator):
"""
Loads a csv file from Google Cloud Storage into a Trino table.
Assumptions:
1. CSV file should not have headers
2. Trino table with requisite columns is already created
3. Optionally, a separate JSON file with headers can be provided
:param source_bucket: Source GCS bucket that contains the csv
:param source_object: csv file including the path
:param trino_table: trino table to upload the data
:param trino_conn_id: destination trino connection
:param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud and
interact with the Google Cloud Storage service.
:param schema_fields: The names of the columns to fill in the table. If schema_fields is
provided, any path provided in the schema object will be
:param schema_object: JSON file with schema fields
:param delegate_to: The account to impersonate using domain-wide delegation of authority,
if any. For this to work, the service account making the request must have
domain-wide delegation enabled.
:param impersonation_chain: Optional 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.
"""

template_fields: Sequence[str] = (
'source_bucket',
'source_object',
'trino_table',
)

def __init__(
self,
*,
source_bucket: str,
source_object: str,
trino_table: str,
trino_conn_id: str = "trino_default",
gcp_conn_id: str = "google_cloud_default",
schema_fields: Optional[Iterable[str]] = None,
schema_object: Optional[str] = None,
delegate_to: Optional[str] = None,
impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
**kwargs,
) -> None:
super().__init__(**kwargs)
self.source_bucket = source_bucket
self.source_object = source_object
self.trino_table = trino_table
self.trino_conn_id = trino_conn_id
self.gcp_conn_id = gcp_conn_id
self.schema_fields = schema_fields
self.schema_object = schema_object
self.delegate_to = delegate_to
self.impersonation_chain = impersonation_chain

def execute(self, context: 'Context') -> None:
gcs_hook = GCSHook(
gcp_conn_id=self.gcp_conn_id,
delegate_to=self.delegate_to,
impersonation_chain=self.impersonation_chain,
)

trino_hook = TrinoHook(trino_conn_id=self.trino_conn_id)

with NamedTemporaryFile("w+") as temp_file:
self.log.info("Downloading data from %s", self.source_object)
gcs_hook.download(
bucket_name=self.source_bucket,
object_name=self.source_object,
filename=temp_file.name,
)

data = csv.reader(temp_file)
rows = (tuple(row) for row in data)
self.log.info("Inserting data into %s", self.trino_table)
if self.schema_fields:
trino_hook.insert_rows(table=self.trino_table, rows=rows, target_fields=self.schema_fields)
elif self.schema_object:
blob = gcs_hook.download(
bucket_name=self.source_bucket,
object_name=self.schema_object,
)
schema_fields = json.loads(blob.decode("utf-8"))
trino_hook.insert_rows(table=self.trino_table, rows=rows, target_fields=schema_fields)
else:
trino_hook.insert_rows(table=self.trino_table, rows=rows)
7 changes: 7 additions & 0 deletions docs/apache-airflow-providers-trino/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,12 @@
Content
-------

.. toctree::
:maxdepth: 1
:caption: Guides

TrinoTransferOperator types <operators/transfer/gcs_to_trino>

.. toctree::
:maxdepth: 1
:caption: References
Expand All @@ -32,6 +38,7 @@ Content
:maxdepth: 1
:caption: Resources

Example DAGs <https://github.com/apache/airflow/tree/main/airflow/providers/trino/example_dags>
PyPI Repository <https://pypi.org/project/apache-airflow-providers-trino/>
Installing from sources <installing-providers-from-sources>

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
.. 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.
Google Cloud Storage to Trino Transfer Operator
===============================================

Google has a service `Google Cloud Storage <https://cloud.google.com/storage/>`__. This service is
used to store large data from various applications.

`Trino <https://trino.io/>`__ is an open source, fast, distributed SQL query engine for running interactive
analytic queries against data sources of all sizes ranging from gigabytes to petabytes. Trino allows
querying data where it lives, including Hive, Cassandra, relational databases or even proprietary data stores.
A single Trino query can combine data from multiple sources, allowing for analytics across your entire
organization.


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

.. include::/operators/_partials/prerequisite_tasks.rst
.. _howto/operator:GCSToPresto:

Load CSV from GCS to Trino Table
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^

To load a CSV file from Google Cloud Storage to a Trino table you can use the
:class:`~airflow.providers.trino.transfers.gcs_to_trino.GCSToTrinoOperator`.

This operator assumes that CSV does not have headers and the data is corresponding to the columns in a
pre-existing presto table. Optionally, you can provide schema as tuple/list of strings or as a path to a
JSON file in the same bucket as the CSV file.

.. exampleinclude:: /../../airflow/providers/trino/example_dags/example_gcs_to_trino.py
:language: python
:dedent: 4
:start-after: [START gcs_csv_to_trino_table]
:end-before: [END gcs_csv_to_trino_table]
17 changes: 17 additions & 0 deletions tests/providers/trino/transfers/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
#
# 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.
Loading

0 comments on commit 942f8fd

Please sign in to comment.