Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
36 commits
Select commit Hold shift + click to select a range
34ccdb2
Created HTTP to GCS Operator
Apr 23, 2025
013c153
Created Unit Test for HTTP to GCS Operator
josuegen Apr 23, 2025
c1f01da
Precommit fixes
josuegen Apr 23, 2025
0c021d7
Added documentation and test as per pre-commit checks
josuegen Apr 23, 2025
2bd5aab
Removed commits on files made by pre-commit
josuegen Apr 23, 2025
9b2ecbd
Fixerd unit test for HTTP to GCS operator
josuegen Apr 28, 2025
6d27946
Merge branch 'main' into feature/http_to_gcs
josuegen Apr 29, 2025
710f4ed
Fixed Unit testing for HTTP to GCS
josuegen Apr 30, 2025
7acdb46
Merge branch 'main' into feature/http_to_gcs
josuegen Apr 30, 2025
4d2b9d9
Merge branch 'main' into feature/http_to_gcs
josuegen Apr 30, 2025
0f41e22
Merge branch 'main' into feature/http_to_gcs
josuegen Apr 30, 2025
618df78
Merge branch 'main' into feature/http_to_gcs
josuegen May 1, 2025
b4e5386
Merge branch 'main' into feature/http_to_gcs
josuegen May 2, 2025
359ac78
Merge branch 'main' into feature/http_to_gcs
josuegen May 6, 2025
a5045ad
Fixed unit testing
josuegen May 6, 2025
084e8c9
Merge branch 'main' into feature/http_to_gcs
josuegen May 6, 2025
fd8c46a
Fixed unit testing
josuegen May 6, 2025
b75144d
Updated cross-dependency specification for breeze checks
josuegen May 7, 2025
f143e92
Fixed provider documentation for HTTP to GCS
josuegen May 9, 2025
28a057a
Fixed breeze unit testing for HTTP to GCS
josuegen May 9, 2025
7d9df9c
Fixed unit testing for HTTP to GCS
josuegen May 9, 2025
51a1f1a
Fixed order in selected-providers-list-as-string for breeze test
josuegen May 13, 2025
8277a0c
Fixed unit test for HTTP Hook in HTTPToGCS
josuegen May 13, 2025
8eac7e4
Fixed unit test for HTTP Hook in HTTPToGCS
josuegen May 13, 2025
4cb7826
Fixed order in selected-providers-list-as-string for breeze test
josuegen May 13, 2025
8a24484
Merge branch 'main' into feature/http_to_gcs
josuegen May 13, 2025
e986b94
Fixed order in selected-providers-list-as-string for breeze test
josuegen May 13, 2025
55e088f
Merge branch 'main' into feature/http_to_gcs
josuegen May 13, 2025
4e78205
Merge branch 'main' into feature/http_to_gcs
josuegen May 14, 2025
716d215
Merge branch 'main' into feature/http_to_gcs
josuegen May 17, 2025
a95e17f
Merge branch 'main' into feature/http_to_gcs
josuegen May 19, 2025
b638fac
Removed ORM calls when managing connections in system tests
josuegen May 19, 2025
b72c3f3
Added fix for breeze unit test individual-providers-test-types-list-a…
josuegen May 22, 2025
359fa8e
Typo fix for breeze unit test individual-providers-test-types-list-as…
josuegen May 22, 2025
ff3ebc8
Merge branch 'main' into feature/http_to_gcs
josuegen May 22, 2025
5a1594b
Merge branch 'main' into feature/http_to_gcs
josuegen May 28, 2025
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
21 changes: 12 additions & 9 deletions dev/breeze/tests/test_selective_checks.py
Original file line number Diff line number Diff line change
Expand Up @@ -690,7 +690,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
"providers/http/tests/file.py",
),
{
"selected-providers-list-as-string": "amazon apache.livy dbt.cloud dingding discord http",
"selected-providers-list-as-string": "amazon apache.livy dbt.cloud dingding discord google http",
"all-python-versions": f"['{DEFAULT_PYTHON_MAJOR_MINOR_VERSION}']",
"all-python-versions-list-as-string": DEFAULT_PYTHON_MAJOR_MINOR_VERSION,
"python-versions": f"['{DEFAULT_PYTHON_MAJOR_MINOR_VERSION}']",
Expand All @@ -710,8 +710,8 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
"providers-test-types-list-as-strings-in-json": json.dumps(
[
{
"description": "amazon...apache.livy,d",
"test_types": "Providers[amazon] Providers[apache.livy,dbt.cloud,dingding,discord,http]",
"description": "amazon...google",
"test_types": "Providers[amazon] Providers[apache.livy,dbt.cloud,dingding,discord,http] Providers[google]",
}
]
),
Expand All @@ -721,9 +721,12 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
"description": "amazon...apache.livy",
"test_types": "Providers[amazon] Providers[apache.livy]",
},
{"description": "dbt.cloud", "test_types": "Providers[dbt.cloud]"},
{"description": "dingding", "test_types": "Providers[dingding]"},
{
"description": "dbt.cloud...dingding",
"test_types": "Providers[dbt.cloud] Providers[dingding]",
},
{"description": "discord", "test_types": "Providers[discord]"},
{"description": "google", "test_types": "Providers[google]"},
{"description": "http", "test_types": "Providers[http]"},
]
),
Expand Down Expand Up @@ -1947,7 +1950,7 @@ def test_expected_output_push(
{
"selected-providers-list-as-string": "amazon apache.beam apache.cassandra apache.kafka "
"cncf.kubernetes common.compat common.sql "
"facebook google hashicorp microsoft.azure microsoft.mssql mysql "
"facebook google hashicorp http microsoft.azure microsoft.mssql mysql "
"openlineage oracle postgres presto salesforce samba sftp ssh trino",
"all-python-versions": f"['{DEFAULT_PYTHON_MAJOR_MINOR_VERSION}']",
"all-python-versions-list-as-string": DEFAULT_PYTHON_MAJOR_MINOR_VERSION,
Expand All @@ -1958,7 +1961,7 @@ def test_expected_output_push(
"skip-providers-tests": "false",
"docs-build": "true",
"docs-list-as-string": "apache-airflow helm-chart amazon apache.beam apache.cassandra "
"apache.kafka cncf.kubernetes common.compat common.sql facebook google hashicorp microsoft.azure "
"apache.kafka cncf.kubernetes common.compat common.sql facebook google hashicorp http microsoft.azure "
"microsoft.mssql mysql openlineage oracle postgres "
"presto salesforce samba sftp ssh trino",
"skip-pre-commits": ALL_SKIPPED_COMMITS_IF_NO_UI,
Expand All @@ -1973,7 +1976,7 @@ def test_expected_output_push(
"description": "amazon...google",
"test_types": "Providers[amazon] Providers[apache.beam,apache.cassandra,"
"apache.kafka,cncf.kubernetes,common.compat,common.sql,facebook,"
"hashicorp,microsoft.azure,microsoft.mssql,mysql,"
"hashicorp,http,microsoft.azure,microsoft.mssql,mysql,"
"openlineage,oracle,postgres,presto,salesforce,samba,sftp,ssh,trino] "
"Providers[google]",
}
Expand Down Expand Up @@ -2235,7 +2238,7 @@ def test_upgrade_to_newer_dependencies(
("providers/google/docs/some_file.rst",),
{
"docs-list-as-string": "amazon apache.beam apache.cassandra apache.kafka "
"cncf.kubernetes common.compat common.sql facebook google hashicorp "
"cncf.kubernetes common.compat common.sql facebook google hashicorp http "
"microsoft.azure microsoft.mssql mysql openlineage oracle "
"postgres presto salesforce samba sftp ssh trino",
},
Expand Down
61 changes: 61 additions & 0 deletions providers/google/docs/operators/transfer/http_to_gcs.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
.. 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.


HTTP to Google Cloud Storage 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.
HTTP (Hypertext Transfer Protocol) HTTP is an application layer protocol
designed to transfer information between networked devices
and runs on top of other layers of the network protocol stack.

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

.. include:: /operators/_partials/prerequisite_tasks.rst

.. _howto/operator:HttpToGCSOperator:

Operator
^^^^^^^^

Transfer files between HTTP and Google Storage is performed with the
:class:`~airflow.providers.google.cloud.transfers.http_to_gcs.HttpToGCSOperator` operator.

Use :ref:`Jinja templating <concepts:jinja-templating>` with
:template-fields:`airflow.providers.google.cloud.transfers.http_to_gcs.HttpToGCSOperator`
to define values dynamically.

Copying single files
--------------------

The following Operator copies a single file.

.. exampleinclude:: /../../google/tests/system/google/cloud/gcs/example_http_to_gcs.py
:language: python
:dedent: 4
:start-after: [START howto_transfer_http_to_gcs]
:end-before: [END howto_transfer_http_to_gcs]

Reference
^^^^^^^^^

For more information, see

* `Google Cloud Storage Documentation <https://cloud.google.com/storage/>`__
4 changes: 4 additions & 0 deletions providers/google/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -1089,6 +1089,10 @@ transfers:
target-integration-name: Google Cloud Storage (GCS)
python-module: airflow.providers.google.cloud.transfers.azure_blob_to_gcs
how-to-guide: /docs/apache-airflow-providers-google/operators/transfer/azure_blob_to_gcs.rst
- source-integration-name: Hypertext Transfer Protocol (HTTP)
target-integration-name: Google Cloud Storage (GCS)
python-module: airflow.providers.google.cloud.transfers.http_to_gcs
how-to-guide: /docs/apache-airflow-providers-google/operators/transfer/http_to_gcs.rst

connection-types:
- hook-class-name: airflow.providers.google.common.hooks.base_google.GoogleBaseHook
Expand Down
4 changes: 4 additions & 0 deletions providers/google/pyproject.toml
Original file line number Diff line number Diff line change
Expand Up @@ -202,6 +202,9 @@ dependencies = [
"trino" = [
"apache-airflow-providers-trino"
]
"http" = [
"apache-airflow-providers-http"
]

[dependency-groups]
dev = [
Expand All @@ -215,6 +218,7 @@ dev = [
"apache-airflow-providers-common-compat",
"apache-airflow-providers-common-sql",
"apache-airflow-providers-facebook",
"apache-airflow-providers-http",
"apache-airflow-providers-microsoft-azure",
"apache-airflow-providers-microsoft-mssql",
"apache-airflow-providers-mysql",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,193 @@
#
# 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 operator to move data from HTTP endpoint to GCS."""

from __future__ import annotations

from functools import cached_property
from typing import TYPE_CHECKING, Any

from airflow.models import BaseOperator
from airflow.providers.google.cloud.hooks.gcs import GCSHook
from airflow.providers.http.hooks.http import HttpHook

if TYPE_CHECKING:
from collections.abc import Sequence

from requests.auth import AuthBase

from airflow.utils.context import Context


class HttpToGCSOperator(BaseOperator):
"""
Calls an endpoint on an HTTP system to execute an action and store the result in GCS.

:param http_conn_id: The :ref:`http connection<howto/connection:http>` to run
the operator against
:param endpoint: The relative part of the full url. (templated)
:param method: The HTTP method to use, default = "POST"
:param data: The data to pass. POST-data in POST/PUT and params
in the URL for a GET request. (templated)
:param headers: The HTTP headers to be added to the GET request
:param response_check: A check against the 'requests' response object.
The callable takes the response object as the first positional argument
and optionally any number of keyword arguments available in the context dictionary.
It should return True for 'pass' and False otherwise.
:param response_filter: A function allowing you to manipulate the response
text. e.g response_filter=lambda response: json.loads(response.text).
The callable takes the response object as the first positional argument
and optionally any number of keyword arguments available in the context dictionary.
:param extra_options: Extra options for the 'requests' library, see the
'requests' documentation (options to modify timeout, ssl, etc.)
:param log_response: Log the response (default: False)
:param auth_type: The auth type for the service
:param tcp_keep_alive: Enable TCP Keep Alive for the connection.
:param tcp_keep_alive_idle: The TCP Keep Alive Idle parameter (corresponds to ``socket.TCP_KEEPIDLE``).
:param tcp_keep_alive_count: The TCP Keep Alive count parameter (corresponds to ``socket.TCP_KEEPCNT``)
:param tcp_keep_alive_interval: The TCP Keep Alive interval parameter (corresponds to
``socket.TCP_KEEPINTVL``)
:param gcp_conn_id: The connection ID to use when fetching connection info.
: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.
:param bucket_name: The bucket to upload to.
:param object_name: The object name to set when uploading the file.
:param mime_type: The file mime type set when uploading the file.
:param gzip: Option to compress local file or file data for upload
:param encoding: bytes encoding for file data if provided as string
:param chunk_size: Blob chunk size.
:param timeout: Request timeout in seconds.
:param num_max_attempts: Number of attempts to try to upload the file.
:param metadata: The metadata to be uploaded with the file.
:param cache_contro: Cache-Control metadata field.
:param user_project: The identifier of the Google Cloud project to bill for the request. Required for Requester Pays buckets.
"""

template_fields: Sequence[str] = (
"http_conn_id",
"endpoint",
"data",
"headers",
"gcp_conn_id",
"bucket_name",
"object_name",
)
template_fields_renderers = {"headers": "json", "data": "py"}
template_ext: Sequence[str] = ()
ui_color = "#f4a460"

def __init__(
self,
*,
endpoint: str | None = None,
method: str = "GET",
data: Any = None,
headers: dict[str, str] | None = None,
extra_options: dict[str, Any] | None = None,
http_conn_id: str = "http_default",
log_response: bool = False,
auth_type: type[AuthBase] | None = None,
tcp_keep_alive: bool = True,
tcp_keep_alive_idle: int = 120,
tcp_keep_alive_count: int = 20,
tcp_keep_alive_interval: int = 30,
gcp_conn_id: str = "google_cloud_default",
impersonation_chain: str | Sequence[str] | None = None,
bucket_name: str,
object_name: str,
mime_type: str | None = None,
gzip: bool = False,
encoding: str | None = None,
chunk_size: int | None = None,
timeout: int | None = None,
num_max_attempts: int = 3,
metadata: dict | None = None,
cache_control: str | None = None,
user_project: str | None = None,
**kwargs,
):
super().__init__(**kwargs)
self.http_conn_id = http_conn_id
self.method = method
self.endpoint = endpoint
self.headers = headers or {}
self.data = data or {}
self.extra_options = extra_options or {}
self.log_response = log_response
self.auth_type = auth_type
self.tcp_keep_alive = tcp_keep_alive
self.tcp_keep_alive_idle = tcp_keep_alive_idle
self.tcp_keep_alive_count = tcp_keep_alive_count
self.tcp_keep_alive_interval = tcp_keep_alive_interval
self.gcp_conn_id = gcp_conn_id
self.impersonation_chain = impersonation_chain
self.bucket_name = bucket_name
self.object_name = object_name
self.mime_type = mime_type
self.gzip = gzip
self.encoding = encoding
self.chunk_size = chunk_size
self.timeout = timeout
self.num_max_attempts = num_max_attempts
self.metadata = metadata
self.cache_control = cache_control
self.user_project = user_project

@cached_property
def http_hook(self) -> HttpHook:
"""Create and return an HttpHook."""
return HttpHook(
self.method,
http_conn_id=self.http_conn_id,
auth_type=self.auth_type,
tcp_keep_alive=self.tcp_keep_alive,
tcp_keep_alive_idle=self.tcp_keep_alive_idle,
tcp_keep_alive_count=self.tcp_keep_alive_count,
tcp_keep_alive_interval=self.tcp_keep_alive_interval,
)

@cached_property
def gcs_hook(self) -> GCSHook:
"""Create and return an GCSHook."""
return GCSHook(gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain)

def execute(self, context: Context):
self.log.info("Calling HTTP method")
response = self.http_hook.run(
endpoint=self.endpoint, data=self.data, headers=self.headers, extra_options=self.extra_options
)

self.log.info("Uploading to GCS")
self.gcs_hook.upload(
data=response.content,
bucket_name=self.bucket_name,
object_name=self.object_name,
mime_type=self.mime_type,
gzip=self.gzip,
encoding=self.encoding or response.encoding,
chunk_size=self.chunk_size,
timeout=self.timeout,
num_max_attempts=self.num_max_attempts,
metadata=self.metadata,
cache_control=self.cache_control,
user_project=self.user_project,
)
Original file line number Diff line number Diff line change
Expand Up @@ -1338,6 +1338,12 @@ def get_provider_info():
"python-module": "airflow.providers.google.cloud.transfers.azure_blob_to_gcs",
"how-to-guide": "/docs/apache-airflow-providers-google/operators/transfer/azure_blob_to_gcs.rst",
},
{
"source-integration-name": "Hypertext Transfer Protocol (HTTP)",
"target-integration-name": "Google Cloud Storage (GCS)",
"python-module": "airflow.providers.google.cloud.transfers.http_to_gcs",
"how-to-guide": "/docs/apache-airflow-providers-google/operators/transfer/http_to_gcs.rst",
},
],
"connection-types": [
{
Expand Down
Loading
Loading