[ 
https://issues.apache.org/jira/browse/BEAM-12479?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Pascal GILLET updated BEAM-12479:
---------------------------------
    Description: 
{{UnsupportedOperationExceptions}} are thrown in

{{org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils#toBeamValue(FieldType, 
Object)}}

when reading from BigQuery tables with 

{{org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO#readTableRowsWithSchema()}} 
and converting the returned {{TableRows}} to Beam {{Rows}}

Example:
{code:java}
PCollection<Row> rows =
 pipeline
 .apply(
 "Read from BigQuery table",
 BigQueryIO.readTableRowsWithSchema().from(String.format("%s:%s.%s", project, 
dataset, table)))
 .apply(Convert.toRows());{code}
 

{{UnsupportedOperationException}} messages that I have encountered are of the 
type:

{{Converting BigQuery type "java.lang.Boolean" to "BOOLEAN" is not supported}}

{{Converting BigQuery type "java.lang.Double" to "DOUBLE" is not supported}}

...While the conversion of these Java types should be straightforward.

Indeed, the method {{BigQueryUtils#toBeamValue(FieldType, Object)}} expects 
only {{String}} objects or {{Collections}} of {{Strings}}.

I had to upgrade {{com.google.cloud:google-cloud-bigquery}} from 1.108.0 to 
1.132.0. So my guess is this latest version is now able to map BiqQuery (SQL) 
types to Java types instead of raw Strings, in particular BOOL to Boolean, 
INT64 to Long and FLOAT64 to Double.

In my understanding though, from BigQuery to Beam, there would be no need to 
manage Java Byte, Short, Integer and Float since BigQuery types are "limited" 
to standard SQL types INT64, and FLOAT64 that encompass them all (on the other 
hand, BQ NUMERIC type is mapped to Java BigDecimal).

I propose a pull request to also support {{Number}} and {{Boolean}} objects in 
method {{BigQueryUtils#toBeamValue(FieldType, Object)}}. It is only added 
behavior and the updated method is still compatible with the current 
functioning.

 

 

  was:
{{UnsupportedOperationExceptions}} are thrown in

{{org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils#toBeamValue(FieldType, 
Object)}}

when reading from BigQuery tables with 

{{org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO#readTableRowsWithSchema()}} 
and converting the returned {{TableRows}} to Beam {{Rows}}

Example:
{code:java}
PCollection<Row> rows =
 pipeline
 .apply(
 "Read from BigQuery table",
 BigQueryIO.readTableRowsWithSchema().from(String.format("%s:%s.%s", project, 
dataset, table)))
 .apply(Convert.toRows());{code}
 

{{UnsupportedOperationException}} messages are of the type:

{{Converting BigQuery type "java.lang.Boolean" to "BOOLEAN" is not supported}}

{{Converting BigQuery type "java.lang.Double" to "DOUBLE" is not supported}}

And so on for other numeric types...while the conversion of these Java types 
should be straightforward.

Indeed, the method {{BigQueryUtils#toBeamValue(FieldType, Object)}} expects 
only {{String}} objects or {{Collections}} of {{Strings}}.

I propose a pull request to also support {{Number}} and {{Boolean}} objects.

 

 


> UnsupportedOperationException when reading from BigQuery tables and 
> converting TableRows to Beam Rows
> -----------------------------------------------------------------------------------------------------
>
>                 Key: BEAM-12479
>                 URL: https://issues.apache.org/jira/browse/BEAM-12479
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-gcp
>    Affects Versions: 2.29.0
>         Environment: Beam Java SDK
>            Reporter: Pascal GILLET
>            Assignee: Pascal GILLET
>            Priority: P2
>              Labels: pull-request-available
>             Fix For: 2.32.0
>
>
> {{UnsupportedOperationExceptions}} are thrown in
> {{org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils#toBeamValue(FieldType, 
> Object)}}
> when reading from BigQuery tables with 
> {{org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO#readTableRowsWithSchema()}} 
> and converting the returned {{TableRows}} to Beam {{Rows}}
> Example:
> {code:java}
> PCollection<Row> rows =
>  pipeline
>  .apply(
>  "Read from BigQuery table",
>  BigQueryIO.readTableRowsWithSchema().from(String.format("%s:%s.%s", project, 
> dataset, table)))
>  .apply(Convert.toRows());{code}
>  
> {{UnsupportedOperationException}} messages that I have encountered are of the 
> type:
> {{Converting BigQuery type "java.lang.Boolean" to "BOOLEAN" is not supported}}
> {{Converting BigQuery type "java.lang.Double" to "DOUBLE" is not supported}}
> ...While the conversion of these Java types should be straightforward.
> Indeed, the method {{BigQueryUtils#toBeamValue(FieldType, Object)}} expects 
> only {{String}} objects or {{Collections}} of {{Strings}}.
> I had to upgrade {{com.google.cloud:google-cloud-bigquery}} from 1.108.0 to 
> 1.132.0. So my guess is this latest version is now able to map BiqQuery (SQL) 
> types to Java types instead of raw Strings, in particular BOOL to Boolean, 
> INT64 to Long and FLOAT64 to Double.
> In my understanding though, from BigQuery to Beam, there would be no need to 
> manage Java Byte, Short, Integer and Float since BigQuery types are "limited" 
> to standard SQL types INT64, and FLOAT64 that encompass them all (on the 
> other hand, BQ NUMERIC type is mapped to Java BigDecimal).
> I propose a pull request to also support {{Number}} and {{Boolean}} objects 
> in method {{BigQueryUtils#toBeamValue(FieldType, Object)}}. It is only added 
> behavior and the updated method is still compatible with the current 
> functioning.
>  
>  



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

Reply via email to