Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
61 changes: 51 additions & 10 deletions airflow/providers/google/cloud/transfers/bigquery_to_gcs.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,9 @@

from airflow import AirflowException
from airflow.models import BaseOperator
from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook
from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook, BigQueryJob
from airflow.providers.google.cloud.links.bigquery import BigQueryTableLink
from airflow.providers.google.cloud.triggers.bigquery import BigQueryInsertJobTrigger

if TYPE_CHECKING:
from airflow.utils.context import Context
Expand Down Expand Up @@ -79,6 +80,7 @@ class BigQueryToGCSOperator(BaseOperator):
:param force_rerun: If True then operator will use hash of uuid as job id suffix
:param reattach_states: Set of BigQuery job's states in case of which we should reattach
to the job. Should be other than final states.
:param deferrable: Run operator in the deferrable mode
"""

template_fields: Sequence[str] = (
Expand Down Expand Up @@ -111,6 +113,7 @@ def __init__(
job_id: str | None = None,
force_rerun: bool = False,
reattach_states: set[str] | None = None,
deferrable: bool = False,
**kwargs,
) -> None:
super().__init__(**kwargs)
Expand All @@ -132,6 +135,7 @@ def __init__(
self.force_rerun = force_rerun
self.reattach_states: set[str] = reattach_states or set()
self.hook: BigQueryHook | None = None
self.deferrable = deferrable

@staticmethod
def _handle_job_error(job: ExtractJob) -> None:
Expand Down Expand Up @@ -169,6 +173,24 @@ def _prepare_configuration(self):
configuration["extract"]["printHeader"] = self.print_header
return configuration

def _submit_job(
self,
hook: BigQueryHook,
job_id: str,
configuration: dict,
) -> BigQueryJob:
# Submit a new job without waiting for it to complete.

return hook.insert_job(
configuration=configuration,
project_id=hook.project_id,
location=self.location,
job_id=job_id,
timeout=self.result_timeout,
retry=self.result_retry,
nowait=True,
)

def execute(self, context: Context):
self.log.info(
"Executing extract of %s into: %s",
Expand All @@ -195,15 +217,7 @@ def execute(self, context: Context):

try:
self.log.info("Executing: %s", configuration)
job: ExtractJob = hook.insert_job(
job_id=job_id,
configuration=configuration,
project_id=self.project_id,
location=self.location,
timeout=self.result_timeout,
retry=self.result_retry,
)
self._handle_job_error(job)
job: ExtractJob = self._submit_job(hook=hook, job_id=job_id, configuration=configuration)
except Conflict:
# If the job already exists retrieve it
job = hook.get_job(
Expand Down Expand Up @@ -232,3 +246,30 @@ def execute(self, context: Context):
project_id=project_id,
table_id=table_id,
)

if self.deferrable:
self.defer(
timeout=self.execution_timeout,
trigger=BigQueryInsertJobTrigger(
conn_id=self.gcp_conn_id,
job_id=job_id,
project_id=self.hook.project_id,
),
method_name="execute_complete",
)
else:
job.result(timeout=self.result_timeout, retry=self.result_retry)

def execute_complete(self, context: Context, event: dict[str, Any]):
"""
Callback for when the trigger fires - returns immediately.
Relies on trigger to throw an exception, otherwise it assumes execution was
successful.
"""
if event["status"] == "error":
raise AirflowException(event["message"])
self.log.info(
"%s completed with response %s ",
self.task_id,
event["message"],
)
70 changes: 69 additions & 1 deletion tests/providers/google/cloud/transfers/test_bigquery_to_gcs.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,12 @@
from unittest import mock
from unittest.mock import MagicMock

import pytest
from google.cloud.bigquery.retry import DEFAULT_RETRY

from airflow.exceptions import TaskDeferred
from airflow.providers.google.cloud.transfers.bigquery_to_gcs import BigQueryToGCSOperator
from airflow.providers.google.cloud.triggers.bigquery import BigQueryInsertJobTrigger

TASK_ID = "test-bq-create-table-operator"
TEST_DATASET = "test-dataset"
Expand Down Expand Up @@ -64,6 +67,7 @@ def test_execute(self, mock_hook):
mock_hook.return_value.split_tablename.return_value = (PROJECT_ID, TEST_DATASET, TEST_TABLE_ID)
mock_hook.return_value.generate_job_id.return_value = real_job_id
mock_hook.return_value.insert_job.return_value = MagicMock(job_id="real_job_id", error_result=False)
mock_hook.return_value.project_id = PROJECT_ID

operator = BigQueryToGCSOperator(
task_id=TASK_ID,
Expand All @@ -80,8 +84,72 @@ def test_execute(self, mock_hook):
mock_hook.return_value.insert_job.assert_called_once_with(
job_id="123456_hash",
configuration=expected_configuration,
project_id=None,
project_id=PROJECT_ID,
location=None,
timeout=None,
retry=DEFAULT_RETRY,
nowait=True,
)

@mock.patch("airflow.providers.google.cloud.transfers.bigquery_to_gcs.BigQueryHook")
def test_execute_deferrable_mode(self, mock_hook):
source_project_dataset_table = f"{PROJECT_ID}:{TEST_DATASET}.{TEST_TABLE_ID}"
destination_cloud_storage_uris = ["gs://some-bucket/some-file.txt"]
compression = "NONE"
export_format = "CSV"
field_delimiter = ","
print_header = True
labels = {"k1": "v1"}
job_id = "123456"
hash_ = "hash"
real_job_id = f"{job_id}_{hash_}"

expected_configuration = {
"extract": {
"sourceTable": {
"projectId": "test-project-id",
"datasetId": "test-dataset",
"tableId": "test-table-id",
},
"compression": "NONE",
"destinationUris": ["gs://some-bucket/some-file.txt"],
"destinationFormat": "CSV",
"fieldDelimiter": ",",
"printHeader": True,
},
"labels": {"k1": "v1"},
}

mock_hook.return_value.split_tablename.return_value = (PROJECT_ID, TEST_DATASET, TEST_TABLE_ID)
mock_hook.return_value.generate_job_id.return_value = real_job_id
mock_hook.return_value.insert_job.return_value = MagicMock(job_id="real_job_id", error_result=False)
mock_hook.return_value.project_id = PROJECT_ID

operator = BigQueryToGCSOperator(
project_id=PROJECT_ID,
task_id=TASK_ID,
source_project_dataset_table=source_project_dataset_table,
destination_cloud_storage_uris=destination_cloud_storage_uris,
compression=compression,
export_format=export_format,
field_delimiter=field_delimiter,
print_header=print_header,
labels=labels,
deferrable=True,
)
with pytest.raises(TaskDeferred) as exc:
operator.execute(context=mock.MagicMock())

assert isinstance(
exc.value.trigger, BigQueryInsertJobTrigger
), "Trigger is not a BigQueryInsertJobTrigger"

mock_hook.return_value.insert_job.assert_called_once_with(
configuration=expected_configuration,
job_id="123456_hash",
project_id=PROJECT_ID,
location=None,
timeout=None,
retry=DEFAULT_RETRY,
nowait=True,
)
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
#
# 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.
"""
Airflow System Test DAG that verifies BigQueryToGCSOperator.
"""
from __future__ import annotations

import os
from datetime import datetime

from airflow import models
from airflow.providers.google.cloud.operators.bigquery import (
BigQueryCreateEmptyDatasetOperator,
BigQueryCreateEmptyTableOperator,
BigQueryDeleteDatasetOperator,
)
from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
from airflow.providers.google.cloud.transfers.bigquery_to_gcs import BigQueryToGCSOperator
from airflow.utils.trigger_rule import TriggerRule

ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
DAG_ID = "bigquery_to_gcs_async"

DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
PROJECT_ID = os.environ.get("SYSTEM_TESTS_GCP_PROJECT")
BUCKET_FILE = "test.csv"
TABLE = "test"


with models.DAG(
DAG_ID,
schedule="@once",
start_date=datetime(2021, 1, 1),
catchup=False,
tags=["example", "bigquery"],
) as dag:
create_bucket = GCSCreateBucketOperator(
task_id="create_bucket", bucket_name=BUCKET_NAME, project_id=PROJECT_ID
)

create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)

create_table = BigQueryCreateEmptyTableOperator(
task_id="create_table",
dataset_id=DATASET_NAME,
table_id=TABLE,
schema_fields=[
{"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
{"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
],
)

bigquery_to_gcs_async = BigQueryToGCSOperator(
task_id="bigquery_to_gcs",
source_project_dataset_table=f"{DATASET_NAME}.{TABLE}",
destination_cloud_storage_uris=[f"gs://{BUCKET_NAME}/{BUCKET_FILE}"],
deferrable=True,
)

delete_bucket = GCSDeleteBucketOperator(
task_id="delete_bucket", bucket_name=BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
)

delete_dataset = BigQueryDeleteDatasetOperator(
task_id="delete_dataset",
dataset_id=DATASET_NAME,
delete_contents=True,
trigger_rule=TriggerRule.ALL_DONE,
)

(
# TEST SETUP
[create_bucket, create_dataset]
>> create_table
# TEST BODY
>> bigquery_to_gcs_async
# TEST TEARDOWN
>> [delete_bucket, delete_dataset]
)

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)