This is an automated email from the ASF dual-hosted git repository.
potiuk pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git
The following commit(s) were added to refs/heads/main by this push:
new 9e49d91a0a Add deferrable mode to `GCSObjectUpdateSensor` (#30579)
9e49d91a0a is described below
commit 9e49d91a0a9462696d9b3cebc55275e704943fbf
Author: Phani Kumar <[email protected]>
AuthorDate: Sat Apr 22 16:23:18 2023 +0530
Add deferrable mode to `GCSObjectUpdateSensor` (#30579)
---
airflow/providers/google/cloud/sensors/gcs.py | 39 +++++-
airflow/providers/google/cloud/triggers/gcs.py | 105 ++++++++++++++-
.../operators/cloud/gcs.rst | 10 ++
tests/providers/google/cloud/sensors/test_gcs.py | 44 ++++++-
tests/providers/google/cloud/triggers/test_gcs.py | 143 ++++++++++++++++++++-
tests/providers/google/cloud/utils/airflow_util.py | 74 +++++++++++
.../google/cloud/gcs/example_gcs_sensor.py | 6 +
7 files changed, 416 insertions(+), 5 deletions(-)
diff --git a/airflow/providers/google/cloud/sensors/gcs.py
b/airflow/providers/google/cloud/sensors/gcs.py
index 397a66542c..fd36e624c0 100644
--- a/airflow/providers/google/cloud/sensors/gcs.py
+++ b/airflow/providers/google/cloud/sensors/gcs.py
@@ -29,7 +29,7 @@ from google.cloud.storage.retry import DEFAULT_RETRY
from airflow.exceptions import AirflowException
from airflow.providers.google.cloud.hooks.gcs import GCSHook
-from airflow.providers.google.cloud.triggers.gcs import GCSBlobTrigger
+from airflow.providers.google.cloud.triggers.gcs import GCSBlobTrigger,
GCSCheckBlobUpdateTimeTrigger
from airflow.sensors.base import BaseSensorOperator, poke_mode_only
if TYPE_CHECKING:
@@ -184,6 +184,7 @@ class GCSObjectUpdateSensor(BaseSensorOperator):
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
(templated).
+ :param deferrable: Run sensor in deferrable mode
"""
template_fields: Sequence[str] = (
@@ -200,6 +201,7 @@ class GCSObjectUpdateSensor(BaseSensorOperator):
ts_func: Callable = ts_function,
google_cloud_conn_id: str = "google_cloud_default",
impersonation_chain: str | Sequence[str] | None = None,
+ deferrable: bool = False,
**kwargs,
) -> None:
@@ -209,6 +211,7 @@ class GCSObjectUpdateSensor(BaseSensorOperator):
self.ts_func = ts_func
self.google_cloud_conn_id = google_cloud_conn_id
self.impersonation_chain = impersonation_chain
+ self.deferrable = deferrable
def poke(self, context: Context) -> bool:
self.log.info("Sensor checks existence of : %s, %s", self.bucket,
self.object)
@@ -218,6 +221,38 @@ class GCSObjectUpdateSensor(BaseSensorOperator):
)
return hook.is_updated_after(self.bucket, self.object,
self.ts_func(context))
+ def execute(self, context: Context) -> None:
+ """Airflow runs this method on the worker and defers using the
trigger."""
+ if self.deferrable is False:
+ super().execute(context)
+ else:
+ self.defer(
+ timeout=timedelta(seconds=self.timeout),
+ trigger=GCSCheckBlobUpdateTimeTrigger(
+ bucket=self.bucket,
+ object_name=self.object,
+ target_date=self.ts_func(context),
+ poke_interval=self.poke_interval,
+ google_cloud_conn_id=self.google_cloud_conn_id,
+ hook_params={
+ "delegate_to": self.delegate_to,
+ "impersonation_chain": self.impersonation_chain,
+ },
+ ),
+ method_name="execute_complete",
+ )
+
+ def execute_complete(self, context: dict[str, Any], event: dict[str, str]
| None = None) -> str:
+ """Callback for when the trigger fires."""
+ if event:
+ if event["status"] == "success":
+ self.log.info(
+ "Checking last updated time for object %s in bucket : %s",
self.object, self.bucket
+ )
+ return event["message"]
+ raise AirflowException(event["message"])
+ raise AirflowException("No event received in trigger callback")
+
class GCSObjectsWithPrefixExistenceSensor(BaseSensorOperator):
"""
@@ -265,7 +300,7 @@ class
GCSObjectsWithPrefixExistenceSensor(BaseSensorOperator):
self.impersonation_chain = impersonation_chain
def poke(self, context: Context) -> bool:
- self.log.info("Sensor checks existence of objects: %s, %s",
self.bucket, self.prefix)
+ self.log.info("Checking for existence of object: %s, %s", self.bucket,
self.prefix)
hook = GCSHook(
gcp_conn_id=self.google_cloud_conn_id,
impersonation_chain=self.impersonation_chain,
diff --git a/airflow/providers/google/cloud/triggers/gcs.py
b/airflow/providers/google/cloud/triggers/gcs.py
index 32ca257eae..7003df5942 100644
--- a/airflow/providers/google/cloud/triggers/gcs.py
+++ b/airflow/providers/google/cloud/triggers/gcs.py
@@ -18,12 +18,14 @@
from __future__ import annotations
import asyncio
+from datetime import datetime
from typing import Any, AsyncIterator
from aiohttp import ClientSession
from airflow.providers.google.cloud.hooks.gcs import GCSAsyncHook
from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.utils import timezone
class GCSBlobTrigger(BaseTrigger):
@@ -65,7 +67,7 @@ class GCSBlobTrigger(BaseTrigger):
)
async def run(self) -> AsyncIterator["TriggerEvent"]:
- """Simple loop until the relevant file/folder is found."""
+ """loop until the relevant file/folder is found."""
try:
hook = self._get_async_hook()
while True:
@@ -97,3 +99,104 @@ class GCSBlobTrigger(BaseTrigger):
if object_response:
return "success"
return "pending"
+
+
+class GCSCheckBlobUpdateTimeTrigger(BaseTrigger):
+ """
+ A trigger that makes an async call to GCS to check whether the object is
updated in a bucket.
+
+ :param bucket: google cloud storage bucket name cloud storage where the
objects are residing.
+ :param object_name: the file or folder present in the bucket
+ :param target_date: context datetime to compare with blob object updated
time
+ :param poke_interval: polling period in seconds to check for file/folder
+ :param google_cloud_conn_id: reference to the Google Connection
+ :param hook_params: dict object that has delegate_to and
impersonation_chain
+ """
+
+ def __init__(
+ self,
+ bucket: str,
+ object_name: str,
+ target_date: datetime,
+ poke_interval: float,
+ google_cloud_conn_id: str,
+ hook_params: dict[str, Any],
+ ):
+ super().__init__()
+ self.bucket = bucket
+ self.object_name = object_name
+ self.target_date = target_date
+ self.poke_interval = poke_interval
+ self.google_cloud_conn_id: str = google_cloud_conn_id
+ self.hook_params = hook_params
+
+ def serialize(self) -> tuple[str, dict[str, Any]]:
+ """Serializes GCSCheckBlobUpdateTimeTrigger arguments and classpath."""
+ return (
+
"airflow.providers.google.cloud.triggers.gcs.GCSCheckBlobUpdateTimeTrigger",
+ {
+ "bucket": self.bucket,
+ "object_name": self.object_name,
+ "target_date": self.target_date,
+ "poke_interval": self.poke_interval,
+ "google_cloud_conn_id": self.google_cloud_conn_id,
+ "hook_params": self.hook_params,
+ },
+ )
+
+ async def run(self) -> AsyncIterator["TriggerEvent"]:
+ """Loop until the object updated time is greater than target
datetime"""
+ try:
+ hook = self._get_async_hook()
+ while True:
+ status, res = await self._is_blob_updated_after(
+ hook=hook,
+ bucket_name=self.bucket,
+ object_name=self.object_name,
+ target_date=self.target_date,
+ )
+ if status:
+ yield TriggerEvent(res)
+ await asyncio.sleep(self.poke_interval)
+ except Exception as e:
+ yield TriggerEvent({"status": "error", "message": str(e)})
+
+ def _get_async_hook(self) -> GCSAsyncHook:
+ return GCSAsyncHook(gcp_conn_id=self.google_cloud_conn_id,
**self.hook_params)
+
+ async def _is_blob_updated_after(
+ self, hook: GCSAsyncHook, bucket_name: str, object_name: str,
target_date: datetime
+ ) -> tuple[bool, dict[str, Any]]:
+ """
+ Checks if the object in the bucket is updated.
+
+ :param hook: GCSAsyncHook Hook class
+ :param bucket_name: The Google Cloud Storage bucket where the object
is.
+ :param object_name: The name of the blob_name to check in the Google
cloud
+ storage bucket.
+ :param target_date: context datetime to compare with blob object
updated time
+ """
+ async with ClientSession() as session:
+ client = await hook.get_storage_client(session)
+ bucket = client.get_bucket(bucket_name)
+ blob = await bucket.get_blob(blob_name=object_name)
+ if blob is None:
+ res = {
+ "message": f"Object ({object_name}) not found in Bucket
({bucket_name})",
+ "status": "error",
+ }
+ return True, res
+
+ blob_updated_date = blob.updated # type: ignore[attr-defined]
+ blob_updated_time = datetime.strptime(blob_updated_date,
"%Y-%m-%dT%H:%M:%S.%fZ").replace(
+ tzinfo=timezone.utc
+ ) # Blob updated time is in string format so converting the
string format
+ # to datetime object to compare the last updated time
+
+ if blob_updated_time is not None:
+ if not target_date.tzinfo:
+ target_date = target_date.replace(tzinfo=timezone.utc)
+ self.log.info("Verify object date: %s > %s",
blob_updated_time, target_date)
+ if blob_updated_time > target_date:
+ return True, {"status": "success", "message": "success"}
+ return False, {"status": "pending", "message": "pending"}
diff --git a/docs/apache-airflow-providers-google/operators/cloud/gcs.rst
b/docs/apache-airflow-providers-google/operators/cloud/gcs.rst
index af32b74627..86f91d757d 100644
--- a/docs/apache-airflow-providers-google/operators/cloud/gcs.rst
+++ b/docs/apache-airflow-providers-google/operators/cloud/gcs.rst
@@ -225,6 +225,16 @@ Use the
:class:`~airflow.providers.google.cloud.sensors.gcs.GCSObjectUpdateSenso
:start-after: [START howto_sensor_object_update_exists_task]
:end-before: [END howto_sensor_object_update_exists_task]
+You can set the ``deferrable`` param to True if you want this sensor to run
asynchronously, leading to efficient
+utilization of resources in your Airflow deployment. However the triggerer
component needs to be enabled
+for this functionality to work.
+
+.. exampleinclude::
/../../tests/system/providers/google/cloud/gcs/example_gcs_sensor.py
+ :language: python
+ :dedent: 4
+ :start-after: [START howto_sensor_object_update_exists_task_async]
+ :end-before: [END howto_sensor_object_update_exists_task_async]
+
More information
""""""""""""""""
diff --git a/tests/providers/google/cloud/sensors/test_gcs.py
b/tests/providers/google/cloud/sensors/test_gcs.py
index 8a66fb0081..bc5db301d6 100644
--- a/tests/providers/google/cloud/sensors/test_gcs.py
+++ b/tests/providers/google/cloud/sensors/test_gcs.py
@@ -34,7 +34,8 @@ from airflow.providers.google.cloud.sensors.gcs import (
GCSUploadSessionCompleteSensor,
ts_function,
)
-from airflow.providers.google.cloud.triggers.gcs import GCSBlobTrigger
+from airflow.providers.google.cloud.triggers.gcs import GCSBlobTrigger,
GCSCheckBlobUpdateTimeTrigger
+from tests.providers.google.cloud.utils.airflow_util import create_context
TEST_BUCKET = "TEST_BUCKET"
@@ -225,6 +226,47 @@ class TestGoogleCloudStorageObjectUpdatedSensor:
assert result is True
+class TestGCSObjectUpdateSensorAsync:
+ OPERATOR = GCSObjectUpdateSensor(
+ task_id="gcs-obj-update",
+ bucket=TEST_BUCKET,
+ object=TEST_OBJECT,
+ google_cloud_conn_id=TEST_GCP_CONN_ID,
+ deferrable=True,
+ )
+
+ def test_gcs_object_update_sensor_async(self, context):
+ """
+ Asserts that a task is deferred and a GCSBlobTrigger will be fired
+ when the GCSObjectUpdateSensorAsync is executed.
+ """
+
+ with pytest.raises(TaskDeferred) as exc:
+ self.OPERATOR.execute(create_context(self.OPERATOR))
+ assert isinstance(
+ exc.value.trigger, GCSCheckBlobUpdateTimeTrigger
+ ), "Trigger is not a GCSCheckBlobUpdateTimeTrigger"
+
+ def test_gcs_object_update_sensor_async_execute_failure(self, context):
+ """Tests that an AirflowException is raised in case of error event"""
+
+ with pytest.raises(AirflowException):
+ self.OPERATOR.execute_complete(
+ context=context, event={"status": "error", "message": "test
failure message"}
+ )
+
+ def test_gcs_object_update_sensor_async_execute_complete(self, context):
+ """Asserts that logging occurs as expected"""
+
+ with mock.patch.object(self.OPERATOR.log, "info") as mock_log_info:
+ self.OPERATOR.execute_complete(
+ context=context, event={"status": "success", "message": "Job
completed"}
+ )
+ mock_log_info.assert_called_with(
+ "Checking last updated time for object %s in bucket : %s",
TEST_OBJECT, TEST_BUCKET
+ )
+
+
class TestGoogleCloudStoragePrefixSensor:
@mock.patch("airflow.providers.google.cloud.sensors.gcs.GCSHook")
def test_should_pass_arguments_to_hook(self, mock_hook):
diff --git a/tests/providers/google/cloud/triggers/test_gcs.py
b/tests/providers/google/cloud/triggers/test_gcs.py
index 2b33198b51..476eeeadd4 100644
--- a/tests/providers/google/cloud/triggers/test_gcs.py
+++ b/tests/providers/google/cloud/triggers/test_gcs.py
@@ -18,12 +18,13 @@
from __future__ import annotations
import asyncio
+from datetime import datetime
import pytest
from gcloud.aio.storage import Bucket, Storage
from airflow.providers.google.cloud.hooks.gcs import GCSAsyncHook
-from airflow.providers.google.cloud.triggers.gcs import GCSBlobTrigger
+from airflow.providers.google.cloud.triggers.gcs import GCSBlobTrigger,
GCSCheckBlobUpdateTimeTrigger
from airflow.triggers.base import TriggerEvent
from tests.providers.google.cloud.utils.compat import AsyncMock, async_mock
@@ -33,6 +34,7 @@ TEST_PREFIX = "TEST_PREFIX"
TEST_GCP_CONN_ID = "TEST_GCP_CONN_ID"
TEST_POLLING_INTERVAL = 3.0
TEST_HOOK_PARAMS = {}
+TEST_TS_OBJECT = datetime.utcnow()
@pytest.fixture
@@ -125,3 +127,142 @@ class TestGCSBlobTrigger:
res = await trigger._object_exists(hook, TEST_BUCKET, TEST_OBJECT)
assert res == response
bucket.blob_exists.assert_called_once_with(blob_name=TEST_OBJECT)
+
+
+class TestGCSCheckBlobUpdateTimeTrigger:
+ TRIGGER = GCSCheckBlobUpdateTimeTrigger(
+ bucket=TEST_BUCKET,
+ object_name=TEST_OBJECT,
+ target_date=TEST_TS_OBJECT,
+ poke_interval=TEST_POLLING_INTERVAL,
+ google_cloud_conn_id=TEST_GCP_CONN_ID,
+ hook_params=TEST_HOOK_PARAMS,
+ )
+
+ def test_gcs_blob_update_trigger_serialization(self):
+ """
+ Asserts that the GCSCheckBlobUpdateTimeTrigger correctly serializes
its arguments
+ and classpath.
+ """
+
+ classpath, kwargs = self.TRIGGER.serialize()
+ assert classpath ==
"airflow.providers.google.cloud.triggers.gcs.GCSCheckBlobUpdateTimeTrigger"
+ assert kwargs == {
+ "bucket": TEST_BUCKET,
+ "object_name": TEST_OBJECT,
+ "target_date": TEST_TS_OBJECT,
+ "poke_interval": TEST_POLLING_INTERVAL,
+ "google_cloud_conn_id": TEST_GCP_CONN_ID,
+ "hook_params": TEST_HOOK_PARAMS,
+ }
+
+ @pytest.mark.asyncio
+ @async_mock.patch(
+
"airflow.providers.google.cloud.triggers.gcs.GCSCheckBlobUpdateTimeTrigger._is_blob_updated_after"
+ )
+ async def test_gcs_blob_update_trigger_success(self, mock_blob_updated):
+ """
+ Tests success case GCSCheckBlobUpdateTimeTrigger
+ """
+ mock_blob_updated.return_value = True, {"status": "success",
"message": "success"}
+
+ generator = self.TRIGGER.run()
+ actual = await generator.asend(None)
+ assert TriggerEvent({"status": "success", "message": "success"}) ==
actual
+
+ @pytest.mark.asyncio
+ @async_mock.patch(
+
"airflow.providers.google.cloud.triggers.gcs.GCSCheckBlobUpdateTimeTrigger._is_blob_updated_after"
+ )
+ async def test_gcs_blob_update_trigger_pending(self, mock_blob_updated):
+ """
+ Test that GCSCheckBlobUpdateTimeTrigger is in loop till file isn't
updated.
+ """
+ mock_blob_updated.return_value = False, {"status": "pending",
"message": "pending"}
+
+ task = asyncio.create_task(self.TRIGGER.run().__anext__())
+ await asyncio.sleep(0.5)
+
+ # TriggerEvent was not returned
+ assert task.done() is False
+ asyncio.get_event_loop().stop()
+
+ @pytest.mark.asyncio
+ @async_mock.patch(
+
"airflow.providers.google.cloud.triggers.gcs.GCSCheckBlobUpdateTimeTrigger._is_blob_updated_after"
+ )
+ async def test_gcs_blob_update_trigger_exception(self, mock_object_exists):
+ """
+ Tests the GCSCheckBlobUpdateTimeTrigger does fire if there is an
exception.
+ """
+ mock_object_exists.side_effect = AsyncMock(side_effect=Exception("Test
exception"))
+
+ task = [i async for i in self.TRIGGER.run()]
+ assert len(task) == 1
+ assert TriggerEvent({"status": "error", "message": "Test exception"})
in task
+
+ @pytest.mark.asyncio
+ @pytest.mark.parametrize(
+ "blob_object_update_datetime, ts_object, expected_response",
+ [
+ (
+ "2022-03-07T10:05:43.535Z",
+ datetime(2022, 1, 1, 1, 1, 1),
+ (True, {"status": "success", "message": "success"}),
+ ),
+ (
+ "2022-03-07T10:05:43.535Z",
+ datetime(2022, 3, 8, 1, 1, 1),
+ (False, {"status": "pending", "message": "pending"}),
+ ),
+ ],
+ )
+ async def test_is_blob_updated_after(self, blob_object_update_datetime,
ts_object, expected_response):
+ """
+ Tests to check if a particular object in Google Cloud Storage
+ is found or not
+ """
+ hook = AsyncMock(GCSAsyncHook)
+ storage = AsyncMock(Storage)
+ hook.get_storage_client.return_value = storage
+ bucket = AsyncMock(Bucket)
+ storage.get_bucket.return_value = bucket
+ bucket.get_blob.return_value.updated = blob_object_update_datetime
+ trigger = GCSCheckBlobUpdateTimeTrigger(
+ bucket=TEST_BUCKET,
+ object_name=TEST_OBJECT,
+ target_date=ts_object,
+ poke_interval=TEST_POLLING_INTERVAL,
+ google_cloud_conn_id=TEST_GCP_CONN_ID,
+ hook_params=TEST_HOOK_PARAMS,
+ )
+ res = await trigger._is_blob_updated_after(hook, TEST_BUCKET,
TEST_OBJECT, ts_object)
+ assert res == expected_response
+
+ @pytest.mark.asyncio
+ @pytest.mark.parametrize(
+ "blob_object, expected_response",
+ [
+ (
+ None,
+ (
+ True,
+ {"status": "error", "message": "Object (TEST_OBJECT) not
found in Bucket (TEST_BUCKET)"},
+ ),
+ ),
+ ],
+ )
+ async def test_is_blob_updated_after_with_none(self, blob_object,
expected_response):
+ """
+ Tests to check if a particular object in Google Cloud Storage
+ is found or not
+ """
+ hook = AsyncMock(GCSAsyncHook)
+ storage = AsyncMock(Storage)
+ hook.get_storage_client.return_value = storage
+ bucket = AsyncMock(Bucket)
+ storage.get_bucket.return_value = bucket
+ bucket.get_blob.return_value = blob_object
+
+ res = await self.TRIGGER._is_blob_updated_after(hook, TEST_BUCKET,
TEST_OBJECT, TEST_TS_OBJECT)
+ assert res == expected_response
diff --git a/tests/providers/google/cloud/utils/airflow_util.py
b/tests/providers/google/cloud/utils/airflow_util.py
new file mode 100644
index 0000000000..da3faa6fcf
--- /dev/null
+++ b/tests/providers/google/cloud/utils/airflow_util.py
@@ -0,0 +1,74 @@
+# 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.
+from __future__ import annotations
+
+from unittest import mock
+
+import pendulum
+
+from airflow.models import DAG, Connection
+from airflow.models.baseoperator import BaseOperator
+from airflow.models.dagrun import DagRun
+from airflow.models.taskinstance import TaskInstance
+from airflow.utils import timezone
+from airflow.utils.types import DagRunType
+
+
+def get_dag_run(dag_id: str = "test_dag_id", run_id: str = "test_dag_id") ->
DagRun:
+ dag_run = DagRun(
+ dag_id=dag_id, run_type="manual",
execution_date=timezone.datetime(2022, 1, 1), run_id=run_id
+ )
+ return dag_run
+
+
+def get_task_instance(task: BaseOperator) -> TaskInstance:
+ return TaskInstance(task, timezone.datetime(2022, 1, 1))
+
+
+def get_conn() -> Connection:
+ return Connection(
+ conn_id="test_conn",
+ extra={},
+ )
+
+
+def create_context(task, dag=None):
+ if dag is None:
+ dag = DAG(dag_id="dag")
+ tzinfo = pendulum.timezone("UTC")
+ execution_date = timezone.datetime(2022, 1, 1, 1, 0, 0, tzinfo=tzinfo)
+ dag_run = DagRun(
+ dag_id=dag.dag_id,
+ execution_date=execution_date,
+ run_id=DagRun.generate_run_id(DagRunType.MANUAL, execution_date),
+ )
+
+ task_instance = TaskInstance(task=task)
+ task_instance.dag_run = dag_run
+ task_instance.xcom_push = mock.Mock()
+ return {
+ "dag": dag,
+ "ts": execution_date.isoformat(),
+ "task": task,
+ "ti": task_instance,
+ "task_instance": task_instance,
+ "run_id": dag_run.run_id,
+ "dag_run": dag_run,
+ "execution_date": execution_date,
+ "data_interval_end": execution_date,
+ "logical_date": execution_date,
+ }
diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_sensor.py
b/tests/system/providers/google/cloud/gcs/example_gcs_sensor.py
index bc7a5dda5d..c081e63e4c 100644
--- a/tests/system/providers/google/cloud/gcs/example_gcs_sensor.py
+++ b/tests/system/providers/google/cloud/gcs/example_gcs_sensor.py
@@ -101,6 +101,12 @@ with models.DAG(
)
# [END howto_sensor_object_update_exists_task]
+ # [START howto_sensor_object_update_exists_task_async]
+ gcs_update_object_exists_async = GCSObjectUpdateSensor(
+ bucket=BUCKET_NAME, object=FILE_NAME,
task_id="gcs_object_update_sensor_task_async", deferrable=True
+ )
+ # [END howto_sensor_object_update_exists_task_async]
+
upload_file = LocalFilesystemToGCSOperator(
task_id="upload_file",
src=UPLOAD_FILE_PATH,