Hi Vachan,
I don't think Beam DECIMAL is really a great mapping for either BigQuery's
NUMERIC or BIGNUMERIC type. Beam's DECIMAL represents arbitrary precision
decimals [1] to map well to Java's BigDecimal class [2].

Maybe we should add a fixed-precision decimal logical type [3], then have
specific instances of it with the appropriate precision that map to NUMERIC
and BIGNUMERIC. We could also shunt Beam DECIMAL to BIGNUMERIC for
compatibility.

[1]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L424
[2] https://docs.oracle.com/javase/8/docs/api/java/math/BigDecimal.html
[3]
https://github.com/apache/beam/tree/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes

On Thu, Mar 18, 2021 at 12:00 PM Vachan Shetty <vac...@google.com> wrote:

> Hello, I am currently trying to add support for BigQuery's new BIGNUMERIC
> datatype [1] in Beam's BigQueryIO. I am currently following the steps that
> were used for adding the NUMERIC datatype [2]. AFAICT Beam's DECIMAL is
> the most appropriate datatype to map to BIGNUMERIC in BQ. However, the
> Beam DECIMAL datatype is already mapped to NUMERIC in BQ [2, 3]. Given
> this, should I simply map all Beam DECIMAL to BQ BIGNUMERIC? Or should this
> conversion be done based on other information? [1]:
> https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types#decimal_types
> [2]: https://issues.apache.org/jira/browse/BEAM-4417 [3]:
> https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java#L188
>

Reply via email to