Skip to content

Commit

Permalink
Fix to CloudBuildRunBuildTriggerOperator fails to find build id. (#…
Browse files Browse the repository at this point in the history
…22419)

* Fix CloudBuildRunBuildTriggerOperator: 'property' object has no attribute 'build' #22398
  • Loading branch information
stephenoken authored Mar 23, 2022
1 parent f51a674 commit 0f0a1a7
Show file tree
Hide file tree
Showing 2 changed files with 18 additions and 6 deletions.
6 changes: 3 additions & 3 deletions airflow/providers/google/cloud/hooks/cloud_build.py
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,7 @@ def create_build(
metadata=metadata,
)

id_ = self._get_build_id_from_operation(Operation)
id_ = self._get_build_id_from_operation(operation)

if not wait:
return self.get_build(id_=id_, project_id=project_id)
Expand Down Expand Up @@ -460,7 +460,7 @@ def retry_build(
metadata=metadata,
)

id_ = self._get_build_id_from_operation(Operation)
id_ = self._get_build_id_from_operation(operation)

if not wait:
return self.get_build(id_=id_, project_id=project_id)
Expand Down Expand Up @@ -510,7 +510,7 @@ def run_build_trigger(
metadata=metadata,
)

id_ = self._get_build_id_from_operation(Operation)
id_ = self._get_build_id_from_operation(operation)

if not wait:
return self.get_build(id_=id_, project_id=project_id)
Expand Down
18 changes: 15 additions & 3 deletions tests/providers/google/cloud/hooks/test_cloud_build.py
Original file line number Diff line number Diff line change
Expand Up @@ -106,14 +106,18 @@ def test_create_build_without_wait(self, get_conn, mock_get_id_from_operation):

self.hook.create_build(build=BUILD, project_id=PROJECT_ID, wait=False)

get_conn.return_value.create_build.assert_called_once_with(
mock_operation = get_conn.return_value.create_build

mock_operation.assert_called_once_with(
request={'project_id': PROJECT_ID, 'build': BUILD}, retry=None, timeout=None, metadata=()
)

get_conn.return_value.get_build.assert_called_once_with(
request={'project_id': PROJECT_ID, 'id': BUILD_ID}, retry=None, timeout=None, metadata=()
)

mock_get_id_from_operation.assert_called_once_with(mock_operation())

@patch("airflow.providers.google.cloud.hooks.cloud_build.CloudBuildHook.get_conn")
def test_create_build_trigger(self, get_conn):
self.hook.create_build_trigger(trigger=BUILD_TRIGGER, project_id=PROJECT_ID)
Expand Down Expand Up @@ -194,7 +198,9 @@ def test_retry_build_with_wait(self, get_conn, wait_time, mock_get_id_from_opera

self.hook.retry_build(id_=BUILD_ID, project_id=PROJECT_ID)

get_conn.return_value.retry_build.assert_called_once_with(
mock_operation = get_conn.return_value.retry_build

mock_operation.assert_called_once_with(
request={'project_id': PROJECT_ID, 'id': BUILD_ID}, retry=None, timeout=None, metadata=()
)

Expand All @@ -204,6 +210,8 @@ def test_retry_build_with_wait(self, get_conn, wait_time, mock_get_id_from_opera
request={'project_id': PROJECT_ID, 'id': BUILD_ID}, retry=None, timeout=None, metadata=()
)

mock_get_id_from_operation.assert_called_once_with(mock_operation())

@patch("airflow.providers.google.cloud.hooks.cloud_build.CloudBuildHook._get_build_id_from_operation")
@patch("airflow.providers.google.cloud.hooks.cloud_build.CloudBuildHook.get_conn")
def test_retry_build_without_wait(self, get_conn, mock_get_id_from_operation):
Expand Down Expand Up @@ -233,7 +241,9 @@ def test_run_build_trigger_with_wait(self, get_conn, wait_time, mock_get_id_from
trigger_id=TRIGGER_ID, source=REPO_SOURCE['repo_source'], project_id=PROJECT_ID
)

get_conn.return_value.run_build_trigger.assert_called_once_with(
mock_operation = get_conn.return_value.run_build_trigger

mock_operation.assert_called_once_with(
request={
'project_id': PROJECT_ID,
'trigger_id': TRIGGER_ID,
Expand All @@ -250,6 +260,8 @@ def test_run_build_trigger_with_wait(self, get_conn, wait_time, mock_get_id_from
request={'project_id': PROJECT_ID, 'id': BUILD_ID}, retry=None, timeout=None, metadata=()
)

mock_get_id_from_operation.assert_called_once_with(mock_operation())

@patch("airflow.providers.google.cloud.hooks.cloud_build.CloudBuildHook._get_build_id_from_operation")
@patch("airflow.providers.google.cloud.hooks.cloud_build.CloudBuildHook.get_conn")
def test_run_build_trigger_without_wait(self, get_conn, mock_get_id_from_operation):
Expand Down

0 comments on commit 0f0a1a7

Please sign in to comment.
  翻译: