diff --git a/google/cloud/bigquery/client.py b/google/cloud/bigquery/client.py index 742ecac2e..8142c59cd 100644 --- a/google/cloud/bigquery/client.py +++ b/google/cloud/bigquery/client.py @@ -86,7 +86,7 @@ from google.cloud.bigquery.model import ModelReference from google.cloud.bigquery.model import _model_arg_to_model_ref from google.cloud.bigquery.query import _QueryResults -from google.cloud.bigquery.retry import DEFAULT_RETRY +from google.cloud.bigquery.retry import DEFAULT_RETRY, DEFAULT_JOB_RETRY from google.cloud.bigquery.routine import Routine from google.cloud.bigquery.routine import RoutineReference from google.cloud.bigquery.schema import SchemaField @@ -3163,6 +3163,7 @@ def query( project: str = None, retry: retries.Retry = DEFAULT_RETRY, timeout: float = None, + job_retry: retries.Retry = DEFAULT_JOB_RETRY, ) -> job.QueryJob: """Run a SQL query. @@ -3192,21 +3193,52 @@ def query( Project ID of the project of where to run the job. Defaults to the client's project. retry (Optional[google.api_core.retry.Retry]): - How to retry the RPC. + How to retry the RPC. This only applies to making RPC + calls. It isn't used to retry failed jobs. This has + a reasonable default that should only be overridden + with care. timeout (Optional[float]): The number of seconds to wait for the underlying HTTP transport before using ``retry``. + job_retry (Optional[google.api_core.retry.Retry]): + How to retry failed jobs. The default retries + rate-limit-exceeded errors. Passing ``None`` disables + job retry. + + Not all jobs can be retried. If ``job_id`` is + provided, then the job returned by the query will not + be retryable, and an exception will be raised if a + non-``None`` (and non-default) value for ``job_retry`` + is also provided. + + Note that errors aren't detected until ``result()`` is + called on the job returned. The ``job_retry`` + specified here becomes the default ``job_retry`` for + ``result()``, where it can also be specified. Returns: google.cloud.bigquery.job.QueryJob: A new query job instance. Raises: TypeError: - If ``job_config`` is not an instance of :class:`~google.cloud.bigquery.job.QueryJobConfig` - class. + If ``job_config`` is not an instance of + :class:`~google.cloud.bigquery.job.QueryJobConfig` + class, or if both ``job_id`` and non-``None`` non-default + ``job_retry`` are provided. """ job_id_given = job_id is not None - job_id = _make_job_id(job_id, job_id_prefix) + if ( + job_id_given + and job_retry is not None + and job_retry is not DEFAULT_JOB_RETRY + ): + raise TypeError( + "`job_retry` was provided, but the returned job is" + " not retryable, because a custom `job_id` was" + " provided." + ) + + job_id_save = job_id if project is None: project = self.project @@ -3214,8 +3246,6 @@ def query( if location is None: location = self.location - job_config = copy.deepcopy(job_config) - if self._default_query_job_config: if job_config: _verify_job_config_type( @@ -3225,6 +3255,8 @@ def query( # that is in the default, # should be filled in with the default # the incoming therefore has precedence + # + # Note that _fill_from_default doesn't mutate the receiver job_config = job_config._fill_from_default( self._default_query_job_config ) @@ -3233,34 +3265,54 @@ def query( self._default_query_job_config, google.cloud.bigquery.job.QueryJobConfig, ) - job_config = copy.deepcopy(self._default_query_job_config) + job_config = self._default_query_job_config - job_ref = job._JobReference(job_id, project=project, location=location) - query_job = job.QueryJob(job_ref, query, client=self, job_config=job_config) + # Note that we haven't modified the original job_config (or + # _default_query_job_config) up to this point. + job_config_save = job_config - try: - query_job._begin(retry=retry, timeout=timeout) - except core_exceptions.Conflict as create_exc: - # The thought is if someone is providing their own job IDs and they get - # their job ID generation wrong, this could end up returning results for - # the wrong query. We thus only try to recover if job ID was not given. - if job_id_given: - raise create_exc + def do_query(): + # Make a copy now, so that original doesn't get changed by the process + # below and to facilitate retry + job_config = copy.deepcopy(job_config_save) + + job_id = _make_job_id(job_id_save, job_id_prefix) + job_ref = job._JobReference(job_id, project=project, location=location) + query_job = job.QueryJob(job_ref, query, client=self, job_config=job_config) try: - query_job = self.get_job( - job_id, - project=project, - location=location, - retry=retry, - timeout=timeout, - ) - except core_exceptions.GoogleAPIError: # (includes RetryError) - raise create_exc + query_job._begin(retry=retry, timeout=timeout) + except core_exceptions.Conflict as create_exc: + # The thought is if someone is providing their own job IDs and they get + # their job ID generation wrong, this could end up returning results for + # the wrong query. We thus only try to recover if job ID was not given. + if job_id_given: + raise create_exc + + try: + query_job = self.get_job( + job_id, + project=project, + location=location, + retry=retry, + timeout=timeout, + ) + except core_exceptions.GoogleAPIError: # (includes RetryError) + raise create_exc + else: + return query_job else: return query_job - else: - return query_job + + future = do_query() + # The future might be in a failed state now, but if it's + # unrecoverable, we'll find out when we ask for it's result, at which + # point, we may retry. + if not job_id_given: + future._retry_do_query = do_query # in case we have to retry later + future._job_retry = job_retry + + return future def insert_rows( self, diff --git a/google/cloud/bigquery/job/query.py b/google/cloud/bigquery/job/query.py index 2cb7ee28e..3ab47b0f9 100644 --- a/google/cloud/bigquery/job/query.py +++ b/google/cloud/bigquery/job/query.py @@ -36,7 +36,7 @@ from google.cloud.bigquery.query import ScalarQueryParameter from google.cloud.bigquery.query import StructQueryParameter from google.cloud.bigquery.query import UDFResource -from google.cloud.bigquery.retry import DEFAULT_RETRY +from google.cloud.bigquery.retry import DEFAULT_RETRY, DEFAULT_JOB_RETRY from google.cloud.bigquery.routine import RoutineReference from google.cloud.bigquery.table import _EmptyRowIterator from google.cloud.bigquery.table import RangePartitioning @@ -1260,6 +1260,7 @@ def result( retry: "retries.Retry" = DEFAULT_RETRY, timeout: float = None, start_index: int = None, + job_retry: "retries.Retry" = DEFAULT_JOB_RETRY, ) -> Union["RowIterator", _EmptyRowIterator]: """Start the job and wait for it to complete and get the result. @@ -1270,9 +1271,13 @@ def result( max_results (Optional[int]): The maximum total number of rows from this request. retry (Optional[google.api_core.retry.Retry]): - How to retry the call that retrieves rows. If the job state is - ``DONE``, retrying is aborted early even if the results are not - available, as this will not change anymore. + How to retry the call that retrieves rows. This only + applies to making RPC calls. It isn't used to retry + failed jobs. This has a reasonable default that + should only be overridden with care. If the job state + is ``DONE``, retrying is aborted early even if the + results are not available, as this will not change + anymore. timeout (Optional[float]): The number of seconds to wait for the underlying HTTP transport before using ``retry``. @@ -1280,6 +1285,16 @@ def result( applies to each individual request. start_index (Optional[int]): The zero-based index of the starting row to read. + job_retry (Optional[google.api_core.retry.Retry]): + How to retry failed jobs. The default retries + rate-limit-exceeded errors. Passing ``None`` disables + job retry. + + Not all jobs can be retried. If ``job_id`` was + provided to the query that created this job, then the + job returned by the query will not be retryable, and + an exception will be raised if non-``None`` + non-default ``job_retry`` is also provided. Returns: google.cloud.bigquery.table.RowIterator: @@ -1295,17 +1310,66 @@ def result( Raises: google.cloud.exceptions.GoogleAPICallError: - If the job failed. + If the job failed and retries aren't successful. concurrent.futures.TimeoutError: If the job did not complete in the given timeout. + TypeError: + If Non-``None`` and non-default ``job_retry`` is + provided and the job is not retryable. """ try: - super(QueryJob, self).result(retry=retry, timeout=timeout) + retry_do_query = getattr(self, "_retry_do_query", None) + if retry_do_query is not None: + if job_retry is DEFAULT_JOB_RETRY: + job_retry = self._job_retry + else: + if job_retry is not None and job_retry is not DEFAULT_JOB_RETRY: + raise TypeError( + "`job_retry` was provided, but this job is" + " not retryable, because a custom `job_id` was" + " provided to the query that created this job." + ) + + first = True + + def do_get_result(): + nonlocal first + + if first: + first = False + else: + # Note that we won't get here if retry_do_query is + # None, because we won't use a retry. + + # The orinal job is failed. Create a new one. + job = retry_do_query() + + # If it's already failed, we might as well stop: + if job.done() and job.exception() is not None: + raise job.exception() + + # Become the new job: + self.__dict__.clear() + self.__dict__.update(job.__dict__) + + # This shouldn't be necessary, because once we have a good + # job, it should stay good,and we shouldn't have to retry. + # But let's be paranoid. :) + self._retry_do_query = retry_do_query + self._job_retry = job_retry + + super(QueryJob, self).result(retry=retry, timeout=timeout) + + # Since the job could already be "done" (e.g. got a finished job + # via client.get_job), the superclass call to done() might not + # set the self._query_results cache. + self._reload_query_results(retry=retry, timeout=timeout) + + if retry_do_query is not None and job_retry is not None: + do_get_result = job_retry(do_get_result) + + do_get_result() - # Since the job could already be "done" (e.g. got a finished job - # via client.get_job), the superclass call to done() might not - # set the self._query_results cache. - self._reload_query_results(retry=retry, timeout=timeout) except exceptions.GoogleAPICallError as exc: exc.message += self._format_for_exception(self.query, self.job_id) exc.query_job = self diff --git a/google/cloud/bigquery/retry.py b/google/cloud/bigquery/retry.py index bab28aacb..e9286055c 100644 --- a/google/cloud/bigquery/retry.py +++ b/google/cloud/bigquery/retry.py @@ -32,6 +32,8 @@ auth_exceptions.TransportError, ) +_DEFAULT_JOB_DEADLINE = 60.0 * 10.0 # seconds + def _should_retry(exc): """Predicate for determining when to retry. @@ -56,3 +58,21 @@ def _should_retry(exc): on ``DEFAULT_RETRY``. For example, to change the deadline to 30 seconds, pass ``retry=bigquery.DEFAULT_RETRY.with_deadline(30)``. """ + +job_retry_reasons = "rateLimitExceeded", "backendError" + + +def _job_should_retry(exc): + if not hasattr(exc, "errors") or len(exc.errors) == 0: + return False + + reason = exc.errors[0]["reason"] + return reason in job_retry_reasons + + +DEFAULT_JOB_RETRY = retry.Retry( + predicate=_job_should_retry, deadline=_DEFAULT_JOB_DEADLINE +) +""" +The default job retry object. +""" diff --git a/tests/system/test_job_retry.py b/tests/system/test_job_retry.py new file mode 100644 index 000000000..520545493 --- /dev/null +++ b/tests/system/test_job_retry.py @@ -0,0 +1,72 @@ +# Copyright 2021 Google LLC +# +# Licensed 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 +# +# https://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. + +import contextlib +import threading +import time + +import google.api_core.exceptions +import google.cloud.bigquery +import pytest + + +def thread(func): + thread = threading.Thread(target=func, daemon=True) + thread.start() + return thread + + +@pytest.mark.parametrize("job_retry_on_query", [True, False]) +def test_query_retry_539(bigquery_client, dataset_id, job_retry_on_query): + """ + Test job_retry + + See: https://github.com/googleapis/python-bigquery/issues/539 + """ + from google.api_core import exceptions + from google.api_core.retry import if_exception_type, Retry + + table_name = f"{dataset_id}.t539" + + # Without a custom retry, we fail: + with pytest.raises(google.api_core.exceptions.NotFound): + bigquery_client.query(f"select count(*) from {table_name}").result() + + retry_notfound = Retry(predicate=if_exception_type(exceptions.NotFound)) + + job_retry = dict(job_retry=retry_notfound) if job_retry_on_query else {} + job = bigquery_client.query(f"select count(*) from {table_name}", **job_retry) + job_id = job.job_id + + # We can already know that the job failed, but we're not supposed + # to find out until we call result, which is where retry happend + assert job.done() + assert job.exception() is not None + + @thread + def create_table(): + time.sleep(1) # Give the first retry attempt time to fail. + with contextlib.closing(google.cloud.bigquery.Client()) as client: + client.query(f"create table {table_name} (id int64)").result() + + job_retry = {} if job_retry_on_query else dict(job_retry=retry_notfound) + [[count]] = list(job.result(**job_retry)) + assert count == 0 + + # The job was retried, and thus got a new job id + assert job.job_id != job_id + + # Make sure we don't leave a thread behind: + create_table.join() + bigquery_client.query(f"drop table {table_name}").result() diff --git a/tests/unit/test_job_retry.py b/tests/unit/test_job_retry.py new file mode 100644 index 000000000..b2095d2f2 --- /dev/null +++ b/tests/unit/test_job_retry.py @@ -0,0 +1,247 @@ +# Copyright 2021 Google LLC +# +# Licensed 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 +# +# https://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. + +import datetime +import re + +import mock +import pytest + +import google.api_core.exceptions +import google.api_core.retry + +from .helpers import make_connection + + +# With job_retry_on_query, we're testing 4 scenarios: +# - No `job_retry` passed, retry on default rateLimitExceeded. +# - Pass NotFound retry to `query`. +# - Pass NotFound retry to `result`. +# - Pass BadRequest retry to query, with the value passed to `result` overriding. +@pytest.mark.parametrize("job_retry_on_query", [None, "Query", "Result", "Both"]) +@mock.patch("time.sleep") +def test_retry_failed_jobs(sleep, client, job_retry_on_query): + """ + Test retry of job failures, as opposed to API-invocation failures. + """ + + retry_notfound = google.api_core.retry.Retry( + predicate=google.api_core.retry.if_exception_type( + google.api_core.exceptions.NotFound + ) + ) + retry_badrequest = google.api_core.retry.Retry( + predicate=google.api_core.retry.if_exception_type( + google.api_core.exceptions.BadRequest + ) + ) + + if job_retry_on_query is None: + reason = "rateLimitExceeded" + else: + reason = "notFound" + + err = dict(reason=reason) + responses = [ + dict(status=dict(state="DONE", errors=[err], errorResult=err)), + dict(status=dict(state="DONE", errors=[err], errorResult=err)), + dict(status=dict(state="DONE", errors=[err], errorResult=err)), + dict(status=dict(state="DONE")), + dict(rows=[{"f": [{"v": "1"}]}], totalRows="1"), + ] + + def api_request(method, path, query_params=None, data=None, **kw): + response = responses.pop(0) + if data: + response["jobReference"] = data["jobReference"] + else: + response["jobReference"] = dict( + jobId=path.split("/")[-1], projectId="PROJECT" + ) + return response + + conn = client._connection = make_connection() + conn.api_request.side_effect = api_request + + if job_retry_on_query == "Query": + job_retry = dict(job_retry=retry_notfound) + elif job_retry_on_query == "Both": + # This will be overridden in `result` + job_retry = dict(job_retry=retry_badrequest) + else: + job_retry = {} + job = client.query("select 1", **job_retry) + + orig_job_id = job.job_id + job_retry = ( + dict(job_retry=retry_notfound) + if job_retry_on_query in ("Result", "Both") + else {} + ) + result = job.result(**job_retry) + assert result.total_rows == 1 + assert not responses # We made all the calls we expected to. + + # The job adjusts it's job id based on the id of the last attempt. + assert job.job_id != orig_job_id + assert job.job_id == conn.mock_calls[3][2]["data"]["jobReference"]["jobId"] + + # We had to sleep three times + assert len(sleep.mock_calls) == 3 + + # Sleeps are random, however they're more than 0 + assert min(c[1][0] for c in sleep.mock_calls) > 0 + + # They're at most 2 * (multiplier**(number of sleeps - 1)) * initial + # The default multiplier is 2 + assert max(c[1][0] for c in sleep.mock_calls) <= 8 + + # We can ask for the result again: + responses = [ + dict(rows=[{"f": [{"v": "1"}]}], totalRows="1"), + ] + orig_job_id = job.job_id + result = job.result() + assert result.total_rows == 1 + assert not responses # We made all the calls we expected to. + + # We wouldn't (and didn't) fail, because we're dealing with a successful job. + # So the job id hasn't changed. + assert job.job_id == orig_job_id + + +# With job_retry_on_query, we're testing 4 scenarios: +# - Pass None retry to `query`. +# - Pass None retry to `result`. +@pytest.mark.parametrize("job_retry_on_query", ["Query", "Result"]) +@mock.patch("time.sleep") +def test_disable_retry_failed_jobs(sleep, client, job_retry_on_query): + """ + Test retry of job failures, as opposed to API-invocation failures. + """ + err = dict(reason="rateLimitExceeded") + responses = [dict(status=dict(state="DONE", errors=[err], errorResult=err))] * 3 + + def api_request(method, path, query_params=None, data=None, **kw): + response = responses.pop(0) + response["jobReference"] = data["jobReference"] + return response + + conn = client._connection = make_connection() + conn.api_request.side_effect = api_request + + if job_retry_on_query == "Query": + job_retry = dict(job_retry=None) + else: + job_retry = {} + job = client.query("select 1", **job_retry) + + orig_job_id = job.job_id + job_retry = dict(job_retry=None) if job_retry_on_query == "Result" else {} + with pytest.raises(google.api_core.exceptions.Forbidden): + job.result(**job_retry) + + assert job.job_id == orig_job_id + assert len(sleep.mock_calls) == 0 + + +@mock.patch("google.api_core.retry.datetime_helpers") +@mock.patch("time.sleep") +def test_retry_failed_jobs_after_retry_failed(sleep, datetime_helpers, client): + """ + If at first you don't succeed, maybe you will later. :) + """ + conn = client._connection = make_connection() + + datetime_helpers.utcnow.return_value = datetime.datetime(2021, 7, 29, 10, 43, 2) + + err = dict(reason="rateLimitExceeded") + + def api_request(method, path, query_params=None, data=None, **kw): + calls = sleep.mock_calls + if calls: + datetime_helpers.utcnow.return_value += datetime.timedelta( + seconds=calls[-1][1][0] + ) + response = dict(status=dict(state="DONE", errors=[err], errorResult=err)) + response["jobReference"] = data["jobReference"] + return response + + conn.api_request.side_effect = api_request + + job = client.query("select 1") + orig_job_id = job.job_id + + with pytest.raises(google.api_core.exceptions.RetryError): + job.result() + + # We never got a successful job, so the job id never changed: + assert job.job_id == orig_job_id + + # We failed because we couldn't succeed after 120 seconds. + # But we can try again: + err2 = dict(reason="backendError") # We also retry on this + responses = [ + dict(status=dict(state="DONE", errors=[err2], errorResult=err2)), + dict(status=dict(state="DONE", errors=[err], errorResult=err)), + dict(status=dict(state="DONE", errors=[err2], errorResult=err2)), + dict(status=dict(state="DONE")), + dict(rows=[{"f": [{"v": "1"}]}], totalRows="1"), + ] + + def api_request(method, path, query_params=None, data=None, **kw): + calls = sleep.mock_calls + datetime_helpers.utcnow.return_value += datetime.timedelta( + seconds=calls[-1][1][0] + ) + response = responses.pop(0) + if data: + response["jobReference"] = data["jobReference"] + else: + response["jobReference"] = dict( + jobId=path.split("/")[-1], projectId="PROJECT" + ) + return response + + conn.api_request.side_effect = api_request + result = job.result() + assert result.total_rows == 1 + assert not responses # We made all the calls we expected to. + assert job.job_id != orig_job_id + + +def test_raises_on_job_retry_on_query_with_non_retryable_jobs(client): + with pytest.raises( + TypeError, + match=re.escape( + "`job_retry` was provided, but the returned job is" + " not retryable, because a custom `job_id` was" + " provided." + ), + ): + client.query("select 42", job_id=42, job_retry=google.api_core.retry.Retry()) + + +def test_raises_on_job_retry_on_result_with_non_retryable_jobs(client): + client._connection = make_connection({}) + job = client.query("select 42", job_id=42) + with pytest.raises( + TypeError, + match=re.escape( + "`job_retry` was provided, but this job is" + " not retryable, because a custom `job_id` was" + " provided to the query that created this job." + ), + ): + job.result(job_retry=google.api_core.retry.Retry()) diff --git a/tests/unit/test_retry.py b/tests/unit/test_retry.py index 6fb7f93fd..c7c25e036 100644 --- a/tests/unit/test_retry.py +++ b/tests/unit/test_retry.py @@ -86,3 +86,27 @@ def test_w_unstructured_bad_gateway(self): exc = BadGateway("testing") self.assertTrue(self._call_fut(exc)) + + +def test_DEFAULT_JOB_RETRY_predicate(): + from google.cloud.bigquery.retry import DEFAULT_JOB_RETRY + from google.api_core.exceptions import ClientError + + assert not DEFAULT_JOB_RETRY._predicate(TypeError()) + assert not DEFAULT_JOB_RETRY._predicate(ClientError("fail")) + assert not DEFAULT_JOB_RETRY._predicate( + ClientError("fail", errors=[dict(reason="idk")]) + ) + + assert DEFAULT_JOB_RETRY._predicate( + ClientError("fail", errors=[dict(reason="rateLimitExceeded")]) + ) + assert DEFAULT_JOB_RETRY._predicate( + ClientError("fail", errors=[dict(reason="backendError")]) + ) + + +def test_DEFAULT_JOB_RETRY_deadline(): + from google.cloud.bigquery.retry import DEFAULT_JOB_RETRY + + assert DEFAULT_JOB_RETRY._deadline == 600