You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Yusuke Shimizu (Jira)" <ji...@apache.org> on 2021/03/09 11:14:00 UTC

[jira] [Created] (BEAM-11939) BigQuery FILE_LOADS failed with 400 error in streaming mode in Python

Yusuke Shimizu created BEAM-11939:
-------------------------------------

             Summary: BigQuery FILE_LOADS failed with 400 error in streaming mode in Python
                 Key: BEAM-11939
                 URL: https://issues.apache.org/jira/browse/BEAM-11939
             Project: Beam
          Issue Type: Bug
          Components: io-py-gcp, sdk-py-core
    Affects Versions: 2.28.0
            Reporter: Yusuke Shimizu


 

We are using FILE_LOADS to write to BigQuery in streaming mode using Python. 
after running for about 1 hours, beam job throws an exception with regards to `RuntimeError: apitools.base.py.exceptions.HttpBadRequestError` including error message "Load configuration must specify at least one source URI".
{code:java}
//
Traceback (most recent call last):
  File "/usr/local/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 289, in _execute
    response = task()
  File "/usr/local/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 362, in <lambda>
    lambda: self.create_worker().do_instruction(request), request)
  File "/usr/local/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 606, in do_instruction
    return getattr(self, request_type)(
  File "/usr/local/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 644, in process_bundle
    bundle_processor.process_bundle(instruction_id))
  File "/usr/local/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py", line 999, in process_bundle
    input_op_by_transform_id[element.transform_id].process_encoded(
  File "/usr/local/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py", line 228, in process_encoded
    self.output(decoded_value)
  File "apache_beam/runners/worker/operations.py", line 357, in apache_beam.runners.worker.operations.Operation.output
  File "apache_beam/runners/worker/operations.py", line 359, in apache_beam.runners.worker.operations.Operation.output
  File "apache_beam/runners/worker/operations.py", line 221, in apache_beam.runners.worker.operations.SingletonConsumerSet.receive
  File "apache_beam/runners/worker/operations.py", line 718, in apache_beam.runners.worker.operations.DoOperation.process
  File "apache_beam/runners/worker/operations.py", line 719, in apache_beam.runners.worker.operations.DoOperation.process
  File "apache_beam/runners/common.py", line 1241, in apache_beam.runners.common.DoFnRunner.process
  File "apache_beam/runners/common.py", line 1321, in apache_beam.runners.common.DoFnRunner._reraise_augmented
  File "/usr/local/lib/python3.8/site-packages/future/utils/__init__.py", line 446, in raise_with_traceback
    raise exc.with_traceback(traceback)
  File "apache_beam/runners/common.py", line 1239, in apache_beam.runners.common.DoFnRunner.process
  File "apache_beam/runners/common.py", line 768, in apache_beam.runners.common.PerWindowInvoker.invoke_process
  File "apache_beam/runners/common.py", line 891, in apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
  File "apache_beam/runners/common.py", line 1374, in apache_beam.runners.common._OutputProcessor.process_outputs
  File "/usr/local/lib/python3.8/site-packages/apache_beam/io/gcp/bigquery_file_loads.py", line 520, in process
    job_reference = self.bq_wrapper.perform_load_job(
  File "/usr/local/lib/python3.8/site-packages/apache_beam/io/gcp/bigquery_tools.py", line 825, in perform_load_job
    return self._insert_load_job(
  File "/usr/local/lib/python3.8/site-packages/apache_beam/utils/retry.py", line 260, in wrapper
    return fun(*args, **kwargs)
  File "/usr/local/lib/python3.8/site-packages/apache_beam/io/gcp/bigquery_tools.py", line 438, in _insert_load_job
    return self._start_job(request).jobReference
  File "/usr/local/lib/python3.8/site-packages/apache_beam/io/gcp/bigquery_tools.py", line 449, in _start_job
    response = self.client.jobs.Insert(request)
  File "/usr/local/lib/python3.8/site-packages/apache_beam/io/gcp/internal/clients/bigquery/bigquery_v2_client.py", line 345, in Insert
    return self._RunMethod(
  File "/usr/local/lib/python3.8/site-packages/apitools/base/py/base_api.py", line 731, in _RunMethod
    return self.ProcessHttpResponse(method_config, http_response, request)
  File "/usr/local/lib/python3.8/site-packages/apitools/base/py/base_api.py", line 737, in ProcessHttpResponse
    self.__ProcessHttpResponse(method_config, http_response, request))
  File "/usr/local/lib/python3.8/site-packages/apitools/base/py/base_api.py", line 603, in __ProcessHttpResponse
    raise exceptions.HttpError.FromResponse(
RuntimeError: apitools.base.py.exceptions.HttpBadRequestError: HttpError accessing <https://bigquery.googleapis.com/bigquery/v2/projects/my-project/jobs?alt=json>: response: <{'vary': 'Origin, X-Origin, Referer', 'content-type': 'application/json; charset=UTF-8', 'date': 'Tue, 09 Mar 2021 09:31:01 GMT', 'server': 'ESF', 'cache-control': 'private', 'x-xss-protection': '0', 'x-frame-options': 'SAMEORIGIN', 'x-content-type-options': 'nosniff', 'transfer-encoding': 'chunked', 'status': '400', 'content-length': '318', '-content-encoding': 'gzip'}>, content <{
  "error": {
    "code": 400,
    "message": "Load configuration must specify at least one source URI",
    "errors": [
      {
        "message": "Load configuration must specify at least one source URI",
        "domain": "global",
        "reason": "invalid"
      }
    ],
    "status": "INVALID_ARGUMENT"
  }
}
{code}
 


Perhaps, this can be fixed by validating the input value `[files(= element[1])|https://github.com/apache/beam/blob/v2.28.0/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py#L469]` is not empty.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)