[
https://issues.apache.org/jira/browse/BEAM-4417?focusedWorklogId=117927&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-117927
]
ASF GitHub Bot logged work on BEAM-4417:
----------------------------------------
Author: ASF GitHub Bot
Created on: 30/Jun/18 23:01
Start Date: 30/Jun/18 23:01
Worklog Time Spent: 10m
Work Description: kennknowles closed pull request #5755: [BEAM-4417]
Support BigQuery's NUMERIC type using Java
URL: https://github.com/apache/beam/pull/5755
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java
index f8bf472bb97..bf2cb63e55c 100644
---
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java
+++
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java
@@ -28,6 +28,7 @@
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.BaseEncoding;
+import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
@@ -53,6 +54,7 @@
.put("BYTES", Type.BYTES)
.put("INTEGER", Type.LONG)
.put("FLOAT", Type.DOUBLE)
+ .put("NUMERIC", Type.STRING)
.put("BOOLEAN", Type.BOOLEAN)
.put("TIMESTAMP", Type.LONG)
.put("RECORD", Type.RECORD)
@@ -60,6 +62,7 @@
.put("DATETIME", Type.STRING)
.put("TIME", Type.STRING)
.build();
+
/**
* Formats BigQuery seconds-since-epoch into String matching JSON export.
Thread-safe and
* immutable.
@@ -194,6 +197,12 @@ private static Object convertRequiredField(
case "FLOAT":
verify(v instanceof Double, "Expected Double, got %s", v.getClass());
return v;
+ case "NUMERIC":
+ verify(
+ v instanceof CharSequence || v instanceof BigDecimal,
+ "Expected CharSequence (String) or BigDecimal, got %s",
+ v.getClass());
+ return v.toString();
case "BOOLEAN":
verify(v instanceof Boolean, "Expected Boolean, got %s", v.getClass());
return v;
diff --git
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
index 218e9c272f5..00b0a7d97b0 100644
---
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
+++
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
@@ -70,7 +70,7 @@
.put(TypeName.INT64, StandardSQLTypeName.INT64)
.put(TypeName.FLOAT, StandardSQLTypeName.FLOAT64)
.put(TypeName.DOUBLE, StandardSQLTypeName.FLOAT64)
- .put(TypeName.DECIMAL, StandardSQLTypeName.FLOAT64)
+ .put(TypeName.DECIMAL, StandardSQLTypeName.NUMERIC)
.put(TypeName.BOOLEAN, StandardSQLTypeName.BOOL)
.put(TypeName.ARRAY, StandardSQLTypeName.ARRAY)
.put(TypeName.ROW, StandardSQLTypeName.STRUCT)
diff --git
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StandardSQLTypeName.java
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StandardSQLTypeName.java
index 632b8aeb1a6..7ffa85c8b66 100644
---
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StandardSQLTypeName.java
+++
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StandardSQLTypeName.java
@@ -31,6 +31,8 @@
INT64,
/** A 64-bit IEEE binary floating-point value. */
FLOAT64,
+ /** A decimal value with 38 digits of precision and 9 digits of scale. */
+ NUMERIC,
/** Variable-length character (Unicode) data. */
STRING,
/** Variable-length binary data. */
diff --git
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java
index 42f9167a009..e000b561b1b 100644
---
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java
+++
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java
@@ -27,6 +27,7 @@
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.io.BaseEncoding;
+import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
@@ -65,6 +66,7 @@
new TableFieldSchema().setName("quality").setType("FLOAT") /*
default to NULLABLE */,
new TableFieldSchema().setName("quantity").setType("INTEGER") /*
default to NULLABLE */,
new
TableFieldSchema().setName("birthday").setType("TIMESTAMP").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("birthdayMoney").setType("NUMERIC").setMode("NULLABLE"),
new
TableFieldSchema().setName("flighted").setType("BOOLEAN").setMode("NULLABLE"),
new
TableFieldSchema().setName("sound").setType("BYTES").setMode("NULLABLE"),
new
TableFieldSchema().setName("anniversaryDate").setType("DATE").setMode("NULLABLE"),
@@ -100,7 +102,7 @@ public void testConvertGenericRecordToTableRow() throws
Exception {
}
{
// Test type conversion for:
- // INTEGER, FLOAT, TIMESTAMP, BOOLEAN, BYTES, DATE, DATETIME, TIME.
+ // INTEGER, FLOAT, NUMERIC, TIMESTAMP, BOOLEAN, BYTES, DATE, DATETIME,
TIME.
GenericRecord record = new GenericData.Record(avroSchema);
byte[] soundBytes = "chirp,chirp".getBytes(StandardCharsets.UTF_8);
ByteBuffer soundByteBuffer = ByteBuffer.wrap(soundBytes);
@@ -108,6 +110,7 @@ public void testConvertGenericRecordToTableRow() throws
Exception {
record.put("number", 5L);
record.put("quality", 5.0);
record.put("birthday", 5L);
+ record.put("birthdayMoney", new String("123456789.123456789"));
record.put("flighted", Boolean.TRUE);
record.put("sound", soundByteBuffer);
record.put("anniversaryDate", new Utf8("2000-01-01"));
@@ -118,6 +121,7 @@ public void testConvertGenericRecordToTableRow() throws
Exception {
new TableRow()
.set("number", "5")
.set("birthday", "1970-01-01 00:00:00.000005 UTC")
+ .set("birthdayMoney", "123456789.123456789")
.set("quality", 5.0)
.set("associates", new ArrayList<TableRow>())
.set("flighted", Boolean.TRUE)
@@ -135,11 +139,13 @@ public void testConvertGenericRecordToTableRow() throws
Exception {
GenericRecord record = new GenericData.Record(avroSchema);
record.put("number", 5L);
record.put("associates", Lists.newArrayList(nestedRecord));
+ record.put("birthdayMoney", new BigDecimal("987654321.987654321"));
TableRow convertedRow =
BigQueryAvroUtils.convertGenericRecordToTableRow(record, tableSchema);
TableRow row =
new TableRow()
.set("associates", Lists.newArrayList(new
TableRow().set("species", "other")))
- .set("number", "5");
+ .set("number", "5")
+ .set("birthdayMoney", "987654321.987654321");
assertEquals(row, convertedRow);
}
}
@@ -164,6 +170,9 @@ public void testConvertBigQuerySchemaToAvroSchema() {
assertThat(
avroSchema.getField("birthday").schema(),
equalTo(Schema.createUnion(Schema.create(Type.NULL),
Schema.create(Type.LONG))));
+ assertThat(
+ avroSchema.getField("birthdayMoney").schema(),
+ equalTo(Schema.createUnion(Schema.create(Type.NULL),
Schema.create(Type.STRING))));
assertThat(
avroSchema.getField("flighted").schema(),
equalTo(Schema.createUnion(Schema.create(Type.NULL),
Schema.create(Type.BOOLEAN))));
@@ -224,6 +233,7 @@ public void testConvertBigQuerySchemaToAvroSchema() {
@Nullable Double quality;
@Nullable Long quantity;
@Nullable Long birthday; // Exercises TIMESTAMP.
+ @Nullable String birthdayMoney; // Exercises NUMERIC.
@Nullable Boolean flighted;
@Nullable ByteBuffer sound;
@Nullable Utf8 anniversaryDate;
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 117927)
Time Spent: 3h 10m (was: 3h)
> BigqueryIO Numeric datatype Support
> -----------------------------------
>
> Key: BEAM-4417
> URL: https://issues.apache.org/jira/browse/BEAM-4417
> Project: Beam
> Issue Type: Improvement
> Components: io-java-gcp
> Affects Versions: 2.4.0
> Reporter: Kishan Kumar
> Assignee: Chamikara Jayalath
> Priority: Critical
> Labels: newbie, patch
> Fix For: 2.6.0
>
> Time Spent: 3h 10m
> Remaining Estimate: 0h
>
> The BigQueryIO.read fails while parsing the data from the avro file generated
> while reading the data from the table which has columns with *Numeric*
> datatypes.
> We have gone through the source code at Git-Hub and noticed that *Numeric
> data type is not yet supported.*
>
> Caused by: com.google.common.base.VerifyException: Unsupported BigQuery type:
> NUMERIC
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)