[
https://issues.apache.org/jira/browse/BEAM-4646?focusedWorklogId=116749&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-116749
]
ASF GitHub Bot logged work on BEAM-4646:
----------------------------------------
Author: ASF GitHub Bot
Created on: 28/Jun/18 03:44
Start Date: 28/Jun/18 03:44
Worklog Time Spent: 10m
Work Description: kennknowles closed pull request #5792: [BEAM-4646][SQL]
Improve BigQuery write integration test
URL: https://github.com/apache/beam/pull/5792
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/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryWriteIT.java
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryWriteIT.java
index 52111afdfcf..d3f967e2472 100644
---
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryWriteIT.java
+++
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryWriteIT.java
@@ -17,6 +17,13 @@
*/
package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.BOOLEAN;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.BYTE;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.DECIMAL;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.DOUBLE;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.FLOAT;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.INT16;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.INT32;
import static org.apache.beam.sdk.schemas.Schema.FieldType.INT64;
import static org.apache.beam.sdk.schemas.Schema.FieldType.STRING;
import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -24,6 +31,7 @@
import com.google.common.collect.ImmutableMap;
import java.io.Serializable;
+import java.math.BigDecimal;
import java.util.Arrays;
import java.util.List;
import org.apache.beam.sdk.Pipeline;
@@ -39,7 +47,9 @@
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.Row;
+import org.joda.time.DateTime;
import org.joda.time.Duration;
+import org.joda.time.chrono.ISOChronology;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -55,8 +65,89 @@
.addNullableField("arr", FieldType.array(STRING))
.build();
+ private static final Schema SOURCE_SCHEMA_TWO =
+ Schema.builder()
+ .addNullableField("c_bigint", INT64)
+ .addNullableField("c_tinyint", BYTE)
+ .addNullableField("c_smallint", INT16)
+ .addNullableField("c_integer", INT32)
+ .addNullableField("c_float", FLOAT)
+ .addNullableField("c_double", DOUBLE)
+ .addNullableField("c_decimal", DECIMAL)
+ .addNullableField("c_boolean", BOOLEAN)
+ .addNullableField("c_timestamp",
FieldType.DATETIME.withMetadata("TS"))
+ .addNullableField("c_varchar", STRING)
+ .addNullableField("c_char", STRING)
+ .addNullableField("c_arr", FieldType.array(STRING))
+ .build();
+
@Rule public transient TestPipeline pipeline = TestPipeline.create();
@Rule public transient TestBigQuery bigQuery =
TestBigQuery.create(SOURCE_SCHEMA);
+ @Rule public transient TestBigQuery bigQueryTestingTypes =
TestBigQuery.create(SOURCE_SCHEMA_TWO);
+
+ @Test
+ public void testSQLTypes() {
+ BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(new BigQueryTableProvider());
+
+ String createTableStatement =
+ "CREATE TABLE TEST( \n"
+ + " c_bigint BIGINT, \n"
+ + " c_tinyint TINYINT, \n"
+ + " c_smallint SMALLINT, \n"
+ + " c_integer INTEGER, \n"
+ + " c_float FLOAT, \n"
+ + " c_double DOUBLE, \n"
+ + " c_decimal DECIMAL, \n"
+ + " c_boolean BOOLEAN, \n"
+ + " c_timestamp TIMESTAMP, \n"
+ + " c_varchar VARCHAR, \n "
+ + " c_char CHAR, \n"
+ + " c_arr ARRAY<VARCHAR> \n"
+ + ") \n"
+ + "TYPE 'bigquery' \n"
+ + "LOCATION '"
+ + bigQueryTestingTypes.tableSpec()
+ + "'";
+ sqlEnv.executeDdl(createTableStatement);
+ System.out.println(bigQueryTestingTypes.tableSpec());
+
+ String insertStatement =
+ "INSERT INTO TEST VALUES ("
+ + "9223372036854775807, "
+ + "127, "
+ + "32767, "
+ + "2147483647, "
+ + "1.0, "
+ + "1.0, "
+ + "123.45, "
+ + "TRUE, "
+ + "TIMESTAMP '2018-05-28 20:17:40.123', "
+ + "'varchar', "
+ + "'char', "
+ + "ARRAY['123', '456']"
+ + ")";
+ sqlEnv.parseQuery(insertStatement);
+ BeamSqlRelUtils.toPCollection(pipeline,
sqlEnv.parseQuery(insertStatement));
+ pipeline.run().waitUntilFinish(Duration.standardMinutes(5));
+
+ assertThat(
+ bigQueryTestingTypes.getFlatJsonRows(SOURCE_SCHEMA_TWO),
+ containsInAnyOrder(
+ row(
+ SOURCE_SCHEMA_TWO,
+ 9223372036854775807L,
+ (byte) 127,
+ (short) 32767,
+ 2147483647,
+ (float) 1.0,
+ 1.0,
+ BigDecimal.valueOf(123.45),
+ true,
+ new DateTime(2018, 05, 28, 20, 17, 40, 123,
ISOChronology.getInstanceUTC()),
+ "varchar",
+ "char",
+ Arrays.asList("123", "456"))));
+ }
@Test
public void testInsertValues() throws Exception {
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 be4e3861cb9..3769a45384c 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
@@ -41,6 +41,11 @@
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.Row;
+import org.joda.time.DateTime;
+import org.joda.time.Instant;
+import org.joda.time.chrono.ISOChronology;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.DateTimeFormatterBuilder;
/**
* Utility methods for BigQuery related operations.
@@ -84,6 +89,8 @@
.put(TypeName.DECIMAL, BigDecimal::new)
.put(TypeName.BOOLEAN, Boolean::valueOf)
.put(TypeName.STRING, str -> str)
+ .put(TypeName.DATETIME, str -> new DateTime((long)
(Double.parseDouble(str) * 1000),
+ ISOChronology.getInstanceUTC()))
.build();
private static final Map<String, StandardSQLTypeName>
BEAM_TO_BIGQUERY_METADATA_MAPPING =
@@ -172,18 +179,30 @@ public static TableRow toTableRow(Row row) {
Field schemaField = row.getSchema().getField(i);
TypeName type = schemaField.getType().getTypeName();
- if (TypeName.ARRAY == type) {
- type = schemaField.getType().getCollectionElementType().getTypeName();
- if (TypeName.ROW == type) {
- List<Row> rows = (List<Row>) value;
- List<TableRow> tableRows = new ArrayList<TableRow>(rows.size());
- for (int j = 0; j < rows.size(); j++) {
- tableRows.add(toTableRow(rows.get(j)));
+ switch (type) {
+ case ARRAY:
+ type =
schemaField.getType().getCollectionElementType().getTypeName();
+ if (TypeName.ROW == type) {
+ List<Row> rows = (List<Row>) value;
+ List<TableRow> tableRows = new ArrayList<TableRow>(rows.size());
+ for (int j = 0; j < rows.size(); j++) {
+ tableRows.add(toTableRow(rows.get(j)));
+ }
+ value = tableRows;
}
- value = tableRows;
- }
- } else if (TypeName.ROW == type) {
- value = toTableRow((Row) value);
+ break;
+ case ROW:
+ value = toTableRow((Row) value);
+ break;
+ case DATETIME:
+ DateTimeFormatter patternFormat = new DateTimeFormatterBuilder()
+ .appendPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZZ")
+ .toFormatter();
+ value = ((Instant) value).toDateTime().toString(patternFormat);
+ break;
+ default:
+ value = row.getValue(i);
+ break;
}
output = output.set(schemaField.getName(), value);
----------------------------------------------------------------
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: 116749)
Time Spent: 5h (was: 4h 50m)
> Improve SQL's BigQuery write integration test
> ---------------------------------------------
>
> Key: BEAM-4646
> URL: https://issues.apache.org/jira/browse/BEAM-4646
> Project: Beam
> Issue Type: Improvement
> Components: dsl-sql
> Reporter: Rui Wang
> Assignee: Rui Wang
> Priority: Major
> Time Spent: 5h
> Remaining Estimate: 0h
>
> Right now SQL's BigQuery write integration test only tests VARCHAR, ARRAY and
> BIGINT, it might be better to test other SQL types which Beam SQL/Calcite is
> supporting.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)