[
https://issues.apache.org/jira/browse/BEAM-6769?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16793507#comment-16793507
]
Juta Staes commented on BEAM-6769:
----------------------------------
When providing a schema, the user has to do the base-64 encoding, this in not
done in beam. When the user passes raw bytes this will be interpreted as a
base-64 string and the wrong data will be contained in bq. (e.g. when writing
b'abc' to bq, with schema BYTES it actually writes b'i\xb7' which is the same
as base64.b64decode('abc=')).
I looked further in what happens when not providing a schema: when doing a file
load job in bq itself (not using beam) and uploading a file with '\{'bytes':
'q6yt'} and schema set to autodetect I get the error
{code:java}
'Invalid schema update. Field bytes has changed type from BYTES to STRING'
{code}
This means this error will happen if we do the base64 encoding of the bytes in
beam regardless of py2 or py3 when no schema is specified.
I am not sure this is what is expected from BigQuery or if this is a bug in
the autodetect shema function.
Given this error I think we should require specifying a schema for both py2
and py3. And then use this schema to handle the encoding and decoding when
reading/writing data.
As for python 3: I think only the bytes have to be base64 encoded before doing
json.dumps(). Why should unicode values be base64 encoded?
> BigQuery IO does not support bytes in Python 3
> ----------------------------------------------
>
> Key: BEAM-6769
> URL: https://issues.apache.org/jira/browse/BEAM-6769
> Project: Beam
> Issue Type: Sub-task
> Components: sdk-py-core
> Reporter: Juta Staes
> Assignee: Juta Staes
> Priority: Major
> Time Spent: 2h 40m
> Remaining Estimate: 0h
>
> In Python 2 you could write bytes data to BigQuery. This is tested in
>
> [https://github.com/apache/beam/blob/master/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py#L186]
> Python 3 does not support
> {noformat}
> json.dumps({'test': b'test'}){noformat}
> which is used to encode the data in
>
> [https://github.com/apache/beam/blob/master/sdks/python/apache_beam/io/gcp/bigquery_tools.py#L959]
>
> How should writing bytes to BigQuery be handled in Python 3?
> * Forbid writing bytes into BigQuery on Python 3
> * Guess the encoding (utf-8?)
> * Pass the encoding to BigQuery
> cc: [~tvalentyn]
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)