Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 6680261

Browse files
authored
Merge pull request #17584 from [BEAM-14415] Exception handling tests and logging for partial failure BQIO
* [BEAM-14415] Exception handling tests and logging for partial failures in BQ IO * fix DLQ integration test * fix lint * fix postcommit * fix formatter * Fixing tests and adding test info * fix skipping tests
1 parent 3cca763 commit 6680261

4 files changed

Lines changed: 346 additions & 24 deletions

File tree

sdks/python/apache_beam/io/gcp/bigquery.py

Lines changed: 61 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -378,6 +378,15 @@ def compute_table_name(row):
378378
NOTE: This job name template does not have backwards compatibility guarantees.
379379
"""
380380
BQ_JOB_NAME_TEMPLATE = "beam_bq_job_{job_type}_{job_id}_{step_id}{random}"
381+
"""
382+
The maximum number of times that a bundle of rows that errors out should be
383+
sent for insertion into BigQuery.
384+
385+
The default is 10,000 with exponential backoffs, so a bundle of rows may be
386+
tried for a very long time. You may reduce this property to reduce the number
387+
of retries.
388+
"""
389+
MAX_INSERT_RETRIES = 10000
381390

382391

383392
@deprecated(since='2.11.0', current="bigquery_tools.parse_table_reference")
@@ -1492,6 +1501,7 @@ class BigQueryWriteFn(DoFn):
14921501
DEFAULT_MAX_BATCH_SIZE = 500
14931502

14941503
FAILED_ROWS = 'FailedRows'
1504+
FAILED_ROWS_WITH_ERRORS = 'FailedRowsWithErrors'
14951505
STREAMING_API_LOGGING_FREQUENCY_SEC = 300
14961506

14971507
def __init__(
@@ -1507,7 +1517,8 @@ def __init__(
15071517
additional_bq_parameters=None,
15081518
ignore_insert_ids=False,
15091519
with_batched_input=False,
1510-
ignore_unknown_columns=False):
1520+
ignore_unknown_columns=False,
1521+
max_retries=MAX_INSERT_RETRIES):
15111522
"""Initialize a WriteToBigQuery transform.
15121523
15131524
Args:
@@ -1555,6 +1566,9 @@ def __init__(
15551566
the schema. The unknown values are ignored. Default is False,
15561567
which treats unknown values as errors. See reference:
15571568
https://cloud.google.com/bigquery/docs/reference/rest/v2/tabledata/insertAll
1569+
max_retries: The number of times that we will retry inserting a group of
1570+
rows into BigQuery. By default, we retry 10000 times with exponential
1571+
backoffs (effectively retry forever).
15581572
15591573
"""
15601574
self.schema = schema
@@ -1592,6 +1606,7 @@ def __init__(
15921606
self.streaming_api_logging_frequency_sec = (
15931607
BigQueryWriteFn.STREAMING_API_LOGGING_FREQUENCY_SEC)
15941608
self.ignore_unknown_columns = ignore_unknown_columns
1609+
self._max_retries = max_retries
15951610

15961611
def display_data(self):
15971612
return {
@@ -1643,7 +1658,9 @@ def start_bundle(self):
16431658

16441659
self._backoff_calculator = iter(
16451660
retry.FuzzedExponentialIntervals(
1646-
initial_delay_secs=0.2, num_retries=10000, max_delay_secs=1500))
1661+
initial_delay_secs=0.2,
1662+
num_retries=self._max_retries,
1663+
max_delay_secs=1500))
16471664

16481665
def _create_table_if_needed(self, table_reference, schema=None):
16491666
str_table_reference = '%s:%s.%s' % (
@@ -1754,41 +1771,57 @@ def _flush_batch(self, destination):
17541771
ignore_unknown_values=self.ignore_unknown_columns)
17551772
self.batch_latency_metric.update((time.time() - start) * 1000)
17561773

1757-
failed_rows = [rows[entry['index']] for entry in errors]
1774+
failed_rows = [(rows[entry['index']], entry["errors"])
1775+
for entry in errors]
1776+
retry_backoff = next(self._backoff_calculator, None)
1777+
1778+
# If retry_backoff is None, then we will not retry and must log.
17581779
should_retry = any(
17591780
RetryStrategy.should_retry(
17601781
self._retry_strategy, entry['errors'][0]['reason'])
1761-
for entry in errors)
1782+
for entry in errors) and retry_backoff is not None
1783+
17621784
if not passed:
17631785
self.failed_rows_metric.update(len(failed_rows))
17641786
message = (
17651787
'There were errors inserting to BigQuery. Will{} retry. '
17661788
'Errors were {}'.format(("" if should_retry else " not"), errors))
1767-
if should_retry:
1768-
_LOGGER.warning(message)
1769-
else:
1770-
_LOGGER.error(message)
17711789

1772-
rows = failed_rows
1790+
# The log level is:
1791+
# - WARNING when we are continuing to retry, and have a deadline.
1792+
# - ERROR when we will no longer retry, or MAY retry forever.
1793+
log_level = (
1794+
logging.WARN if should_retry or
1795+
self._retry_strategy != RetryStrategy.RETRY_ALWAYS else
1796+
logging.ERROR)
1797+
1798+
_LOGGER.log(log_level, message)
17731799

17741800
if not should_retry:
17751801
break
17761802
else:
1777-
retry_backoff = next(self._backoff_calculator)
17781803
_LOGGER.info(
17791804
'Sleeping %s seconds before retrying insertion.', retry_backoff)
17801805
time.sleep(retry_backoff)
1806+
rows = [fr[0] for fr in failed_rows]
17811807
self._throttled_secs.inc(retry_backoff)
17821808

17831809
self._total_buffered_rows -= len(self._rows_buffer[destination])
17841810
del self._rows_buffer[destination]
17851811

1786-
return [
1812+
return itertools.chain([
17871813
pvalue.TaggedOutput(
1788-
BigQueryWriteFn.FAILED_ROWS,
1789-
GlobalWindows.windowed_value((destination, row)))
1790-
for row in failed_rows
1791-
]
1814+
BigQueryWriteFn.FAILED_ROWS_WITH_ERRORS,
1815+
GlobalWindows.windowed_value((destination, row, err))) for row,
1816+
err in failed_rows
1817+
],
1818+
[
1819+
pvalue.TaggedOutput(
1820+
BigQueryWriteFn.FAILED_ROWS,
1821+
GlobalWindows.windowed_value(
1822+
(destination, row))) for row,
1823+
unused_err in failed_rows
1824+
])
17921825

17931826

17941827
# The number of shards per destination when writing via streaming inserts.
@@ -1815,7 +1848,8 @@ def __init__(
18151848
ignore_insert_ids,
18161849
ignore_unknown_columns,
18171850
with_auto_sharding,
1818-
test_client=None):
1851+
test_client=None,
1852+
max_retries=None):
18191853
self.table_reference = table_reference
18201854
self.table_side_inputs = table_side_inputs
18211855
self.schema_side_inputs = schema_side_inputs
@@ -1831,6 +1865,7 @@ def __init__(
18311865
self.ignore_insert_ids = ignore_insert_ids
18321866
self.ignore_unknown_columns = ignore_unknown_columns
18331867
self.with_auto_sharding = with_auto_sharding
1868+
self.max_retries = max_retries or MAX_INSERT_RETRIES
18341869

18351870
class InsertIdPrefixFn(DoFn):
18361871
def start_bundle(self):
@@ -1856,7 +1891,8 @@ def expand(self, input):
18561891
additional_bq_parameters=self.additional_bq_parameters,
18571892
ignore_insert_ids=self.ignore_insert_ids,
18581893
ignore_unknown_columns=self.ignore_unknown_columns,
1859-
with_batched_input=self.with_auto_sharding)
1894+
with_batched_input=self.with_auto_sharding,
1895+
max_retries=self.max_retries)
18601896

18611897
def _add_random_shard(element):
18621898
key = element[0]
@@ -1905,7 +1941,9 @@ def _restore_table_ref(sharded_table_ref_elems_kv):
19051941
| 'FromHashableTableRef' >> beam.Map(_restore_table_ref)
19061942
| 'StreamInsertRows' >> ParDo(
19071943
bigquery_write_fn, *self.schema_side_inputs).with_outputs(
1908-
BigQueryWriteFn.FAILED_ROWS, main='main'))
1944+
BigQueryWriteFn.FAILED_ROWS,
1945+
BigQueryWriteFn.FAILED_ROWS_WITH_ERRORS,
1946+
main='main'))
19091947

19101948

19111949
# Flag to be passed to WriteToBigQuery to force schema autodetection
@@ -2194,7 +2232,11 @@ def expand(self, pcoll):
21942232
with_auto_sharding=self.with_auto_sharding,
21952233
test_client=self.test_client)
21962234

2197-
return {BigQueryWriteFn.FAILED_ROWS: outputs[BigQueryWriteFn.FAILED_ROWS]}
2235+
return {
2236+
BigQueryWriteFn.FAILED_ROWS: outputs[BigQueryWriteFn.FAILED_ROWS],
2237+
BigQueryWriteFn.FAILED_ROWS_WITH_ERRORS: outputs[
2238+
BigQueryWriteFn.FAILED_ROWS_WITH_ERRORS],
2239+
}
21982240
else:
21992241
if self._temp_file_format == bigquery_tools.FileFormat.AVRO:
22002242
if self.schema == SCHEMA_AUTODETECT:

0 commit comments

Comments
 (0)