You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@beam.apache.org by Bruno Quinart <br...@quimail.eu> on 2022/02/13 21:36:01 UTC

Re: BigQuery sink & Storage Write API

Hello

Thanks for the confirmation that we can use the Big Query Storage Write API.

I was able to get it working but noticed some things.

1. You need to specify withTriggeringFrequency. The compiler complains if not specified.
The Storage Write API also supports streaming. Is there a reason for not supporting this within Beam?
This API will replace the legacy Streaming API, does this mean low latency inserts to BigQuery will no longer be supported from Beam?

2. Doesn’t work with Dataflow Prime. I get an "Error processing pipeline.” failure when launching with Dataflow Prime. Note that Prime is currently still in Preview.

3. You have to specify withNumStorageWriteApiStreams. When not specified, you get a run time error. Even though the method documentation mentions withAutoSharding() is an alternative. You get a compiler message that it is not yet supported with STORAGE_WRITE_API method.

4. The documentation hasn’t been updated everywhere to include the new write methods. For example for withAutoSharding().

I also noticed another mail last week on user mail list regarding some weird messages with the storage write api.
It seems the new BigQuery insert method is not yet ready for production use.

Rgds
Bruno


> On 25 Jan 2022, at 22:46, Reuven Lax <re...@google.com> wrote:
> 
> Yes, you can use the storage write API from Java. Native Python support has not yet been implemented.
> 
> On Mon, Jan 24, 2022 at 1:47 AM Bruno Quinart <bruno@quimail.eu <ma...@quimail.eu>> wrote:
> Hello
> 
> The BigQuery Storage Write API is GA since October 2021 (docs at [1]).
> BEAM-11648 was created to adapt the Beam BigQuery sink.
> 
> That Jira issue is still marked as open. However, it seems that the functionality has already been added.
> The Javadoc has the STORAGE_WRITE_API method added since release 2.29.0 (see [2]).
> The latest release (2.35.0), removed the notion that it is an experimental API (at BigQuery side) and also added STORAGE_API_AT_LEAST_ONCE method [3].
> 
> However the Beam documentation at [4] does not mention the Storage Write API option at all.
> 
> Can we consider this development done and start using these features?
> 
> What would be the best approach for a Python pipeline?
> I found BEAM-10917 for the Storage Read API with Python SDK. That Jira is also open, but again seems functionality has been added.
> As from release 2.34.0 I see that it is possible to provide method = DIRECT_READ to use the Storage Read API [5] (always in Arvo it seems, not clear how you could use Arrow).
> But didn’t find anything for the Storage Write API.
> 
> Is it better (and even possible) to use the Java BigQuery sink using the multi language features?
> 
> Apologies for these annoying questions. As a dataflow user, I am a bit lost to understand what is the reference (Google docs are limited and refer to Beam, but seems the docs lag a bit versus the code).
> 
> Thanks a lot!
> Bruno
> 
> [1] https://cloud.google.com/bigquery/docs/write-api <https://cloud.google.com/bigquery/docs/write-api>
> [2] https://beam.apache.org/releases/javadoc/2.29.0/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.Method.html <https://beam.apache.org/releases/javadoc/2.29.0/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.Method.html>
> [3] https://beam.apache.org/releases/javadoc/2.35.0/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.Method.html <https://beam.apache.org/releases/javadoc/2.35.0/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.Method.html>
> [4] https://beam.apache.org/documentation/io/built-in/google-bigquery/#writing-to-bigquery <https://beam.apache.org/documentation/io/built-in/google-bigquery/#writing-to-bigquery>
> [5] https://beam.apache.org/releases/pydoc/2.34.0/apache_beam.io.gcp.bigquery.html <https://beam.apache.org/releases/pydoc/2.34.0/apache_beam.io.gcp.bigquery.html>
> 


Re: BigQuery sink & Storage Write API

Posted by Reuven Lax <re...@google.com>.
On Sun, Feb 13, 2022 at 1:36 PM Bruno Quinart <br...@quimail.eu> wrote:

> Hello
>
> Thanks for the confirmation that we can use the Big Query Storage Write
> API.
>
> I was able to get it working but noticed some things.
>
> 1. You need to specify withTriggeringFrequency. The compiler complains if
> not specified.
> The Storage Write API also supports streaming. Is there a reason for not
> supporting this within Beam?
>

We do write records immediately, however they are written to a buffered
stream which is flushed at the specified interval. However you can set the
flush interval fairly small - i.e. less than one second - which provides
fairly low latency.

You can achieve even lower latency (and lower cost) if you specify the
at-least-once storage write method. In this case, you don't need to specify
withTriggeringFrequency, and the records are visible immediately in
BigQuery. The tradeoff is that when using this method, some duplicates
might be written to BigQuery.


> This API will replace the legacy Streaming API, does this mean low latency
> inserts to BigQuery will no longer be supported from Beam?
>

See above: the latency should be low (< 1 second) and you can achieve super
low latency if you are willing to sacrifice exactly-once writes.

>
> 2. Doesn’t work with Dataflow Prime. I get an "Error processing pipeline.”
> failure when launching with Dataflow Prime. Note that Prime is currently
> still in Preview.
>

This sink uses Beam features not yet supported by Dataflow Prime. Support
is in progress.

3. You have to specify withNumStorageWriteApiStreams. When not specified,
> you get a run time error. Even though the method documentation mentions
> withAutoSharding() is an alternative. You get a compiler message that it is
> not yet supported with STORAGE_WRITE_API method.
>

Auto sharding work for the storage write API is in progress. In the
meantime, you must specify the number of parallel streams.

>
> 4. The documentation hasn’t been updated everywhere to include the new
> write methods. For example for withAutoSharding().
>
> I also noticed another mail last week on user mail list regarding some
> weird messages with the storage write api.
>
What messages? I'm not aware of any on the current version of Beam.


> It seems the new BigQuery insert method is not yet ready for production
> use.
>
> Rgds
> Bruno
>
>
> On 25 Jan 2022, at 22:46, Reuven Lax <re...@google.com> wrote:
>
> Yes, you can use the storage write API from Java. Native Python support
> has not yet been implemented.
>
> On Mon, Jan 24, 2022 at 1:47 AM Bruno Quinart <br...@quimail.eu> wrote:
>
>> Hello
>>
>> The BigQuery Storage Write API is GA since October 2021 (docs at [1]).
>> BEAM-11648 was created to adapt the Beam BigQuery sink.
>>
>> That Jira issue is still marked as open. However, it seems that the
>> functionality has already been added.
>> The Javadoc has the STORAGE_WRITE_API method added since release 2.29.0
>> (see [2]).
>> The latest release (2.35.0), removed the notion that it is an
>> experimental API (at BigQuery side) and also added
>> STORAGE_API_AT_LEAST_ONCE method [3].
>>
>> However the Beam documentation at [4] does not mention the Storage Write
>> API option at all.
>>
>> Can we consider this development done and start using these features?
>>
>> What would be the best approach for a Python pipeline?
>> I found BEAM-10917 for the Storage Read API with Python SDK. That Jira is
>> also open, but again seems functionality has been added.
>> As from release 2.34.0 I see that it is possible to provide method =
>> DIRECT_READ to use the Storage Read API [5] (always in Arvo it seems, not
>> clear how you could use Arrow).
>> But didn’t find anything for the Storage Write API.
>>
>> Is it better (and even possible) to use the Java BigQuery sink using the
>> multi language features?
>>
>> Apologies for these annoying questions. As a dataflow user, I am a bit
>> lost to understand what is the reference (Google docs are limited and refer
>> to Beam, but seems the docs lag a bit versus the code).
>>
>> Thanks a lot!
>> Bruno
>>
>> [1] https://cloud.google.com/bigquery/docs/write-api
>> [2]
>> https://beam.apache.org/releases/javadoc/2.29.0/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.Method.html
>> [3]
>> https://beam.apache.org/releases/javadoc/2.35.0/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.Method.html
>> [4]
>> https://beam.apache.org/documentation/io/built-in/google-bigquery/#writing-to-bigquery
>> [5]
>> https://beam.apache.org/releases/pydoc/2.34.0/apache_beam.io.gcp.bigquery.html
>>
>>
>