[
https://issues.apache.org/jira/browse/BEAM-3983?focusedWorklogId=96840&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-96840
]
ASF GitHub Bot logged work on BEAM-3983:
----------------------------------------
Author: ASF GitHub Bot
Created on: 30/Apr/18 20:23
Start Date: 30/Apr/18 20:23
Worklog Time Spent: 10m
Work Description: kennknowles closed pull request #5220: [BEAM-3983][SQL]
Add BigQuery table provider
URL: https://github.com/apache/beam/pull/5220
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/build.gradle
b/sdks/java/extensions/sql/build.gradle
index bdd4f3e2a63..a286cf437f2 100644
--- a/sdks/java/extensions/sql/build.gradle
+++ b/sdks/java/extensions/sql/build.gradle
@@ -65,6 +65,7 @@ dependencies {
shadow library.java.joda_time
shadow project(path: ":beam-runners-direct-java", configuration: "shadow")
provided project(path: ":beam-sdks-java-io-kafka", configuration: "shadow")
+ provided project(path: ":beam-sdks-java-io-google-cloud-platform",
configuration: "shadow")
provided library.java.kafka_clients
testCompile library.java.slf4j_jdk14
testCompile library.java.junit
diff --git a/sdks/java/extensions/sql/pom.xml b/sdks/java/extensions/sql/pom.xml
index 61fcb319f68..586847eaf4e 100644
--- a/sdks/java/extensions/sql/pom.xml
+++ b/sdks/java/extensions/sql/pom.xml
@@ -396,6 +396,12 @@
<scope>provided</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.beam</groupId>
+ <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
+ <scope>provided</scope>
+ </dependency>
+
<!-- for tests -->
<dependency>
<groupId>junit</groupId>
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQueryTable.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQueryTable.java
new file mode 100644
index 00000000000..6bfd839c018
--- /dev/null
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQueryTable.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.extensions.sql.impl.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.impl.schema.BeamIOType;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils;
+import org.apache.beam.sdk.io.gcp.bigquery.WriteResult;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * {@code BeamBigQueryTable} represent a BigQuery table as a target.
+ * This provider does not currently support being a source.
+ *
+ */
+@Experimental
+public class BeamBigQueryTable extends BaseBeamTable implements Serializable {
+ private String tableSpec;
+
+ public BeamBigQueryTable(Schema beamSchema, String tableSpec) {
+ super(beamSchema);
+ this.tableSpec = tableSpec;
+ }
+
+ @Override
+ public BeamIOType getSourceType() {
+ return BeamIOType.BOUNDED;
+ }
+
+ @Override
+ public PCollection<Row> buildIOReader(Pipeline pipeline) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public PTransform<? super PCollection<Row>, POutput> buildIOWriter() {
+ return new PTransform<PCollection<Row>, POutput>() {
+ @Override
+ public WriteResult expand(PCollection<Row> input) {
+ return input.apply(BigQueryIO.<Row>write()
+ .withSchema(BigQueryUtils.toTableSchema(getSchema()))
+ .withFormatFunction(BigQueryUtils.toTableRow())
+ .to(tableSpec));
+ }
+ };
+ }
+
+ public String getTableSpec() {
+ return tableSpec;
+ }
+}
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTableProvider.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTableProvider.java
new file mode 100644
index 00000000000..dd728d0faeb
--- /dev/null
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTableProvider.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
+
+import static
org.apache.beam.sdk.extensions.sql.meta.provider.MetaUtils.getRowTypeFromTable;
+
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlTable;
+import org.apache.beam.sdk.extensions.sql.meta.Table;
+import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
+import org.apache.beam.sdk.schemas.Schema;
+
+/**
+ * BigQuery table provider.
+ *
+ * <p>A sample of text table is:
+ * <pre>{@code
+ * CREATE TABLE ORDERS(
+ * ID INT COMMENT 'this is the primary key',
+ * NAME VARCHAR(127) COMMENT 'this is the name'
+ * )
+ * TYPE 'bigquery'
+ * COMMENT 'this is the table orders'
+ * LOCATION '[PROJECT_ID]:[DATASET].[TABLE]'
+ * }</pre>
+ */
+public class BigQueryTableProvider implements TableProvider {
+
+ @Override public String getTableType() {
+ return "bigquery";
+ }
+
+ @Override public BeamSqlTable buildBeamSqlTable(Table table) {
+ Schema schema = getRowTypeFromTable(table);
+ String filePattern = table.getLocation();
+
+ return new BeamBigQueryTable(schema, filePattern);
+ }
+
+ @Override public void createTable(Table table) {
+ // empty
+ }
+
+ @Override public void dropTable(String tableName) {
+ // empty
+ }
+
+ @Override public List<Table> listTables() {
+ return Collections.emptyList();
+ }
+
+ @Override public void init() {
+ // empty
+ }
+
+ @Override public void close() {
+ // empty
+ }
+}
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/package-info.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/package-info.java
new file mode 100644
index 00000000000..129714104b2
--- /dev/null
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Table schema for BigQuery.
+ */
+package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/package-info.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/package-info.java
index 4101da775d5..19e88601545 100644
---
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/package-info.java
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/package-info.java
@@ -17,6 +17,6 @@
*/
/**
- * table schema for KafkaIO.
+ * Table schema for KafkaIO.
*/
package org.apache.beam.sdk.extensions.sql.meta.provider.kafka;
diff --git
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTableProviderTest.java
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTableProviderTest.java
new file mode 100644
index 00000000000..51b3452247d
--- /dev/null
+++
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTableProviderTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
+
+import static org.apache.beam.sdk.extensions.sql.RowSqlTypes.INTEGER;
+import static org.apache.beam.sdk.extensions.sql.RowSqlTypes.VARCHAR;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.beam.sdk.extensions.sql.BeamSqlTable;
+import org.apache.beam.sdk.extensions.sql.meta.Column;
+import org.apache.beam.sdk.extensions.sql.meta.Table;
+import org.junit.Test;
+
+/**
+ * UnitTest for {@link BigQueryTableProvider}.
+ */
+public class BigQueryTableProviderTest {
+ private BigQueryTableProvider provider = new BigQueryTableProvider();
+
+ @Test
+ public void testGetTableType() throws Exception {
+ assertEquals("bigquery", provider.getTableType());
+ }
+
+ @Test
+ public void testBuildBeamSqlTable() throws Exception {
+ Table table = fakeTable("hello");
+ BeamSqlTable sqlTable = provider.buildBeamSqlTable(table);
+
+ assertNotNull(sqlTable);
+ assertTrue(sqlTable instanceof BeamBigQueryTable);
+
+ BeamBigQueryTable bqTable = (BeamBigQueryTable) sqlTable;
+ assertEquals("project:dataset.table", bqTable.getTableSpec());
+ }
+
+ private static Table fakeTable(String name) {
+ return Table.builder()
+ .name(name)
+ .comment(name + " table")
+ .location("project:dataset.table")
+ .columns(ImmutableList.of(
+ Column.builder()
+ .name("id")
+ .fieldType(INTEGER)
+ .nullable(true)
+ .build(),
+ Column.builder()
+ .name("name")
+ .fieldType(VARCHAR)
+ .nullable(true)
+ .build()
+ ))
+ .type("bigquery")
+ .build();
+ }
+}
----------------------------------------------------------------
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: 96840)
Time Spent: 12h 50m (was: 12h 40m)
> BigQuery writes from pure SQL
> -----------------------------
>
> Key: BEAM-3983
> URL: https://issues.apache.org/jira/browse/BEAM-3983
> Project: Beam
> Issue Type: New Feature
> Components: dsl-sql
> Reporter: Andrew Pilloud
> Assignee: Andrew Pilloud
> Priority: Major
> Time Spent: 12h 50m
> Remaining Estimate: 0h
>
> It would be nice if you could write to BigQuery in SQL without writing any
> java code. For example:
> {code:java}
> INSERT INTO bigquery SELECT * FROM PCOLLECTION{code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)