Skip to content

Commit

Permalink
Support num_retries field in env var for GCP connection (#8700)
Browse files Browse the repository at this point in the history
  • Loading branch information
mik-laj committed May 5, 2020
1 parent 41b4c27 commit 8d6f1aa
Show file tree
Hide file tree
Showing 2 changed files with 59 additions and 1 deletion.
14 changes: 13 additions & 1 deletion airflow/providers/google/common/hooks/base_google.py
Expand Up @@ -250,7 +250,19 @@ def num_retries(self) -> int:
:return: the number of times each API request should be retried
:rtype: int
"""
return self._get_field('num_retries') or 5
field_value = self._get_field('num_retries', default=5)
if field_value is None:
return 5
if isinstance(field_value, str) and field_value.strip() == '':
return 5
try:
return int(field_value)
except ValueError:
raise AirflowException(
f"The num_retries field should be a integer. "
f"Current value: \"{field_value}\" (type: {type(field_value)}). "
f"Please check the connection configuration."
)

@property
def client_info(self) -> ClientInfo:
Expand Down
46 changes: 46 additions & 0 deletions tests/providers/google/common/hooks/test_base_google.py
Expand Up @@ -724,3 +724,49 @@ def test_provide_authorized_gcloud_via_gcloud_application_default(
],
any_order=False
)


class TestNumRetry(unittest.TestCase):

def test_should_return_int_when_set_int_via_connection(self):
instance = hook.GoogleBaseHook(gcp_conn_id="google_cloud_default")
instance.extras = {
'extra__google_cloud_platform__num_retries': 10,
}

self.assertIsInstance(instance.num_retries, int)
self.assertEqual(10, instance.num_retries)

@mock.patch.dict(
'os.environ',
AIRFLOW_CONN_GOOGLE_CLOUD_DEFAULT=(
'google-cloud-platform://?extra__google_cloud_platform__num_retries=5'
)
)
def test_should_return_int_when_set_via_env_var(self):
instance = hook.GoogleBaseHook(gcp_conn_id="google_cloud_default")
self.assertIsInstance(instance.num_retries, int)

@mock.patch.dict(
'os.environ',
AIRFLOW_CONN_GOOGLE_CLOUD_DEFAULT=(
'google-cloud-platform://?extra__google_cloud_platform__num_retries=cat'
)
)
def test_should_raise_when_invalid_value_via_env_var(self):
instance = hook.GoogleBaseHook(gcp_conn_id="google_cloud_default")
with self.assertRaisesRegex(
AirflowException, re.escape("The num_retries field should be a integer.")
):
self.assertIsInstance(instance.num_retries, int)

@mock.patch.dict(
'os.environ',
AIRFLOW_CONN_GOOGLE_CLOUD_DEFAULT=(
'google-cloud-platform://?extra__google_cloud_platform__num_retries='
)
)
def test_should_fallback_when_empty_string_in_env_var(self):
instance = hook.GoogleBaseHook(gcp_conn_id="google_cloud_default")
self.assertIsInstance(instance.num_retries, int)
self.assertEqual(5, instance.num_retries)

0 comments on commit 8d6f1aa

Please sign in to comment.