You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by dh...@apache.org on 2016/07/20 20:02:56 UTC
[01/10] incubator-beam git commit: BigQueryIO: move to
google-cloud-platform module
Repository: incubator-beam
Updated Branches:
refs/heads/master 436e4a34e -> 7ec8781a2
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java
new file mode 100644
index 0000000..e0c353b
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java
@@ -0,0 +1,484 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Sum;
+
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableCell;
+import com.google.api.services.bigquery.model.TableDataInsertAllRequest;
+import com.google.api.services.bigquery.model.TableDataInsertAllResponse;
+import com.google.api.services.bigquery.model.TableDataList;
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.Matchers;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Tests for util classes related to BigQuery.
+ */
+@RunWith(JUnit4.class)
+public class BigQueryUtilTest {
+
+ @Rule
+ public ExpectedException thrown = ExpectedException.none();
+
+ @Mock private Bigquery mockClient;
+ @Mock private Bigquery.Tables mockTables;
+ @Mock private Bigquery.Tables.Get mockTablesGet;
+ @Mock private Bigquery.Tabledata mockTabledata;
+ @Mock private Bigquery.Tabledata.List mockTabledataList;
+ private PipelineOptions options;
+
+ @Before
+ public void setUp() {
+ MockitoAnnotations.initMocks(this);
+ this.options = PipelineOptionsFactory.create();
+ }
+
+ @After
+ public void tearDown() {
+ verifyNoMoreInteractions(mockClient);
+ verifyNoMoreInteractions(mockTables);
+ verifyNoMoreInteractions(mockTablesGet);
+ verifyNoMoreInteractions(mockTabledata);
+ verifyNoMoreInteractions(mockTabledataList);
+ }
+
+ private void onInsertAll(List<List<Long>> errorIndicesSequence) throws Exception {
+ when(mockClient.tabledata())
+ .thenReturn(mockTabledata);
+
+ final List<TableDataInsertAllResponse> responses = new ArrayList<>();
+ for (List<Long> errorIndices : errorIndicesSequence) {
+ List<TableDataInsertAllResponse.InsertErrors> errors = new ArrayList<>();
+ for (long i : errorIndices) {
+ TableDataInsertAllResponse.InsertErrors error =
+ new TableDataInsertAllResponse.InsertErrors();
+ error.setIndex(i);
+ }
+ TableDataInsertAllResponse response = new TableDataInsertAllResponse();
+ response.setInsertErrors(errors);
+ responses.add(response);
+ }
+
+ doAnswer(
+ new Answer<Bigquery.Tabledata.InsertAll>() {
+ @Override
+ public Bigquery.Tabledata.InsertAll answer(InvocationOnMock invocation) throws Throwable {
+ Bigquery.Tabledata.InsertAll mockInsertAll = mock(Bigquery.Tabledata.InsertAll.class);
+ when(mockInsertAll.execute())
+ .thenReturn(responses.get(0),
+ responses.subList(1, responses.size()).toArray(
+ new TableDataInsertAllResponse[responses.size() - 1]));
+ return mockInsertAll;
+ }
+ })
+ .when(mockTabledata)
+ .insertAll(anyString(), anyString(), anyString(), any(TableDataInsertAllRequest.class));
+ }
+
+ private void verifyInsertAll(int expectedRetries) throws IOException {
+ verify(mockClient, times(expectedRetries)).tabledata();
+ verify(mockTabledata, times(expectedRetries))
+ .insertAll(anyString(), anyString(), anyString(), any(TableDataInsertAllRequest.class));
+ }
+
+ private void onTableGet(Table table) throws IOException {
+ when(mockClient.tables())
+ .thenReturn(mockTables);
+ when(mockTables.get(anyString(), anyString(), anyString()))
+ .thenReturn(mockTablesGet);
+ when(mockTablesGet.execute())
+ .thenReturn(table);
+ }
+
+ private void verifyTableGet() throws IOException {
+ verify(mockClient).tables();
+ verify(mockTables).get("project", "dataset", "table");
+ verify(mockTablesGet, atLeastOnce()).execute();
+ }
+
+ private void onTableList(TableDataList result) throws IOException {
+ when(mockClient.tabledata())
+ .thenReturn(mockTabledata);
+ when(mockTabledata.list(anyString(), anyString(), anyString()))
+ .thenReturn(mockTabledataList);
+ when(mockTabledataList.execute())
+ .thenReturn(result);
+ }
+
+ private void verifyTabledataList() throws IOException {
+ verify(mockClient, atLeastOnce()).tabledata();
+ verify(mockTabledata, atLeastOnce()).list("project", "dataset", "table");
+ verify(mockTabledataList, atLeastOnce()).execute();
+ // Max results may be set when testing for an empty table.
+ verify(mockTabledataList, atLeast(0)).setMaxResults(anyLong());
+ }
+
+ private Table basicTableSchema() {
+ return new Table()
+ .setSchema(new TableSchema()
+ .setFields(Arrays.asList(
+ new TableFieldSchema()
+ .setName("name")
+ .setType("STRING"),
+ new TableFieldSchema()
+ .setName("answer")
+ .setType("INTEGER")
+ )));
+ }
+
+ private Table basicTableSchemaWithTime() {
+ return new Table()
+ .setSchema(new TableSchema()
+ .setFields(Arrays.asList(
+ new TableFieldSchema()
+ .setName("time")
+ .setType("TIMESTAMP")
+ )));
+ }
+
+ @Test
+ public void testReadWithTime() throws IOException, InterruptedException {
+ // The BigQuery JSON API returns timestamps in the following format: floating-point seconds
+ // since epoch (UTC) with microsecond precision. Test that we faithfully preserve a set of
+ // known values.
+ TableDataList input = rawDataList(
+ rawRow("1.430397296789E9"),
+ rawRow("1.45206228E9"),
+ rawRow("1.452062291E9"),
+ rawRow("1.4520622911E9"),
+ rawRow("1.45206229112E9"),
+ rawRow("1.452062291123E9"),
+ rawRow("1.4520622911234E9"),
+ rawRow("1.45206229112345E9"),
+ rawRow("1.452062291123456E9"));
+ onTableGet(basicTableSchemaWithTime());
+ onTableList(input);
+
+ // Known results verified from BigQuery's export to JSON on GCS API.
+ List<String> expected = ImmutableList.of(
+ "2015-04-30 12:34:56.789 UTC",
+ "2016-01-06 06:38:00 UTC",
+ "2016-01-06 06:38:11 UTC",
+ "2016-01-06 06:38:11.1 UTC",
+ "2016-01-06 06:38:11.12 UTC",
+ "2016-01-06 06:38:11.123 UTC",
+ "2016-01-06 06:38:11.1234 UTC",
+ "2016-01-06 06:38:11.12345 UTC",
+ "2016-01-06 06:38:11.123456 UTC");
+
+ // Download the rows, verify the interactions.
+ List<TableRow> rows = new ArrayList<>();
+ try (BigQueryTableRowIterator iterator =
+ BigQueryTableRowIterator.fromTable(
+ BigQueryIO.parseTableSpec("project:dataset.table"), mockClient)) {
+ iterator.open();
+ while (iterator.advance()) {
+ rows.add(iterator.getCurrent());
+ }
+ }
+ verifyTableGet();
+ verifyTabledataList();
+
+ // Verify the timestamp converted as desired.
+ assertEquals("Expected input and output rows to have the same size",
+ expected.size(), rows.size());
+ for (int i = 0; i < expected.size(); ++i) {
+ assertEquals("i=" + i, expected.get(i), rows.get(i).get("time"));
+ }
+
+ }
+
+ private TableRow rawRow(Object...args) {
+ List<TableCell> cells = new LinkedList<>();
+ for (Object a : args) {
+ cells.add(new TableCell().setV(a));
+ }
+ return new TableRow().setF(cells);
+ }
+
+ private TableDataList rawDataList(TableRow...rows) {
+ return new TableDataList()
+ .setRows(Arrays.asList(rows));
+ }
+
+ @Test
+ public void testRead() throws IOException, InterruptedException {
+ onTableGet(basicTableSchema());
+
+ TableDataList dataList = rawDataList(rawRow("Arthur", 42));
+ onTableList(dataList);
+
+ try (BigQueryTableRowIterator iterator = BigQueryTableRowIterator.fromTable(
+ BigQueryIO.parseTableSpec("project:dataset.table"),
+ mockClient)) {
+ iterator.open();
+ Assert.assertTrue(iterator.advance());
+ TableRow row = iterator.getCurrent();
+
+ Assert.assertTrue(row.containsKey("name"));
+ Assert.assertTrue(row.containsKey("answer"));
+ Assert.assertEquals("Arthur", row.get("name"));
+ Assert.assertEquals(42, row.get("answer"));
+
+ Assert.assertFalse(iterator.advance());
+
+ verifyTableGet();
+ verifyTabledataList();
+ }
+ }
+
+ @Test
+ public void testReadEmpty() throws IOException, InterruptedException {
+ onTableGet(basicTableSchema());
+
+ // BigQuery may respond with a page token for an empty table, ensure we
+ // handle it.
+ TableDataList dataList = new TableDataList()
+ .setPageToken("FEED==")
+ .setTotalRows(0L);
+ onTableList(dataList);
+
+ try (BigQueryTableRowIterator iterator = BigQueryTableRowIterator.fromTable(
+ BigQueryIO.parseTableSpec("project:dataset.table"),
+ mockClient)) {
+ iterator.open();
+
+ Assert.assertFalse(iterator.advance());
+
+ verifyTableGet();
+ verifyTabledataList();
+ }
+ }
+
+ @Test
+ public void testReadMultiPage() throws IOException, InterruptedException {
+ onTableGet(basicTableSchema());
+
+ TableDataList page1 = rawDataList(rawRow("Row1", 1))
+ .setPageToken("page2");
+ TableDataList page2 = rawDataList(rawRow("Row2", 2))
+ .setTotalRows(2L);
+
+ when(mockClient.tabledata())
+ .thenReturn(mockTabledata);
+ when(mockTabledata.list(anyString(), anyString(), anyString()))
+ .thenReturn(mockTabledataList);
+ when(mockTabledataList.execute())
+ .thenReturn(page1)
+ .thenReturn(page2);
+
+ try (BigQueryTableRowIterator iterator = BigQueryTableRowIterator.fromTable(
+ BigQueryIO.parseTableSpec("project:dataset.table"),
+ mockClient)) {
+ iterator.open();
+
+ List<String> names = new LinkedList<>();
+ while (iterator.advance()) {
+ names.add((String) iterator.getCurrent().get("name"));
+ }
+
+ Assert.assertThat(names, Matchers.hasItems("Row1", "Row2"));
+
+ verifyTableGet();
+ verifyTabledataList();
+ // The second call should have used a page token.
+ verify(mockTabledataList).setPageToken("page2");
+ }
+ }
+
+ @Test
+ public void testReadOpenFailure() throws IOException, InterruptedException {
+ thrown.expect(IOException.class);
+
+ when(mockClient.tables())
+ .thenReturn(mockTables);
+ when(mockTables.get(anyString(), anyString(), anyString()))
+ .thenReturn(mockTablesGet);
+ when(mockTablesGet.execute())
+ .thenThrow(new IOException("No such table"));
+
+ try (BigQueryTableRowIterator iterator = BigQueryTableRowIterator.fromTable(
+ BigQueryIO.parseTableSpec("project:dataset.table"),
+ mockClient)) {
+ try {
+ iterator.open(); // throws.
+ } finally {
+ verifyTableGet();
+ }
+ }
+ }
+
+ @Test
+ public void testWriteAppend() throws IOException {
+ onTableGet(basicTableSchema());
+
+ TableReference ref = BigQueryIO
+ .parseTableSpec("project:dataset.table");
+
+ BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options);
+
+ inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_APPEND,
+ BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null);
+
+ verifyTableGet();
+ }
+
+ @Test
+ public void testWriteEmpty() throws IOException {
+ onTableGet(basicTableSchema());
+
+ TableDataList dataList = new TableDataList().setTotalRows(0L);
+ onTableList(dataList);
+
+ TableReference ref = BigQueryIO
+ .parseTableSpec("project:dataset.table");
+
+ BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options);
+
+ inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_EMPTY,
+ BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null);
+
+ verifyTableGet();
+ verifyTabledataList();
+ }
+
+ @Test
+ public void testWriteEmptyFail() throws IOException {
+ thrown.expect(IOException.class);
+
+ onTableGet(basicTableSchema());
+
+ TableDataList dataList = rawDataList(rawRow("Arthur", 42));
+ onTableList(dataList);
+
+ TableReference ref = BigQueryIO
+ .parseTableSpec("project:dataset.table");
+
+ BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options);
+
+ try {
+ inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_EMPTY,
+ BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null);
+ } finally {
+ verifyTableGet();
+ verifyTabledataList();
+ }
+ }
+
+ @Test
+ public void testInsertAll() throws Exception, IOException {
+ // Build up a list of indices to fail on each invocation. This should result in
+ // 5 calls to insertAll.
+ List<List<Long>> errorsIndices = new ArrayList<>();
+ errorsIndices.add(Arrays.asList(0L, 5L, 10L, 15L, 20L));
+ errorsIndices.add(Arrays.asList(0L, 2L, 4L));
+ errorsIndices.add(Arrays.asList(0L, 2L));
+ errorsIndices.add(new ArrayList<Long>());
+ onInsertAll(errorsIndices);
+
+ TableReference ref = BigQueryIO
+ .parseTableSpec("project:dataset.table");
+ BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options, 5);
+
+ List<TableRow> rows = new ArrayList<>();
+ List<String> ids = new ArrayList<>();
+ for (int i = 0; i < 25; ++i) {
+ rows.add(rawRow("foo", 1234));
+ ids.add(new String());
+ }
+
+ InMemoryLongSumAggregator byteCountAggregator = new InMemoryLongSumAggregator("ByteCount");
+ try {
+ inserter.insertAll(ref, rows, ids, byteCountAggregator);
+ } finally {
+ verifyInsertAll(5);
+ // Each of the 25 rows is 23 bytes: "{f=[{v=foo}, {v=1234}]}"
+ assertEquals("Incorrect byte count", 25L * 23L, byteCountAggregator.getSum());
+ }
+ }
+
+ private static class InMemoryLongSumAggregator implements Aggregator<Long, Long> {
+ private final String name;
+ private long sum = 0;
+
+ public InMemoryLongSumAggregator(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public void addValue(Long value) {
+ sum += value;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public CombineFn<Long, ?, Long> getCombineFn() {
+ return new Sum.SumLongFn();
+ }
+
+ public long getSum() {
+ return sum;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
index 4561cb1..8677755 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
@@ -101,6 +101,17 @@
</plugin>
</plugins>
</pluginManagement>
+
+ <extensions>
+ <!-- Use os-maven-plugin to initialize the "os.detected" properties for Bigtable
+ in beam-sdks-java-io-google-cloud-platform.
+ See: https://cloud.google.com/bigtable/docs/using-maven#encryption -->
+ <extension>
+ <groupId>kr.motd.maven</groupId>
+ <artifactId>os-maven-plugin</artifactId>
+ <version>1.4.0.Final</version>
+ </extension>
+ </extensions>
</build>
<dependencies>
@@ -124,6 +135,13 @@
<version>[0-incubating, 2-incubating)</version>
</dependency>
+ <!-- Adds a dependency on a specific version of the Beam Google Cloud Platform IO module. -->
+ <dependency>
+ <groupId>org.apache.beam</groupId>
+ <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
+ <version>[0-incubating, 2-incubating)</version>
+ </dependency>
+
<dependency>
<groupId>com.google.api-client</groupId>
<artifactId>google-api-client</artifactId>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
index 8e56b03..c2defa7 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
@@ -21,9 +21,9 @@ import ${package}.common.DataflowExampleUtils;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.io.BigQueryIO;
import org.apache.beam.sdk.io.PubsubIO;
import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
[10/10] incubator-beam git commit: Closes #681
Posted by dh...@apache.org.
Closes #681
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/7ec8781a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/7ec8781a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/7ec8781a
Branch: refs/heads/master
Commit: 7ec8781a2e18548a23c882329f0b50f7254202ec
Parents: 436e4a3 b240525
Author: Dan Halperin <dh...@google.com>
Authored: Wed Jul 20 13:02:43 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Jul 20 13:02:43 2016 -0700
----------------------------------------------------------------------
.../apache/beam/examples/WindowedWordCount.java | 2 +-
.../beam/examples/complete/AutoComplete.java | 2 +-
.../examples/complete/StreamingWordExtract.java | 2 +-
.../examples/complete/TrafficMaxLaneFlow.java | 2 +-
.../beam/examples/complete/TrafficRoutes.java | 2 +-
.../examples/cookbook/BigQueryTornadoes.java | 2 +-
.../cookbook/CombinePerKeyExamples.java | 2 +-
.../beam/examples/cookbook/FilterExamples.java | 2 +-
.../beam/examples/cookbook/JoinExamples.java | 2 +-
.../examples/cookbook/MaxPerKeyExamples.java | 2 +-
.../beam/examples/cookbook/TriggerExample.java | 2 +-
examples/java8/pom.xml | 5 +
.../complete/game/utils/WriteToBigQuery.java | 6 +-
.../game/utils/WriteWindowedToBigQuery.java | 6 +-
.../beam/runners/dataflow/DataflowRunner.java | 9 -
.../java/org/apache/beam/sdk/io/BigQueryIO.java | 2447 ------------------
.../org/apache/beam/sdk/util/AvroUtils.java | 207 --
.../apache/beam/sdk/util/BigQueryServices.java | 177 --
.../beam/sdk/util/BigQueryServicesImpl.java | 515 ----
.../beam/sdk/util/BigQueryTableInserter.java | 469 ----
.../beam/sdk/util/BigQueryTableRowIterator.java | 472 ----
.../org/apache/beam/sdk/values/PCollection.java | 34 +-
.../org/apache/beam/sdk/io/BigQueryIOTest.java | 1231 ---------
.../org/apache/beam/sdk/util/AvroUtilsTest.java | 78 -
.../beam/sdk/util/BigQueryServicesImplTest.java | 303 ---
.../sdk/util/BigQueryTableInserterTest.java | 311 ---
.../sdk/util/BigQueryTableRowIteratorTest.java | 256 --
.../apache/beam/sdk/util/BigQueryUtilTest.java | 485 ----
.../util/RetryHttpRequestInitializerTest.java | 20 +-
sdks/java/io/google-cloud-platform/pom.xml | 35 +
.../sdk/io/gcp/bigquery/BigQueryAvroUtils.java | 236 ++
.../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2446 +++++++++++++++++
.../sdk/io/gcp/bigquery/BigQueryServices.java | 177 ++
.../io/gcp/bigquery/BigQueryServicesImpl.java | 511 ++++
.../io/gcp/bigquery/BigQueryTableInserter.java | 413 +++
.../gcp/bigquery/BigQueryTableRowIterator.java | 474 ++++
.../beam/sdk/io/gcp/bigquery/package-info.java | 24 +
.../io/gcp/bigquery/BigQueryAvroUtilsTest.java | 143 +
.../sdk/io/gcp/bigquery/BigQueryIOTest.java | 1231 +++++++++
.../gcp/bigquery/BigQueryServicesImplTest.java | 289 +++
.../gcp/bigquery/BigQueryTableInserterTest.java | 313 +++
.../bigquery/BigQueryTableRowIteratorTest.java | 256 ++
.../sdk/io/gcp/bigquery/BigQueryUtilTest.java | 484 ++++
.../main/resources/archetype-resources/pom.xml | 18 +
.../src/main/java/WindowedWordCount.java | 2 +-
45 files changed, 7092 insertions(+), 7013 deletions(-)
----------------------------------------------------------------------
[04/10] incubator-beam git commit: BigQueryIO: move to
google-cloud-platform module
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
new file mode 100644
index 0000000..130d444
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -0,0 +1,2446 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.TableRowJsonCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.AvroSource;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.FileBasedSink;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService;
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory;
+import org.apache.beam.sdk.util.IOChannelFactory;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.MimeTypes;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.Reshuffle;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.Transport;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.model.Job;
+import com.google.api.services.bigquery.model.JobConfigurationExtract;
+import com.google.api.services.bigquery.model.JobConfigurationLoad;
+import com.google.api.services.bigquery.model.JobConfigurationQuery;
+import com.google.api.services.bigquery.model.JobReference;
+import com.google.api.services.bigquery.model.JobStatistics;
+import com.google.api.services.bigquery.model.JobStatus;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.hadoop.util.ApiErrorExtractor;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.apache.avro.generic.GenericRecord;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import javax.annotation.Nullable;
+
+/**
+ * {@link PTransform}s for reading and writing
+ * <a href="https://developers.google.com/bigquery/">BigQuery</a> tables.
+ *
+ * <h3>Table References</h3>
+ * <p>A fully-qualified BigQuery table name consists of three components:
+ * <ul>
+ * <li>{@code projectId}: the Cloud project id (defaults to
+ * {@link GcpOptions#getProject()}).
+ * <li>{@code datasetId}: the BigQuery dataset id, unique within a project.
+ * <li>{@code tableId}: a table id, unique within a dataset.
+ * </ul>
+ *
+ * <p>BigQuery table references are stored as a {@link TableReference}, which comes
+ * from the <a href="https://cloud.google.com/bigquery/client-libraries">
+ * BigQuery Java Client API</a>.
+ * Tables can be referred to as Strings, with or without the {@code projectId}.
+ * A helper function is provided ({@link BigQueryIO#parseTableSpec(String)})
+ * that parses the following string forms into a {@link TableReference}:
+ *
+ * <ul>
+ * <li>[{@code project_id}]:[{@code dataset_id}].[{@code table_id}]
+ * <li>[{@code dataset_id}].[{@code table_id}]
+ * </ul>
+ *
+ * <h3>Reading</h3>
+ * <p>To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation.
+ * This produces a {@link PCollection} of {@link TableRow TableRows} as output:
+ * <pre>{@code
+ * PCollection<TableRow> shakespeare = pipeline.apply(
+ * BigQueryIO.Read.from("clouddataflow-readonly:samples.weather_stations"));
+ * }</pre>
+ *
+ * <p>See {@link TableRow} for more information on the {@link TableRow} object.
+ *
+ * <p>Users may provide a query to read from rather than reading all of a BigQuery table. If
+ * specified, the result obtained by executing the specified query will be used as the data of the
+ * input transform.
+ *
+ * <pre>{@code
+ * PCollection<TableRow> shakespeare = pipeline.apply(
+ * BigQueryIO.Read.fromQuery("SELECT year, mean_temp FROM samples.weather_stations"));
+ * }</pre>
+ *
+ * <p>When creating a BigQuery input transform, users should provide either a query or a table.
+ * Pipeline construction will fail with a validation error if neither or both are specified.
+ *
+ * <h3>Writing</h3>
+ * <p>To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation.
+ * This consumes a {@link PCollection} of {@link TableRow TableRows} as input.
+ * <pre>{@code
+ * PCollection<TableRow> quotes = ...
+ *
+ * List<TableFieldSchema> fields = new ArrayList<>();
+ * fields.add(new TableFieldSchema().setName("source").setType("STRING"));
+ * fields.add(new TableFieldSchema().setName("quote").setType("STRING"));
+ * TableSchema schema = new TableSchema().setFields(fields);
+ *
+ * quotes.apply(BigQueryIO.Write
+ * .to("my-project:output.output_table")
+ * .withSchema(schema)
+ * .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
+ * }</pre>
+ *
+ * <p>See {@link BigQueryIO.Write} for details on how to specify if a write should
+ * append to an existing table, replace the table, or verify that the table is
+ * empty. Note that the dataset being written to must already exist. Unbounded PCollections can only
+ * be written using {@link WriteDisposition#WRITE_EMPTY} or {@link WriteDisposition#WRITE_APPEND}.
+ *
+ * <h3>Sharding BigQuery output tables</h3>
+ * <p>A common use case is to dynamically generate BigQuery table names based on
+ * the current window. To support this,
+ * {@link BigQueryIO.Write#to(SerializableFunction)}
+ * accepts a function mapping the current window to a tablespec. For example,
+ * here's code that outputs daily tables to BigQuery:
+ * <pre>{@code
+ * PCollection<TableRow> quotes = ...
+ * quotes.apply(Window.<TableRow>into(CalendarWindows.days(1)))
+ * .apply(BigQueryIO.Write
+ * .withSchema(schema)
+ * .to(new SerializableFunction<BoundedWindow, String>() {
+ * public String apply(BoundedWindow window) {
+ * // The cast below is safe because CalendarWindows.days(1) produces IntervalWindows.
+ * String dayString = DateTimeFormat.forPattern("yyyy_MM_dd")
+ * .withZone(DateTimeZone.UTC)
+ * .print(((IntervalWindow) window).start());
+ * return "my-project:output.output_table_" + dayString;
+ * }
+ * }));
+ * }</pre>
+ *
+ * <p>Per-window tables are not yet supported in batch mode.
+ *
+ * <h3>Permissions</h3>
+ * <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
+ * Dataflow job. Please refer to the documentation of corresponding {@link PipelineRunner}s for
+ * more details.
+ *
+ * <p>Please see <a href="https://cloud.google.com/bigquery/access-control">BigQuery Access Control
+ * </a> for security and permission related information specific to BigQuery.
+ */
+public class BigQueryIO {
+ private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class);
+
+ /**
+ * Singleton instance of the JSON factory used to read and write JSON
+ * formatted rows.
+ */
+ private static final JsonFactory JSON_FACTORY = Transport.getJsonFactory();
+
+ /**
+ * Project IDs must contain 6-63 lowercase letters, digits, or dashes.
+ * IDs must start with a letter and may not end with a dash.
+ * This regex isn't exact - this allows for patterns that would be rejected by
+ * the service, but this is sufficient for basic parsing of table references.
+ */
+ private static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]{4,61}[a-z0-9]";
+
+ /**
+ * Regular expression that matches Dataset IDs.
+ */
+ private static final String DATASET_REGEXP = "[-\\w.]{1,1024}";
+
+ /**
+ * Regular expression that matches Table IDs.
+ */
+ private static final String TABLE_REGEXP = "[-\\w$@]{1,1024}";
+
+ /**
+ * Matches table specifications in the form {@code "[project_id]:[dataset_id].[table_id]"} or
+ * {@code "[dataset_id].[table_id]"}.
+ */
+ private static final String DATASET_TABLE_REGEXP =
+ String.format("((?<PROJECT>%s):)?(?<DATASET>%s)\\.(?<TABLE>%s)", PROJECT_ID_REGEXP,
+ DATASET_REGEXP, TABLE_REGEXP);
+
+ private static final Pattern TABLE_SPEC = Pattern.compile(DATASET_TABLE_REGEXP);
+
+ // TODO: make this private and remove improper access from BigQueryIOTranslator.
+ public static final String SET_PROJECT_FROM_OPTIONS_WARNING =
+ "No project specified for BigQuery table \"%1$s.%2$s\". Assuming it is in \"%3$s\". If the"
+ + " table is in a different project please specify it as a part of the BigQuery table"
+ + " definition.";
+
+ private static final String RESOURCE_NOT_FOUND_ERROR =
+ "BigQuery %1$s not found for table \"%2$s\" . Please create the %1$s before pipeline"
+ + " execution. If the %1$s is created by an earlier stage of the pipeline, this"
+ + " validation can be disabled using #withoutValidation.";
+
+ private static final String UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR =
+ "Unable to confirm BigQuery %1$s presence for table \"%2$s\". If the %1$s is created by"
+ + " an earlier stage of the pipeline, this validation can be disabled using"
+ + " #withoutValidation.";
+
+ /**
+ * Parse a table specification in the form
+ * {@code "[project_id]:[dataset_id].[table_id]"} or {@code "[dataset_id].[table_id]"}.
+ *
+ * <p>If the project id is omitted, the default project id is used.
+ */
+ public static TableReference parseTableSpec(String tableSpec) {
+ Matcher match = TABLE_SPEC.matcher(tableSpec);
+ if (!match.matches()) {
+ throw new IllegalArgumentException(
+ "Table reference is not in [project_id]:[dataset_id].[table_id] "
+ + "format: " + tableSpec);
+ }
+
+ TableReference ref = new TableReference();
+ ref.setProjectId(match.group("PROJECT"));
+
+ return ref.setDatasetId(match.group("DATASET")).setTableId(match.group("TABLE"));
+ }
+
+ /**
+ * Returns a canonical string representation of the {@link TableReference}.
+ */
+ public static String toTableSpec(TableReference ref) {
+ StringBuilder sb = new StringBuilder();
+ if (ref.getProjectId() != null) {
+ sb.append(ref.getProjectId());
+ sb.append(":");
+ }
+
+ sb.append(ref.getDatasetId()).append('.').append(ref.getTableId());
+ return sb.toString();
+ }
+
+ /**
+ * A {@link PTransform} that reads from a BigQuery table and returns a
+ * {@link PCollection} of {@link TableRow TableRows} containing each of the rows of the table.
+ *
+ * <p>Each {@link TableRow} contains values indexed by column name. Here is a
+ * sample processing function that processes a "line" column from rows:
+ * <pre>{@code
+ * static class ExtractWordsFn extends DoFn<TableRow, String> {
+ * public void processElement(ProcessContext c) {
+ * // Get the "line" field of the TableRow object, split it into words, and emit them.
+ * TableRow row = c.element();
+ * String[] words = row.get("line").toString().split("[^a-zA-Z']+");
+ * for (String word : words) {
+ * if (!word.isEmpty()) {
+ * c.output(word);
+ * }
+ * }
+ * }
+ * }}</pre>
+ */
+ public static class Read {
+
+ /**
+ * Reads a BigQuery table specified as {@code "[project_id]:[dataset_id].[table_id]"} or
+ * {@code "[dataset_id].[table_id]"} for tables within the current project.
+ */
+ public static Bound from(String tableSpec) {
+ return new Bound().from(tableSpec);
+ }
+
+ /**
+ * Reads results received after executing the given query.
+ */
+ public static Bound fromQuery(String query) {
+ return new Bound().fromQuery(query);
+ }
+
+ /**
+ * Reads a BigQuery table specified as a {@link TableReference} object.
+ */
+ public static Bound from(TableReference table) {
+ return new Bound().from(table);
+ }
+
+ /**
+ * Disables BigQuery table validation, which is enabled by default.
+ */
+ public static Bound withoutValidation() {
+ return new Bound().withoutValidation();
+ }
+
+ /**
+ * A {@link PTransform} that reads from a BigQuery table and returns a bounded
+ * {@link PCollection} of {@link TableRow TableRows}.
+ */
+ public static class Bound extends PTransform<PInput, PCollection<TableRow>> {
+ @Nullable final String jsonTableRef;
+ @Nullable final String query;
+
+ /**
+ * Disable validation that the table exists or the query succeeds prior to pipeline
+ * submission. Basic validation (such as ensuring that a query or table is specified) still
+ * occurs.
+ */
+ final boolean validate;
+ @Nullable final Boolean flattenResults;
+ @Nullable BigQueryServices bigQueryServices;
+
+ private static final String QUERY_VALIDATION_FAILURE_ERROR =
+ "Validation of query \"%1$s\" failed. If the query depends on an earlier stage of the"
+ + " pipeline, This validation can be disabled using #withoutValidation.";
+
+ // The maximum number of retries to poll a BigQuery job in the cleanup phase.
+ // We expect the jobs have already DONE, and don't need a high max retires.
+ private static final int CLEANUP_JOB_POLL_MAX_RETRIES = 10;
+
+ private Bound() {
+ this(
+ null /* name */,
+ null /* query */,
+ null /* jsonTableRef */,
+ true /* validate */,
+ null /* flattenResults */,
+ null /* bigQueryServices */);
+ }
+
+ private Bound(
+ String name, @Nullable String query, @Nullable String jsonTableRef, boolean validate,
+ @Nullable Boolean flattenResults, @Nullable BigQueryServices bigQueryServices) {
+ super(name);
+ this.jsonTableRef = jsonTableRef;
+ this.query = query;
+ this.validate = validate;
+ this.flattenResults = flattenResults;
+ this.bigQueryServices = bigQueryServices;
+ }
+
+ /**
+ * Returns a copy of this transform that reads from the specified table. Refer to
+ * {@link #parseTableSpec(String)} for the specification format.
+ *
+ * <p>Does not modify this object.
+ */
+ public Bound from(String tableSpec) {
+ return from(parseTableSpec(tableSpec));
+ }
+
+ /**
+ * Returns a copy of this transform that reads from the specified table.
+ *
+ * <p>Does not modify this object.
+ */
+ public Bound from(TableReference table) {
+ return new Bound(
+ name, query, toJsonString(table), validate, flattenResults, bigQueryServices);
+ }
+
+ /**
+ * Returns a copy of this transform that reads the results of the specified query.
+ *
+ * <p>Does not modify this object.
+ *
+ * <p>By default, the query results will be flattened -- see
+ * "flattenResults" in the <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs">
+ * Jobs documentation</a> for more information. To disable flattening, use
+ * {@link BigQueryIO.Read.Bound#withoutResultFlattening}.
+ */
+ public Bound fromQuery(String query) {
+ return new Bound(name, query, jsonTableRef, validate,
+ MoreObjects.firstNonNull(flattenResults, Boolean.TRUE), bigQueryServices);
+ }
+
+ /**
+ * Disable validation that the table exists or the query succeeds prior to pipeline
+ * submission. Basic validation (such as ensuring that a query or table is specified) still
+ * occurs.
+ */
+ public Bound withoutValidation() {
+ return new Bound(name, query, jsonTableRef, false, flattenResults, bigQueryServices);
+ }
+
+ /**
+ * Disable <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs">
+ * flattening of query results</a>.
+ *
+ * <p>Only valid when a query is used ({@link #fromQuery}). Setting this option when reading
+ * from a table will cause an error during validation.
+ */
+ public Bound withoutResultFlattening() {
+ return new Bound(name, query, jsonTableRef, validate, false, bigQueryServices);
+ }
+
+ @VisibleForTesting
+ Bound withTestServices(BigQueryServices testServices) {
+ return new Bound(name, query, jsonTableRef, validate, flattenResults, testServices);
+ }
+
+ @Override
+ public void validate(PInput input) {
+ // Even if existence validation is disabled, we need to make sure that the BigQueryIO
+ // read is properly specified.
+ BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class);
+
+ TableReference table = getTableWithDefaultProject(bqOptions);
+ if (table == null && query == null) {
+ throw new IllegalStateException(
+ "Invalid BigQuery read operation, either table reference or query has to be set");
+ } else if (table != null && query != null) {
+ throw new IllegalStateException("Invalid BigQuery read operation. Specifies both a"
+ + " query and a table, only one of these should be provided");
+ } else if (table != null && flattenResults != null) {
+ throw new IllegalStateException("Invalid BigQuery read operation. Specifies a"
+ + " table with a result flattening preference, which is not configurable");
+ } else if (query != null && flattenResults == null) {
+ throw new IllegalStateException("Invalid BigQuery read operation. Specifies a"
+ + " query without a result flattening preference");
+ }
+
+ if (validate) {
+ BigQueryServices bqServices = getBigQueryServices();
+ // Check for source table/query presence for early failure notification.
+ // Note that a presence check can fail if the table or dataset are created by earlier
+ // stages of the pipeline or if a query depends on earlier stages of a pipeline. For these
+ // cases the withoutValidation method can be used to disable the check.
+ if (table != null) {
+ DatasetService datasetService = bqServices.getDatasetService(bqOptions);
+ verifyDatasetPresence(datasetService, table);
+ verifyTablePresence(datasetService, table);
+ }
+ if (query != null) {
+ JobService jobService = bqServices.getJobService(bqOptions);
+ try {
+ jobService.dryRunQuery(bqOptions.getProject(), query);
+ } catch (Exception e) {
+ throw new IllegalArgumentException(
+ String.format(QUERY_VALIDATION_FAILURE_ERROR, query), e);
+ }
+ }
+ }
+ }
+
+ @Override
+ public PCollection<TableRow> apply(PInput input) {
+ String uuid = randomUUIDString();
+ final String jobIdToken = "beam_job_" + uuid;
+
+ BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class);
+
+ BoundedSource<TableRow> source;
+ final BigQueryServices bqServices = getBigQueryServices();
+
+ final String extractDestinationDir;
+ String tempLocation = bqOptions.getTempLocation();
+ try {
+ IOChannelFactory factory = IOChannelUtils.getFactory(tempLocation);
+ extractDestinationDir = factory.resolve(tempLocation, uuid);
+ } catch (IOException e) {
+ throw new RuntimeException(
+ String.format("Failed to resolve extract destination directory in %s", tempLocation));
+ }
+
+ final String executingProject = bqOptions.getProject();
+ if (!Strings.isNullOrEmpty(query)) {
+ String queryTempDatasetId = "temp_dataset_" + uuid;
+ String queryTempTableId = "temp_table_" + uuid;
+
+ TableReference queryTempTableRef = new TableReference()
+ .setProjectId(executingProject)
+ .setDatasetId(queryTempDatasetId)
+ .setTableId(queryTempTableId);
+
+ source = BigQueryQuerySource.create(
+ jobIdToken, query, queryTempTableRef, flattenResults,
+ extractDestinationDir, bqServices);
+ } else {
+ TableReference inputTable = getTableWithDefaultProject(bqOptions);
+ source = BigQueryTableSource.create(
+ jobIdToken, inputTable, extractDestinationDir, bqServices, executingProject);
+ }
+ PassThroughThenCleanup.CleanupOperation cleanupOperation =
+ new PassThroughThenCleanup.CleanupOperation() {
+ @Override
+ void cleanup(PipelineOptions options) throws Exception {
+ BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
+
+ JobReference jobRef = new JobReference()
+ .setProjectId(executingProject)
+ .setJobId(getExtractJobId(jobIdToken));
+ Job extractJob = bqServices.getJobService(bqOptions).pollJob(
+ jobRef, CLEANUP_JOB_POLL_MAX_RETRIES);
+
+ Collection<String> extractFiles = null;
+ if (extractJob != null) {
+ extractFiles = getExtractFilePaths(extractDestinationDir, extractJob);
+ } else {
+ IOChannelFactory factory = IOChannelUtils.getFactory(extractDestinationDir);
+ Collection<String> dirMatch = factory.match(extractDestinationDir);
+ if (!dirMatch.isEmpty()) {
+ extractFiles = factory.match(factory.resolve(extractDestinationDir, "*"));
+ }
+ }
+ if (extractFiles != null && !extractFiles.isEmpty()) {
+ new GcsUtilFactory().create(options).remove(extractFiles);
+ }
+ }};
+ return input.getPipeline()
+ .apply(org.apache.beam.sdk.io.Read.from(source))
+ .setCoder(getDefaultOutputCoder())
+ .apply(new PassThroughThenCleanup<TableRow>(cleanupOperation));
+ }
+
+ @Override
+ protected Coder<TableRow> getDefaultOutputCoder() {
+ return TableRowJsonCoder.of();
+ }
+
+ @Override
+ public void populateDisplayData(DisplayData.Builder builder) {
+ super.populateDisplayData(builder);
+ TableReference table = getTable();
+
+ if (table != null) {
+ builder.add(DisplayData.item("table", toTableSpec(table))
+ .withLabel("Table"));
+ }
+
+ builder
+ .addIfNotNull(DisplayData.item("query", query)
+ .withLabel("Query"))
+ .addIfNotNull(DisplayData.item("flattenResults", flattenResults)
+ .withLabel("Flatten Query Results"))
+ .addIfNotDefault(DisplayData.item("validation", validate)
+ .withLabel("Validation Enabled"),
+ true);
+ }
+
+ /**
+ * Returns the table to read, or {@code null} if reading from a query instead.
+ *
+ * <p>If the table's project is not specified, use the executing project.
+ */
+ @Nullable private TableReference getTableWithDefaultProject(BigQueryOptions bqOptions) {
+ TableReference table = getTable();
+ if (table != null && Strings.isNullOrEmpty(table.getProjectId())) {
+ // If user does not specify a project we assume the table to be located in
+ // the default project.
+ table.setProjectId(bqOptions.getProject());
+ }
+ return table;
+ }
+
+ /**
+ * Returns the table to read, or {@code null} if reading from a query instead.
+ */
+ @Nullable
+ public TableReference getTable() {
+ return fromJsonString(jsonTableRef, TableReference.class);
+ }
+
+ /**
+ * Returns the query to be read, or {@code null} if reading from a table instead.
+ */
+ public String getQuery() {
+ return query;
+ }
+
+ /**
+ * Returns true if table validation is enabled.
+ */
+ public boolean getValidate() {
+ return validate;
+ }
+
+ /**
+ * Returns true/false if result flattening is enabled/disabled, or null if not applicable.
+ */
+ public Boolean getFlattenResults() {
+ return flattenResults;
+ }
+
+ private BigQueryServices getBigQueryServices() {
+ if (bigQueryServices == null) {
+ bigQueryServices = new BigQueryServicesImpl();
+ }
+ return bigQueryServices;
+ }
+ }
+
+ /** Disallow construction of utility class. */
+ private Read() {}
+ }
+
+ /**
+ * A {@link PTransform} that invokes {@link CleanupOperation} after the input {@link PCollection}
+ * has been processed.
+ */
+ @VisibleForTesting
+ static class PassThroughThenCleanup<T> extends PTransform<PCollection<T>, PCollection<T>> {
+
+ private CleanupOperation cleanupOperation;
+
+ PassThroughThenCleanup(CleanupOperation cleanupOperation) {
+ this.cleanupOperation = cleanupOperation;
+ }
+
+ @Override
+ public PCollection<T> apply(PCollection<T> input) {
+ TupleTag<T> mainOutput = new TupleTag<>();
+ TupleTag<Void> cleanupSignal = new TupleTag<>();
+ PCollectionTuple outputs = input.apply(ParDo.of(new IdentityFn<T>())
+ .withOutputTags(mainOutput, TupleTagList.of(cleanupSignal)));
+
+ PCollectionView<Void> cleanupSignalView = outputs.get(cleanupSignal)
+ .setCoder(VoidCoder.of())
+ .apply(View.<Void>asSingleton().withDefaultValue(null));
+
+ input.getPipeline()
+ .apply("Create(CleanupOperation)", Create.of(cleanupOperation))
+ .apply("Cleanup", ParDo.of(
+ new DoFn<CleanupOperation, Void>() {
+ @Override
+ public void processElement(ProcessContext c)
+ throws Exception {
+ c.element().cleanup(c.getPipelineOptions());
+ }
+ }).withSideInputs(cleanupSignalView));
+
+ return outputs.get(mainOutput);
+ }
+
+ private static class IdentityFn<T> extends DoFn<T, T> {
+ @Override
+ public void processElement(ProcessContext c) {
+ c.output(c.element());
+ }
+ }
+
+ abstract static class CleanupOperation implements Serializable {
+ abstract void cleanup(PipelineOptions options) throws Exception;
+ }
+ }
+
+ /**
+ * A {@link BigQuerySourceBase} for reading BigQuery tables.
+ */
+ @VisibleForTesting
+ static class BigQueryTableSource extends BigQuerySourceBase {
+
+ static BigQueryTableSource create(
+ String jobIdToken,
+ TableReference table,
+ String extractDestinationDir,
+ BigQueryServices bqServices,
+ String executingProject) {
+ return new BigQueryTableSource(
+ jobIdToken, table, extractDestinationDir, bqServices, executingProject);
+ }
+
+ private final String jsonTable;
+ private final AtomicReference<Long> tableSizeBytes;
+
+ private BigQueryTableSource(
+ String jobIdToken,
+ TableReference table,
+ String extractDestinationDir,
+ BigQueryServices bqServices,
+ String executingProject) {
+ super(jobIdToken, extractDestinationDir, bqServices, executingProject);
+ checkNotNull(table, "table");
+ this.jsonTable = toJsonString(table);
+ this.tableSizeBytes = new AtomicReference<>();
+ }
+
+ @Override
+ protected TableReference getTableToExtract(BigQueryOptions bqOptions) throws IOException {
+ return JSON_FACTORY.fromString(jsonTable, TableReference.class);
+ }
+
+ @Override
+ public BoundedReader<TableRow> createReader(PipelineOptions options) throws IOException {
+ BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
+ TableReference tableRef = JSON_FACTORY.fromString(jsonTable, TableReference.class);
+ return new BigQueryReader(this, bqServices.getReaderFromTable(bqOptions, tableRef));
+ }
+
+ @Override
+ public synchronized long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+ if (tableSizeBytes.get() == null) {
+ TableReference table = JSON_FACTORY.fromString(jsonTable, TableReference.class);
+
+ Long numBytes = bqServices.getDatasetService(options.as(BigQueryOptions.class))
+ .getTable(table.getProjectId(), table.getDatasetId(), table.getTableId())
+ .getNumBytes();
+ tableSizeBytes.compareAndSet(null, numBytes);
+ }
+ return tableSizeBytes.get();
+ }
+
+ @Override
+ protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception {
+ // Do nothing.
+ }
+
+ @Override
+ public void populateDisplayData(DisplayData.Builder builder) {
+ super.populateDisplayData(builder);
+ builder.add(DisplayData.item("table", jsonTable));
+ }
+ }
+
+ /**
+ * A {@link BigQuerySourceBase} for querying BigQuery tables.
+ */
+ @VisibleForTesting
+ static class BigQueryQuerySource extends BigQuerySourceBase {
+
+ static BigQueryQuerySource create(
+ String jobIdToken,
+ String query,
+ TableReference queryTempTableRef,
+ Boolean flattenResults,
+ String extractDestinationDir,
+ BigQueryServices bqServices) {
+ return new BigQueryQuerySource(
+ jobIdToken,
+ query,
+ queryTempTableRef,
+ flattenResults,
+ extractDestinationDir,
+ bqServices);
+ }
+
+ private final String query;
+ private final String jsonQueryTempTable;
+ private final Boolean flattenResults;
+ private transient AtomicReference<JobStatistics> dryRunJobStats;
+
+ private BigQueryQuerySource(
+ String jobIdToken,
+ String query,
+ TableReference queryTempTableRef,
+ Boolean flattenResults,
+ String extractDestinationDir,
+ BigQueryServices bqServices) {
+ super(jobIdToken, extractDestinationDir, bqServices,
+ checkNotNull(queryTempTableRef, "queryTempTableRef").getProjectId());
+ this.query = checkNotNull(query, "query");
+ this.jsonQueryTempTable = toJsonString(queryTempTableRef);
+ this.flattenResults = checkNotNull(flattenResults, "flattenResults");
+ this.dryRunJobStats = new AtomicReference<>();
+ }
+
+ @Override
+ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+ BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
+ return dryRunQueryIfNeeded(bqOptions).getTotalBytesProcessed();
+ }
+
+ @Override
+ public BoundedReader<TableRow> createReader(PipelineOptions options) throws IOException {
+ BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
+ return new BigQueryReader(this, bqServices.getReaderFromQuery(
+ bqOptions, query, executingProject, flattenResults));
+ }
+
+ @Override
+ protected TableReference getTableToExtract(BigQueryOptions bqOptions)
+ throws IOException, InterruptedException {
+ // 1. Find the location of the query.
+ TableReference dryRunTempTable = dryRunQueryIfNeeded(bqOptions)
+ .getQuery()
+ .getReferencedTables()
+ .get(0);
+ DatasetService tableService = bqServices.getDatasetService(bqOptions);
+ String location = tableService.getTable(
+ dryRunTempTable.getProjectId(),
+ dryRunTempTable.getDatasetId(),
+ dryRunTempTable.getTableId()).getLocation();
+
+ // 2. Create the temporary dataset in the query location.
+ TableReference tableToExtract =
+ JSON_FACTORY.fromString(jsonQueryTempTable, TableReference.class);
+ tableService.createDataset(
+ tableToExtract.getProjectId(), tableToExtract.getDatasetId(), location, "");
+
+ // 3. Execute the query.
+ String queryJobId = jobIdToken + "-query";
+ executeQuery(
+ executingProject,
+ queryJobId,
+ query,
+ tableToExtract,
+ flattenResults,
+ bqServices.getJobService(bqOptions));
+ return tableToExtract;
+ }
+
+ @Override
+ protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception {
+ TableReference tableToRemove =
+ JSON_FACTORY.fromString(jsonQueryTempTable, TableReference.class);
+
+ DatasetService tableService = bqServices.getDatasetService(bqOptions);
+ tableService.deleteTable(
+ tableToRemove.getProjectId(),
+ tableToRemove.getDatasetId(),
+ tableToRemove.getTableId());
+ tableService.deleteDataset(tableToRemove.getProjectId(), tableToRemove.getDatasetId());
+ }
+
+ @Override
+ public void populateDisplayData(DisplayData.Builder builder) {
+ super.populateDisplayData(builder);
+ builder.add(DisplayData.item("query", query));
+ }
+ private synchronized JobStatistics dryRunQueryIfNeeded(BigQueryOptions bqOptions)
+ throws InterruptedException, IOException {
+ if (dryRunJobStats.get() == null) {
+ JobStatistics jobStats =
+ bqServices.getJobService(bqOptions).dryRunQuery(executingProject, query);
+ dryRunJobStats.compareAndSet(null, jobStats);
+ }
+ return dryRunJobStats.get();
+ }
+
+ private static void executeQuery(
+ String executingProject,
+ String jobId,
+ String query,
+ TableReference destinationTable,
+ boolean flattenResults,
+ JobService jobService) throws IOException, InterruptedException {
+ JobReference jobRef = new JobReference()
+ .setProjectId(executingProject)
+ .setJobId(jobId);
+ JobConfigurationQuery queryConfig = new JobConfigurationQuery();
+ queryConfig
+ .setQuery(query)
+ .setAllowLargeResults(true)
+ .setCreateDisposition("CREATE_IF_NEEDED")
+ .setDestinationTable(destinationTable)
+ .setFlattenResults(flattenResults)
+ .setPriority("BATCH")
+ .setWriteDisposition("WRITE_EMPTY");
+ jobService.startQueryJob(jobRef, queryConfig);
+ Job job = jobService.pollJob(jobRef, JOB_POLL_MAX_RETRIES);
+ if (parseStatus(job) != Status.SUCCEEDED) {
+ throw new IOException("Query job failed: " + jobId);
+ }
+ return;
+ }
+
+ private void readObject(ObjectInputStream in) throws ClassNotFoundException, IOException {
+ in.defaultReadObject();
+ dryRunJobStats = new AtomicReference<>();
+ }
+ }
+
+ /**
+ * An abstract {@link BoundedSource} to read a table from BigQuery.
+ *
+ * <p>This source uses a BigQuery export job to take a snapshot of the table on GCS, and then
+ * reads in parallel from each produced file. It is implemented by {@link BigQueryTableSource},
+ * and {@link BigQueryQuerySource}, depending on the configuration of the read.
+ * Specifically,
+ * <ul>
+ * <li>{@link BigQueryTableSource} is for reading BigQuery tables</li>
+ * <li>{@link BigQueryQuerySource} is for querying BigQuery tables</li>
+ * </ul>
+ * ...
+ */
+ private abstract static class BigQuerySourceBase extends BoundedSource<TableRow> {
+ // The maximum number of attempts to verify temp files.
+ private static final int MAX_FILES_VERIFY_ATTEMPTS = 10;
+
+ // The maximum number of retries to poll a BigQuery job.
+ protected static final int JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE;
+
+ // The initial backoff for verifying temp files.
+ private static final long INITIAL_FILES_VERIFY_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1);
+
+ protected final String jobIdToken;
+ protected final String extractDestinationDir;
+ protected final BigQueryServices bqServices;
+ protected final String executingProject;
+
+ private BigQuerySourceBase(
+ String jobIdToken,
+ String extractDestinationDir,
+ BigQueryServices bqServices,
+ String executingProject) {
+ this.jobIdToken = checkNotNull(jobIdToken, "jobIdToken");
+ this.extractDestinationDir = checkNotNull(extractDestinationDir, "extractDestinationDir");
+ this.bqServices = checkNotNull(bqServices, "bqServices");
+ this.executingProject = checkNotNull(executingProject, "executingProject");
+ }
+
+ @Override
+ public List<BoundedSource<TableRow>> splitIntoBundles(
+ long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
+ BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
+ TableReference tableToExtract = getTableToExtract(bqOptions);
+ JobService jobService = bqServices.getJobService(bqOptions);
+ String extractJobId = getExtractJobId(jobIdToken);
+ List<String> tempFiles = executeExtract(extractJobId, tableToExtract, jobService);
+
+ TableSchema tableSchema = bqServices.getDatasetService(bqOptions).getTable(
+ tableToExtract.getProjectId(),
+ tableToExtract.getDatasetId(),
+ tableToExtract.getTableId()).getSchema();
+
+ cleanupTempResource(bqOptions);
+ return createSources(tempFiles, tableSchema);
+ }
+
+ protected abstract TableReference getTableToExtract(BigQueryOptions bqOptions) throws Exception;
+
+ protected abstract void cleanupTempResource(BigQueryOptions bqOptions) throws Exception;
+
+ @Override
+ public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+ return false;
+ }
+
+ @Override
+ public void validate() {
+ // Do nothing, validation is done in BigQuery.Read.
+ }
+
+ @Override
+ public Coder<TableRow> getDefaultOutputCoder() {
+ return TableRowJsonCoder.of();
+ }
+
+ private List<String> executeExtract(
+ String jobId, TableReference table, JobService jobService)
+ throws InterruptedException, IOException {
+ JobReference jobRef = new JobReference()
+ .setProjectId(executingProject)
+ .setJobId(jobId);
+
+ String destinationUri = getExtractDestinationUri(extractDestinationDir);
+ JobConfigurationExtract extract = new JobConfigurationExtract()
+ .setSourceTable(table)
+ .setDestinationFormat("AVRO")
+ .setDestinationUris(ImmutableList.of(destinationUri));
+
+ LOG.info("Starting BigQuery extract job: {}", jobId);
+ jobService.startExtractJob(jobRef, extract);
+ Job extractJob =
+ jobService.pollJob(jobRef, JOB_POLL_MAX_RETRIES);
+ if (parseStatus(extractJob) != Status.SUCCEEDED) {
+ throw new IOException(String.format(
+ "Extract job %s failed, status: %s",
+ extractJob.getJobReference().getJobId(), extractJob.getStatus()));
+ }
+
+ List<String> tempFiles = getExtractFilePaths(extractDestinationDir, extractJob);
+ return ImmutableList.copyOf(tempFiles);
+ }
+
+ private List<BoundedSource<TableRow>> createSources(
+ List<String> files, TableSchema tableSchema) throws IOException, InterruptedException {
+ final String jsonSchema = JSON_FACTORY.toString(tableSchema);
+
+ SerializableFunction<GenericRecord, TableRow> function =
+ new SerializableFunction<GenericRecord, TableRow>() {
+ @Override
+ public TableRow apply(GenericRecord input) {
+ return BigQueryAvroUtils.convertGenericRecordToTableRow(
+ input, fromJsonString(jsonSchema, TableSchema.class));
+ }};
+
+ List<BoundedSource<TableRow>> avroSources = Lists.newArrayList();
+ BackOff backoff = new AttemptBoundedExponentialBackOff(
+ MAX_FILES_VERIFY_ATTEMPTS, INITIAL_FILES_VERIFY_BACKOFF_MILLIS);
+ for (String fileName : files) {
+ while (BackOffUtils.next(Sleeper.DEFAULT, backoff)) {
+ if (IOChannelUtils.getFactory(fileName).getSizeBytes(fileName) != -1) {
+ break;
+ }
+ }
+ avroSources.add(new TransformingSource<>(
+ AvroSource.from(fileName), function, getDefaultOutputCoder()));
+ }
+ return ImmutableList.copyOf(avroSources);
+ }
+
+ protected static class BigQueryReader extends BoundedSource.BoundedReader<TableRow> {
+ private final BigQuerySourceBase source;
+ private final BigQueryServices.BigQueryJsonReader reader;
+
+ private BigQueryReader(
+ BigQuerySourceBase source, BigQueryServices.BigQueryJsonReader reader) {
+ this.source = source;
+ this.reader = reader;
+ }
+
+ @Override
+ public BoundedSource<TableRow> getCurrentSource() {
+ return source;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ return reader.start();
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ return reader.advance();
+ }
+
+ @Override
+ public TableRow getCurrent() throws NoSuchElementException {
+ return reader.getCurrent();
+ }
+
+ @Override
+ public void close() throws IOException {
+ reader.close();
+ }
+ }
+ }
+
+ /**
+ * A {@link BoundedSource} that reads from {@code BoundedSource<T>}
+ * and transforms elements to type {@code V}.
+ */
+ @VisibleForTesting
+ static class TransformingSource<T, V> extends BoundedSource<V> {
+ private final BoundedSource<T> boundedSource;
+ private final SerializableFunction<T, V> function;
+ private final Coder<V> outputCoder;
+
+ TransformingSource(
+ BoundedSource<T> boundedSource,
+ SerializableFunction<T, V> function,
+ Coder<V> outputCoder) {
+ this.boundedSource = boundedSource;
+ this.function = function;
+ this.outputCoder = outputCoder;
+ }
+
+ @Override
+ public List<? extends BoundedSource<V>> splitIntoBundles(
+ long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
+ return Lists.transform(
+ boundedSource.splitIntoBundles(desiredBundleSizeBytes, options),
+ new Function<BoundedSource<T>, BoundedSource<V>>() {
+ @Override
+ public BoundedSource<V> apply(BoundedSource<T> input) {
+ return new TransformingSource<>(input, function, outputCoder);
+ }
+ });
+ }
+
+ @Override
+ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+ return boundedSource.getEstimatedSizeBytes(options);
+ }
+
+ @Override
+ public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+ return boundedSource.producesSortedKeys(options);
+ }
+
+ @Override
+ public BoundedReader<V> createReader(PipelineOptions options) throws IOException {
+ return new TransformingReader(boundedSource.createReader(options));
+ }
+
+ @Override
+ public void validate() {
+ boundedSource.validate();
+ }
+
+ @Override
+ public Coder<V> getDefaultOutputCoder() {
+ return outputCoder;
+ }
+
+ private class TransformingReader extends BoundedReader<V> {
+ private final BoundedReader<T> boundedReader;
+
+ private TransformingReader(BoundedReader<T> boundedReader) {
+ this.boundedReader = boundedReader;
+ }
+
+ @Override
+ public synchronized BoundedSource<V> getCurrentSource() {
+ return new TransformingSource<>(boundedReader.getCurrentSource(), function, outputCoder);
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ return boundedReader.start();
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ return boundedReader.advance();
+ }
+
+ @Override
+ public V getCurrent() throws NoSuchElementException {
+ T current = boundedReader.getCurrent();
+ return function.apply(current);
+ }
+
+ @Override
+ public void close() throws IOException {
+ boundedReader.close();
+ }
+
+ @Override
+ public synchronized BoundedSource<V> splitAtFraction(double fraction) {
+ return new TransformingSource<>(
+ boundedReader.splitAtFraction(fraction), function, outputCoder);
+ }
+
+ @Override
+ public Double getFractionConsumed() {
+ return boundedReader.getFractionConsumed();
+ }
+
+ @Override
+ public Instant getCurrentTimestamp() throws NoSuchElementException {
+ return boundedReader.getCurrentTimestamp();
+ }
+ }
+ }
+
+ private static String getExtractJobId(String jobIdToken) {
+ return jobIdToken + "-extract";
+ }
+
+ private static String getExtractDestinationUri(String extractDestinationDir) {
+ return String.format("%s/%s", extractDestinationDir, "*.avro");
+ }
+
+ private static List<String> getExtractFilePaths(String extractDestinationDir, Job extractJob)
+ throws IOException {
+ JobStatistics jobStats = extractJob.getStatistics();
+ List<Long> counts = jobStats.getExtract().getDestinationUriFileCounts();
+ if (counts.size() != 1) {
+ String errorMessage = (counts.size() == 0
+ ? "No destination uri file count received."
+ : String.format("More than one destination uri file count received. First two are %s, %s",
+ counts.get(0), counts.get(1)));
+ throw new RuntimeException(errorMessage);
+ }
+ long filesCount = counts.get(0);
+
+ ImmutableList.Builder<String> paths = ImmutableList.builder();
+ IOChannelFactory factory = IOChannelUtils.getFactory(extractDestinationDir);
+ for (long i = 0; i < filesCount; ++i) {
+ String filePath =
+ factory.resolve(extractDestinationDir, String.format("%012d%s", i, ".avro"));
+ paths.add(filePath);
+ }
+ return paths.build();
+ }
+
+ /////////////////////////////////////////////////////////////////////////////
+
+ /**
+ * A {@link PTransform} that writes a {@link PCollection} containing {@link TableRow TableRows}
+ * to a BigQuery table.
+ *
+ * <p>In BigQuery, each table has an encosing dataset. The dataset being written must already
+ * exist.
+ *
+ * <p>By default, tables will be created if they do not exist, which corresponds to a
+ * {@link CreateDisposition#CREATE_IF_NEEDED} disposition that matches the default of BigQuery's
+ * Jobs API. A schema must be provided (via {@link BigQueryIO.Write#withSchema(TableSchema)}),
+ * or else the transform may fail at runtime with an {@link IllegalArgumentException}.
+ *
+ * <p>By default, writes require an empty table, which corresponds to
+ * a {@link WriteDisposition#WRITE_EMPTY} disposition that matches the
+ * default of BigQuery's Jobs API.
+ *
+ * <p>Here is a sample transform that produces TableRow values containing
+ * "word" and "count" columns:
+ * <pre>{@code
+ * static class FormatCountsFn extends DoFn<KV<String, Long>, TableRow> {
+ * public void processElement(ProcessContext c) {
+ * TableRow row = new TableRow()
+ * .set("word", c.element().getKey())
+ * .set("count", c.element().getValue().intValue());
+ * c.output(row);
+ * }
+ * }}</pre>
+ */
+ public static class Write {
+ /**
+ * An enumeration type for the BigQuery create disposition strings.
+ *
+ * @see <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.query.createDisposition">
+ * <code>configuration.query.createDisposition</code> in the BigQuery Jobs API</a>
+ */
+ public enum CreateDisposition {
+ /**
+ * Specifics that tables should not be created.
+ *
+ * <p>If the output table does not exist, the write fails.
+ */
+ CREATE_NEVER,
+
+ /**
+ * Specifies that tables should be created if needed. This is the default
+ * behavior.
+ *
+ * <p>Requires that a table schema is provided via {@link BigQueryIO.Write#withSchema}.
+ * This precondition is checked before starting a job. The schema is
+ * not required to match an existing table's schema.
+ *
+ * <p>When this transformation is executed, if the output table does not
+ * exist, the table is created from the provided schema. Note that even if
+ * the table exists, it may be recreated if necessary when paired with a
+ * {@link WriteDisposition#WRITE_TRUNCATE}.
+ */
+ CREATE_IF_NEEDED
+ }
+
+ /**
+ * An enumeration type for the BigQuery write disposition strings.
+ *
+ * @see <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.query.writeDisposition">
+ * <code>configuration.query.writeDisposition</code> in the BigQuery Jobs API</a>
+ */
+ public enum WriteDisposition {
+ /**
+ * Specifies that write should replace a table.
+ *
+ * <p>The replacement may occur in multiple steps - for instance by first
+ * removing the existing table, then creating a replacement, then filling
+ * it in. This is not an atomic operation, and external programs may
+ * see the table in any of these intermediate steps.
+ */
+ WRITE_TRUNCATE,
+
+ /**
+ * Specifies that rows may be appended to an existing table.
+ */
+ WRITE_APPEND,
+
+ /**
+ * Specifies that the output table must be empty. This is the default
+ * behavior.
+ *
+ * <p>If the output table is not empty, the write fails at runtime.
+ *
+ * <p>This check may occur long before data is written, and does not
+ * guarantee exclusive access to the table. If two programs are run
+ * concurrently, each specifying the same output table and
+ * a {@link WriteDisposition} of {@link WriteDisposition#WRITE_EMPTY}, it is possible
+ * for both to succeed.
+ */
+ WRITE_EMPTY
+ }
+
+ /**
+ * Creates a write transformation for the given table specification.
+ *
+ * <p>Refer to {@link #parseTableSpec(String)} for the specification format.
+ */
+ public static Bound to(String tableSpec) {
+ return new Bound().to(tableSpec);
+ }
+
+ /** Creates a write transformation for the given table. */
+ public static Bound to(TableReference table) {
+ return new Bound().to(table);
+ }
+
+ /**
+ * Creates a write transformation from a function that maps windows to table specifications.
+ * Each time a new window is encountered, this function will be called and the resulting table
+ * will be created. Records within that window will be written to the associated table.
+ *
+ * <p>See {@link #parseTableSpec(String)} for the format that {@code tableSpecFunction} should
+ * return.
+ *
+ * <p>{@code tableSpecFunction} should be deterministic. When given the same window, it should
+ * always return the same table specification.
+ */
+ public static Bound to(SerializableFunction<BoundedWindow, String> tableSpecFunction) {
+ return new Bound().to(tableSpecFunction);
+ }
+
+ /**
+ * Creates a write transformation from a function that maps windows to {@link TableReference}
+ * objects.
+ *
+ * <p>{@code tableRefFunction} should be deterministic. When given the same window, it should
+ * always return the same table reference.
+ */
+ public static Bound toTableReference(
+ SerializableFunction<BoundedWindow, TableReference> tableRefFunction) {
+ return new Bound().toTableReference(tableRefFunction);
+ }
+
+ /**
+ * Creates a write transformation with the specified schema to use in table creation.
+ *
+ * <p>The schema is <i>required</i> only if writing to a table that does not already
+ * exist, and {@link CreateDisposition} is set to
+ * {@link CreateDisposition#CREATE_IF_NEEDED}.
+ */
+ public static Bound withSchema(TableSchema schema) {
+ return new Bound().withSchema(schema);
+ }
+
+ /** Creates a write transformation with the specified options for creating the table. */
+ public static Bound withCreateDisposition(CreateDisposition disposition) {
+ return new Bound().withCreateDisposition(disposition);
+ }
+
+ /** Creates a write transformation with the specified options for writing to the table. */
+ public static Bound withWriteDisposition(WriteDisposition disposition) {
+ return new Bound().withWriteDisposition(disposition);
+ }
+
+ /**
+ * Creates a write transformation with BigQuery table validation disabled.
+ */
+ public static Bound withoutValidation() {
+ return new Bound().withoutValidation();
+ }
+
+ /**
+ * A {@link PTransform} that can write either a bounded or unbounded
+ * {@link PCollection} of {@link TableRow TableRows} to a BigQuery table.
+ */
+ public static class Bound extends PTransform<PCollection<TableRow>, PDone> {
+ @Nullable final String jsonTableRef;
+
+ @Nullable final SerializableFunction<BoundedWindow, TableReference> tableRefFunction;
+
+ // Table schema. The schema is required only if the table does not exist.
+ @Nullable final String jsonSchema;
+
+ // Options for creating the table. Valid values are CREATE_IF_NEEDED and
+ // CREATE_NEVER.
+ final CreateDisposition createDisposition;
+
+ // Options for writing to the table. Valid values are WRITE_TRUNCATE,
+ // WRITE_APPEND and WRITE_EMPTY.
+ final WriteDisposition writeDisposition;
+
+ // An option to indicate if table validation is desired. Default is true.
+ final boolean validate;
+
+ @Nullable private BigQueryServices bigQueryServices;
+
+ private static class TranslateTableSpecFunction implements
+ SerializableFunction<BoundedWindow, TableReference> {
+ private SerializableFunction<BoundedWindow, String> tableSpecFunction;
+
+ TranslateTableSpecFunction(SerializableFunction<BoundedWindow, String> tableSpecFunction) {
+ this.tableSpecFunction = tableSpecFunction;
+ }
+
+ @Override
+ public TableReference apply(BoundedWindow value) {
+ return parseTableSpec(tableSpecFunction.apply(value));
+ }
+ }
+
+ /**
+ * @deprecated Should be private. Instead, use one of the factory methods in
+ * {@link BigQueryIO.Write}, such as {@link BigQueryIO.Write#to(String)}, to create an
+ * instance of this class.
+ */
+ @Deprecated
+ public Bound() {
+ this(
+ null /* name */,
+ null /* jsonTableRef */,
+ null /* tableRefFunction */,
+ null /* jsonSchema */,
+ CreateDisposition.CREATE_IF_NEEDED,
+ WriteDisposition.WRITE_EMPTY,
+ true /* validate */,
+ null /* bigQueryServices */);
+ }
+
+ private Bound(String name, @Nullable String jsonTableRef,
+ @Nullable SerializableFunction<BoundedWindow, TableReference> tableRefFunction,
+ @Nullable String jsonSchema,
+ CreateDisposition createDisposition, WriteDisposition writeDisposition, boolean validate,
+ @Nullable BigQueryServices bigQueryServices) {
+ super(name);
+ this.jsonTableRef = jsonTableRef;
+ this.tableRefFunction = tableRefFunction;
+ this.jsonSchema = jsonSchema;
+ this.createDisposition = checkNotNull(createDisposition, "createDisposition");
+ this.writeDisposition = checkNotNull(writeDisposition, "writeDisposition");
+ this.validate = validate;
+ this.bigQueryServices = bigQueryServices;
+ }
+
+ /**
+ * Returns a copy of this write transformation, but writing to the specified table. Refer to
+ * {@link #parseTableSpec(String)} for the specification format.
+ *
+ * <p>Does not modify this object.
+ */
+ public Bound to(String tableSpec) {
+ return to(parseTableSpec(tableSpec));
+ }
+
+ /**
+ * Returns a copy of this write transformation, but writing to the specified table.
+ *
+ * <p>Does not modify this object.
+ */
+ public Bound to(TableReference table) {
+ return new Bound(name, toJsonString(table), tableRefFunction, jsonSchema, createDisposition,
+ writeDisposition, validate, bigQueryServices);
+ }
+
+ /**
+ * Returns a copy of this write transformation, but using the specified function to determine
+ * which table to write to for each window.
+ *
+ * <p>Does not modify this object.
+ *
+ * <p>{@code tableSpecFunction} should be deterministic. When given the same window, it
+ * should always return the same table specification.
+ */
+ public Bound to(
+ SerializableFunction<BoundedWindow, String> tableSpecFunction) {
+ return toTableReference(new TranslateTableSpecFunction(tableSpecFunction));
+ }
+
+ /**
+ * Returns a copy of this write transformation, but using the specified function to determine
+ * which table to write to for each window.
+ *
+ * <p>Does not modify this object.
+ *
+ * <p>{@code tableRefFunction} should be deterministic. When given the same window, it should
+ * always return the same table reference.
+ */
+ public Bound toTableReference(
+ SerializableFunction<BoundedWindow, TableReference> tableRefFunction) {
+ return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
+ writeDisposition, validate, bigQueryServices);
+ }
+
+ /**
+ * Returns a copy of this write transformation, but using the specified schema for rows
+ * to be written.
+ *
+ * <p>Does not modify this object.
+ */
+ public Bound withSchema(TableSchema schema) {
+ return new Bound(name, jsonTableRef, tableRefFunction, toJsonString(schema),
+ createDisposition, writeDisposition, validate, bigQueryServices);
+ }
+
+ /**
+ * Returns a copy of this write transformation, but using the specified create disposition.
+ *
+ * <p>Does not modify this object.
+ */
+ public Bound withCreateDisposition(CreateDisposition createDisposition) {
+ return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
+ writeDisposition, validate, bigQueryServices);
+ }
+
+ /**
+ * Returns a copy of this write transformation, but using the specified write disposition.
+ *
+ * <p>Does not modify this object.
+ */
+ public Bound withWriteDisposition(WriteDisposition writeDisposition) {
+ return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
+ writeDisposition, validate, bigQueryServices);
+ }
+
+ /**
+ * Returns a copy of this write transformation, but without BigQuery table validation.
+ *
+ * <p>Does not modify this object.
+ */
+ public Bound withoutValidation() {
+ return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
+ writeDisposition, false, bigQueryServices);
+ }
+
+ @VisibleForTesting
+ Bound withTestServices(BigQueryServices testServices) {
+ return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
+ writeDisposition, validate, testServices);
+ }
+
+ private static void verifyTableEmpty(
+ DatasetService datasetService,
+ TableReference table) {
+ try {
+ boolean isEmpty = datasetService.isTableEmpty(
+ table.getProjectId(), table.getDatasetId(), table.getTableId());
+ if (!isEmpty) {
+ throw new IllegalArgumentException(
+ "BigQuery table is not empty: " + BigQueryIO.toTableSpec(table));
+ }
+ } catch (IOException | InterruptedException e) {
+ ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
+ if (e instanceof IOException && errorExtractor.itemNotFound((IOException) e)) {
+ // Nothing to do. If the table does not exist, it is considered empty.
+ } else {
+ throw new RuntimeException(
+ "unable to confirm BigQuery table emptiness for table "
+ + BigQueryIO.toTableSpec(table), e);
+ }
+ }
+ }
+
+ @Override
+ public void validate(PCollection<TableRow> input) {
+ BigQueryOptions options = input.getPipeline().getOptions().as(BigQueryOptions.class);
+
+ // Exactly one of the table and table reference can be configured.
+ checkState(
+ jsonTableRef != null || tableRefFunction != null,
+ "must set the table reference of a BigQueryIO.Write transform");
+ checkState(
+ jsonTableRef == null || tableRefFunction == null,
+ "Cannot set both a table reference and a table function for a BigQueryIO.Write"
+ + " transform");
+
+ // Require a schema if creating one or more tables.
+ checkArgument(
+ createDisposition != CreateDisposition.CREATE_IF_NEEDED || jsonSchema != null,
+ "CreateDisposition is CREATE_IF_NEEDED, however no schema was provided.");
+
+ // The user specified a table.
+ if (jsonTableRef != null && validate) {
+ TableReference table = getTableWithDefaultProject(options);
+
+ DatasetService datasetService = getBigQueryServices().getDatasetService(options);
+ // Check for destination table presence and emptiness for early failure notification.
+ // Note that a presence check can fail when the table or dataset is created by an earlier
+ // stage of the pipeline. For these cases the #withoutValidation method can be used to
+ // disable the check.
+ verifyDatasetPresence(datasetService, table);
+ if (getCreateDisposition() == BigQueryIO.Write.CreateDisposition.CREATE_NEVER) {
+ verifyTablePresence(datasetService, table);
+ }
+ if (getWriteDisposition() == BigQueryIO.Write.WriteDisposition.WRITE_EMPTY) {
+ verifyTableEmpty(datasetService, table);
+ }
+ }
+
+ if (input.isBounded() == PCollection.IsBounded.UNBOUNDED || tableRefFunction != null) {
+ // We will use BigQuery's streaming write API -- validate supported dispositions.
+ checkArgument(
+ createDisposition != CreateDisposition.CREATE_NEVER,
+ "CreateDisposition.CREATE_NEVER is not supported for an unbounded PCollection or when"
+ + " using a tablespec function.");
+
+ checkArgument(
+ writeDisposition != WriteDisposition.WRITE_TRUNCATE,
+ "WriteDisposition.WRITE_TRUNCATE is not supported for an unbounded PCollection or"
+ + " when using a tablespec function.");
+ } else {
+ // We will use a BigQuery load job -- validate the temp location.
+ String tempLocation = options.getTempLocation();
+ checkArgument(
+ !Strings.isNullOrEmpty(tempLocation),
+ "BigQueryIO.Write needs a GCS temp location to store temp files.");
+ if (bigQueryServices == null) {
+ try {
+ GcsPath.fromUri(tempLocation);
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgumentException(
+ String.format(
+ "BigQuery temp location expected a valid 'gs://' path, but was given '%s'",
+ tempLocation),
+ e);
+ }
+ }
+ }
+ }
+
+ @Override
+ public PDone apply(PCollection<TableRow> input) {
+ BigQueryOptions options = input.getPipeline().getOptions().as(BigQueryOptions.class);
+
+ // In a streaming job, or when a tablespec function is defined, we use StreamWithDeDup
+ // and BigQuery's streaming import API.
+ if (options.isStreaming() || tableRefFunction != null) {
+ return input.apply(new StreamWithDeDup(getTable(), tableRefFunction, getSchema()));
+ }
+
+ TableReference table = fromJsonString(jsonTableRef, TableReference.class);
+ if (Strings.isNullOrEmpty(table.getProjectId())) {
+ table.setProjectId(options.getProject());
+ }
+ String jobIdToken = randomUUIDString();
+ String tempLocation = options.getTempLocation();
+ String tempFilePrefix;
+ try {
+ IOChannelFactory factory = IOChannelUtils.getFactory(tempLocation);
+ tempFilePrefix = factory.resolve(
+ factory.resolve(tempLocation, "BigQuerySinkTemp"),
+ jobIdToken);
+ } catch (IOException e) {
+ throw new RuntimeException(
+ String.format("Failed to resolve BigQuery temp location in %s", tempLocation),
+ e);
+ }
+
+ BigQueryServices bqServices = getBigQueryServices();
+ return input.apply("Write", org.apache.beam.sdk.io.Write.to(
+ new BigQuerySink(
+ jobIdToken,
+ table,
+ jsonSchema,
+ getWriteDisposition(),
+ getCreateDisposition(),
+ tempFilePrefix,
+ input.getCoder(),
+ bqServices)));
+ }
+
+ @Override
+ protected Coder<Void> getDefaultOutputCoder() {
+ return VoidCoder.of();
+ }
+
+ @Override
+ public void populateDisplayData(DisplayData.Builder builder) {
+ super.populateDisplayData(builder);
+
+ builder
+ .addIfNotNull(DisplayData.item("table", jsonTableRef)
+ .withLabel("Table Reference"))
+ .addIfNotNull(DisplayData.item("schema", jsonSchema)
+ .withLabel("Table Schema"));
+
+ if (tableRefFunction != null) {
+ builder.add(DisplayData.item("tableFn", tableRefFunction.getClass())
+ .withLabel("Table Reference Function"));
+ }
+
+ builder
+ .add(DisplayData.item("createDisposition", createDisposition.toString())
+ .withLabel("Table CreateDisposition"))
+ .add(DisplayData.item("writeDisposition", writeDisposition.toString())
+ .withLabel("Table WriteDisposition"))
+ .addIfNotDefault(DisplayData.item("validation", validate)
+ .withLabel("Validation Enabled"), true);
+ }
+
+ /** Returns the create disposition. */
+ public CreateDisposition getCreateDisposition() {
+ return createDisposition;
+ }
+
+ /** Returns the write disposition. */
+ public WriteDisposition getWriteDisposition() {
+ return writeDisposition;
+ }
+
+ /** Returns the table schema. */
+ public TableSchema getSchema() {
+ return fromJsonString(jsonSchema, TableSchema.class);
+ }
+
+ /**
+ * Returns the table to write, or {@code null} if writing with {@code tableRefFunction}.
+ *
+ * <p>If the table's project is not specified, use the executing project.
+ */
+ @Nullable private TableReference getTableWithDefaultProject(BigQueryOptions bqOptions) {
+ TableReference table = getTable();
+ if (table != null && Strings.isNullOrEmpty(table.getProjectId())) {
+ // If user does not specify a project we assume the table to be located in
+ // the default project.
+ table.setProjectId(bqOptions.getProject());
+ }
+ return table;
+ }
+
+ /** Returns the table reference, or {@code null}. */
+ @Nullable
+ public TableReference getTable() {
+ return fromJsonString(jsonTableRef, TableReference.class);
+ }
+
+ /** Returns {@code true} if table validation is enabled. */
+ public boolean getValidate() {
+ return validate;
+ }
+
+ private BigQueryServices getBigQueryServices() {
+ if (bigQueryServices == null) {
+ bigQueryServices = new BigQueryServicesImpl();
+ }
+ return bigQueryServices;
+ }
+ }
+
+ /** Disallow construction of utility class. */
+ private Write() {}
+ }
+
+ /**
+ * {@link BigQuerySink} is implemented as a {@link FileBasedSink}.
+ *
+ * <p>It uses BigQuery load job to import files into BigQuery.
+ */
+ static class BigQuerySink extends FileBasedSink<TableRow> {
+ private final String jobIdToken;
+ @Nullable private final String jsonTable;
+ @Nullable private final String jsonSchema;
+ private final WriteDisposition writeDisposition;
+ private final CreateDisposition createDisposition;
+ private final Coder<TableRow> coder;
+ private final BigQueryServices bqServices;
+
+ public BigQuerySink(
+ String jobIdToken,
+ @Nullable TableReference table,
+ @Nullable String jsonSchema,
+ WriteDisposition writeDisposition,
+ CreateDisposition createDisposition,
+ String tempFile,
+ Coder<TableRow> coder,
+ BigQueryServices bqServices) {
+ super(tempFile, ".json");
+ this.jobIdToken = checkNotNull(jobIdToken, "jobIdToken");
+ if (table == null) {
+ this.jsonTable = null;
+ } else {
+ checkArgument(!Strings.isNullOrEmpty(table.getProjectId()),
+ "Table %s should have a project specified", table);
+ this.jsonTable = toJsonString(table);
+ }
+ this.jsonSchema = jsonSchema;
+ this.writeDisposition = checkNotNull(writeDisposition, "writeDisposition");
+ this.createDisposition = checkNotNull(createDisposition, "createDisposition");
+ this.coder = checkNotNull(coder, "coder");
+ this.bqServices = checkNotNull(bqServices, "bqServices");
+ }
+
+ @Override
+ public FileBasedSink.FileBasedWriteOperation<TableRow> createWriteOperation(
+ PipelineOptions options) {
+ return new BigQueryWriteOperation(this);
+ }
+
+ @Override
+ public void populateDisplayData(DisplayData.Builder builder) {
+ super.populateDisplayData(builder);
+
+ builder
+ .addIfNotNull(DisplayData.item("schema", jsonSchema)
+ .withLabel("Table Schema"))
+ .addIfNotNull(DisplayData.item("tableSpec", jsonTable)
+ .withLabel("Table Specification"));
+ }
+
+ private static class BigQueryWriteOperation extends FileBasedWriteOperation<TableRow> {
+ // The maximum number of retry load jobs.
+ private static final int MAX_RETRY_LOAD_JOBS = 3;
+
+ // The maximum number of retries to poll the status of a load job.
+ // It sets to {@code Integer.MAX_VALUE} to block until the BigQuery job finishes.
+ private static final int LOAD_JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE;
+
+ private final BigQuerySink bigQuerySink;
+
+ private BigQueryWriteOperation(BigQuerySink sink) {
+ super(checkNotNull(sink, "sink"));
+ this.bigQuerySink = sink;
+ }
+
+ @Override
+ public FileBasedWriter<TableRow> createWriter(PipelineOptions options) throws Exception {
+ return new TableRowWriter(this, bigQuerySink.coder);
+ }
+
+ @Override
+ public void finalize(Iterable<FileResult> writerResults, PipelineOptions options)
+ throws IOException, InterruptedException {
+ try {
+ BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
+ List<String> tempFiles = Lists.newArrayList();
+ for (FileResult result : writerResults) {
+ tempFiles.add(result.getFilename());
+ }
+ if (!tempFiles.isEmpty()) {
+ load(
+ bigQuerySink.bqServices.getJobService(bqOptions),
+ bigQuerySink.jobIdToken,
+ fromJsonString(bigQuerySink.jsonTable, TableReference.class),
+ tempFiles,
+ fromJsonString(bigQuerySink.jsonSchema, TableSchema.class),
+ bigQuerySink.writeDisposition,
+ bigQuerySink.createDisposition);
+ }
+ } finally {
+ removeTemporaryFiles(options);
+ }
+ }
+
+ /**
+ * Import files into BigQuery with load jobs.
+ *
+ * <p>Returns if files are successfully loaded into BigQuery.
+ * Throws a RuntimeException if:
+ * 1. The status of one load job is UNKNOWN. This is to avoid duplicating data.
+ * 2. It exceeds {@code MAX_RETRY_LOAD_JOBS}.
+ *
+ * <p>If a load job failed, it will try another load job with a different job id.
+ */
+ private void load(
+ JobService jobService,
+ String jobIdPrefix,
+ TableReference ref,
+ List<String> gcsUris,
+ @Nullable TableSchema schema,
+ WriteDisposition writeDisposition,
+ CreateDisposition createDisposition) throws InterruptedException, IOException {
+ JobConfigurationLoad loadConfig = new JobConfigurationLoad()
+ .setSourceUris(gcsUris)
+ .setDestinationTable(ref)
+ .setSchema(schema)
+ .setWriteDisposition(writeDisposition.name())
+ .setCreateDisposition(createDisposition.name())
+ .setSourceFormat("NEWLINE_DELIMITED_JSON");
+
+ boolean retrying = false;
+ String projectId = ref.getProjectId();
+ for (int i = 0; i < MAX_RETRY_LOAD_JOBS; ++i) {
+ String jobId = jobIdPrefix + "-" + i;
+ if (retrying) {
+ LOG.info("Previous load jobs failed, retrying.");
+ }
+ LOG.info("Starting BigQuery load job: {}", jobId);
+ JobReference jobRef = new JobReference()
+ .setProjectId(projectId)
+ .setJobId(jobId);
+ jobService.startLoadJob(jobRef, loadConfig);
+ Status jobStatus =
+ parseStatus(jobService.pollJob(jobRef, LOAD_JOB_POLL_MAX_RETRIES));
+ switch (jobStatus) {
+ case SUCCEEDED:
+ return;
+ case UNKNOWN:
+ throw new RuntimeException("Failed to poll the load job status.");
+ case FAILED:
+ LOG.info("BigQuery load job failed: {}", jobId);
+ retrying = true;
+ continue;
+ default:
+ throw new IllegalStateException("Unexpected job status: " + jobStatus);
+ }
+ }
+ throw new RuntimeException(
+ "Failed to create the load job, reached max retries: " + MAX_RETRY_LOAD_JOBS);
+ }
+ }
+
+ private static class TableRowWriter extends FileBasedWriter<TableRow> {
+ private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8);
+ private final Coder<TableRow> coder;
+ private OutputStream out;
+
+ public TableRowWriter(
+ FileBasedWriteOperation<TableRow> writeOperation, Coder<TableRow> coder) {
+ super(writeOperation);
+ this.mimeType = MimeTypes.TEXT;
+ this.coder = coder;
+ }
+
+ @Override
+ protected void prepareWrite(WritableByteChannel channel) throws Exception {
+ out = Channels.newOutputStream(channel);
+ }
+
+ @Override
+ public void write(TableRow value) throws Exception {
+ // Use Context.OUTER to encode and NEWLINE as the delimeter.
+ coder.encode(value, out, Context.OUTER);
+ out.write(NEWLINE);
+ }
+ }
+ }
+
+ private static void verifyDatasetPresence(DatasetService datasetService, TableReference table) {
+ try {
+ datasetService.getDataset(table.getProjectId(), table.getDatasetId());
+ } catch (Exception e) {
+ ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
+ if ((e instanceof IOException) && errorExtractor.itemNotFound((IOException) e)) {
+ throw new IllegalArgumentException(
+ String.format(RESOURCE_NOT_FOUND_ERROR, "dataset", BigQueryIO.toTableSpec(table)),
+ e);
+ } else {
+ throw new RuntimeException(
+ String.format(UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR, "dataset",
+ BigQueryIO.toTableSpec(table)),
+ e);
+ }
+ }
+ }
+
+ private static void verifyTablePresence(DatasetService datasetService, TableReference table) {
+ try {
+ datasetService.getTable(table.getProjectId(), table.getDatasetId(), table.getTableId());
+ } catch (Exception e) {
+ ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
+ if ((e instanceof IOException) && errorExtractor.itemNotFound((IOException) e)) {
+ throw new IllegalArgumentException(
+ String.format(RESOURCE_NOT_FOUND_ERROR, "table", BigQueryIO.toTableSpec(table)), e);
+ } else {
+ throw new RuntimeException(
+ String.format(UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR, "table",
+ BigQueryIO.toTableSpec(table)),
+ e);
+ }
+ }
+ }
+
+ /////////////////////////////////////////////////////////////////////////////
+
+ /**
+ * Implementation of DoFn to perform streaming BigQuery write.
+ */
+ @SystemDoFnInternal
+ private static class StreamingWriteFn
+ extends DoFn<KV<ShardedKey<String>, TableRowInfo>, Void> {
+ /** TableSchema in JSON. Use String to make the class Serializable. */
+ private final String jsonTableSchema;
+
+ /** JsonTableRows to accumulate BigQuery rows in order to batch writes. */
+ private transient Map<String, List<TableRow>> tableRows;
+
+ /** The list of unique ids for each BigQuery table row. */
+ private transient Map<String, List<String>> uniqueIdsForTableRows;
+
+ /** The list of tables created so far, so we don't try the creation
+ each time. */
+ private static Set<String> createdTables =
+ Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
+
+ /** Tracks bytes written, exposed as "ByteCount" Counter. */
+ private Aggregator<Long, Long> byteCountAggregator =
+ createAggregator("ByteCount", new Sum.SumLongFn());
+
+ /** Constructor. */
+ StreamingWriteFn(TableSchema schema) {
+ jsonTableSchema = toJsonString(schema);
+ }
+
+ /** Prepares a target BigQuery table. */
+ @Override
+ public void startBundle(Context context) {
+ tableRows = new HashMap<>();
+ uniqueIdsForTableRows = new HashMap<>();
+ }
+
+ /** Accumulates the input into JsonTableRows and uniqueIdsForTableRows. */
+ @Override
+ public void processElement(ProcessContext context) {
+ String tableSpec = context.element().getKey().getKey();
+ List<TableRow> rows = getOrCreateMapListValue(tableRows, tableSpec);
+ List<String> uniqueIds = getOrCreateMapListValue(uniqueIdsForTableRows, tableSpec);
+
+ rows.add(context.element().getValue().tableRow);
+ uniqueIds.add(context.element().getValue().uniqueId);
+ }
+
+ /** Writes the accumulated rows into BigQuery with streaming API. */
+ @Override
+ public void finishBundle(Context context) throws Exception {
+ BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class);
+ Bigquery client = Transport.newBigQueryClient(options).build();
+
+ for (Map.Entry<String, List<TableRow>> entry : tableRows.entrySet()) {
+ TableReference tableReference = getOrCreateTable(options, entry.getKey());
+ flushRows(client, tableReference, entry.getValue(),
+ uniqueIdsForTableRows.get(entry.getKey()), options);
+ }
+ tableRows.clear();
+ uniqueIdsForTableRows.clear();
+ }
+
+ @Override
+ public void populateDisplayData(DisplayData.Builder builder) {
+ super.populateDisplayData(builder);
+
+ builder.addIfNotNull(DisplayData.item("schema", jsonTableSchema)
+ .withLabel("Table Schema"));
+ }
+
+ public TableReference getOrCreateTable(BigQueryOptions options, String tableSpec)
+ throws IOException {
+ TableReference tableReference = parseTableSpec(tableSpec);
+ if (!createdTables.contains(tableSpec)) {
+ synchronized (createdTables) {
+ // Another thread may have succeeded in creating the table in the meanwhile, so
+ // check again. This check isn't needed for correctness, but we add it to prevent
+ // every thread from attempting a create and overwhelming our BigQuery quota.
+ if (!createdTables.contains(tableSpec)) {
+ TableSchema tableSchema = JSON_FACTORY.fromString(jsonTableSchema, TableSchema.class);
+ Bigquery client = Transport.newBigQueryClient(options).build();
+ BigQueryTableInserter inserter = new BigQueryTableInserter(client, options);
+ inserter.getOrCreateTable(tableReference, WriteDisposition.WRITE_APPEND,
+ CreateDisposition.CREATE_IF_NEEDED, tableSchema);
+ createdTables.add(tableSpec);
+ }
+ }
+ }
+ return tableReference;
+ }
+
+ /** Writes the accumulated rows into BigQuery with streaming API. */
+ private void flushRows(Bigquery client, TableReference tableReference,
+ List<TableRow> tableRows, List<String> uniqueIds, BigQueryOptions options) {
+ if (!tableRows.isEmpty()) {
+ try {
+ BigQueryTableInserter inserter = new BigQueryTableInserter(client, options);
+ inserter.insertAll(tableReference, tableRows, uniqueIds, byteCountAggregator);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+ }
+
+ private static class ShardedKey<K> {
+ private final K key;
+ private final int shardNumber;
+
+ public static <K> ShardedKey<K> of(K key, int shardNumber) {
+ return new ShardedKey<K>(key, shardNumber);
+ }
+
+ private ShardedKey(K key, int shardNumber) {
+ this.key = key;
+ this.shardNumber = shardNumber;
+ }
+
+ public K getKey() {
+ return key;
+ }
+
+ public int getShardNumber() {
+ return shardNumber;
+ }
+ }
+
+ /**
+ * A {@link Coder} for {@link ShardedKey}, using a wrapped key {@link Coder}.
+ */
+ private static class ShardedKeyCoder<KeyT>
+ extends StandardCoder<ShardedKey<KeyT>> {
+ public static <KeyT> ShardedKeyCoder<KeyT> of(Coder<KeyT> keyCoder) {
+ return new ShardedKeyCoder<>(keyCoder);
+ }
+
+ @JsonCreator
+ public static <KeyT> ShardedKeyCoder<KeyT> of(
+ @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
+ List<Coder<KeyT>> components) {
+ checkArgument(components.size() == 1, "Expecting 1 component, got %s", components.size());
+ return of(components.get(0));
+ }
+
+ protected ShardedKeyCoder(Coder<KeyT> keyCoder) {
+ this.keyCoder = keyCoder;
+ this.shardNumberCoder = VarIntCoder.of();
+ }
+
+ @Override
+ public List<? extends Coder<?>> getCoderArguments() {
+ return Arrays.asList(keyCoder);
+ }
+
+ @Override
+ public void encode(ShardedKey<KeyT> key, OutputStream outStream, Context context)
+ throws IOException {
+ keyCoder.encode(key.getKey(), outStream, context.nested());
+ shardNumberCoder.encode(key.getShardNumber(), outStream, context);
+ }
+
+ @Override
+ public ShardedKey<KeyT> decode(InputStream inStream, Context context)
+ throws IOException {
+ return new ShardedKey<KeyT>(
+ keyCoder.decode(inStream, context.nested()),
+ shardNumberCoder.decode(inStream, context));
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ keyCoder.verifyDeterministic();
+ }
+
+ Coder<KeyT> keyCoder;
+ VarIntCoder shardNumberCoder;
+ }
+
+ private static class TableRowInfoCoder extends AtomicCoder<TableRowInfo> {
+ private static final TableRowInfoCoder INSTANCE = new TableRowInfoCoder();
+
+ @JsonCreator
+ public static TableRowInfoCoder of() {
+ return INSTANCE;
+ }
+
+ @Override
+ public void encode(TableRowInfo value, OutputStream outStream, Context context)
+ throws IOException {
+ if (value == null) {
+ throw new CoderException("cannot encode a null value");
+ }
+ tableRowCoder.encode(value.tableRow, outStream, context.nested());
+ idCoder.encode(value.uniqueId, outStream, context.nested());
+ }
+
+ @Override
+ public TableRowInfo decode(InputStream inStream, Context context)
+ throws IOException {
+ return new TableRowInfo(
+ tableRowCoder.decode(inStream, context.nested()),
+ idCoder.decode(inStream, context.nested()));
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ throw new NonDeterministicException(this, "TableRows are not deterministic.");
+ }
+
+ TableRowJsonCoder tableRowCoder = TableRowJsonCoder.of()
<TRUNCATED>
[05/10] incubator-beam git commit: BigQueryIO: move to
google-cloud-platform module
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableRowIteratorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableRowIteratorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableRowIteratorTest.java
deleted file mode 100644
index e92f2c6..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableRowIteratorTest.java
+++ /dev/null
@@ -1,256 +0,0 @@
-/*
- * 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.util;
-
-import static org.hamcrest.Matchers.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
-
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.model.Dataset;
-import com.google.api.services.bigquery.model.ErrorProto;
-import com.google.api.services.bigquery.model.Job;
-import com.google.api.services.bigquery.model.JobConfiguration;
-import com.google.api.services.bigquery.model.JobConfigurationQuery;
-import com.google.api.services.bigquery.model.JobReference;
-import com.google.api.services.bigquery.model.JobStatus;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableCell;
-import com.google.api.services.bigquery.model.TableDataList;
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * Tests for {@link BigQueryTableRowIterator}.
- */
-@RunWith(JUnit4.class)
-public class BigQueryTableRowIteratorTest {
-
- @Rule public ExpectedException thrown = ExpectedException.none();
-
- @Mock private Bigquery mockClient;
- @Mock private Bigquery.Datasets mockDatasets;
- @Mock private Bigquery.Datasets.Delete mockDatasetsDelete;
- @Mock private Bigquery.Datasets.Insert mockDatasetsInsert;
- @Mock private Bigquery.Jobs mockJobs;
- @Mock private Bigquery.Jobs.Get mockJobsGet;
- @Mock private Bigquery.Jobs.Insert mockJobsInsert;
- @Mock private Bigquery.Tables mockTables;
- @Mock private Bigquery.Tables.Get mockTablesGet;
- @Mock private Bigquery.Tables.Delete mockTablesDelete;
- @Mock private Bigquery.Tabledata mockTabledata;
- @Mock private Bigquery.Tabledata.List mockTabledataList;
-
- @Before
- public void setUp() throws IOException {
- MockitoAnnotations.initMocks(this);
- when(mockClient.tabledata()).thenReturn(mockTabledata);
- when(mockTabledata.list(anyString(), anyString(), anyString())).thenReturn(mockTabledataList);
-
- when(mockClient.tables()).thenReturn(mockTables);
- when(mockTables.delete(anyString(), anyString(), anyString())).thenReturn(mockTablesDelete);
- when(mockTables.get(anyString(), anyString(), anyString())).thenReturn(mockTablesGet);
-
- when(mockClient.datasets()).thenReturn(mockDatasets);
- when(mockDatasets.delete(anyString(), anyString())).thenReturn(mockDatasetsDelete);
- when(mockDatasets.insert(anyString(), any(Dataset.class))).thenReturn(mockDatasetsInsert);
-
- when(mockClient.jobs()).thenReturn(mockJobs);
- when(mockJobs.insert(anyString(), any(Job.class))).thenReturn(mockJobsInsert);
- when(mockJobs.get(anyString(), anyString())).thenReturn(mockJobsGet);
- }
-
- @After
- public void tearDown() {
- verifyNoMoreInteractions(mockClient);
- verifyNoMoreInteractions(mockDatasets);
- verifyNoMoreInteractions(mockDatasetsDelete);
- verifyNoMoreInteractions(mockDatasetsInsert);
- verifyNoMoreInteractions(mockJobs);
- verifyNoMoreInteractions(mockJobsGet);
- verifyNoMoreInteractions(mockJobsInsert);
- verifyNoMoreInteractions(mockTables);
- verifyNoMoreInteractions(mockTablesDelete);
- verifyNoMoreInteractions(mockTablesGet);
- verifyNoMoreInteractions(mockTabledata);
- verifyNoMoreInteractions(mockTabledataList);
- }
-
- private static Table tableWithBasicSchema() {
- return new Table()
- .setSchema(
- new TableSchema()
- .setFields(
- Arrays.asList(
- new TableFieldSchema().setName("name").setType("STRING"),
- new TableFieldSchema().setName("answer").setType("INTEGER"))));
- }
-
- private TableRow rawRow(Object... args) {
- List<TableCell> cells = new LinkedList<>();
- for (Object a : args) {
- cells.add(new TableCell().setV(a));
- }
- return new TableRow().setF(cells);
- }
-
- private TableDataList rawDataList(TableRow... rows) {
- return new TableDataList().setRows(Arrays.asList(rows));
- }
-
- /**
- * Verifies that when the query runs, the correct data is returned and the temporary dataset and
- * table are both cleaned up.
- */
- @Test
- public void testReadFromQuery() throws IOException, InterruptedException {
- // Mock job inserting.
- Job insertedJob = new Job().setJobReference(new JobReference());
- when(mockJobsInsert.execute()).thenReturn(insertedJob);
-
- // Mock job polling.
- JobStatus status = new JobStatus().setState("DONE");
- TableReference tableRef =
- new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
- JobConfigurationQuery queryConfig = new JobConfigurationQuery().setDestinationTable(tableRef);
- Job getJob =
- new Job()
- .setJobReference(new JobReference())
- .setStatus(status)
- .setConfiguration(new JobConfiguration().setQuery(queryConfig));
- when(mockJobsGet.execute()).thenReturn(getJob);
-
- // Mock table schema fetch.
- when(mockTablesGet.execute()).thenReturn(tableWithBasicSchema());
-
- // Mock table data fetch.
- when(mockTabledataList.execute()).thenReturn(rawDataList(rawRow("Arthur", 42)));
-
- // Run query and verify
- String query = "SELECT name, count from table";
- try (BigQueryTableRowIterator iterator =
- BigQueryTableRowIterator.fromQuery(query, "project", mockClient, null)) {
- iterator.open();
- assertTrue(iterator.advance());
- TableRow row = iterator.getCurrent();
-
- assertTrue(row.containsKey("name"));
- assertTrue(row.containsKey("answer"));
- assertEquals("Arthur", row.get("name"));
- assertEquals(42, row.get("answer"));
-
- assertFalse(iterator.advance());
- }
-
- // Temp dataset created and later deleted.
- verify(mockClient, times(2)).datasets();
- verify(mockDatasets).insert(anyString(), any(Dataset.class));
- verify(mockDatasetsInsert).execute();
- verify(mockDatasets).delete(anyString(), anyString());
- verify(mockDatasetsDelete).execute();
- // Job inserted to run the query, polled once.
- verify(mockClient, times(2)).jobs();
- verify(mockJobs).insert(anyString(), any(Job.class));
- verify(mockJobsInsert).execute();
- verify(mockJobs).get(anyString(), anyString());
- verify(mockJobsGet).execute();
- // Temp table get after query finish, deleted after reading.
- verify(mockClient, times(2)).tables();
- verify(mockTables).get("project", "dataset", "table");
- verify(mockTablesGet).execute();
- verify(mockTables).delete(anyString(), anyString(), anyString());
- verify(mockTablesDelete).execute();
- // Table data read.
- verify(mockClient).tabledata();
- verify(mockTabledata).list("project", "dataset", "table");
- verify(mockTabledataList).execute();
- }
-
- /**
- * Verifies that when the query fails, the user gets a useful exception and the temporary dataset
- * is cleaned up. Also verifies that the temporary table (which is never created) is not
- * erroneously attempted to be deleted.
- */
- @Test
- public void testQueryFailed() throws IOException {
- // Job can be created.
- JobReference ref = new JobReference();
- Job insertedJob = new Job().setJobReference(ref);
- when(mockJobsInsert.execute()).thenReturn(insertedJob);
-
- // Job state polled with an error.
- String errorReason = "bad query";
- JobStatus status =
- new JobStatus().setState("DONE").setErrorResult(new ErrorProto().setMessage(errorReason));
- Job getJob = new Job().setJobReference(ref).setStatus(status);
- when(mockJobsGet.execute()).thenReturn(getJob);
-
- String query = "NOT A QUERY";
- try (BigQueryTableRowIterator iterator =
- BigQueryTableRowIterator.fromQuery(query, "project", mockClient, null)) {
- try {
- iterator.open();
- fail();
- } catch (Exception expected) {
- // Verify message explains cause and reports the query.
- assertThat(expected.getMessage(), containsString("failed"));
- assertThat(expected.getMessage(), containsString(errorReason));
- assertThat(expected.getMessage(), containsString(query));
- }
- }
-
- // Temp dataset created and then later deleted.
- verify(mockClient, times(2)).datasets();
- verify(mockDatasets).insert(anyString(), any(Dataset.class));
- verify(mockDatasetsInsert).execute();
- verify(mockDatasets).delete(anyString(), anyString());
- verify(mockDatasetsDelete).execute();
- // Job inserted to run the query, then polled once.
- verify(mockClient, times(2)).jobs();
- verify(mockJobs).insert(anyString(), any(Job.class));
- verify(mockJobsInsert).execute();
- verify(mockJobs).get(anyString(), anyString());
- verify(mockJobsGet).execute();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryUtilTest.java
deleted file mode 100644
index c033a7d..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryUtilTest.java
+++ /dev/null
@@ -1,485 +0,0 @@
-/*
- * 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.util;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.atLeast;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.Sum;
-
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableCell;
-import com.google.api.services.bigquery.model.TableDataInsertAllRequest;
-import com.google.api.services.bigquery.model.TableDataInsertAllResponse;
-import com.google.api.services.bigquery.model.TableDataList;
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.Matchers;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * Tests for util classes related to BigQuery.
- */
-@RunWith(JUnit4.class)
-public class BigQueryUtilTest {
-
- @Rule
- public ExpectedException thrown = ExpectedException.none();
-
- @Mock private Bigquery mockClient;
- @Mock private Bigquery.Tables mockTables;
- @Mock private Bigquery.Tables.Get mockTablesGet;
- @Mock private Bigquery.Tabledata mockTabledata;
- @Mock private Bigquery.Tabledata.List mockTabledataList;
- private PipelineOptions options;
-
- @Before
- public void setUp() {
- MockitoAnnotations.initMocks(this);
- this.options = PipelineOptionsFactory.create();
- }
-
- @After
- public void tearDown() {
- verifyNoMoreInteractions(mockClient);
- verifyNoMoreInteractions(mockTables);
- verifyNoMoreInteractions(mockTablesGet);
- verifyNoMoreInteractions(mockTabledata);
- verifyNoMoreInteractions(mockTabledataList);
- }
-
- private void onInsertAll(List<List<Long>> errorIndicesSequence) throws Exception {
- when(mockClient.tabledata())
- .thenReturn(mockTabledata);
-
- final List<TableDataInsertAllResponse> responses = new ArrayList<>();
- for (List<Long> errorIndices : errorIndicesSequence) {
- List<TableDataInsertAllResponse.InsertErrors> errors = new ArrayList<>();
- for (long i : errorIndices) {
- TableDataInsertAllResponse.InsertErrors error =
- new TableDataInsertAllResponse.InsertErrors();
- error.setIndex(i);
- }
- TableDataInsertAllResponse response = new TableDataInsertAllResponse();
- response.setInsertErrors(errors);
- responses.add(response);
- }
-
- doAnswer(
- new Answer<Bigquery.Tabledata.InsertAll>() {
- @Override
- public Bigquery.Tabledata.InsertAll answer(InvocationOnMock invocation) throws Throwable {
- Bigquery.Tabledata.InsertAll mockInsertAll = mock(Bigquery.Tabledata.InsertAll.class);
- when(mockInsertAll.execute())
- .thenReturn(responses.get(0),
- responses.subList(1, responses.size()).toArray(
- new TableDataInsertAllResponse[responses.size() - 1]));
- return mockInsertAll;
- }
- })
- .when(mockTabledata)
- .insertAll(anyString(), anyString(), anyString(), any(TableDataInsertAllRequest.class));
- }
-
- private void verifyInsertAll(int expectedRetries) throws IOException {
- verify(mockClient, times(expectedRetries)).tabledata();
- verify(mockTabledata, times(expectedRetries))
- .insertAll(anyString(), anyString(), anyString(), any(TableDataInsertAllRequest.class));
- }
-
- private void onTableGet(Table table) throws IOException {
- when(mockClient.tables())
- .thenReturn(mockTables);
- when(mockTables.get(anyString(), anyString(), anyString()))
- .thenReturn(mockTablesGet);
- when(mockTablesGet.execute())
- .thenReturn(table);
- }
-
- private void verifyTableGet() throws IOException {
- verify(mockClient).tables();
- verify(mockTables).get("project", "dataset", "table");
- verify(mockTablesGet, atLeastOnce()).execute();
- }
-
- private void onTableList(TableDataList result) throws IOException {
- when(mockClient.tabledata())
- .thenReturn(mockTabledata);
- when(mockTabledata.list(anyString(), anyString(), anyString()))
- .thenReturn(mockTabledataList);
- when(mockTabledataList.execute())
- .thenReturn(result);
- }
-
- private void verifyTabledataList() throws IOException {
- verify(mockClient, atLeastOnce()).tabledata();
- verify(mockTabledata, atLeastOnce()).list("project", "dataset", "table");
- verify(mockTabledataList, atLeastOnce()).execute();
- // Max results may be set when testing for an empty table.
- verify(mockTabledataList, atLeast(0)).setMaxResults(anyLong());
- }
-
- private Table basicTableSchema() {
- return new Table()
- .setSchema(new TableSchema()
- .setFields(Arrays.asList(
- new TableFieldSchema()
- .setName("name")
- .setType("STRING"),
- new TableFieldSchema()
- .setName("answer")
- .setType("INTEGER")
- )));
- }
-
- private Table basicTableSchemaWithTime() {
- return new Table()
- .setSchema(new TableSchema()
- .setFields(Arrays.asList(
- new TableFieldSchema()
- .setName("time")
- .setType("TIMESTAMP")
- )));
- }
-
- @Test
- public void testReadWithTime() throws IOException, InterruptedException {
- // The BigQuery JSON API returns timestamps in the following format: floating-point seconds
- // since epoch (UTC) with microsecond precision. Test that we faithfully preserve a set of
- // known values.
- TableDataList input = rawDataList(
- rawRow("1.430397296789E9"),
- rawRow("1.45206228E9"),
- rawRow("1.452062291E9"),
- rawRow("1.4520622911E9"),
- rawRow("1.45206229112E9"),
- rawRow("1.452062291123E9"),
- rawRow("1.4520622911234E9"),
- rawRow("1.45206229112345E9"),
- rawRow("1.452062291123456E9"));
- onTableGet(basicTableSchemaWithTime());
- onTableList(input);
-
- // Known results verified from BigQuery's export to JSON on GCS API.
- List<String> expected = ImmutableList.of(
- "2015-04-30 12:34:56.789 UTC",
- "2016-01-06 06:38:00 UTC",
- "2016-01-06 06:38:11 UTC",
- "2016-01-06 06:38:11.1 UTC",
- "2016-01-06 06:38:11.12 UTC",
- "2016-01-06 06:38:11.123 UTC",
- "2016-01-06 06:38:11.1234 UTC",
- "2016-01-06 06:38:11.12345 UTC",
- "2016-01-06 06:38:11.123456 UTC");
-
- // Download the rows, verify the interactions.
- List<TableRow> rows = new ArrayList<>();
- try (BigQueryTableRowIterator iterator =
- BigQueryTableRowIterator.fromTable(
- BigQueryIO.parseTableSpec("project:dataset.table"), mockClient)) {
- iterator.open();
- while (iterator.advance()) {
- rows.add(iterator.getCurrent());
- }
- }
- verifyTableGet();
- verifyTabledataList();
-
- // Verify the timestamp converted as desired.
- assertEquals("Expected input and output rows to have the same size",
- expected.size(), rows.size());
- for (int i = 0; i < expected.size(); ++i) {
- assertEquals("i=" + i, expected.get(i), rows.get(i).get("time"));
- }
-
- }
-
- private TableRow rawRow(Object...args) {
- List<TableCell> cells = new LinkedList<>();
- for (Object a : args) {
- cells.add(new TableCell().setV(a));
- }
- return new TableRow().setF(cells);
- }
-
- private TableDataList rawDataList(TableRow...rows) {
- return new TableDataList()
- .setRows(Arrays.asList(rows));
- }
-
- @Test
- public void testRead() throws IOException, InterruptedException {
- onTableGet(basicTableSchema());
-
- TableDataList dataList = rawDataList(rawRow("Arthur", 42));
- onTableList(dataList);
-
- try (BigQueryTableRowIterator iterator = BigQueryTableRowIterator.fromTable(
- BigQueryIO.parseTableSpec("project:dataset.table"),
- mockClient)) {
- iterator.open();
- Assert.assertTrue(iterator.advance());
- TableRow row = iterator.getCurrent();
-
- Assert.assertTrue(row.containsKey("name"));
- Assert.assertTrue(row.containsKey("answer"));
- Assert.assertEquals("Arthur", row.get("name"));
- Assert.assertEquals(42, row.get("answer"));
-
- Assert.assertFalse(iterator.advance());
-
- verifyTableGet();
- verifyTabledataList();
- }
- }
-
- @Test
- public void testReadEmpty() throws IOException, InterruptedException {
- onTableGet(basicTableSchema());
-
- // BigQuery may respond with a page token for an empty table, ensure we
- // handle it.
- TableDataList dataList = new TableDataList()
- .setPageToken("FEED==")
- .setTotalRows(0L);
- onTableList(dataList);
-
- try (BigQueryTableRowIterator iterator = BigQueryTableRowIterator.fromTable(
- BigQueryIO.parseTableSpec("project:dataset.table"),
- mockClient)) {
- iterator.open();
-
- Assert.assertFalse(iterator.advance());
-
- verifyTableGet();
- verifyTabledataList();
- }
- }
-
- @Test
- public void testReadMultiPage() throws IOException, InterruptedException {
- onTableGet(basicTableSchema());
-
- TableDataList page1 = rawDataList(rawRow("Row1", 1))
- .setPageToken("page2");
- TableDataList page2 = rawDataList(rawRow("Row2", 2))
- .setTotalRows(2L);
-
- when(mockClient.tabledata())
- .thenReturn(mockTabledata);
- when(mockTabledata.list(anyString(), anyString(), anyString()))
- .thenReturn(mockTabledataList);
- when(mockTabledataList.execute())
- .thenReturn(page1)
- .thenReturn(page2);
-
- try (BigQueryTableRowIterator iterator = BigQueryTableRowIterator.fromTable(
- BigQueryIO.parseTableSpec("project:dataset.table"),
- mockClient)) {
- iterator.open();
-
- List<String> names = new LinkedList<>();
- while (iterator.advance()) {
- names.add((String) iterator.getCurrent().get("name"));
- }
-
- Assert.assertThat(names, Matchers.hasItems("Row1", "Row2"));
-
- verifyTableGet();
- verifyTabledataList();
- // The second call should have used a page token.
- verify(mockTabledataList).setPageToken("page2");
- }
- }
-
- @Test
- public void testReadOpenFailure() throws IOException, InterruptedException {
- thrown.expect(IOException.class);
-
- when(mockClient.tables())
- .thenReturn(mockTables);
- when(mockTables.get(anyString(), anyString(), anyString()))
- .thenReturn(mockTablesGet);
- when(mockTablesGet.execute())
- .thenThrow(new IOException("No such table"));
-
- try (BigQueryTableRowIterator iterator = BigQueryTableRowIterator.fromTable(
- BigQueryIO.parseTableSpec("project:dataset.table"),
- mockClient)) {
- try {
- iterator.open(); // throws.
- } finally {
- verifyTableGet();
- }
- }
- }
-
- @Test
- public void testWriteAppend() throws IOException {
- onTableGet(basicTableSchema());
-
- TableReference ref = BigQueryIO
- .parseTableSpec("project:dataset.table");
-
- BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options);
-
- inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_APPEND,
- BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null);
-
- verifyTableGet();
- }
-
- @Test
- public void testWriteEmpty() throws IOException {
- onTableGet(basicTableSchema());
-
- TableDataList dataList = new TableDataList().setTotalRows(0L);
- onTableList(dataList);
-
- TableReference ref = BigQueryIO
- .parseTableSpec("project:dataset.table");
-
- BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options);
-
- inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_EMPTY,
- BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null);
-
- verifyTableGet();
- verifyTabledataList();
- }
-
- @Test
- public void testWriteEmptyFail() throws IOException {
- thrown.expect(IOException.class);
-
- onTableGet(basicTableSchema());
-
- TableDataList dataList = rawDataList(rawRow("Arthur", 42));
- onTableList(dataList);
-
- TableReference ref = BigQueryIO
- .parseTableSpec("project:dataset.table");
-
- BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options);
-
- try {
- inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_EMPTY,
- BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null);
- } finally {
- verifyTableGet();
- verifyTabledataList();
- }
- }
-
- @Test
- public void testInsertAll() throws Exception, IOException {
- // Build up a list of indices to fail on each invocation. This should result in
- // 5 calls to insertAll.
- List<List<Long>> errorsIndices = new ArrayList<>();
- errorsIndices.add(Arrays.asList(0L, 5L, 10L, 15L, 20L));
- errorsIndices.add(Arrays.asList(0L, 2L, 4L));
- errorsIndices.add(Arrays.asList(0L, 2L));
- errorsIndices.add(new ArrayList<Long>());
- onInsertAll(errorsIndices);
-
- TableReference ref = BigQueryIO
- .parseTableSpec("project:dataset.table");
- BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options, 5);
-
- List<TableRow> rows = new ArrayList<>();
- List<String> ids = new ArrayList<>();
- for (int i = 0; i < 25; ++i) {
- rows.add(rawRow("foo", 1234));
- ids.add(new String());
- }
-
- InMemoryLongSumAggregator byteCountAggregator = new InMemoryLongSumAggregator("ByteCount");
- try {
- inserter.insertAll(ref, rows, ids, byteCountAggregator);
- } finally {
- verifyInsertAll(5);
- // Each of the 25 rows is 23 bytes: "{f=[{v=foo}, {v=1234}]}"
- assertEquals("Incorrect byte count", 25L * 23L, byteCountAggregator.getSum());
- }
- }
-
- private static class InMemoryLongSumAggregator implements Aggregator<Long, Long> {
- private final String name;
- private long sum = 0;
-
- public InMemoryLongSumAggregator(String name) {
- this.name = name;
- }
-
- @Override
- public void addValue(Long value) {
- sum += value;
- }
-
- @Override
- public String getName() {
- return name;
- }
-
- @Override
- public CombineFn<Long, ?, Long> getCombineFn() {
- return new Sum.SumLongFn();
- }
-
- public long getSum() {
- return sum;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
index 83ffaa1..91d74db 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
@@ -30,8 +30,6 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
import com.google.api.client.auth.oauth2.Credential;
import com.google.api.client.http.HttpRequest;
import com.google.api.client.http.HttpResponse;
@@ -46,11 +44,8 @@ import com.google.api.client.testing.http.MockHttpTransport;
import com.google.api.client.testing.http.MockLowLevelHttpRequest;
import com.google.api.client.util.NanoClock;
import com.google.api.client.util.Sleeper;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
import com.google.api.services.storage.Storage;
-import com.google.common.collect.ImmutableList;
+import com.google.api.services.storage.Storage.Objects.Get;
import org.hamcrest.Matchers;
import org.junit.After;
@@ -279,21 +274,18 @@ public class RetryHttpRequestInitializerTest {
}
}).build();
- // A sample HTTP request to BigQuery that uses both default Transport and default
+ // A sample HTTP request to Google Cloud Storage that uses both default Transport and default
// RetryHttpInitializer.
- Bigquery b = new Bigquery.Builder(
+ Storage storage = new Storage.Builder(
transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer()).build();
- BigQueryTableInserter inserter = new BigQueryTableInserter(b, PipelineOptionsFactory.create());
- TableReference t = new TableReference()
- .setProjectId("project").setDatasetId("dataset").setTableId("table");
+ Get getRequest = storage.objects().get("gs://fake", "file");
try {
- inserter.insertAll(t, ImmutableList.of(new TableRow()));
+ getRequest.execute();
fail();
} catch (Throwable e) {
- assertThat(e, Matchers.<Throwable>instanceOf(RuntimeException.class));
- assertThat(e.getCause(), Matchers.<Throwable>instanceOf(SocketTimeoutException.class));
+ assertThat(e, Matchers.<Throwable>instanceOf(SocketTimeoutException.class));
assertEquals(1 + defaultNumberOfRetries, executeCount.get());
}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml
index b86f3b9..4ee83cf 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -107,6 +107,21 @@
<!-- Build dependencies -->
<dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-annotations</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>com.google.apis</groupId>
+ <artifactId>google-api-services-bigquery</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>com.google.cloud.bigdataoss</groupId>
+ <artifactId>util</artifactId>
+ </dependency>
+
+ <dependency>
<groupId>com.google.cloud.datastore</groupId>
<artifactId>datastore-v1beta3-proto-client</artifactId>
</dependency>
@@ -122,6 +137,11 @@
</dependency>
<dependency>
+ <groupId>joda-time</groupId>
+ <artifactId>joda-time</artifactId>
+ </dependency>
+
+ <dependency>
<groupId>com.google.cloud.bigtable</groupId>
<artifactId>bigtable-protos</artifactId>
<version>${bigtable.version}</version>
@@ -146,11 +166,21 @@
</dependency>
<dependency>
+ <groupId>com.google.api-client</groupId>
+ <artifactId>google-api-client</artifactId>
+ </dependency>
+
+ <dependency>
<groupId>com.google.http-client</groupId>
<artifactId>google-http-client</artifactId>
</dependency>
<dependency>
+ <groupId>com.google.http-client</groupId>
+ <artifactId>google-http-client-jackson2</artifactId>
+ </dependency>
+
+ <dependency>
<groupId>com.google.oauth-client</groupId>
<artifactId>google-oauth-client</artifactId>
</dependency>
@@ -183,6 +213,11 @@
<scope>runtime</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.avro</groupId>
+ <artifactId>avro</artifactId>
+ </dependency>
+
<!-- test -->
<dependency>
<groupId>org.apache.beam</groupId>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..48e2258
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java
@@ -0,0 +1,236 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import static com.google.common.base.MoreObjects.firstNonNull;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Verify.verify;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.Schema.Type;
+import org.apache.avro.generic.GenericRecord;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.util.List;
+
+import javax.annotation.Nullable;
+
+/**
+ * A set of utilities for working with Avro files.
+ *
+ * <p>These utilities are based on the <a
+ * href="https://avro.apache.org/docs/1.8.1/spec.html">Avro 1.8.1</a> specification.
+ */
+class BigQueryAvroUtils {
+
+ /**
+ * Formats BigQuery seconds-since-epoch into String matching JSON export. Thread-safe and
+ * immutable.
+ */
+ private static final DateTimeFormatter DATE_AND_SECONDS_FORMATTER =
+ DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").withZoneUTC();
+ // Package private for BigQueryTableRowIterator to use.
+ static String formatTimestamp(String timestamp) {
+ // timestamp is in "seconds since epoch" format, with scientific notation.
+ // e.g., "1.45206229112345E9" to mean "2016-01-06 06:38:11.123456 UTC".
+ // Separate into seconds and microseconds.
+ double timestampDoubleMicros = Double.parseDouble(timestamp) * 1000000;
+ long timestampMicros = (long) timestampDoubleMicros;
+ long seconds = timestampMicros / 1000000;
+ int micros = (int) (timestampMicros % 1000000);
+ String dayAndTime = DATE_AND_SECONDS_FORMATTER.print(seconds * 1000);
+
+ // No sub-second component.
+ if (micros == 0) {
+ return String.format("%s UTC", dayAndTime);
+ }
+
+ // Sub-second component.
+ int digits = 6;
+ int subsecond = micros;
+ while (subsecond % 10 == 0) {
+ digits--;
+ subsecond /= 10;
+ }
+ String formatString = String.format("%%0%dd", digits);
+ String fractionalSeconds = String.format(formatString, subsecond);
+ return String.format("%s.%s UTC", dayAndTime, fractionalSeconds);
+ }
+
+ /**
+ * Utility function to convert from an Avro {@link GenericRecord} to a BigQuery {@link TableRow}.
+ *
+ * See <a href="https://cloud.google.com/bigquery/exporting-data-from-bigquery#config">
+ * "Avro format"</a> for more information.
+ */
+ static TableRow convertGenericRecordToTableRow(GenericRecord record, TableSchema schema) {
+ return convertGenericRecordToTableRow(record, schema.getFields());
+ }
+
+ private static TableRow convertGenericRecordToTableRow(
+ GenericRecord record, List<TableFieldSchema> fields) {
+ TableRow row = new TableRow();
+ for (TableFieldSchema subSchema : fields) {
+ // Per https://cloud.google.com/bigquery/docs/reference/v2/tables#schema, the name field
+ // is required, so it may not be null.
+ Field field = record.getSchema().getField(subSchema.getName());
+ Object convertedValue =
+ getTypedCellValue(field.schema(), subSchema, record.get(field.name()));
+ if (convertedValue != null) {
+ // To match the JSON files exported by BigQuery, do not include null values in the output.
+ row.set(field.name(), convertedValue);
+ }
+ }
+ return row;
+ }
+
+ @Nullable
+ private static Object getTypedCellValue(Schema schema, TableFieldSchema fieldSchema, Object v) {
+ // Per https://cloud.google.com/bigquery/docs/reference/v2/tables#schema, the mode field
+ // is optional (and so it may be null), but defaults to "NULLABLE".
+ String mode = firstNonNull(fieldSchema.getMode(), "NULLABLE");
+ switch (mode) {
+ case "REQUIRED":
+ return convertRequiredField(schema.getType(), fieldSchema, v);
+ case "REPEATED":
+ return convertRepeatedField(schema, fieldSchema, v);
+ case "NULLABLE":
+ return convertNullableField(schema, fieldSchema, v);
+ default:
+ throw new UnsupportedOperationException(
+ "Parsing a field with BigQuery field schema mode " + fieldSchema.getMode());
+ }
+ }
+
+ private static List<Object> convertRepeatedField(
+ Schema schema, TableFieldSchema fieldSchema, Object v) {
+ Type arrayType = schema.getType();
+ verify(
+ arrayType == Type.ARRAY,
+ "BigQuery REPEATED field %s should be Avro ARRAY, not %s",
+ fieldSchema.getName(),
+ arrayType);
+ // REPEATED fields are represented as Avro arrays.
+ if (v == null) {
+ // Handle the case of an empty repeated field.
+ return ImmutableList.of();
+ }
+ @SuppressWarnings("unchecked")
+ List<Object> elements = (List<Object>) v;
+ ImmutableList.Builder<Object> values = ImmutableList.builder();
+ Type elementType = schema.getElementType().getType();
+ for (Object element : elements) {
+ values.add(convertRequiredField(elementType, fieldSchema, element));
+ }
+ return values.build();
+ }
+
+ private static Object convertRequiredField(
+ Type avroType, TableFieldSchema fieldSchema, Object v) {
+ // REQUIRED fields are represented as the corresponding Avro types. For example, a BigQuery
+ // INTEGER type maps to an Avro LONG type.
+ checkNotNull(v, "REQUIRED field %s should not be null", fieldSchema.getName());
+ ImmutableMap<String, Type> fieldMap =
+ ImmutableMap.<String, Type>builder()
+ .put("STRING", Type.STRING)
+ .put("INTEGER", Type.LONG)
+ .put("FLOAT", Type.DOUBLE)
+ .put("BOOLEAN", Type.BOOLEAN)
+ .put("TIMESTAMP", Type.LONG)
+ .put("RECORD", Type.RECORD)
+ .build();
+ // Per https://cloud.google.com/bigquery/docs/reference/v2/tables#schema, the type field
+ // is required, so it may not be null.
+ String bqType = fieldSchema.getType();
+ Type expectedAvroType = fieldMap.get(bqType);
+ verify(
+ avroType == expectedAvroType,
+ "Expected Avro schema type %s, not %s, for BigQuery %s field %s",
+ expectedAvroType,
+ avroType,
+ bqType,
+ fieldSchema.getName());
+ switch (fieldSchema.getType()) {
+ case "STRING":
+ // Avro will use a CharSequence to represent String objects, but it may not always use
+ // java.lang.String; for example, it may prefer org.apache.avro.util.Utf8.
+ verify(v instanceof CharSequence, "Expected CharSequence (String), got %s", v.getClass());
+ return v.toString();
+ case "INTEGER":
+ verify(v instanceof Long, "Expected Long, got %s", v.getClass());
+ return ((Long) v).toString();
+ case "FLOAT":
+ verify(v instanceof Double, "Expected Double, got %s", v.getClass());
+ return v;
+ case "BOOLEAN":
+ verify(v instanceof Boolean, "Expected Boolean, got %s", v.getClass());
+ return v;
+ case "TIMESTAMP":
+ // TIMESTAMP data types are represented as Avro LONG types. They are converted back to
+ // Strings with variable-precision (up to six digits) to match the JSON files export
+ // by BigQuery.
+ verify(v instanceof Long, "Expected Long, got %s", v.getClass());
+ Double doubleValue = ((Long) v) / 1000000.0;
+ return formatTimestamp(doubleValue.toString());
+ case "RECORD":
+ verify(v instanceof GenericRecord, "Expected GenericRecord, got %s", v.getClass());
+ return convertGenericRecordToTableRow((GenericRecord) v, fieldSchema.getFields());
+ default:
+ throw new UnsupportedOperationException(
+ String.format(
+ "Unexpected BigQuery field schema type %s for field named %s",
+ fieldSchema.getType(),
+ fieldSchema.getName()));
+ }
+ }
+
+ @Nullable
+ private static Object convertNullableField(
+ Schema avroSchema, TableFieldSchema fieldSchema, Object v) {
+ // NULLABLE fields are represented as an Avro Union of the corresponding type and "null".
+ verify(
+ avroSchema.getType() == Type.UNION,
+ "Expected Avro schema type UNION, not %s, for BigQuery NULLABLE field %s",
+ avroSchema.getType(),
+ fieldSchema.getName());
+ List<Schema> unionTypes = avroSchema.getTypes();
+ verify(
+ unionTypes.size() == 2,
+ "BigQuery NULLABLE field %s should be an Avro UNION of NULL and another type, not %s",
+ fieldSchema.getName(),
+ unionTypes);
+
+ if (v == null) {
+ return null;
+ }
+
+ Type firstType = unionTypes.get(0).getType();
+ if (!firstType.equals(Type.NULL)) {
+ return convertRequiredField(firstType, fieldSchema, v);
+ }
+ return convertRequiredField(unionTypes.get(1).getType(), fieldSchema, v);
+ }
+}
[02/10] incubator-beam git commit: BigQueryIO: move to
google-cloud-platform module
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
new file mode 100644
index 0000000..00e7891
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
@@ -0,0 +1,1231 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.fromJsonString;
+import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.toJsonString;
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.hamcrest.Matchers.hasItem;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.TableRowJsonCoder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.BigQueryQuerySource;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.BigQueryTableSource;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.PassThroughThenCleanup;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.PassThroughThenCleanup.CleanupOperation;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Status;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TransformingSource;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService;
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.StreamingOptions;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.RunnableOnService;
+import org.apache.beam.sdk.testing.SourceTestUtils;
+import org.apache.beam.sdk.testing.SourceTestUtils.ExpectedSplitOutcome;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.IOChannelFactory;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.api.client.util.Data;
+import com.google.api.client.util.Strings;
+import com.google.api.services.bigquery.model.ErrorProto;
+import com.google.api.services.bigquery.model.Job;
+import com.google.api.services.bigquery.model.JobConfigurationExtract;
+import com.google.api.services.bigquery.model.JobConfigurationLoad;
+import com.google.api.services.bigquery.model.JobConfigurationQuery;
+import com.google.api.services.bigquery.model.JobReference;
+import com.google.api.services.bigquery.model.JobStatistics;
+import com.google.api.services.bigquery.model.JobStatistics2;
+import com.google.api.services.bigquery.model.JobStatistics4;
+import com.google.api.services.bigquery.model.JobStatus;
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+
+import org.hamcrest.CoreMatchers;
+import org.hamcrest.Matchers;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+import javax.annotation.Nullable;
+
+/**
+ * Tests for BigQueryIO.
+ */
+@RunWith(JUnit4.class)
+public class BigQueryIOTest implements Serializable {
+
+ // Status.UNKNOWN maps to null
+ private static final Map<Status, Job> JOB_STATUS_MAP = ImmutableMap.of(
+ Status.SUCCEEDED, new Job().setStatus(new JobStatus()),
+ Status.FAILED, new Job().setStatus(new JobStatus().setErrorResult(new ErrorProto())));
+
+ private static class FakeBigQueryServices implements BigQueryServices {
+
+ private String[] jsonTableRowReturns = new String[0];
+ private JobService jobService;
+ private DatasetService datasetService;
+
+ public FakeBigQueryServices withJobService(JobService jobService) {
+ this.jobService = jobService;
+ return this;
+ }
+
+ public FakeBigQueryServices withDatasetService(DatasetService datasetService) {
+ this.datasetService = datasetService;
+ return this;
+ }
+
+ public FakeBigQueryServices readerReturns(String... jsonTableRowReturns) {
+ this.jsonTableRowReturns = jsonTableRowReturns;
+ return this;
+ }
+
+ @Override
+ public JobService getJobService(BigQueryOptions bqOptions) {
+ return jobService;
+ }
+
+ @Override
+ public DatasetService getDatasetService(BigQueryOptions bqOptions) {
+ return datasetService;
+ }
+
+ @Override
+ public BigQueryJsonReader getReaderFromTable(
+ BigQueryOptions bqOptions, TableReference tableRef) {
+ return new FakeBigQueryReader(jsonTableRowReturns);
+ }
+
+ @Override
+ public BigQueryJsonReader getReaderFromQuery(
+ BigQueryOptions bqOptions, String query, String projectId, @Nullable Boolean flatten) {
+ return new FakeBigQueryReader(jsonTableRowReturns);
+ }
+
+ private static class FakeBigQueryReader implements BigQueryJsonReader {
+ private static final int UNSTARTED = -1;
+ private static final int CLOSED = Integer.MAX_VALUE;
+
+ private String[] jsonTableRowReturns;
+ private int currIndex;
+
+ FakeBigQueryReader(String[] jsonTableRowReturns) {
+ this.jsonTableRowReturns = jsonTableRowReturns;
+ this.currIndex = UNSTARTED;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ assertEquals(UNSTARTED, currIndex);
+ currIndex = 0;
+ return currIndex < jsonTableRowReturns.length;
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ return ++currIndex < jsonTableRowReturns.length;
+ }
+
+ @Override
+ public TableRow getCurrent() throws NoSuchElementException {
+ if (currIndex >= jsonTableRowReturns.length) {
+ throw new NoSuchElementException();
+ }
+ return fromJsonString(jsonTableRowReturns[currIndex], TableRow.class);
+ }
+
+ @Override
+ public void close() throws IOException {
+ currIndex = CLOSED;
+ }
+ }
+ }
+
+ private static class FakeJobService implements JobService, Serializable {
+
+ private Object[] startJobReturns;
+ private Object[] pollJobReturns;
+ private String executingProject;
+ // Both counts will be reset back to zeros after serialization.
+ // This is a work around for DoFn's verifyUnmodified check.
+ private transient int startJobCallsCount;
+ private transient int pollJobStatusCallsCount;
+
+ public FakeJobService() {
+ this.startJobReturns = new Object[0];
+ this.pollJobReturns = new Object[0];
+ this.startJobCallsCount = 0;
+ this.pollJobStatusCallsCount = 0;
+ }
+
+ /**
+ * Sets the return values to mock {@link JobService#startLoadJob},
+ * {@link JobService#startExtractJob} and {@link JobService#startQueryJob}.
+ *
+ * <p>Throws if the {@link Object} is a {@link Exception}, returns otherwise.
+ */
+ public FakeJobService startJobReturns(Object... startJobReturns) {
+ this.startJobReturns = startJobReturns;
+ return this;
+ }
+
+ /**
+ * Sets the return values to mock {@link JobService#pollJob}.
+ *
+ * <p>Throws if the {@link Object} is a {@link Exception}, returns otherwise.
+ */
+ public FakeJobService pollJobReturns(Object... pollJobReturns) {
+ this.pollJobReturns = pollJobReturns;
+ return this;
+ }
+
+ /**
+ * Verifies executing project.
+ */
+ public FakeJobService verifyExecutingProject(String executingProject) {
+ this.executingProject = executingProject;
+ return this;
+ }
+
+ @Override
+ public void startLoadJob(JobReference jobRef, JobConfigurationLoad loadConfig)
+ throws InterruptedException, IOException {
+ startJob(jobRef);
+ }
+
+ @Override
+ public void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig)
+ throws InterruptedException, IOException {
+ startJob(jobRef);
+ }
+
+ @Override
+ public void startQueryJob(JobReference jobRef, JobConfigurationQuery query)
+ throws IOException, InterruptedException {
+ startJob(jobRef);
+ }
+
+ @Override
+ public Job pollJob(JobReference jobRef, int maxAttempts)
+ throws InterruptedException {
+ if (!Strings.isNullOrEmpty(executingProject)) {
+ checkArgument(
+ jobRef.getProjectId().equals(executingProject),
+ "Project id: %s is not equal to executing project: %s",
+ jobRef.getProjectId(), executingProject);
+ }
+
+ if (pollJobStatusCallsCount < pollJobReturns.length) {
+ Object ret = pollJobReturns[pollJobStatusCallsCount++];
+ if (ret instanceof Job) {
+ return (Job) ret;
+ } else if (ret instanceof Status) {
+ return JOB_STATUS_MAP.get(ret);
+ } else if (ret instanceof InterruptedException) {
+ throw (InterruptedException) ret;
+ } else {
+ throw new RuntimeException("Unexpected return type: " + ret.getClass());
+ }
+ } else {
+ throw new RuntimeException(
+ "Exceeded expected number of calls: " + pollJobReturns.length);
+ }
+ }
+
+ private void startJob(JobReference jobRef) throws IOException, InterruptedException {
+ if (!Strings.isNullOrEmpty(executingProject)) {
+ checkArgument(
+ jobRef.getProjectId().equals(executingProject),
+ "Project id: %s is not equal to executing project: %s",
+ jobRef.getProjectId(), executingProject);
+ }
+
+ if (startJobCallsCount < startJobReturns.length) {
+ Object ret = startJobReturns[startJobCallsCount++];
+ if (ret instanceof IOException) {
+ throw (IOException) ret;
+ } else if (ret instanceof InterruptedException) {
+ throw (InterruptedException) ret;
+ } else {
+ return;
+ }
+ } else {
+ throw new RuntimeException(
+ "Exceeded expected number of calls: " + startJobReturns.length);
+ }
+ }
+
+ @Override
+ public JobStatistics dryRunQuery(String projectId, String query)
+ throws InterruptedException, IOException {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ @Rule public transient ExpectedException thrown = ExpectedException.none();
+ @Rule public transient ExpectedLogs logged = ExpectedLogs.none(BigQueryIO.class);
+ @Rule public transient TemporaryFolder testFolder = new TemporaryFolder();
+ @Mock(extraInterfaces = Serializable.class)
+ public transient BigQueryServices.JobService mockJobService;
+ @Mock private transient IOChannelFactory mockIOChannelFactory;
+ @Mock(extraInterfaces = Serializable.class) private transient DatasetService mockDatasetService;
+
+ private transient BigQueryOptions bqOptions;
+
+ private void checkReadTableObject(
+ BigQueryIO.Read.Bound bound, String project, String dataset, String table) {
+ checkReadTableObjectWithValidate(bound, project, dataset, table, true);
+ }
+
+ private void checkReadQueryObject(BigQueryIO.Read.Bound bound, String query) {
+ checkReadQueryObjectWithValidate(bound, query, true);
+ }
+
+ private void checkReadTableObjectWithValidate(
+ BigQueryIO.Read.Bound bound, String project, String dataset, String table, boolean validate) {
+ assertEquals(project, bound.getTable().getProjectId());
+ assertEquals(dataset, bound.getTable().getDatasetId());
+ assertEquals(table, bound.getTable().getTableId());
+ assertNull(bound.query);
+ assertEquals(validate, bound.getValidate());
+ }
+
+ private void checkReadQueryObjectWithValidate(
+ BigQueryIO.Read.Bound bound, String query, boolean validate) {
+ assertNull(bound.getTable());
+ assertEquals(query, bound.query);
+ assertEquals(validate, bound.getValidate());
+ }
+
+ private void checkWriteObject(
+ BigQueryIO.Write.Bound bound, String project, String dataset, String table,
+ TableSchema schema, CreateDisposition createDisposition,
+ WriteDisposition writeDisposition) {
+ checkWriteObjectWithValidate(
+ bound, project, dataset, table, schema, createDisposition, writeDisposition, true);
+ }
+
+ private void checkWriteObjectWithValidate(
+ BigQueryIO.Write.Bound bound, String project, String dataset, String table,
+ TableSchema schema, CreateDisposition createDisposition,
+ WriteDisposition writeDisposition, boolean validate) {
+ assertEquals(project, bound.getTable().getProjectId());
+ assertEquals(dataset, bound.getTable().getDatasetId());
+ assertEquals(table, bound.getTable().getTableId());
+ assertEquals(schema, bound.getSchema());
+ assertEquals(createDisposition, bound.createDisposition);
+ assertEquals(writeDisposition, bound.writeDisposition);
+ assertEquals(validate, bound.validate);
+ }
+
+ @Before
+ public void setUp() throws IOException {
+ bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
+ bqOptions.setProject("defaultProject");
+ bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath());
+
+ MockitoAnnotations.initMocks(this);
+ }
+
+ @Test
+ public void testBuildTableBasedSource() {
+ BigQueryIO.Read.Bound bound = BigQueryIO.Read.from("foo.com:project:somedataset.sometable");
+ checkReadTableObject(bound, "foo.com:project", "somedataset", "sometable");
+ }
+
+ @Test
+ public void testBuildQueryBasedSource() {
+ BigQueryIO.Read.Bound bound = BigQueryIO.Read.fromQuery("foo_query");
+ checkReadQueryObject(bound, "foo_query");
+ }
+
+ @Test
+ public void testBuildTableBasedSourceWithoutValidation() {
+ // This test just checks that using withoutValidation will not trigger object
+ // construction errors.
+ BigQueryIO.Read.Bound bound =
+ BigQueryIO.Read.from("foo.com:project:somedataset.sometable").withoutValidation();
+ checkReadTableObjectWithValidate(bound, "foo.com:project", "somedataset", "sometable", false);
+ }
+
+ @Test
+ public void testBuildQueryBasedSourceWithoutValidation() {
+ // This test just checks that using withoutValidation will not trigger object
+ // construction errors.
+ BigQueryIO.Read.Bound bound =
+ BigQueryIO.Read.fromQuery("some_query").withoutValidation();
+ checkReadQueryObjectWithValidate(bound, "some_query", false);
+ }
+
+ @Test
+ public void testBuildTableBasedSourceWithDefaultProject() {
+ BigQueryIO.Read.Bound bound =
+ BigQueryIO.Read.from("somedataset.sometable");
+ checkReadTableObject(bound, null, "somedataset", "sometable");
+ }
+
+ @Test
+ public void testBuildSourceWithTableReference() {
+ TableReference table = new TableReference()
+ .setProjectId("foo.com:project")
+ .setDatasetId("somedataset")
+ .setTableId("sometable");
+ BigQueryIO.Read.Bound bound = BigQueryIO.Read.from(table);
+ checkReadTableObject(bound, "foo.com:project", "somedataset", "sometable");
+ }
+
+ @Test
+ public void testValidateReadSetsDefaultProject() throws Exception {
+ String projectId = "someproject";
+ String datasetId = "somedataset";
+ BigQueryOptions options = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
+ options.setProject(projectId);
+
+ FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+ .withJobService(mockJobService)
+ .withDatasetService(mockDatasetService);
+ when(mockDatasetService.getDataset(projectId, datasetId)).thenThrow(
+ new RuntimeException("Unable to confirm BigQuery dataset presence"));
+
+ Pipeline p = TestPipeline.create(options);
+
+ TableReference tableRef = new TableReference();
+ tableRef.setDatasetId(datasetId);
+ tableRef.setTableId("sometable");
+
+ thrown.expect(RuntimeException.class);
+ // Message will be one of following depending on the execution environment.
+ thrown.expectMessage(
+ Matchers.either(Matchers.containsString("Unable to confirm BigQuery dataset presence"))
+ .or(Matchers.containsString("BigQuery dataset not found for table")));
+ p.apply(BigQueryIO.Read.from(tableRef)
+ .withTestServices(fakeBqServices));
+ }
+
+ @Test
+ @Category(RunnableOnService.class)
+ public void testBuildSourceWithoutTableQueryOrValidation() {
+ Pipeline p = TestPipeline.create();
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage(
+ "Invalid BigQuery read operation, either table reference or query has to be set");
+ p.apply(BigQueryIO.Read.withoutValidation());
+ p.run();
+ }
+
+ @Test
+ @Category(RunnableOnService.class)
+ public void testBuildSourceWithTableAndQuery() {
+ Pipeline p = TestPipeline.create();
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage(
+ "Invalid BigQuery read operation. Specifies both a query and a table, only one of these"
+ + " should be provided");
+ p.apply("ReadMyTable",
+ BigQueryIO.Read
+ .from("foo.com:project:somedataset.sometable")
+ .fromQuery("query"));
+ p.run();
+ }
+
+ @Test
+ @Category(RunnableOnService.class)
+ public void testBuildSourceWithTableAndFlatten() {
+ Pipeline p = TestPipeline.create();
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage(
+ "Invalid BigQuery read operation. Specifies a"
+ + " table with a result flattening preference, which is not configurable");
+ p.apply("ReadMyTable",
+ BigQueryIO.Read
+ .from("foo.com:project:somedataset.sometable")
+ .withoutResultFlattening());
+ p.run();
+ }
+
+ @Test
+ @Category(RunnableOnService.class)
+ public void testBuildSourceWithTableAndFlattenWithoutValidation() {
+ Pipeline p = TestPipeline.create();
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage(
+ "Invalid BigQuery read operation. Specifies a"
+ + " table with a result flattening preference, which is not configurable");
+ p.apply(
+ BigQueryIO.Read
+ .from("foo.com:project:somedataset.sometable")
+ .withoutValidation()
+ .withoutResultFlattening());
+ p.run();
+ }
+
+ @Test
+ @Category(NeedsRunner.class)
+ public void testReadFromTable() {
+ FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+ .withJobService(new FakeJobService()
+ .startJobReturns("done", "done")
+ .pollJobReturns(Status.UNKNOWN)
+ .verifyExecutingProject(bqOptions.getProject()))
+ .readerReturns(
+ toJsonString(new TableRow().set("name", "a").set("number", 1)),
+ toJsonString(new TableRow().set("name", "b").set("number", 2)),
+ toJsonString(new TableRow().set("name", "c").set("number", 3)));
+
+ Pipeline p = TestPipeline.create(bqOptions);
+ PCollection<String> output = p
+ .apply(BigQueryIO.Read.from("non-executing-project:somedataset.sometable")
+ .withTestServices(fakeBqServices)
+ .withoutValidation())
+ .apply(ParDo.of(new DoFn<TableRow, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ c.output((String) c.element().get("name"));
+ }
+ }));
+
+ PAssert.that(output)
+ .containsInAnyOrder(ImmutableList.of("a", "b", "c"));
+
+ p.run();
+ }
+
+ @Test
+ @Category(NeedsRunner.class)
+ public void testCustomSink() throws Exception {
+ FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+ .withJobService(new FakeJobService()
+ .startJobReturns("done", "done", "done")
+ .pollJobReturns(Status.FAILED, Status.FAILED, Status.SUCCEEDED));
+
+ Pipeline p = TestPipeline.create(bqOptions);
+ p.apply(Create.of(
+ new TableRow().set("name", "a").set("number", 1),
+ new TableRow().set("name", "b").set("number", 2),
+ new TableRow().set("name", "c").set("number", 3))
+ .withCoder(TableRowJsonCoder.of()))
+ .apply(BigQueryIO.Write.to("dataset-id.table-id")
+ .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+ .withSchema(new TableSchema().setFields(
+ ImmutableList.of(
+ new TableFieldSchema().setName("name").setType("STRING"),
+ new TableFieldSchema().setName("number").setType("INTEGER"))))
+ .withTestServices(fakeBqServices)
+ .withoutValidation());
+ p.run();
+
+ logged.verifyInfo("Starting BigQuery load job");
+ logged.verifyInfo("Previous load jobs failed, retrying.");
+ File tempDir = new File(bqOptions.getTempLocation());
+ assertEquals(0, tempDir.listFiles(new FileFilter() {
+ @Override
+ public boolean accept(File pathname) {
+ return pathname.isFile();
+ }}).length);
+ }
+
+ @Test
+ @Category(NeedsRunner.class)
+ public void testCustomSinkUnknown() throws Exception {
+ FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+ .withJobService(new FakeJobService()
+ .startJobReturns("done", "done")
+ .pollJobReturns(Status.FAILED, Status.UNKNOWN));
+
+ Pipeline p = TestPipeline.create(bqOptions);
+ p.apply(Create.of(
+ new TableRow().set("name", "a").set("number", 1),
+ new TableRow().set("name", "b").set("number", 2),
+ new TableRow().set("name", "c").set("number", 3))
+ .withCoder(TableRowJsonCoder.of()))
+ .apply(BigQueryIO.Write.to("project-id:dataset-id.table-id")
+ .withCreateDisposition(CreateDisposition.CREATE_NEVER)
+ .withTestServices(fakeBqServices)
+ .withoutValidation());
+
+ thrown.expect(RuntimeException.class);
+ thrown.expectMessage("Failed to poll the load job status.");
+ p.run();
+
+ File tempDir = new File(bqOptions.getTempLocation());
+ assertEquals(0, tempDir.listFiles(new FileFilter() {
+ @Override
+ public boolean accept(File pathname) {
+ return pathname.isFile();
+ }}).length);
+ }
+
+ @Test
+ public void testBuildSourceDisplayData() {
+ String tableSpec = "project:dataset.tableid";
+
+ BigQueryIO.Read.Bound read = BigQueryIO.Read
+ .from(tableSpec)
+ .fromQuery("myQuery")
+ .withoutResultFlattening()
+ .withoutValidation();
+
+ DisplayData displayData = DisplayData.from(read);
+
+ assertThat(displayData, hasDisplayItem("table", tableSpec));
+ assertThat(displayData, hasDisplayItem("query", "myQuery"));
+ assertThat(displayData, hasDisplayItem("flattenResults", false));
+ assertThat(displayData, hasDisplayItem("validation", false));
+ }
+
+ @Test
+ @Category(RunnableOnService.class)
+ @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient")
+ public void testTableSourcePrimitiveDisplayData() throws IOException, InterruptedException {
+ DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
+ BigQueryIO.Read.Bound read = BigQueryIO.Read
+ .from("project:dataset.tableId")
+ .withTestServices(new FakeBigQueryServices()
+ .withDatasetService(mockDatasetService)
+ .withJobService(mockJobService))
+ .withoutValidation();
+
+ Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
+ assertThat("BigQueryIO.Read should include the table spec in its primitive display data",
+ displayData, hasItem(hasDisplayItem("table")));
+ }
+
+ @Test
+ @Category(RunnableOnService.class)
+ @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient")
+ public void testQuerySourcePrimitiveDisplayData() throws IOException, InterruptedException {
+ DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
+ BigQueryIO.Read.Bound read = BigQueryIO.Read
+ .fromQuery("foobar")
+ .withTestServices(new FakeBigQueryServices()
+ .withDatasetService(mockDatasetService)
+ .withJobService(mockJobService))
+ .withoutValidation();
+
+ Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
+ assertThat("BigQueryIO.Read should include the query in its primitive display data",
+ displayData, hasItem(hasDisplayItem("query")));
+ }
+
+
+ @Test
+ public void testBuildSink() {
+ BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("foo.com:project:somedataset.sometable");
+ checkWriteObject(
+ bound, "foo.com:project", "somedataset", "sometable",
+ null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
+ }
+
+ @Test
+ @Category(RunnableOnService.class)
+ @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient")
+ public void testBatchSinkPrimitiveDisplayData() throws IOException, InterruptedException {
+ testSinkPrimitiveDisplayData(/* streaming: */ false);
+ }
+
+ @Test
+ @Category(RunnableOnService.class)
+ @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient")
+ public void testStreamingSinkPrimitiveDisplayData() throws IOException, InterruptedException {
+ testSinkPrimitiveDisplayData(/* streaming: */ true);
+ }
+
+ private void testSinkPrimitiveDisplayData(boolean streaming) throws IOException,
+ InterruptedException {
+ PipelineOptions options = TestPipeline.testingPipelineOptions();
+ options.as(StreamingOptions.class).setStreaming(streaming);
+ DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(options);
+
+ BigQueryIO.Write.Bound write = BigQueryIO.Write
+ .to("project:dataset.table")
+ .withSchema(new TableSchema().set("col1", "type1").set("col2", "type2"))
+ .withTestServices(new FakeBigQueryServices()
+ .withDatasetService(mockDatasetService)
+ .withJobService(mockJobService))
+ .withoutValidation();
+
+ Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(write);
+ assertThat("BigQueryIO.Write should include the table spec in its primitive display data",
+ displayData, hasItem(hasDisplayItem("tableSpec")));
+
+ assertThat("BigQueryIO.Write should include the table schema in its primitive display data",
+ displayData, hasItem(hasDisplayItem("schema")));
+ }
+
+ @Test
+ public void testBuildSinkwithoutValidation() {
+ // This test just checks that using withoutValidation will not trigger object
+ // construction errors.
+ BigQueryIO.Write.Bound bound =
+ BigQueryIO.Write.to("foo.com:project:somedataset.sometable").withoutValidation();
+ checkWriteObjectWithValidate(
+ bound, "foo.com:project", "somedataset", "sometable",
+ null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, false);
+ }
+
+ @Test
+ public void testBuildSinkDefaultProject() {
+ BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("somedataset.sometable");
+ checkWriteObject(
+ bound, null, "somedataset", "sometable",
+ null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
+ }
+
+ @Test
+ public void testBuildSinkWithTableReference() {
+ TableReference table = new TableReference()
+ .setProjectId("foo.com:project")
+ .setDatasetId("somedataset")
+ .setTableId("sometable");
+ BigQueryIO.Write.Bound bound = BigQueryIO.Write.to(table);
+ checkWriteObject(
+ bound, "foo.com:project", "somedataset", "sometable",
+ null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
+ }
+
+ @Test
+ @Category(RunnableOnService.class)
+ public void testBuildSinkWithoutTable() {
+ Pipeline p = TestPipeline.create();
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("must set the table reference");
+ p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()))
+ .apply(BigQueryIO.Write.withoutValidation());
+ }
+
+ @Test
+ public void testBuildSinkWithSchema() {
+ TableSchema schema = new TableSchema();
+ BigQueryIO.Write.Bound bound =
+ BigQueryIO.Write.to("foo.com:project:somedataset.sometable").withSchema(schema);
+ checkWriteObject(
+ bound, "foo.com:project", "somedataset", "sometable",
+ schema, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
+ }
+
+ @Test
+ public void testBuildSinkWithCreateDispositionNever() {
+ BigQueryIO.Write.Bound bound = BigQueryIO.Write
+ .to("foo.com:project:somedataset.sometable")
+ .withCreateDisposition(CreateDisposition.CREATE_NEVER);
+ checkWriteObject(
+ bound, "foo.com:project", "somedataset", "sometable",
+ null, CreateDisposition.CREATE_NEVER, WriteDisposition.WRITE_EMPTY);
+ }
+
+ @Test
+ public void testBuildSinkWithCreateDispositionIfNeeded() {
+ BigQueryIO.Write.Bound bound = BigQueryIO.Write
+ .to("foo.com:project:somedataset.sometable")
+ .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED);
+ checkWriteObject(
+ bound, "foo.com:project", "somedataset", "sometable",
+ null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
+ }
+
+ @Test
+ public void testBuildSinkWithWriteDispositionTruncate() {
+ BigQueryIO.Write.Bound bound = BigQueryIO.Write
+ .to("foo.com:project:somedataset.sometable")
+ .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE);
+ checkWriteObject(
+ bound, "foo.com:project", "somedataset", "sometable",
+ null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_TRUNCATE);
+ }
+
+ @Test
+ public void testBuildSinkWithWriteDispositionAppend() {
+ BigQueryIO.Write.Bound bound = BigQueryIO.Write
+ .to("foo.com:project:somedataset.sometable")
+ .withWriteDisposition(WriteDisposition.WRITE_APPEND);
+ checkWriteObject(
+ bound, "foo.com:project", "somedataset", "sometable",
+ null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_APPEND);
+ }
+
+ @Test
+ public void testBuildSinkWithWriteDispositionEmpty() {
+ BigQueryIO.Write.Bound bound = BigQueryIO.Write
+ .to("foo.com:project:somedataset.sometable")
+ .withWriteDisposition(WriteDisposition.WRITE_EMPTY);
+ checkWriteObject(
+ bound, "foo.com:project", "somedataset", "sometable",
+ null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
+ }
+
+ @Test
+ public void testBuildSinkDisplayData() {
+ String tableSpec = "project:dataset.table";
+ TableSchema schema = new TableSchema().set("col1", "type1").set("col2", "type2");
+
+ BigQueryIO.Write.Bound write = BigQueryIO.Write
+ .to(tableSpec)
+ .withSchema(schema)
+ .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+ .withWriteDisposition(WriteDisposition.WRITE_APPEND)
+ .withoutValidation();
+
+ DisplayData displayData = DisplayData.from(write);
+
+ assertThat(displayData, hasDisplayItem("table"));
+ assertThat(displayData, hasDisplayItem("schema"));
+ assertThat(displayData,
+ hasDisplayItem("createDisposition", CreateDisposition.CREATE_IF_NEEDED.toString()));
+ assertThat(displayData,
+ hasDisplayItem("writeDisposition", WriteDisposition.WRITE_APPEND.toString()));
+ assertThat(displayData, hasDisplayItem("validation", false));
+ }
+
+ private void testWriteValidatesDataset(boolean streaming) throws Exception {
+ String projectId = "someproject";
+ String datasetId = "somedataset";
+
+ BigQueryOptions options = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
+ options.setProject(projectId);
+ options.setStreaming(streaming);
+
+ FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+ .withJobService(mockJobService)
+ .withDatasetService(mockDatasetService);
+ when(mockDatasetService.getDataset(projectId, datasetId)).thenThrow(
+ new RuntimeException("Unable to confirm BigQuery dataset presence"));
+
+ Pipeline p = TestPipeline.create(options);
+
+ TableReference tableRef = new TableReference();
+ tableRef.setDatasetId(datasetId);
+ tableRef.setTableId("sometable");
+
+ thrown.expect(RuntimeException.class);
+ // Message will be one of following depending on the execution environment.
+ thrown.expectMessage(
+ Matchers.either(Matchers.containsString("Unable to confirm BigQuery dataset presence"))
+ .or(Matchers.containsString("BigQuery dataset not found for table")));
+ p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()))
+ .apply(BigQueryIO.Write
+ .to(tableRef)
+ .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+ .withSchema(new TableSchema())
+ .withTestServices(fakeBqServices));
+ }
+
+ @Test
+ public void testWriteValidatesDatasetBatch() throws Exception {
+ testWriteValidatesDataset(false);
+ }
+
+ @Test
+ public void testWriteValidatesDatasetStreaming() throws Exception {
+ testWriteValidatesDataset(true);
+ }
+
+ @Test
+ public void testTableParsing() {
+ TableReference ref = BigQueryIO
+ .parseTableSpec("my-project:data_set.table_name");
+ Assert.assertEquals("my-project", ref.getProjectId());
+ Assert.assertEquals("data_set", ref.getDatasetId());
+ Assert.assertEquals("table_name", ref.getTableId());
+ }
+
+ @Test
+ public void testTableParsing_validPatterns() {
+ BigQueryIO.parseTableSpec("a123-456:foo_bar.d");
+ BigQueryIO.parseTableSpec("a12345:b.c");
+ BigQueryIO.parseTableSpec("b12345.c");
+ }
+
+ @Test
+ public void testTableParsing_noProjectId() {
+ TableReference ref = BigQueryIO
+ .parseTableSpec("data_set.table_name");
+ Assert.assertEquals(null, ref.getProjectId());
+ Assert.assertEquals("data_set", ref.getDatasetId());
+ Assert.assertEquals("table_name", ref.getTableId());
+ }
+
+ @Test
+ public void testTableParsingError() {
+ thrown.expect(IllegalArgumentException.class);
+ BigQueryIO.parseTableSpec("0123456:foo.bar");
+ }
+
+ @Test
+ public void testTableParsingError_2() {
+ thrown.expect(IllegalArgumentException.class);
+ BigQueryIO.parseTableSpec("myproject:.bar");
+ }
+
+ @Test
+ public void testTableParsingError_3() {
+ thrown.expect(IllegalArgumentException.class);
+ BigQueryIO.parseTableSpec(":a.b");
+ }
+
+ @Test
+ public void testTableParsingError_slash() {
+ thrown.expect(IllegalArgumentException.class);
+ BigQueryIO.parseTableSpec("a\\b12345:c.d");
+ }
+
+ // Test that BigQuery's special null placeholder objects can be encoded.
+ @Test
+ public void testCoder_nullCell() throws CoderException {
+ TableRow row = new TableRow();
+ row.set("temperature", Data.nullOf(Object.class));
+ row.set("max_temperature", Data.nullOf(Object.class));
+
+ byte[] bytes = CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), row);
+
+ TableRow newRow = CoderUtils.decodeFromByteArray(TableRowJsonCoder.of(), bytes);
+ byte[] newBytes = CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), newRow);
+
+ Assert.assertArrayEquals(bytes, newBytes);
+ }
+
+ @Test
+ public void testBigQueryIOGetName() {
+ assertEquals("BigQueryIO.Read", BigQueryIO.Read.from("somedataset.sometable").getName());
+ assertEquals("BigQueryIO.Write", BigQueryIO.Write.to("somedataset.sometable").getName());
+ }
+
+ @Test
+ public void testWriteValidateFailsCreateNoSchema() {
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("no schema was provided");
+ TestPipeline.create()
+ .apply(Create.<TableRow>of())
+ .apply(BigQueryIO.Write
+ .to("dataset.table")
+ .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED));
+ }
+
+ @Test
+ public void testWriteValidateFailsTableAndTableSpec() {
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("Cannot set both a table reference and a table function");
+ TestPipeline.create()
+ .apply(Create.<TableRow>of())
+ .apply(BigQueryIO.Write
+ .to("dataset.table")
+ .to(new SerializableFunction<BoundedWindow, String>() {
+ @Override
+ public String apply(BoundedWindow input) {
+ return null;
+ }
+ }));
+ }
+
+ @Test
+ public void testWriteValidateFailsNoTableAndNoTableSpec() {
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("must set the table reference of a BigQueryIO.Write transform");
+ TestPipeline.create()
+ .apply(Create.<TableRow>of())
+ .apply("name", BigQueryIO.Write.withoutValidation());
+ }
+
+ @Test
+ public void testBigQueryTableSourceThroughJsonAPI() throws Exception {
+ FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+ .withJobService(mockJobService)
+ .readerReturns(
+ toJsonString(new TableRow().set("name", "a").set("number", "1")),
+ toJsonString(new TableRow().set("name", "b").set("number", "2")),
+ toJsonString(new TableRow().set("name", "c").set("number", "3")));
+
+ String jobIdToken = "testJobIdToken";
+ TableReference table = BigQueryIO.parseTableSpec("project.data_set.table_name");
+ String extractDestinationDir = "mock://tempLocation";
+ BoundedSource<TableRow> bqSource = BigQueryTableSource.create(
+ jobIdToken, table, extractDestinationDir, fakeBqServices, "project");
+
+ List<TableRow> expected = ImmutableList.of(
+ new TableRow().set("name", "a").set("number", "1"),
+ new TableRow().set("name", "b").set("number", "2"),
+ new TableRow().set("name", "c").set("number", "3"));
+
+ PipelineOptions options = PipelineOptionsFactory.create();
+ Assert.assertThat(
+ SourceTestUtils.readFromSource(bqSource, options),
+ CoreMatchers.is(expected));
+ SourceTestUtils.assertSplitAtFractionBehavior(
+ bqSource, 2, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options);
+ }
+
+ @Test
+ public void testBigQueryTableSourceInitSplit() throws Exception {
+ Job extractJob = new Job();
+ JobStatistics jobStats = new JobStatistics();
+ JobStatistics4 extractStats = new JobStatistics4();
+ extractStats.setDestinationUriFileCounts(ImmutableList.of(1L));
+ jobStats.setExtract(extractStats);
+ extractJob.setStatus(new JobStatus())
+ .setStatistics(jobStats);
+
+ FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+ .withJobService(mockJobService)
+ .withDatasetService(mockDatasetService)
+ .readerReturns(
+ toJsonString(new TableRow().set("name", "a").set("number", "1")),
+ toJsonString(new TableRow().set("name", "b").set("number", "2")),
+ toJsonString(new TableRow().set("name", "c").set("number", "3")));
+
+ String jobIdToken = "testJobIdToken";
+ TableReference table = BigQueryIO.parseTableSpec("project:data_set.table_name");
+ String extractDestinationDir = "mock://tempLocation";
+ BoundedSource<TableRow> bqSource = BigQueryTableSource.create(
+ jobIdToken, table, extractDestinationDir, fakeBqServices, "project");
+
+ List<TableRow> expected = ImmutableList.of(
+ new TableRow().set("name", "a").set("number", "1"),
+ new TableRow().set("name", "b").set("number", "2"),
+ new TableRow().set("name", "c").set("number", "3"));
+
+ when(mockJobService.pollJob(Mockito.<JobReference>any(), Mockito.anyInt()))
+ .thenReturn(extractJob);
+ PipelineOptions options = PipelineOptionsFactory.create();
+ options.setTempLocation("mock://tempLocation");
+
+ IOChannelUtils.setIOFactory("mock", mockIOChannelFactory);
+ when(mockIOChannelFactory.resolve(anyString(), anyString()))
+ .thenReturn("mock://tempLocation/output");
+ when(mockDatasetService.getTable(anyString(), anyString(), anyString()))
+ .thenReturn(new Table().setSchema(new TableSchema()));
+
+ Assert.assertThat(
+ SourceTestUtils.readFromSource(bqSource, options),
+ CoreMatchers.is(expected));
+ SourceTestUtils.assertSplitAtFractionBehavior(
+ bqSource, 2, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options);
+
+ List<? extends BoundedSource<TableRow>> sources = bqSource.splitIntoBundles(100, options);
+ assertEquals(1, sources.size());
+ BoundedSource<TableRow> actual = sources.get(0);
+ assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class));
+
+ Mockito.verify(mockJobService)
+ .startExtractJob(Mockito.<JobReference>any(), Mockito.<JobConfigurationExtract>any());
+ }
+
+ @Test
+ public void testBigQueryQuerySourceInitSplit() throws Exception {
+ TableReference dryRunTable = new TableReference();
+
+ Job queryJob = new Job();
+ JobStatistics queryJobStats = new JobStatistics();
+ JobStatistics2 queryStats = new JobStatistics2();
+ queryStats.setReferencedTables(ImmutableList.of(dryRunTable));
+ queryJobStats.setQuery(queryStats);
+ queryJob.setStatus(new JobStatus())
+ .setStatistics(queryJobStats);
+
+ Job extractJob = new Job();
+ JobStatistics extractJobStats = new JobStatistics();
+ JobStatistics4 extractStats = new JobStatistics4();
+ extractStats.setDestinationUriFileCounts(ImmutableList.of(1L));
+ extractJobStats.setExtract(extractStats);
+ extractJob.setStatus(new JobStatus())
+ .setStatistics(extractJobStats);
+
+ FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+ .withJobService(mockJobService)
+ .withDatasetService(mockDatasetService)
+ .readerReturns(
+ toJsonString(new TableRow().set("name", "a").set("number", "1")),
+ toJsonString(new TableRow().set("name", "b").set("number", "2")),
+ toJsonString(new TableRow().set("name", "c").set("number", "3")));
+
+ String jobIdToken = "testJobIdToken";
+ String extractDestinationDir = "mock://tempLocation";
+ TableReference destinationTable = BigQueryIO.parseTableSpec("project:data_set.table_name");
+ BoundedSource<TableRow> bqSource = BigQueryQuerySource.create(
+ jobIdToken, "query", destinationTable, true /* flattenResults */,
+ extractDestinationDir, fakeBqServices);
+
+ List<TableRow> expected = ImmutableList.of(
+ new TableRow().set("name", "a").set("number", "1"),
+ new TableRow().set("name", "b").set("number", "2"),
+ new TableRow().set("name", "c").set("number", "3"));
+
+ PipelineOptions options = PipelineOptionsFactory.create();
+ options.setTempLocation(extractDestinationDir);
+
+ TableReference queryTable = new TableReference()
+ .setProjectId("testProejct")
+ .setDatasetId("testDataset")
+ .setTableId("testTable");
+ when(mockJobService.dryRunQuery(anyString(), anyString()))
+ .thenReturn(new JobStatistics().setQuery(
+ new JobStatistics2()
+ .setTotalBytesProcessed(100L)
+ .setReferencedTables(ImmutableList.of(queryTable))));
+ when(mockDatasetService.getTable(
+ eq(queryTable.getProjectId()), eq(queryTable.getDatasetId()), eq(queryTable.getTableId())))
+ .thenReturn(new Table().setSchema(new TableSchema()));
+ when(mockDatasetService.getTable(
+ eq(destinationTable.getProjectId()),
+ eq(destinationTable.getDatasetId()),
+ eq(destinationTable.getTableId())))
+ .thenReturn(new Table().setSchema(new TableSchema()));
+ IOChannelUtils.setIOFactory("mock", mockIOChannelFactory);
+ when(mockIOChannelFactory.resolve(anyString(), anyString()))
+ .thenReturn("mock://tempLocation/output");
+ when(mockJobService.pollJob(Mockito.<JobReference>any(), Mockito.anyInt()))
+ .thenReturn(extractJob);
+
+ Assert.assertThat(
+ SourceTestUtils.readFromSource(bqSource, options),
+ CoreMatchers.is(expected));
+ SourceTestUtils.assertSplitAtFractionBehavior(
+ bqSource, 2, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options);
+
+ List<? extends BoundedSource<TableRow>> sources = bqSource.splitIntoBundles(100, options);
+ assertEquals(1, sources.size());
+ BoundedSource<TableRow> actual = sources.get(0);
+ assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class));
+
+ Mockito.verify(mockJobService)
+ .startQueryJob(
+ Mockito.<JobReference>any(), Mockito.<JobConfigurationQuery>any());
+ Mockito.verify(mockJobService)
+ .startExtractJob(Mockito.<JobReference>any(), Mockito.<JobConfigurationExtract>any());
+ Mockito.verify(mockDatasetService)
+ .createDataset(anyString(), anyString(), anyString(), anyString());
+ }
+
+ @Test
+ public void testTransformingSource() throws Exception {
+ int numElements = 10000;
+ @SuppressWarnings("deprecation")
+ BoundedSource<Long> longSource = CountingSource.upTo(numElements);
+ SerializableFunction<Long, String> toStringFn =
+ new SerializableFunction<Long, String>() {
+ @Override
+ public String apply(Long input) {
+ return input.toString();
+ }};
+ BoundedSource<String> stringSource = new TransformingSource<>(
+ longSource, toStringFn, StringUtf8Coder.of());
+
+ List<String> expected = Lists.newArrayList();
+ for (int i = 0; i < numElements; i++) {
+ expected.add(String.valueOf(i));
+ }
+
+ PipelineOptions options = PipelineOptionsFactory.create();
+ Assert.assertThat(
+ SourceTestUtils.readFromSource(stringSource, options),
+ CoreMatchers.is(expected));
+ SourceTestUtils.assertSplitAtFractionBehavior(
+ stringSource, 100, 0.3, ExpectedSplitOutcome.MUST_SUCCEED_AND_BE_CONSISTENT, options);
+
+ SourceTestUtils.assertSourcesEqualReferenceSource(
+ stringSource, stringSource.splitIntoBundles(100, options), options);
+ }
+
+ @Test
+ @Category(RunnableOnService.class)
+ public void testPassThroughThenCleanup() throws Exception {
+ Pipeline p = TestPipeline.create();
+
+ PCollection<Integer> output = p
+ .apply(Create.of(1, 2, 3))
+ .apply(new PassThroughThenCleanup<Integer>(new CleanupOperation() {
+ @Override
+ void cleanup(PipelineOptions options) throws Exception {
+ // no-op
+ }}));
+
+ PAssert.that(output).containsInAnyOrder(1, 2, 3);
+
+ p.run();
+ }
+
+ @Test
+ @Category(NeedsRunner.class)
+ public void testPassThroughThenCleanupExecuted() throws Exception {
+ Pipeline p = TestPipeline.create();
+
+ p.apply(Create.<Integer>of())
+ .apply(new PassThroughThenCleanup<Integer>(new CleanupOperation() {
+ @Override
+ void cleanup(PipelineOptions options) throws Exception {
+ throw new RuntimeException("cleanup executed");
+ }}));
+
+ thrown.expect(RuntimeException.class);
+ thrown.expectMessage("cleanup executed");
+
+ p.run();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
new file mode 100644
index 0000000..2cdf511
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
@@ -0,0 +1,289 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.JobServiceImpl;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
+import org.apache.beam.sdk.util.Transport;
+
+import com.google.api.client.googleapis.json.GoogleJsonError;
+import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
+import com.google.api.client.googleapis.json.GoogleJsonErrorContainer;
+import com.google.api.client.http.LowLevelHttpResponse;
+import com.google.api.client.json.GenericJson;
+import com.google.api.client.json.Json;
+import com.google.api.client.json.jackson2.JacksonFactory;
+import com.google.api.client.testing.http.MockHttpTransport;
+import com.google.api.client.testing.http.MockLowLevelHttpRequest;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.model.ErrorProto;
+import com.google.api.services.bigquery.model.Job;
+import com.google.api.services.bigquery.model.JobReference;
+import com.google.api.services.bigquery.model.JobStatus;
+import com.google.api.services.bigquery.model.Table;
+import com.google.cloud.hadoop.util.ApiErrorExtractor;
+import com.google.common.collect.ImmutableList;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Tests for {@link BigQueryServicesImpl}.
+ */
+@RunWith(JUnit4.class)
+public class BigQueryServicesImplTest {
+ @Rule public ExpectedException thrown = ExpectedException.none();
+ @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(BigQueryServicesImpl.class);
+ @Mock private LowLevelHttpResponse response;
+ private Bigquery bigquery;
+
+ @Before
+ public void setUp() {
+ MockitoAnnotations.initMocks(this);
+
+ // A mock transport that lets us mock the API responses.
+ MockHttpTransport transport =
+ new MockHttpTransport.Builder()
+ .setLowLevelHttpRequest(
+ new MockLowLevelHttpRequest() {
+ @Override
+ public LowLevelHttpResponse execute() throws IOException {
+ return response;
+ }
+ })
+ .build();
+
+ // A sample BigQuery API client that uses default JsonFactory and RetryHttpInitializer.
+ bigquery =
+ new Bigquery.Builder(
+ transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer())
+ .build();
+ }
+
+ /**
+ * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds.
+ */
+ @Test
+ public void testStartLoadJobSucceeds() throws IOException, InterruptedException {
+ Job testJob = new Job();
+ JobReference jobRef = new JobReference();
+ jobRef.setJobId("jobId");
+ jobRef.setProjectId("projectId");
+ testJob.setJobReference(jobRef);
+
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(200);
+ when(response.getContent()).thenReturn(toStream(testJob));
+
+ Sleeper sleeper = new FastNanoClockAndSleeper();
+ BackOff backoff = new AttemptBoundedExponentialBackOff(
+ 5 /* attempts */, 1000 /* initialIntervalMillis */);
+ JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff);
+
+ verify(response, times(1)).getStatusCode();
+ verify(response, times(1)).getContent();
+ verify(response, times(1)).getContentType();
+ }
+
+ /**
+ * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds
+ * with an already exist job.
+ */
+ @Test
+ public void testStartLoadJobSucceedsAlreadyExists() throws IOException, InterruptedException {
+ Job testJob = new Job();
+ JobReference jobRef = new JobReference();
+ jobRef.setJobId("jobId");
+ jobRef.setProjectId("projectId");
+ testJob.setJobReference(jobRef);
+
+ when(response.getStatusCode()).thenReturn(409); // 409 means already exists
+
+ Sleeper sleeper = new FastNanoClockAndSleeper();
+ BackOff backoff = new AttemptBoundedExponentialBackOff(
+ 5 /* attempts */, 1000 /* initialIntervalMillis */);
+ JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff);
+
+ verify(response, times(1)).getStatusCode();
+ verify(response, times(1)).getContent();
+ verify(response, times(1)).getContentType();
+ }
+
+ /**
+ * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds with a retry.
+ */
+ @Test
+ public void testStartLoadJobRetry() throws IOException, InterruptedException {
+ Job testJob = new Job();
+ JobReference jobRef = new JobReference();
+ jobRef.setJobId("jobId");
+ jobRef.setProjectId("projectId");
+ testJob.setJobReference(jobRef);
+
+ // First response is 403 rate limited, second response has valid payload.
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(403).thenReturn(200);
+ when(response.getContent())
+ .thenReturn(toStream(errorWithReasonAndStatus("rateLimitExceeded", 403)))
+ .thenReturn(toStream(testJob));
+
+ Sleeper sleeper = new FastNanoClockAndSleeper();
+ BackOff backoff = new AttemptBoundedExponentialBackOff(
+ 5 /* attempts */, 1000 /* initialIntervalMillis */);
+ JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff);
+
+ verify(response, times(2)).getStatusCode();
+ verify(response, times(2)).getContent();
+ verify(response, times(2)).getContentType();
+ }
+
+ /**
+ * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} succeeds.
+ */
+ @Test
+ public void testPollJobSucceeds() throws IOException, InterruptedException {
+ Job testJob = new Job();
+ testJob.setStatus(new JobStatus().setState("DONE"));
+
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(200);
+ when(response.getContent()).thenReturn(toStream(testJob));
+
+ BigQueryServicesImpl.JobServiceImpl jobService =
+ new BigQueryServicesImpl.JobServiceImpl(bigquery);
+ JobReference jobRef = new JobReference()
+ .setProjectId("projectId")
+ .setJobId("jobId");
+ Job job = jobService.pollJob(jobRef, Sleeper.DEFAULT, BackOff.ZERO_BACKOFF);
+
+ assertEquals(testJob, job);
+ verify(response, times(1)).getStatusCode();
+ verify(response, times(1)).getContent();
+ verify(response, times(1)).getContentType();
+ }
+
+ /**
+ * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} fails.
+ */
+ @Test
+ public void testPollJobFailed() throws IOException, InterruptedException {
+ Job testJob = new Job();
+ testJob.setStatus(new JobStatus().setState("DONE").setErrorResult(new ErrorProto()));
+
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(200);
+ when(response.getContent()).thenReturn(toStream(testJob));
+
+ BigQueryServicesImpl.JobServiceImpl jobService =
+ new BigQueryServicesImpl.JobServiceImpl(bigquery);
+ JobReference jobRef = new JobReference()
+ .setProjectId("projectId")
+ .setJobId("jobId");
+ Job job = jobService.pollJob(jobRef, Sleeper.DEFAULT, BackOff.ZERO_BACKOFF);
+
+ assertEquals(testJob, job);
+ verify(response, times(1)).getStatusCode();
+ verify(response, times(1)).getContent();
+ verify(response, times(1)).getContentType();
+ }
+
+ /**
+ * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} returns UNKNOWN.
+ */
+ @Test
+ public void testPollJobUnknown() throws IOException, InterruptedException {
+ Job testJob = new Job();
+ testJob.setStatus(new JobStatus());
+
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(200);
+ when(response.getContent()).thenReturn(toStream(testJob));
+
+ BigQueryServicesImpl.JobServiceImpl jobService =
+ new BigQueryServicesImpl.JobServiceImpl(bigquery);
+ JobReference jobRef = new JobReference()
+ .setProjectId("projectId")
+ .setJobId("jobId");
+ Job job = jobService.pollJob(jobRef, Sleeper.DEFAULT, BackOff.STOP_BACKOFF);
+
+ assertEquals(null, job);
+ verify(response, times(1)).getStatusCode();
+ verify(response, times(1)).getContent();
+ verify(response, times(1)).getContentType();
+ }
+
+ @Test
+ public void testExecuteWithRetries() throws IOException, InterruptedException {
+ Table testTable = new Table();
+
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(200);
+ when(response.getContent()).thenReturn(toStream(testTable));
+
+ Table table = BigQueryServicesImpl.executeWithRetries(
+ bigquery.tables().get("projectId", "datasetId", "tableId"),
+ "Failed to get table.",
+ Sleeper.DEFAULT,
+ BackOff.STOP_BACKOFF);
+
+ assertEquals(testTable, table);
+ verify(response, times(1)).getStatusCode();
+ verify(response, times(1)).getContent();
+ verify(response, times(1)).getContentType();
+ }
+
+ /** A helper to wrap a {@link GenericJson} object in a content stream. */
+ private static InputStream toStream(GenericJson content) throws IOException {
+ return new ByteArrayInputStream(JacksonFactory.getDefaultInstance().toByteArray(content));
+ }
+
+ /** A helper that generates the error JSON payload that Google APIs produce. */
+ private static GoogleJsonErrorContainer errorWithReasonAndStatus(String reason, int status) {
+ ErrorInfo info = new ErrorInfo();
+ info.setReason(reason);
+ info.setDomain("global");
+ // GoogleJsonError contains one or more ErrorInfo objects; our utiities read the first one.
+ GoogleJsonError error = new GoogleJsonError();
+ error.setErrors(ImmutableList.of(info));
+ error.setCode(status);
+ // The actual JSON response is an error container.
+ GoogleJsonErrorContainer container = new GoogleJsonErrorContainer();
+ container.setError(error);
+ return container;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java
new file mode 100644
index 0000000..c29da91
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java
@@ -0,0 +1,313 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import static com.google.common.base.Verify.verifyNotNull;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
+import org.apache.beam.sdk.util.Transport;
+
+import com.google.api.client.googleapis.json.GoogleJsonError;
+import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
+import com.google.api.client.googleapis.json.GoogleJsonErrorContainer;
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.http.LowLevelHttpResponse;
+import com.google.api.client.json.GenericJson;
+import com.google.api.client.json.Json;
+import com.google.api.client.json.jackson2.JacksonFactory;
+import com.google.api.client.testing.http.MockHttpTransport;
+import com.google.api.client.testing.http.MockLowLevelHttpRequest;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableDataInsertAllResponse;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.hadoop.util.RetryBoundedBackOff;
+import com.google.common.collect.ImmutableList;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests of {@link BigQueryTableInserter}.
+ */
+@RunWith(JUnit4.class)
+public class BigQueryTableInserterTest {
+ @Rule public ExpectedException thrown = ExpectedException.none();
+ @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(BigQueryTableInserter.class);
+ @Mock private LowLevelHttpResponse response;
+ private Bigquery bigquery;
+ private PipelineOptions options;
+
+ @Before
+ public void setUp() {
+ MockitoAnnotations.initMocks(this);
+
+ // A mock transport that lets us mock the API responses.
+ MockHttpTransport transport =
+ new MockHttpTransport.Builder()
+ .setLowLevelHttpRequest(
+ new MockLowLevelHttpRequest() {
+ @Override
+ public LowLevelHttpResponse execute() throws IOException {
+ return response;
+ }
+ })
+ .build();
+
+ // A sample BigQuery API client that uses default JsonFactory and RetryHttpInitializer.
+ bigquery =
+ new Bigquery.Builder(
+ transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer())
+ .build();
+
+ options = PipelineOptionsFactory.create();
+ }
+
+ @After
+ public void tearDown() throws IOException {
+ // These three interactions happen for every request in the normal response parsing.
+ verify(response, atLeastOnce()).getContentEncoding();
+ verify(response, atLeastOnce()).getHeaderCount();
+ verify(response, atLeastOnce()).getReasonPhrase();
+ verifyNoMoreInteractions(response);
+ }
+
+ /** A helper to wrap a {@link GenericJson} object in a content stream. */
+ private static InputStream toStream(GenericJson content) throws IOException {
+ return new ByteArrayInputStream(JacksonFactory.getDefaultInstance().toByteArray(content));
+ }
+
+ /** A helper that generates the error JSON payload that Google APIs produce. */
+ private static GoogleJsonErrorContainer errorWithReasonAndStatus(String reason, int status) {
+ ErrorInfo info = new ErrorInfo();
+ info.setReason(reason);
+ info.setDomain("global");
+ // GoogleJsonError contains one or more ErrorInfo objects; our utiities read the first one.
+ GoogleJsonError error = new GoogleJsonError();
+ error.setErrors(ImmutableList.of(info));
+ error.setCode(status);
+ // The actual JSON response is an error container.
+ GoogleJsonErrorContainer container = new GoogleJsonErrorContainer();
+ container.setError(error);
+ return container;
+ }
+
+ /**
+ * Tests that {@link BigQueryTableInserter} succeeds on the first try.
+ */
+ @Test
+ public void testCreateTableSucceeds() throws IOException {
+ Table testTable = new Table().setDescription("a table");
+
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(200);
+ when(response.getContent()).thenReturn(toStream(testTable));
+
+ BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
+ Table ret =
+ inserter.tryCreateTable(
+ new Table(),
+ "project",
+ "dataset",
+ new RetryBoundedBackOff(0, BackOff.ZERO_BACKOFF),
+ Sleeper.DEFAULT);
+ assertEquals(testTable, ret);
+ verify(response, times(1)).getStatusCode();
+ verify(response, times(1)).getContent();
+ verify(response, times(1)).getContentType();
+ }
+
+ /**
+ * Tests that {@link BigQueryTableInserter} succeeds when the table already exists.
+ */
+ @Test
+ public void testCreateTableSucceedsAlreadyExists() throws IOException {
+ when(response.getStatusCode()).thenReturn(409); // 409 means already exists
+
+ BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
+ Table ret =
+ inserter.tryCreateTable(
+ new Table(),
+ "project",
+ "dataset",
+ new RetryBoundedBackOff(0, BackOff.ZERO_BACKOFF),
+ Sleeper.DEFAULT);
+
+ assertNull(ret);
+ verify(response, times(1)).getStatusCode();
+ verify(response, times(1)).getContent();
+ verify(response, times(1)).getContentType();
+ }
+
+ /**
+ * Tests that {@link BigQueryTableInserter} retries quota rate limited attempts.
+ */
+ @Test
+ public void testCreateTableRetry() throws IOException {
+ TableReference ref =
+ new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
+ Table testTable = new Table().setTableReference(ref);
+
+ // First response is 403 rate limited, second response has valid payload.
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(403).thenReturn(200);
+ when(response.getContent())
+ .thenReturn(toStream(errorWithReasonAndStatus("rateLimitExceeded", 403)))
+ .thenReturn(toStream(testTable));
+
+ BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
+ Table ret =
+ inserter.tryCreateTable(
+ testTable,
+ "project",
+ "dataset",
+ new RetryBoundedBackOff(3, BackOff.ZERO_BACKOFF),
+ Sleeper.DEFAULT);
+ assertEquals(testTable, ret);
+ verify(response, times(2)).getStatusCode();
+ verify(response, times(2)).getContent();
+ verify(response, times(2)).getContentType();
+ verifyNotNull(ret.getTableReference());
+ expectedLogs.verifyInfo(
+ "Quota limit reached when creating table project:dataset.table, "
+ + "retrying up to 5.0 minutes");
+ }
+
+ /**
+ * Tests that {@link BigQueryTableInserter} does not retry non-rate-limited attempts.
+ */
+ @Test
+ public void testCreateTableDoesNotRetry() throws IOException {
+ Table testTable = new Table().setDescription("a table");
+
+ // First response is 403 not-rate-limited, second response has valid payload but should not
+ // be invoked.
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(403).thenReturn(200);
+ when(response.getContent())
+ .thenReturn(toStream(errorWithReasonAndStatus("actually forbidden", 403)))
+ .thenReturn(toStream(testTable));
+
+ thrown.expect(GoogleJsonResponseException.class);
+ thrown.expectMessage("actually forbidden");
+
+ BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
+ try {
+ inserter.tryCreateTable(
+ new Table(),
+ "project",
+ "dataset",
+ new RetryBoundedBackOff(3, BackOff.ZERO_BACKOFF),
+ Sleeper.DEFAULT);
+ fail();
+ } catch (IOException e) {
+ verify(response, times(1)).getStatusCode();
+ verify(response, times(1)).getContent();
+ verify(response, times(1)).getContentType();
+ throw e;
+ }
+ }
+
+ /**
+ * Tests that {@link BigQueryTableInserter#insertAll} retries quota rate limited attempts.
+ */
+ @Test
+ public void testInsertRetry() throws IOException {
+ TableReference ref =
+ new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
+ List<TableRow> rows = new ArrayList<>();
+ rows.add(new TableRow());
+
+ // First response is 403 rate limited, second response has valid payload.
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(403).thenReturn(200);
+ when(response.getContent())
+ .thenReturn(toStream(errorWithReasonAndStatus("rateLimitExceeded", 403)))
+ .thenReturn(toStream(new TableDataInsertAllResponse()));
+
+ BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
+
+ inserter.insertAll(ref, rows);
+ verify(response, times(2)).getStatusCode();
+ verify(response, times(2)).getContent();
+ verify(response, times(2)).getContentType();
+ expectedLogs.verifyInfo("BigQuery insertAll exceeded rate limit, retrying");
+ }
+
+ /**
+ * Tests that {@link BigQueryTableInserter#insertAll} does not retry non-rate-limited attempts.
+ */
+ @Test
+ public void testInsertDoesNotRetry() throws Throwable {
+ TableReference ref =
+ new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
+ List<TableRow> rows = new ArrayList<>();
+ rows.add(new TableRow());
+
+ // First response is 403 not-rate-limited, second response has valid payload but should not
+ // be invoked.
+ when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
+ when(response.getStatusCode()).thenReturn(403).thenReturn(200);
+ when(response.getContent())
+ .thenReturn(toStream(errorWithReasonAndStatus("actually forbidden", 403)))
+ .thenReturn(toStream(new TableDataInsertAllResponse()));
+
+ thrown.expect(GoogleJsonResponseException.class);
+ thrown.expectMessage("actually forbidden");
+
+ BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
+
+ try {
+ inserter.insertAll(ref, rows);
+ fail();
+ } catch (RuntimeException e) {
+ verify(response, times(1)).getStatusCode();
+ verify(response, times(1)).getContent();
+ verify(response, times(1)).getContentType();
+ throw e.getCause();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java
new file mode 100644
index 0000000..457b071
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java
@@ -0,0 +1,256 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.model.Dataset;
+import com.google.api.services.bigquery.model.ErrorProto;
+import com.google.api.services.bigquery.model.Job;
+import com.google.api.services.bigquery.model.JobConfiguration;
+import com.google.api.services.bigquery.model.JobConfigurationQuery;
+import com.google.api.services.bigquery.model.JobReference;
+import com.google.api.services.bigquery.model.JobStatus;
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableCell;
+import com.google.api.services.bigquery.model.TableDataList;
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Tests for {@link BigQueryTableRowIterator}.
+ */
+@RunWith(JUnit4.class)
+public class BigQueryTableRowIteratorTest {
+
+ @Rule public ExpectedException thrown = ExpectedException.none();
+
+ @Mock private Bigquery mockClient;
+ @Mock private Bigquery.Datasets mockDatasets;
+ @Mock private Bigquery.Datasets.Delete mockDatasetsDelete;
+ @Mock private Bigquery.Datasets.Insert mockDatasetsInsert;
+ @Mock private Bigquery.Jobs mockJobs;
+ @Mock private Bigquery.Jobs.Get mockJobsGet;
+ @Mock private Bigquery.Jobs.Insert mockJobsInsert;
+ @Mock private Bigquery.Tables mockTables;
+ @Mock private Bigquery.Tables.Get mockTablesGet;
+ @Mock private Bigquery.Tables.Delete mockTablesDelete;
+ @Mock private Bigquery.Tabledata mockTabledata;
+ @Mock private Bigquery.Tabledata.List mockTabledataList;
+
+ @Before
+ public void setUp() throws IOException {
+ MockitoAnnotations.initMocks(this);
+ when(mockClient.tabledata()).thenReturn(mockTabledata);
+ when(mockTabledata.list(anyString(), anyString(), anyString())).thenReturn(mockTabledataList);
+
+ when(mockClient.tables()).thenReturn(mockTables);
+ when(mockTables.delete(anyString(), anyString(), anyString())).thenReturn(mockTablesDelete);
+ when(mockTables.get(anyString(), anyString(), anyString())).thenReturn(mockTablesGet);
+
+ when(mockClient.datasets()).thenReturn(mockDatasets);
+ when(mockDatasets.delete(anyString(), anyString())).thenReturn(mockDatasetsDelete);
+ when(mockDatasets.insert(anyString(), any(Dataset.class))).thenReturn(mockDatasetsInsert);
+
+ when(mockClient.jobs()).thenReturn(mockJobs);
+ when(mockJobs.insert(anyString(), any(Job.class))).thenReturn(mockJobsInsert);
+ when(mockJobs.get(anyString(), anyString())).thenReturn(mockJobsGet);
+ }
+
+ @After
+ public void tearDown() {
+ verifyNoMoreInteractions(mockClient);
+ verifyNoMoreInteractions(mockDatasets);
+ verifyNoMoreInteractions(mockDatasetsDelete);
+ verifyNoMoreInteractions(mockDatasetsInsert);
+ verifyNoMoreInteractions(mockJobs);
+ verifyNoMoreInteractions(mockJobsGet);
+ verifyNoMoreInteractions(mockJobsInsert);
+ verifyNoMoreInteractions(mockTables);
+ verifyNoMoreInteractions(mockTablesDelete);
+ verifyNoMoreInteractions(mockTablesGet);
+ verifyNoMoreInteractions(mockTabledata);
+ verifyNoMoreInteractions(mockTabledataList);
+ }
+
+ private static Table tableWithBasicSchema() {
+ return new Table()
+ .setSchema(
+ new TableSchema()
+ .setFields(
+ Arrays.asList(
+ new TableFieldSchema().setName("name").setType("STRING"),
+ new TableFieldSchema().setName("answer").setType("INTEGER"))));
+ }
+
+ private TableRow rawRow(Object... args) {
+ List<TableCell> cells = new LinkedList<>();
+ for (Object a : args) {
+ cells.add(new TableCell().setV(a));
+ }
+ return new TableRow().setF(cells);
+ }
+
+ private TableDataList rawDataList(TableRow... rows) {
+ return new TableDataList().setRows(Arrays.asList(rows));
+ }
+
+ /**
+ * Verifies that when the query runs, the correct data is returned and the temporary dataset and
+ * table are both cleaned up.
+ */
+ @Test
+ public void testReadFromQuery() throws IOException, InterruptedException {
+ // Mock job inserting.
+ Job insertedJob = new Job().setJobReference(new JobReference());
+ when(mockJobsInsert.execute()).thenReturn(insertedJob);
+
+ // Mock job polling.
+ JobStatus status = new JobStatus().setState("DONE");
+ TableReference tableRef =
+ new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
+ JobConfigurationQuery queryConfig = new JobConfigurationQuery().setDestinationTable(tableRef);
+ Job getJob =
+ new Job()
+ .setJobReference(new JobReference())
+ .setStatus(status)
+ .setConfiguration(new JobConfiguration().setQuery(queryConfig));
+ when(mockJobsGet.execute()).thenReturn(getJob);
+
+ // Mock table schema fetch.
+ when(mockTablesGet.execute()).thenReturn(tableWithBasicSchema());
+
+ // Mock table data fetch.
+ when(mockTabledataList.execute()).thenReturn(rawDataList(rawRow("Arthur", 42)));
+
+ // Run query and verify
+ String query = "SELECT name, count from table";
+ try (BigQueryTableRowIterator iterator =
+ BigQueryTableRowIterator.fromQuery(query, "project", mockClient, null)) {
+ iterator.open();
+ assertTrue(iterator.advance());
+ TableRow row = iterator.getCurrent();
+
+ assertTrue(row.containsKey("name"));
+ assertTrue(row.containsKey("answer"));
+ assertEquals("Arthur", row.get("name"));
+ assertEquals(42, row.get("answer"));
+
+ assertFalse(iterator.advance());
+ }
+
+ // Temp dataset created and later deleted.
+ verify(mockClient, times(2)).datasets();
+ verify(mockDatasets).insert(anyString(), any(Dataset.class));
+ verify(mockDatasetsInsert).execute();
+ verify(mockDatasets).delete(anyString(), anyString());
+ verify(mockDatasetsDelete).execute();
+ // Job inserted to run the query, polled once.
+ verify(mockClient, times(2)).jobs();
+ verify(mockJobs).insert(anyString(), any(Job.class));
+ verify(mockJobsInsert).execute();
+ verify(mockJobs).get(anyString(), anyString());
+ verify(mockJobsGet).execute();
+ // Temp table get after query finish, deleted after reading.
+ verify(mockClient, times(2)).tables();
+ verify(mockTables).get("project", "dataset", "table");
+ verify(mockTablesGet).execute();
+ verify(mockTables).delete(anyString(), anyString(), anyString());
+ verify(mockTablesDelete).execute();
+ // Table data read.
+ verify(mockClient).tabledata();
+ verify(mockTabledata).list("project", "dataset", "table");
+ verify(mockTabledataList).execute();
+ }
+
+ /**
+ * Verifies that when the query fails, the user gets a useful exception and the temporary dataset
+ * is cleaned up. Also verifies that the temporary table (which is never created) is not
+ * erroneously attempted to be deleted.
+ */
+ @Test
+ public void testQueryFailed() throws IOException {
+ // Job can be created.
+ JobReference ref = new JobReference();
+ Job insertedJob = new Job().setJobReference(ref);
+ when(mockJobsInsert.execute()).thenReturn(insertedJob);
+
+ // Job state polled with an error.
+ String errorReason = "bad query";
+ JobStatus status =
+ new JobStatus().setState("DONE").setErrorResult(new ErrorProto().setMessage(errorReason));
+ Job getJob = new Job().setJobReference(ref).setStatus(status);
+ when(mockJobsGet.execute()).thenReturn(getJob);
+
+ String query = "NOT A QUERY";
+ try (BigQueryTableRowIterator iterator =
+ BigQueryTableRowIterator.fromQuery(query, "project", mockClient, null)) {
+ try {
+ iterator.open();
+ fail();
+ } catch (Exception expected) {
+ // Verify message explains cause and reports the query.
+ assertThat(expected.getMessage(), containsString("failed"));
+ assertThat(expected.getMessage(), containsString(errorReason));
+ assertThat(expected.getMessage(), containsString(query));
+ }
+ }
+
+ // Temp dataset created and then later deleted.
+ verify(mockClient, times(2)).datasets();
+ verify(mockDatasets).insert(anyString(), any(Dataset.class));
+ verify(mockDatasetsInsert).execute();
+ verify(mockDatasets).delete(anyString(), anyString());
+ verify(mockDatasetsDelete).execute();
+ // Job inserted to run the query, then polled once.
+ verify(mockClient, times(2)).jobs();
+ verify(mockJobs).insert(anyString(), any(Job.class));
+ verify(mockJobsInsert).execute();
+ verify(mockJobs).get(anyString(), anyString());
+ verify(mockJobsGet).execute();
+ }
+}
[06/10] incubator-beam git commit: BigQueryIO: move to
google-cloud-platform module
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
deleted file mode 100644
index 0d1a9f8..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
+++ /dev/null
@@ -1,1231 +0,0 @@
-/*
- * 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.io;
-
-import static org.apache.beam.sdk.io.BigQueryIO.fromJsonString;
-import static org.apache.beam.sdk.io.BigQueryIO.toJsonString;
-import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import static org.hamcrest.Matchers.hasItem;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.TableRowJsonCoder;
-import org.apache.beam.sdk.io.BigQueryIO.BigQueryQuerySource;
-import org.apache.beam.sdk.io.BigQueryIO.BigQueryTableSource;
-import org.apache.beam.sdk.io.BigQueryIO.PassThroughThenCleanup;
-import org.apache.beam.sdk.io.BigQueryIO.PassThroughThenCleanup.CleanupOperation;
-import org.apache.beam.sdk.io.BigQueryIO.Status;
-import org.apache.beam.sdk.io.BigQueryIO.TransformingSource;
-import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition;
-import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition;
-import org.apache.beam.sdk.options.BigQueryOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.options.StreamingOptions;
-import org.apache.beam.sdk.testing.ExpectedLogs;
-import org.apache.beam.sdk.testing.NeedsRunner;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.RunnableOnService;
-import org.apache.beam.sdk.testing.SourceTestUtils;
-import org.apache.beam.sdk.testing.SourceTestUtils.ExpectedSplitOutcome;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.BigQueryServices;
-import org.apache.beam.sdk.util.BigQueryServices.DatasetService;
-import org.apache.beam.sdk.util.BigQueryServices.JobService;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.IOChannelFactory;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.api.client.util.Data;
-import com.google.api.client.util.Strings;
-import com.google.api.services.bigquery.model.ErrorProto;
-import com.google.api.services.bigquery.model.Job;
-import com.google.api.services.bigquery.model.JobConfigurationExtract;
-import com.google.api.services.bigquery.model.JobConfigurationLoad;
-import com.google.api.services.bigquery.model.JobConfigurationQuery;
-import com.google.api.services.bigquery.model.JobReference;
-import com.google.api.services.bigquery.model.JobStatistics;
-import com.google.api.services.bigquery.model.JobStatistics2;
-import com.google.api.services.bigquery.model.JobStatistics4;
-import com.google.api.services.bigquery.model.JobStatus;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-
-import org.hamcrest.CoreMatchers;
-import org.hamcrest.Matchers;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-/**
- * Tests for BigQueryIO.
- */
-@RunWith(JUnit4.class)
-public class BigQueryIOTest implements Serializable {
-
- // Status.UNKNOWN maps to null
- private static final Map<Status, Job> JOB_STATUS_MAP = ImmutableMap.of(
- Status.SUCCEEDED, new Job().setStatus(new JobStatus()),
- Status.FAILED, new Job().setStatus(new JobStatus().setErrorResult(new ErrorProto())));
-
- private static class FakeBigQueryServices implements BigQueryServices {
-
- private String[] jsonTableRowReturns = new String[0];
- private JobService jobService;
- private DatasetService datasetService;
-
- public FakeBigQueryServices withJobService(JobService jobService) {
- this.jobService = jobService;
- return this;
- }
-
- public FakeBigQueryServices withDatasetService(DatasetService datasetService) {
- this.datasetService = datasetService;
- return this;
- }
-
- public FakeBigQueryServices readerReturns(String... jsonTableRowReturns) {
- this.jsonTableRowReturns = jsonTableRowReturns;
- return this;
- }
-
- @Override
- public JobService getJobService(BigQueryOptions bqOptions) {
- return jobService;
- }
-
- @Override
- public DatasetService getDatasetService(BigQueryOptions bqOptions) {
- return datasetService;
- }
-
- @Override
- public BigQueryJsonReader getReaderFromTable(
- BigQueryOptions bqOptions, TableReference tableRef) {
- return new FakeBigQueryReader(jsonTableRowReturns);
- }
-
- @Override
- public BigQueryJsonReader getReaderFromQuery(
- BigQueryOptions bqOptions, String query, String projectId, @Nullable Boolean flatten) {
- return new FakeBigQueryReader(jsonTableRowReturns);
- }
-
- private static class FakeBigQueryReader implements BigQueryJsonReader {
- private static final int UNSTARTED = -1;
- private static final int CLOSED = Integer.MAX_VALUE;
-
- private String[] jsonTableRowReturns;
- private int currIndex;
-
- FakeBigQueryReader(String[] jsonTableRowReturns) {
- this.jsonTableRowReturns = jsonTableRowReturns;
- this.currIndex = UNSTARTED;
- }
-
- @Override
- public boolean start() throws IOException {
- assertEquals(UNSTARTED, currIndex);
- currIndex = 0;
- return currIndex < jsonTableRowReturns.length;
- }
-
- @Override
- public boolean advance() throws IOException {
- return ++currIndex < jsonTableRowReturns.length;
- }
-
- @Override
- public TableRow getCurrent() throws NoSuchElementException {
- if (currIndex >= jsonTableRowReturns.length) {
- throw new NoSuchElementException();
- }
- return fromJsonString(jsonTableRowReturns[currIndex], TableRow.class);
- }
-
- @Override
- public void close() throws IOException {
- currIndex = CLOSED;
- }
- }
- }
-
- private static class FakeJobService implements JobService, Serializable {
-
- private Object[] startJobReturns;
- private Object[] pollJobReturns;
- private String executingProject;
- // Both counts will be reset back to zeros after serialization.
- // This is a work around for DoFn's verifyUnmodified check.
- private transient int startJobCallsCount;
- private transient int pollJobStatusCallsCount;
-
- public FakeJobService() {
- this.startJobReturns = new Object[0];
- this.pollJobReturns = new Object[0];
- this.startJobCallsCount = 0;
- this.pollJobStatusCallsCount = 0;
- }
-
- /**
- * Sets the return values to mock {@link JobService#startLoadJob},
- * {@link JobService#startExtractJob} and {@link JobService#startQueryJob}.
- *
- * <p>Throws if the {@link Object} is a {@link Exception}, returns otherwise.
- */
- public FakeJobService startJobReturns(Object... startJobReturns) {
- this.startJobReturns = startJobReturns;
- return this;
- }
-
- /**
- * Sets the return values to mock {@link JobService#pollJob}.
- *
- * <p>Throws if the {@link Object} is a {@link Exception}, returns otherwise.
- */
- public FakeJobService pollJobReturns(Object... pollJobReturns) {
- this.pollJobReturns = pollJobReturns;
- return this;
- }
-
- /**
- * Verifies executing project.
- */
- public FakeJobService verifyExecutingProject(String executingProject) {
- this.executingProject = executingProject;
- return this;
- }
-
- @Override
- public void startLoadJob(JobReference jobRef, JobConfigurationLoad loadConfig)
- throws InterruptedException, IOException {
- startJob(jobRef);
- }
-
- @Override
- public void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig)
- throws InterruptedException, IOException {
- startJob(jobRef);
- }
-
- @Override
- public void startQueryJob(JobReference jobRef, JobConfigurationQuery query)
- throws IOException, InterruptedException {
- startJob(jobRef);
- }
-
- @Override
- public Job pollJob(JobReference jobRef, int maxAttempts)
- throws InterruptedException {
- if (!Strings.isNullOrEmpty(executingProject)) {
- checkArgument(
- jobRef.getProjectId().equals(executingProject),
- "Project id: %s is not equal to executing project: %s",
- jobRef.getProjectId(), executingProject);
- }
-
- if (pollJobStatusCallsCount < pollJobReturns.length) {
- Object ret = pollJobReturns[pollJobStatusCallsCount++];
- if (ret instanceof Job) {
- return (Job) ret;
- } else if (ret instanceof Status) {
- return JOB_STATUS_MAP.get(ret);
- } else if (ret instanceof InterruptedException) {
- throw (InterruptedException) ret;
- } else {
- throw new RuntimeException("Unexpected return type: " + ret.getClass());
- }
- } else {
- throw new RuntimeException(
- "Exceeded expected number of calls: " + pollJobReturns.length);
- }
- }
-
- private void startJob(JobReference jobRef) throws IOException, InterruptedException {
- if (!Strings.isNullOrEmpty(executingProject)) {
- checkArgument(
- jobRef.getProjectId().equals(executingProject),
- "Project id: %s is not equal to executing project: %s",
- jobRef.getProjectId(), executingProject);
- }
-
- if (startJobCallsCount < startJobReturns.length) {
- Object ret = startJobReturns[startJobCallsCount++];
- if (ret instanceof IOException) {
- throw (IOException) ret;
- } else if (ret instanceof InterruptedException) {
- throw (InterruptedException) ret;
- } else {
- return;
- }
- } else {
- throw new RuntimeException(
- "Exceeded expected number of calls: " + startJobReturns.length);
- }
- }
-
- @Override
- public JobStatistics dryRunQuery(String projectId, String query)
- throws InterruptedException, IOException {
- throw new UnsupportedOperationException();
- }
- }
-
- @Rule public transient ExpectedException thrown = ExpectedException.none();
- @Rule public transient ExpectedLogs logged = ExpectedLogs.none(BigQueryIO.class);
- @Rule public transient TemporaryFolder testFolder = new TemporaryFolder();
- @Mock(extraInterfaces = Serializable.class)
- public transient BigQueryServices.JobService mockJobService;
- @Mock private transient IOChannelFactory mockIOChannelFactory;
- @Mock(extraInterfaces = Serializable.class) private transient DatasetService mockDatasetService;
-
- private transient BigQueryOptions bqOptions;
-
- private void checkReadTableObject(
- BigQueryIO.Read.Bound bound, String project, String dataset, String table) {
- checkReadTableObjectWithValidate(bound, project, dataset, table, true);
- }
-
- private void checkReadQueryObject(BigQueryIO.Read.Bound bound, String query) {
- checkReadQueryObjectWithValidate(bound, query, true);
- }
-
- private void checkReadTableObjectWithValidate(
- BigQueryIO.Read.Bound bound, String project, String dataset, String table, boolean validate) {
- assertEquals(project, bound.getTable().getProjectId());
- assertEquals(dataset, bound.getTable().getDatasetId());
- assertEquals(table, bound.getTable().getTableId());
- assertNull(bound.query);
- assertEquals(validate, bound.getValidate());
- }
-
- private void checkReadQueryObjectWithValidate(
- BigQueryIO.Read.Bound bound, String query, boolean validate) {
- assertNull(bound.getTable());
- assertEquals(query, bound.query);
- assertEquals(validate, bound.getValidate());
- }
-
- private void checkWriteObject(
- BigQueryIO.Write.Bound bound, String project, String dataset, String table,
- TableSchema schema, CreateDisposition createDisposition,
- WriteDisposition writeDisposition) {
- checkWriteObjectWithValidate(
- bound, project, dataset, table, schema, createDisposition, writeDisposition, true);
- }
-
- private void checkWriteObjectWithValidate(
- BigQueryIO.Write.Bound bound, String project, String dataset, String table,
- TableSchema schema, CreateDisposition createDisposition,
- WriteDisposition writeDisposition, boolean validate) {
- assertEquals(project, bound.getTable().getProjectId());
- assertEquals(dataset, bound.getTable().getDatasetId());
- assertEquals(table, bound.getTable().getTableId());
- assertEquals(schema, bound.getSchema());
- assertEquals(createDisposition, bound.createDisposition);
- assertEquals(writeDisposition, bound.writeDisposition);
- assertEquals(validate, bound.validate);
- }
-
- @Before
- public void setUp() throws IOException {
- bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
- bqOptions.setProject("defaultProject");
- bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath());
-
- MockitoAnnotations.initMocks(this);
- }
-
- @Test
- public void testBuildTableBasedSource() {
- BigQueryIO.Read.Bound bound = BigQueryIO.Read.from("foo.com:project:somedataset.sometable");
- checkReadTableObject(bound, "foo.com:project", "somedataset", "sometable");
- }
-
- @Test
- public void testBuildQueryBasedSource() {
- BigQueryIO.Read.Bound bound = BigQueryIO.Read.fromQuery("foo_query");
- checkReadQueryObject(bound, "foo_query");
- }
-
- @Test
- public void testBuildTableBasedSourceWithoutValidation() {
- // This test just checks that using withoutValidation will not trigger object
- // construction errors.
- BigQueryIO.Read.Bound bound =
- BigQueryIO.Read.from("foo.com:project:somedataset.sometable").withoutValidation();
- checkReadTableObjectWithValidate(bound, "foo.com:project", "somedataset", "sometable", false);
- }
-
- @Test
- public void testBuildQueryBasedSourceWithoutValidation() {
- // This test just checks that using withoutValidation will not trigger object
- // construction errors.
- BigQueryIO.Read.Bound bound =
- BigQueryIO.Read.fromQuery("some_query").withoutValidation();
- checkReadQueryObjectWithValidate(bound, "some_query", false);
- }
-
- @Test
- public void testBuildTableBasedSourceWithDefaultProject() {
- BigQueryIO.Read.Bound bound =
- BigQueryIO.Read.from("somedataset.sometable");
- checkReadTableObject(bound, null, "somedataset", "sometable");
- }
-
- @Test
- public void testBuildSourceWithTableReference() {
- TableReference table = new TableReference()
- .setProjectId("foo.com:project")
- .setDatasetId("somedataset")
- .setTableId("sometable");
- BigQueryIO.Read.Bound bound = BigQueryIO.Read.from(table);
- checkReadTableObject(bound, "foo.com:project", "somedataset", "sometable");
- }
-
- @Test
- public void testValidateReadSetsDefaultProject() throws Exception {
- String projectId = "someproject";
- String datasetId = "somedataset";
- BigQueryOptions options = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
- options.setProject(projectId);
-
- FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
- .withJobService(mockJobService)
- .withDatasetService(mockDatasetService);
- when(mockDatasetService.getDataset(projectId, datasetId)).thenThrow(
- new RuntimeException("Unable to confirm BigQuery dataset presence"));
-
- Pipeline p = TestPipeline.create(options);
-
- TableReference tableRef = new TableReference();
- tableRef.setDatasetId(datasetId);
- tableRef.setTableId("sometable");
-
- thrown.expect(RuntimeException.class);
- // Message will be one of following depending on the execution environment.
- thrown.expectMessage(
- Matchers.either(Matchers.containsString("Unable to confirm BigQuery dataset presence"))
- .or(Matchers.containsString("BigQuery dataset not found for table")));
- p.apply(BigQueryIO.Read.from(tableRef)
- .withTestServices(fakeBqServices));
- }
-
- @Test
- @Category(RunnableOnService.class)
- public void testBuildSourceWithoutTableQueryOrValidation() {
- Pipeline p = TestPipeline.create();
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage(
- "Invalid BigQuery read operation, either table reference or query has to be set");
- p.apply(BigQueryIO.Read.withoutValidation());
- p.run();
- }
-
- @Test
- @Category(RunnableOnService.class)
- public void testBuildSourceWithTableAndQuery() {
- Pipeline p = TestPipeline.create();
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage(
- "Invalid BigQuery read operation. Specifies both a query and a table, only one of these"
- + " should be provided");
- p.apply("ReadMyTable",
- BigQueryIO.Read
- .from("foo.com:project:somedataset.sometable")
- .fromQuery("query"));
- p.run();
- }
-
- @Test
- @Category(RunnableOnService.class)
- public void testBuildSourceWithTableAndFlatten() {
- Pipeline p = TestPipeline.create();
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage(
- "Invalid BigQuery read operation. Specifies a"
- + " table with a result flattening preference, which is not configurable");
- p.apply("ReadMyTable",
- BigQueryIO.Read
- .from("foo.com:project:somedataset.sometable")
- .withoutResultFlattening());
- p.run();
- }
-
- @Test
- @Category(RunnableOnService.class)
- public void testBuildSourceWithTableAndFlattenWithoutValidation() {
- Pipeline p = TestPipeline.create();
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage(
- "Invalid BigQuery read operation. Specifies a"
- + " table with a result flattening preference, which is not configurable");
- p.apply(
- BigQueryIO.Read
- .from("foo.com:project:somedataset.sometable")
- .withoutValidation()
- .withoutResultFlattening());
- p.run();
- }
-
- @Test
- @Category(NeedsRunner.class)
- public void testReadFromTable() {
- FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
- .withJobService(new FakeJobService()
- .startJobReturns("done", "done")
- .pollJobReturns(Status.UNKNOWN)
- .verifyExecutingProject(bqOptions.getProject()))
- .readerReturns(
- toJsonString(new TableRow().set("name", "a").set("number", 1)),
- toJsonString(new TableRow().set("name", "b").set("number", 2)),
- toJsonString(new TableRow().set("name", "c").set("number", 3)));
-
- Pipeline p = TestPipeline.create(bqOptions);
- PCollection<String> output = p
- .apply(BigQueryIO.Read.from("non-executing-project:somedataset.sometable")
- .withTestServices(fakeBqServices)
- .withoutValidation())
- .apply(ParDo.of(new DoFn<TableRow, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- c.output((String) c.element().get("name"));
- }
- }));
-
- PAssert.that(output)
- .containsInAnyOrder(ImmutableList.of("a", "b", "c"));
-
- p.run();
- }
-
- @Test
- @Category(NeedsRunner.class)
- public void testCustomSink() throws Exception {
- FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
- .withJobService(new FakeJobService()
- .startJobReturns("done", "done", "done")
- .pollJobReturns(Status.FAILED, Status.FAILED, Status.SUCCEEDED));
-
- Pipeline p = TestPipeline.create(bqOptions);
- p.apply(Create.of(
- new TableRow().set("name", "a").set("number", 1),
- new TableRow().set("name", "b").set("number", 2),
- new TableRow().set("name", "c").set("number", 3))
- .withCoder(TableRowJsonCoder.of()))
- .apply(BigQueryIO.Write.to("dataset-id.table-id")
- .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
- .withSchema(new TableSchema().setFields(
- ImmutableList.of(
- new TableFieldSchema().setName("name").setType("STRING"),
- new TableFieldSchema().setName("number").setType("INTEGER"))))
- .withTestServices(fakeBqServices)
- .withoutValidation());
- p.run();
-
- logged.verifyInfo("Starting BigQuery load job");
- logged.verifyInfo("Previous load jobs failed, retrying.");
- File tempDir = new File(bqOptions.getTempLocation());
- assertEquals(0, tempDir.listFiles(new FileFilter() {
- @Override
- public boolean accept(File pathname) {
- return pathname.isFile();
- }}).length);
- }
-
- @Test
- @Category(NeedsRunner.class)
- public void testCustomSinkUnknown() throws Exception {
- FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
- .withJobService(new FakeJobService()
- .startJobReturns("done", "done")
- .pollJobReturns(Status.FAILED, Status.UNKNOWN));
-
- Pipeline p = TestPipeline.create(bqOptions);
- p.apply(Create.of(
- new TableRow().set("name", "a").set("number", 1),
- new TableRow().set("name", "b").set("number", 2),
- new TableRow().set("name", "c").set("number", 3))
- .withCoder(TableRowJsonCoder.of()))
- .apply(BigQueryIO.Write.to("project-id:dataset-id.table-id")
- .withCreateDisposition(CreateDisposition.CREATE_NEVER)
- .withTestServices(fakeBqServices)
- .withoutValidation());
-
- thrown.expect(RuntimeException.class);
- thrown.expectMessage("Failed to poll the load job status.");
- p.run();
-
- File tempDir = new File(bqOptions.getTempLocation());
- assertEquals(0, tempDir.listFiles(new FileFilter() {
- @Override
- public boolean accept(File pathname) {
- return pathname.isFile();
- }}).length);
- }
-
- @Test
- public void testBuildSourceDisplayData() {
- String tableSpec = "project:dataset.tableid";
-
- BigQueryIO.Read.Bound read = BigQueryIO.Read
- .from(tableSpec)
- .fromQuery("myQuery")
- .withoutResultFlattening()
- .withoutValidation();
-
- DisplayData displayData = DisplayData.from(read);
-
- assertThat(displayData, hasDisplayItem("table", tableSpec));
- assertThat(displayData, hasDisplayItem("query", "myQuery"));
- assertThat(displayData, hasDisplayItem("flattenResults", false));
- assertThat(displayData, hasDisplayItem("validation", false));
- }
-
- @Test
- @Category(RunnableOnService.class)
- @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient")
- public void testTableSourcePrimitiveDisplayData() throws IOException, InterruptedException {
- DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
- BigQueryIO.Read.Bound read = BigQueryIO.Read
- .from("project:dataset.tableId")
- .withTestServices(new FakeBigQueryServices()
- .withDatasetService(mockDatasetService)
- .withJobService(mockJobService))
- .withoutValidation();
-
- Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
- assertThat("BigQueryIO.Read should include the table spec in its primitive display data",
- displayData, hasItem(hasDisplayItem("table")));
- }
-
- @Test
- @Category(RunnableOnService.class)
- @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient")
- public void testQuerySourcePrimitiveDisplayData() throws IOException, InterruptedException {
- DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
- BigQueryIO.Read.Bound read = BigQueryIO.Read
- .fromQuery("foobar")
- .withTestServices(new FakeBigQueryServices()
- .withDatasetService(mockDatasetService)
- .withJobService(mockJobService))
- .withoutValidation();
-
- Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
- assertThat("BigQueryIO.Read should include the query in its primitive display data",
- displayData, hasItem(hasDisplayItem("query")));
- }
-
-
- @Test
- public void testBuildSink() {
- BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("foo.com:project:somedataset.sometable");
- checkWriteObject(
- bound, "foo.com:project", "somedataset", "sometable",
- null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
- }
-
- @Test
- @Category(RunnableOnService.class)
- @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient")
- public void testBatchSinkPrimitiveDisplayData() throws IOException, InterruptedException {
- testSinkPrimitiveDisplayData(/* streaming: */ false);
- }
-
- @Test
- @Category(RunnableOnService.class)
- @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient")
- public void testStreamingSinkPrimitiveDisplayData() throws IOException, InterruptedException {
- testSinkPrimitiveDisplayData(/* streaming: */ true);
- }
-
- private void testSinkPrimitiveDisplayData(boolean streaming) throws IOException,
- InterruptedException {
- PipelineOptions options = TestPipeline.testingPipelineOptions();
- options.as(StreamingOptions.class).setStreaming(streaming);
- DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(options);
-
- BigQueryIO.Write.Bound write = BigQueryIO.Write
- .to("project:dataset.table")
- .withSchema(new TableSchema().set("col1", "type1").set("col2", "type2"))
- .withTestServices(new FakeBigQueryServices()
- .withDatasetService(mockDatasetService)
- .withJobService(mockJobService))
- .withoutValidation();
-
- Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(write);
- assertThat("BigQueryIO.Write should include the table spec in its primitive display data",
- displayData, hasItem(hasDisplayItem("tableSpec")));
-
- assertThat("BigQueryIO.Write should include the table schema in its primitive display data",
- displayData, hasItem(hasDisplayItem("schema")));
- }
-
- @Test
- public void testBuildSinkwithoutValidation() {
- // This test just checks that using withoutValidation will not trigger object
- // construction errors.
- BigQueryIO.Write.Bound bound =
- BigQueryIO.Write.to("foo.com:project:somedataset.sometable").withoutValidation();
- checkWriteObjectWithValidate(
- bound, "foo.com:project", "somedataset", "sometable",
- null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, false);
- }
-
- @Test
- public void testBuildSinkDefaultProject() {
- BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("somedataset.sometable");
- checkWriteObject(
- bound, null, "somedataset", "sometable",
- null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
- }
-
- @Test
- public void testBuildSinkWithTableReference() {
- TableReference table = new TableReference()
- .setProjectId("foo.com:project")
- .setDatasetId("somedataset")
- .setTableId("sometable");
- BigQueryIO.Write.Bound bound = BigQueryIO.Write.to(table);
- checkWriteObject(
- bound, "foo.com:project", "somedataset", "sometable",
- null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
- }
-
- @Test
- @Category(RunnableOnService.class)
- public void testBuildSinkWithoutTable() {
- Pipeline p = TestPipeline.create();
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("must set the table reference");
- p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()))
- .apply(BigQueryIO.Write.withoutValidation());
- }
-
- @Test
- public void testBuildSinkWithSchema() {
- TableSchema schema = new TableSchema();
- BigQueryIO.Write.Bound bound =
- BigQueryIO.Write.to("foo.com:project:somedataset.sometable").withSchema(schema);
- checkWriteObject(
- bound, "foo.com:project", "somedataset", "sometable",
- schema, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
- }
-
- @Test
- public void testBuildSinkWithCreateDispositionNever() {
- BigQueryIO.Write.Bound bound = BigQueryIO.Write
- .to("foo.com:project:somedataset.sometable")
- .withCreateDisposition(CreateDisposition.CREATE_NEVER);
- checkWriteObject(
- bound, "foo.com:project", "somedataset", "sometable",
- null, CreateDisposition.CREATE_NEVER, WriteDisposition.WRITE_EMPTY);
- }
-
- @Test
- public void testBuildSinkWithCreateDispositionIfNeeded() {
- BigQueryIO.Write.Bound bound = BigQueryIO.Write
- .to("foo.com:project:somedataset.sometable")
- .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED);
- checkWriteObject(
- bound, "foo.com:project", "somedataset", "sometable",
- null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
- }
-
- @Test
- public void testBuildSinkWithWriteDispositionTruncate() {
- BigQueryIO.Write.Bound bound = BigQueryIO.Write
- .to("foo.com:project:somedataset.sometable")
- .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE);
- checkWriteObject(
- bound, "foo.com:project", "somedataset", "sometable",
- null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_TRUNCATE);
- }
-
- @Test
- public void testBuildSinkWithWriteDispositionAppend() {
- BigQueryIO.Write.Bound bound = BigQueryIO.Write
- .to("foo.com:project:somedataset.sometable")
- .withWriteDisposition(WriteDisposition.WRITE_APPEND);
- checkWriteObject(
- bound, "foo.com:project", "somedataset", "sometable",
- null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_APPEND);
- }
-
- @Test
- public void testBuildSinkWithWriteDispositionEmpty() {
- BigQueryIO.Write.Bound bound = BigQueryIO.Write
- .to("foo.com:project:somedataset.sometable")
- .withWriteDisposition(WriteDisposition.WRITE_EMPTY);
- checkWriteObject(
- bound, "foo.com:project", "somedataset", "sometable",
- null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
- }
-
- @Test
- public void testBuildSinkDisplayData() {
- String tableSpec = "project:dataset.table";
- TableSchema schema = new TableSchema().set("col1", "type1").set("col2", "type2");
-
- BigQueryIO.Write.Bound write = BigQueryIO.Write
- .to(tableSpec)
- .withSchema(schema)
- .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
- .withWriteDisposition(WriteDisposition.WRITE_APPEND)
- .withoutValidation();
-
- DisplayData displayData = DisplayData.from(write);
-
- assertThat(displayData, hasDisplayItem("table"));
- assertThat(displayData, hasDisplayItem("schema"));
- assertThat(displayData,
- hasDisplayItem("createDisposition", CreateDisposition.CREATE_IF_NEEDED.toString()));
- assertThat(displayData,
- hasDisplayItem("writeDisposition", WriteDisposition.WRITE_APPEND.toString()));
- assertThat(displayData, hasDisplayItem("validation", false));
- }
-
- private void testWriteValidatesDataset(boolean streaming) throws Exception {
- String projectId = "someproject";
- String datasetId = "somedataset";
-
- BigQueryOptions options = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
- options.setProject(projectId);
- options.setStreaming(streaming);
-
- FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
- .withJobService(mockJobService)
- .withDatasetService(mockDatasetService);
- when(mockDatasetService.getDataset(projectId, datasetId)).thenThrow(
- new RuntimeException("Unable to confirm BigQuery dataset presence"));
-
- Pipeline p = TestPipeline.create(options);
-
- TableReference tableRef = new TableReference();
- tableRef.setDatasetId(datasetId);
- tableRef.setTableId("sometable");
-
- thrown.expect(RuntimeException.class);
- // Message will be one of following depending on the execution environment.
- thrown.expectMessage(
- Matchers.either(Matchers.containsString("Unable to confirm BigQuery dataset presence"))
- .or(Matchers.containsString("BigQuery dataset not found for table")));
- p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()))
- .apply(BigQueryIO.Write
- .to(tableRef)
- .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
- .withSchema(new TableSchema())
- .withTestServices(fakeBqServices));
- }
-
- @Test
- public void testWriteValidatesDatasetBatch() throws Exception {
- testWriteValidatesDataset(false);
- }
-
- @Test
- public void testWriteValidatesDatasetStreaming() throws Exception {
- testWriteValidatesDataset(true);
- }
-
- @Test
- public void testTableParsing() {
- TableReference ref = BigQueryIO
- .parseTableSpec("my-project:data_set.table_name");
- Assert.assertEquals("my-project", ref.getProjectId());
- Assert.assertEquals("data_set", ref.getDatasetId());
- Assert.assertEquals("table_name", ref.getTableId());
- }
-
- @Test
- public void testTableParsing_validPatterns() {
- BigQueryIO.parseTableSpec("a123-456:foo_bar.d");
- BigQueryIO.parseTableSpec("a12345:b.c");
- BigQueryIO.parseTableSpec("b12345.c");
- }
-
- @Test
- public void testTableParsing_noProjectId() {
- TableReference ref = BigQueryIO
- .parseTableSpec("data_set.table_name");
- Assert.assertEquals(null, ref.getProjectId());
- Assert.assertEquals("data_set", ref.getDatasetId());
- Assert.assertEquals("table_name", ref.getTableId());
- }
-
- @Test
- public void testTableParsingError() {
- thrown.expect(IllegalArgumentException.class);
- BigQueryIO.parseTableSpec("0123456:foo.bar");
- }
-
- @Test
- public void testTableParsingError_2() {
- thrown.expect(IllegalArgumentException.class);
- BigQueryIO.parseTableSpec("myproject:.bar");
- }
-
- @Test
- public void testTableParsingError_3() {
- thrown.expect(IllegalArgumentException.class);
- BigQueryIO.parseTableSpec(":a.b");
- }
-
- @Test
- public void testTableParsingError_slash() {
- thrown.expect(IllegalArgumentException.class);
- BigQueryIO.parseTableSpec("a\\b12345:c.d");
- }
-
- // Test that BigQuery's special null placeholder objects can be encoded.
- @Test
- public void testCoder_nullCell() throws CoderException {
- TableRow row = new TableRow();
- row.set("temperature", Data.nullOf(Object.class));
- row.set("max_temperature", Data.nullOf(Object.class));
-
- byte[] bytes = CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), row);
-
- TableRow newRow = CoderUtils.decodeFromByteArray(TableRowJsonCoder.of(), bytes);
- byte[] newBytes = CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), newRow);
-
- Assert.assertArrayEquals(bytes, newBytes);
- }
-
- @Test
- public void testBigQueryIOGetName() {
- assertEquals("BigQueryIO.Read", BigQueryIO.Read.from("somedataset.sometable").getName());
- assertEquals("BigQueryIO.Write", BigQueryIO.Write.to("somedataset.sometable").getName());
- }
-
- @Test
- public void testWriteValidateFailsCreateNoSchema() {
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("no schema was provided");
- TestPipeline.create()
- .apply(Create.<TableRow>of())
- .apply(BigQueryIO.Write
- .to("dataset.table")
- .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED));
- }
-
- @Test
- public void testWriteValidateFailsTableAndTableSpec() {
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("Cannot set both a table reference and a table function");
- TestPipeline.create()
- .apply(Create.<TableRow>of())
- .apply(BigQueryIO.Write
- .to("dataset.table")
- .to(new SerializableFunction<BoundedWindow, String>() {
- @Override
- public String apply(BoundedWindow input) {
- return null;
- }
- }));
- }
-
- @Test
- public void testWriteValidateFailsNoTableAndNoTableSpec() {
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("must set the table reference of a BigQueryIO.Write transform");
- TestPipeline.create()
- .apply(Create.<TableRow>of())
- .apply("name", BigQueryIO.Write.withoutValidation());
- }
-
- @Test
- public void testBigQueryTableSourceThroughJsonAPI() throws Exception {
- FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
- .withJobService(mockJobService)
- .readerReturns(
- toJsonString(new TableRow().set("name", "a").set("number", "1")),
- toJsonString(new TableRow().set("name", "b").set("number", "2")),
- toJsonString(new TableRow().set("name", "c").set("number", "3")));
-
- String jobIdToken = "testJobIdToken";
- TableReference table = BigQueryIO.parseTableSpec("project.data_set.table_name");
- String extractDestinationDir = "mock://tempLocation";
- BoundedSource<TableRow> bqSource = BigQueryTableSource.create(
- jobIdToken, table, extractDestinationDir, fakeBqServices, "project");
-
- List<TableRow> expected = ImmutableList.of(
- new TableRow().set("name", "a").set("number", "1"),
- new TableRow().set("name", "b").set("number", "2"),
- new TableRow().set("name", "c").set("number", "3"));
-
- PipelineOptions options = PipelineOptionsFactory.create();
- Assert.assertThat(
- SourceTestUtils.readFromSource(bqSource, options),
- CoreMatchers.is(expected));
- SourceTestUtils.assertSplitAtFractionBehavior(
- bqSource, 2, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options);
- }
-
- @Test
- public void testBigQueryTableSourceInitSplit() throws Exception {
- Job extractJob = new Job();
- JobStatistics jobStats = new JobStatistics();
- JobStatistics4 extractStats = new JobStatistics4();
- extractStats.setDestinationUriFileCounts(ImmutableList.of(1L));
- jobStats.setExtract(extractStats);
- extractJob.setStatus(new JobStatus())
- .setStatistics(jobStats);
-
- FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
- .withJobService(mockJobService)
- .withDatasetService(mockDatasetService)
- .readerReturns(
- toJsonString(new TableRow().set("name", "a").set("number", "1")),
- toJsonString(new TableRow().set("name", "b").set("number", "2")),
- toJsonString(new TableRow().set("name", "c").set("number", "3")));
-
- String jobIdToken = "testJobIdToken";
- TableReference table = BigQueryIO.parseTableSpec("project:data_set.table_name");
- String extractDestinationDir = "mock://tempLocation";
- BoundedSource<TableRow> bqSource = BigQueryTableSource.create(
- jobIdToken, table, extractDestinationDir, fakeBqServices, "project");
-
- List<TableRow> expected = ImmutableList.of(
- new TableRow().set("name", "a").set("number", "1"),
- new TableRow().set("name", "b").set("number", "2"),
- new TableRow().set("name", "c").set("number", "3"));
-
- when(mockJobService.pollJob(Mockito.<JobReference>any(), Mockito.anyInt()))
- .thenReturn(extractJob);
- PipelineOptions options = PipelineOptionsFactory.create();
- options.setTempLocation("mock://tempLocation");
-
- IOChannelUtils.setIOFactory("mock", mockIOChannelFactory);
- when(mockIOChannelFactory.resolve(anyString(), anyString()))
- .thenReturn("mock://tempLocation/output");
- when(mockDatasetService.getTable(anyString(), anyString(), anyString()))
- .thenReturn(new Table().setSchema(new TableSchema()));
-
- Assert.assertThat(
- SourceTestUtils.readFromSource(bqSource, options),
- CoreMatchers.is(expected));
- SourceTestUtils.assertSplitAtFractionBehavior(
- bqSource, 2, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options);
-
- List<? extends BoundedSource<TableRow>> sources = bqSource.splitIntoBundles(100, options);
- assertEquals(1, sources.size());
- BoundedSource<TableRow> actual = sources.get(0);
- assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class));
-
- Mockito.verify(mockJobService)
- .startExtractJob(Mockito.<JobReference>any(), Mockito.<JobConfigurationExtract>any());
- }
-
- @Test
- public void testBigQueryQuerySourceInitSplit() throws Exception {
- TableReference dryRunTable = new TableReference();
-
- Job queryJob = new Job();
- JobStatistics queryJobStats = new JobStatistics();
- JobStatistics2 queryStats = new JobStatistics2();
- queryStats.setReferencedTables(ImmutableList.of(dryRunTable));
- queryJobStats.setQuery(queryStats);
- queryJob.setStatus(new JobStatus())
- .setStatistics(queryJobStats);
-
- Job extractJob = new Job();
- JobStatistics extractJobStats = new JobStatistics();
- JobStatistics4 extractStats = new JobStatistics4();
- extractStats.setDestinationUriFileCounts(ImmutableList.of(1L));
- extractJobStats.setExtract(extractStats);
- extractJob.setStatus(new JobStatus())
- .setStatistics(extractJobStats);
-
- FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
- .withJobService(mockJobService)
- .withDatasetService(mockDatasetService)
- .readerReturns(
- toJsonString(new TableRow().set("name", "a").set("number", "1")),
- toJsonString(new TableRow().set("name", "b").set("number", "2")),
- toJsonString(new TableRow().set("name", "c").set("number", "3")));
-
- String jobIdToken = "testJobIdToken";
- String extractDestinationDir = "mock://tempLocation";
- TableReference destinationTable = BigQueryIO.parseTableSpec("project:data_set.table_name");
- BoundedSource<TableRow> bqSource = BigQueryQuerySource.create(
- jobIdToken, "query", destinationTable, true /* flattenResults */,
- extractDestinationDir, fakeBqServices);
-
- List<TableRow> expected = ImmutableList.of(
- new TableRow().set("name", "a").set("number", "1"),
- new TableRow().set("name", "b").set("number", "2"),
- new TableRow().set("name", "c").set("number", "3"));
-
- PipelineOptions options = PipelineOptionsFactory.create();
- options.setTempLocation(extractDestinationDir);
-
- TableReference queryTable = new TableReference()
- .setProjectId("testProejct")
- .setDatasetId("testDataset")
- .setTableId("testTable");
- when(mockJobService.dryRunQuery(anyString(), anyString()))
- .thenReturn(new JobStatistics().setQuery(
- new JobStatistics2()
- .setTotalBytesProcessed(100L)
- .setReferencedTables(ImmutableList.of(queryTable))));
- when(mockDatasetService.getTable(
- eq(queryTable.getProjectId()), eq(queryTable.getDatasetId()), eq(queryTable.getTableId())))
- .thenReturn(new Table().setSchema(new TableSchema()));
- when(mockDatasetService.getTable(
- eq(destinationTable.getProjectId()),
- eq(destinationTable.getDatasetId()),
- eq(destinationTable.getTableId())))
- .thenReturn(new Table().setSchema(new TableSchema()));
- IOChannelUtils.setIOFactory("mock", mockIOChannelFactory);
- when(mockIOChannelFactory.resolve(anyString(), anyString()))
- .thenReturn("mock://tempLocation/output");
- when(mockJobService.pollJob(Mockito.<JobReference>any(), Mockito.anyInt()))
- .thenReturn(extractJob);
-
- Assert.assertThat(
- SourceTestUtils.readFromSource(bqSource, options),
- CoreMatchers.is(expected));
- SourceTestUtils.assertSplitAtFractionBehavior(
- bqSource, 2, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options);
-
- List<? extends BoundedSource<TableRow>> sources = bqSource.splitIntoBundles(100, options);
- assertEquals(1, sources.size());
- BoundedSource<TableRow> actual = sources.get(0);
- assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class));
-
- Mockito.verify(mockJobService)
- .startQueryJob(
- Mockito.<JobReference>any(), Mockito.<JobConfigurationQuery>any());
- Mockito.verify(mockJobService)
- .startExtractJob(Mockito.<JobReference>any(), Mockito.<JobConfigurationExtract>any());
- Mockito.verify(mockDatasetService)
- .createDataset(anyString(), anyString(), anyString(), anyString());
- }
-
- @Test
- public void testTransformingSource() throws Exception {
- int numElements = 10000;
- @SuppressWarnings("deprecation")
- BoundedSource<Long> longSource = CountingSource.upTo(numElements);
- SerializableFunction<Long, String> toStringFn =
- new SerializableFunction<Long, String>() {
- @Override
- public String apply(Long input) {
- return input.toString();
- }};
- BoundedSource<String> stringSource = new TransformingSource<>(
- longSource, toStringFn, StringUtf8Coder.of());
-
- List<String> expected = Lists.newArrayList();
- for (int i = 0; i < numElements; i++) {
- expected.add(String.valueOf(i));
- }
-
- PipelineOptions options = PipelineOptionsFactory.create();
- Assert.assertThat(
- SourceTestUtils.readFromSource(stringSource, options),
- CoreMatchers.is(expected));
- SourceTestUtils.assertSplitAtFractionBehavior(
- stringSource, 100, 0.3, ExpectedSplitOutcome.MUST_SUCCEED_AND_BE_CONSISTENT, options);
-
- SourceTestUtils.assertSourcesEqualReferenceSource(
- stringSource, stringSource.splitIntoBundles(100, options), options);
- }
-
- @Test
- @Category(RunnableOnService.class)
- public void testPassThroughThenCleanup() throws Exception {
- Pipeline p = TestPipeline.create();
-
- PCollection<Integer> output = p
- .apply(Create.of(1, 2, 3))
- .apply(new PassThroughThenCleanup<Integer>(new CleanupOperation() {
- @Override
- void cleanup(PipelineOptions options) throws Exception {
- // no-op
- }}));
-
- PAssert.that(output).containsInAnyOrder(1, 2, 3);
-
- p.run();
- }
-
- @Test
- @Category(NeedsRunner.class)
- public void testPassThroughThenCleanupExecuted() throws Exception {
- Pipeline p = TestPipeline.create();
-
- p.apply(Create.<Integer>of())
- .apply(new PassThroughThenCleanup<Integer>(new CleanupOperation() {
- @Override
- void cleanup(PipelineOptions options) throws Exception {
- throw new RuntimeException("cleanup executed");
- }}));
-
- thrown.expect(RuntimeException.class);
- thrown.expectMessage("cleanup executed");
-
- p.run();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java
index 38e921a..b72ab9a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java
@@ -23,17 +23,10 @@ import org.apache.beam.sdk.coders.AvroCoder;
import org.apache.beam.sdk.coders.DefaultCoder;
import org.apache.beam.sdk.util.AvroUtils.AvroMetadata;
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.common.collect.Lists;
-
import org.apache.avro.Schema;
import org.apache.avro.file.CodecFactory;
import org.apache.avro.file.DataFileConstants;
import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumWriter;
import org.apache.avro.reflect.Nullable;
import org.junit.Rule;
@@ -109,77 +102,6 @@ public class AvroUtilsTest {
assertEquals(8, schema.getFields().size());
}
- @Test
- public void testConvertGenericRecordToTableRow() throws Exception {
- TableSchema tableSchema = new TableSchema();
- List<TableFieldSchema> subFields = Lists.<TableFieldSchema>newArrayList(
- new TableFieldSchema().setName("species").setType("STRING").setMode("NULLABLE"));
- /*
- * Note that the quality and quantity fields do not have their mode set, so they should default
- * to NULLABLE. This is an important test of BigQuery semantics.
- *
- * All the other fields we set in this function are required on the Schema response.
- *
- * See https://cloud.google.com/bigquery/docs/reference/v2/tables#schema
- */
- List<TableFieldSchema> fields =
- Lists.<TableFieldSchema>newArrayList(
- new TableFieldSchema().setName("number").setType("INTEGER").setMode("REQUIRED"),
- new TableFieldSchema().setName("species").setType("STRING").setMode("NULLABLE"),
- 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("flighted").setType("BOOLEAN").setMode("NULLABLE"),
- new TableFieldSchema().setName("scion").setType("RECORD").setMode("NULLABLE")
- .setFields(subFields),
- new TableFieldSchema().setName("associates").setType("RECORD").setMode("REPEATED")
- .setFields(subFields));
- tableSchema.setFields(fields);
- Schema avroSchema = AvroCoder.of(Bird.class).getSchema();
-
- {
- // Test nullable fields.
- GenericRecord record = new GenericData.Record(avroSchema);
- record.put("number", 5L);
- TableRow convertedRow = AvroUtils.convertGenericRecordToTableRow(record, tableSchema);
- TableRow row = new TableRow()
- .set("number", "5")
- .set("associates", new ArrayList<TableRow>());
- assertEquals(row, convertedRow);
- }
- {
- // Test type conversion for TIMESTAMP, INTEGER, BOOLEAN, and FLOAT.
- GenericRecord record = new GenericData.Record(avroSchema);
- record.put("number", 5L);
- record.put("quality", 5.0);
- record.put("birthday", 5L);
- record.put("flighted", Boolean.TRUE);
- TableRow convertedRow = AvroUtils.convertGenericRecordToTableRow(record, tableSchema);
- TableRow row = new TableRow()
- .set("number", "5")
- .set("birthday", "1970-01-01 00:00:00.000005 UTC")
- .set("quality", 5.0)
- .set("associates", new ArrayList<TableRow>())
- .set("flighted", Boolean.TRUE);
- assertEquals(row, convertedRow);
- }
- {
- // Test repeated fields.
- Schema subBirdSchema = AvroCoder.of(Bird.SubBird.class).getSchema();
- GenericRecord nestedRecord = new GenericData.Record(subBirdSchema);
- nestedRecord.put("species", "other");
- GenericRecord record = new GenericData.Record(avroSchema);
- record.put("number", 5L);
- record.put("associates", Lists.<GenericRecord>newArrayList(nestedRecord));
- TableRow convertedRow = AvroUtils.convertGenericRecordToTableRow(record, tableSchema);
- TableRow row = new TableRow()
- .set("associates", Lists.<TableRow>newArrayList(
- new TableRow().set("species", "other")))
- .set("number", "5");
- assertEquals(row, convertedRow);
- }
- }
-
/**
* Pojo class used as the record type in tests.
*/
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java
deleted file mode 100644
index 3ec2b37..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java
+++ /dev/null
@@ -1,303 +0,0 @@
-/*
- * 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.util;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.testing.ExpectedLogs;
-import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
-import org.apache.beam.sdk.util.BigQueryServicesImpl.JobServiceImpl;
-
-import com.google.api.client.googleapis.json.GoogleJsonError;
-import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
-import com.google.api.client.googleapis.json.GoogleJsonErrorContainer;
-import com.google.api.client.http.LowLevelHttpResponse;
-import com.google.api.client.json.GenericJson;
-import com.google.api.client.json.Json;
-import com.google.api.client.json.jackson2.JacksonFactory;
-import com.google.api.client.testing.http.MockHttpTransport;
-import com.google.api.client.testing.http.MockLowLevelHttpRequest;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.model.ErrorProto;
-import com.google.api.services.bigquery.model.Job;
-import com.google.api.services.bigquery.model.JobConfigurationLoad;
-import com.google.api.services.bigquery.model.JobReference;
-import com.google.api.services.bigquery.model.JobStatus;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.cloud.hadoop.util.ApiErrorExtractor;
-import com.google.common.collect.ImmutableList;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * Tests for {@link BigQueryServicesImpl}.
- */
-@RunWith(JUnit4.class)
-public class BigQueryServicesImplTest {
- @Rule public ExpectedException thrown = ExpectedException.none();
- @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(BigQueryServicesImpl.class);
- @Mock private LowLevelHttpResponse response;
- private Bigquery bigquery;
-
- @Before
- public void setUp() {
- MockitoAnnotations.initMocks(this);
-
- // A mock transport that lets us mock the API responses.
- MockHttpTransport transport =
- new MockHttpTransport.Builder()
- .setLowLevelHttpRequest(
- new MockLowLevelHttpRequest() {
- @Override
- public LowLevelHttpResponse execute() throws IOException {
- return response;
- }
- })
- .build();
-
- // A sample BigQuery API client that uses default JsonFactory and RetryHttpInitializer.
- bigquery =
- new Bigquery.Builder(
- transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer())
- .build();
- }
-
- /**
- * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds.
- */
- @Test
- public void testStartLoadJobSucceeds() throws IOException, InterruptedException {
- Job testJob = new Job();
- JobReference jobRef = new JobReference();
- jobRef.setJobId("jobId");
- jobRef.setProjectId("projectId");
- testJob.setJobReference(jobRef);
-
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(200);
- when(response.getContent()).thenReturn(toStream(testJob));
-
- TableReference ref = new TableReference();
- ref.setProjectId("projectId");
- JobConfigurationLoad loadConfig = new JobConfigurationLoad();
- loadConfig.setDestinationTable(ref);
-
- Sleeper sleeper = new FastNanoClockAndSleeper();
- BackOff backoff = new AttemptBoundedExponentialBackOff(
- 5 /* attempts */, 1000 /* initialIntervalMillis */);
- JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff);
-
- verify(response, times(1)).getStatusCode();
- verify(response, times(1)).getContent();
- verify(response, times(1)).getContentType();
- }
-
- /**
- * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds
- * with an already exist job.
- */
- @Test
- public void testStartLoadJobSucceedsAlreadyExists() throws IOException, InterruptedException {
- Job testJob = new Job();
- JobReference jobRef = new JobReference();
- jobRef.setJobId("jobId");
- jobRef.setProjectId("projectId");
- testJob.setJobReference(jobRef);
-
- when(response.getStatusCode()).thenReturn(409); // 409 means already exists
-
- TableReference ref = new TableReference();
- ref.setProjectId("projectId");
- JobConfigurationLoad loadConfig = new JobConfigurationLoad();
- loadConfig.setDestinationTable(ref);
-
- Sleeper sleeper = new FastNanoClockAndSleeper();
- BackOff backoff = new AttemptBoundedExponentialBackOff(
- 5 /* attempts */, 1000 /* initialIntervalMillis */);
- JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff);
-
- verify(response, times(1)).getStatusCode();
- verify(response, times(1)).getContent();
- verify(response, times(1)).getContentType();
- }
-
- /**
- * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds with a retry.
- */
- @Test
- public void testStartLoadJobRetry() throws IOException, InterruptedException {
- Job testJob = new Job();
- JobReference jobRef = new JobReference();
- jobRef.setJobId("jobId");
- jobRef.setProjectId("projectId");
- testJob.setJobReference(jobRef);
-
- // First response is 403 rate limited, second response has valid payload.
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(403).thenReturn(200);
- when(response.getContent())
- .thenReturn(toStream(errorWithReasonAndStatus("rateLimitExceeded", 403)))
- .thenReturn(toStream(testJob));
-
- TableReference ref = new TableReference();
- ref.setProjectId("projectId");
- JobConfigurationLoad loadConfig = new JobConfigurationLoad();
- loadConfig.setDestinationTable(ref);
-
- Sleeper sleeper = new FastNanoClockAndSleeper();
- BackOff backoff = new AttemptBoundedExponentialBackOff(
- 5 /* attempts */, 1000 /* initialIntervalMillis */);
- JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff);
-
- verify(response, times(2)).getStatusCode();
- verify(response, times(2)).getContent();
- verify(response, times(2)).getContentType();
- }
-
- /**
- * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} succeeds.
- */
- @Test
- public void testPollJobSucceeds() throws IOException, InterruptedException {
- Job testJob = new Job();
- testJob.setStatus(new JobStatus().setState("DONE"));
-
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(200);
- when(response.getContent()).thenReturn(toStream(testJob));
-
- BigQueryServicesImpl.JobServiceImpl jobService =
- new BigQueryServicesImpl.JobServiceImpl(bigquery);
- JobReference jobRef = new JobReference()
- .setProjectId("projectId")
- .setJobId("jobId");
- Job job = jobService.pollJob(jobRef, Sleeper.DEFAULT, BackOff.ZERO_BACKOFF);
-
- assertEquals(testJob, job);
- verify(response, times(1)).getStatusCode();
- verify(response, times(1)).getContent();
- verify(response, times(1)).getContentType();
- }
-
- /**
- * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} fails.
- */
- @Test
- public void testPollJobFailed() throws IOException, InterruptedException {
- Job testJob = new Job();
- testJob.setStatus(new JobStatus().setState("DONE").setErrorResult(new ErrorProto()));
-
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(200);
- when(response.getContent()).thenReturn(toStream(testJob));
-
- BigQueryServicesImpl.JobServiceImpl jobService =
- new BigQueryServicesImpl.JobServiceImpl(bigquery);
- JobReference jobRef = new JobReference()
- .setProjectId("projectId")
- .setJobId("jobId");
- Job job = jobService.pollJob(jobRef, Sleeper.DEFAULT, BackOff.ZERO_BACKOFF);
-
- assertEquals(testJob, job);
- verify(response, times(1)).getStatusCode();
- verify(response, times(1)).getContent();
- verify(response, times(1)).getContentType();
- }
-
- /**
- * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} returns UNKNOWN.
- */
- @Test
- public void testPollJobUnknown() throws IOException, InterruptedException {
- Job testJob = new Job();
- testJob.setStatus(new JobStatus());
-
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(200);
- when(response.getContent()).thenReturn(toStream(testJob));
-
- BigQueryServicesImpl.JobServiceImpl jobService =
- new BigQueryServicesImpl.JobServiceImpl(bigquery);
- JobReference jobRef = new JobReference()
- .setProjectId("projectId")
- .setJobId("jobId");
- Job job = jobService.pollJob(jobRef, Sleeper.DEFAULT, BackOff.STOP_BACKOFF);
-
- assertEquals(null, job);
- verify(response, times(1)).getStatusCode();
- verify(response, times(1)).getContent();
- verify(response, times(1)).getContentType();
- }
-
- @Test
- public void testExecuteWithRetries() throws IOException, InterruptedException {
- Table testTable = new Table();
-
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(200);
- when(response.getContent()).thenReturn(toStream(testTable));
-
- Table table = BigQueryServicesImpl.executeWithRetries(
- bigquery.tables().get("projectId", "datasetId", "tableId"),
- "Failed to get table.",
- Sleeper.DEFAULT,
- BackOff.STOP_BACKOFF);
-
- assertEquals(testTable, table);
- verify(response, times(1)).getStatusCode();
- verify(response, times(1)).getContent();
- verify(response, times(1)).getContentType();
- }
-
- /** A helper to wrap a {@link GenericJson} object in a content stream. */
- private static InputStream toStream(GenericJson content) throws IOException {
- return new ByteArrayInputStream(JacksonFactory.getDefaultInstance().toByteArray(content));
- }
-
- /** A helper that generates the error JSON payload that Google APIs produce. */
- private static GoogleJsonErrorContainer errorWithReasonAndStatus(String reason, int status) {
- ErrorInfo info = new ErrorInfo();
- info.setReason(reason);
- info.setDomain("global");
- // GoogleJsonError contains one or more ErrorInfo objects; our utiities read the first one.
- GoogleJsonError error = new GoogleJsonError();
- error.setErrors(ImmutableList.of(info));
- error.setCode(status);
- // The actual JSON response is an error container.
- GoogleJsonErrorContainer container = new GoogleJsonErrorContainer();
- container.setError(error);
- return container;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableInserterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableInserterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableInserterTest.java
deleted file mode 100644
index 344e916..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableInserterTest.java
+++ /dev/null
@@ -1,311 +0,0 @@
-/*
- * 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.util;
-
-import static com.google.common.base.Verify.verifyNotNull;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.ExpectedLogs;
-
-import com.google.api.client.googleapis.json.GoogleJsonError;
-import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
-import com.google.api.client.googleapis.json.GoogleJsonErrorContainer;
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.http.LowLevelHttpResponse;
-import com.google.api.client.json.GenericJson;
-import com.google.api.client.json.Json;
-import com.google.api.client.json.jackson2.JacksonFactory;
-import com.google.api.client.testing.http.MockHttpTransport;
-import com.google.api.client.testing.http.MockLowLevelHttpRequest;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableDataInsertAllResponse;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.hadoop.util.RetryBoundedBackOff;
-import com.google.common.collect.ImmutableList;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Tests of {@link BigQueryTableInserter}.
- */
-@RunWith(JUnit4.class)
-public class BigQueryTableInserterTest {
- @Rule public ExpectedException thrown = ExpectedException.none();
- @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(BigQueryTableInserter.class);
- @Mock private LowLevelHttpResponse response;
- private Bigquery bigquery;
- private PipelineOptions options;
-
- @Before
- public void setUp() {
- MockitoAnnotations.initMocks(this);
-
- // A mock transport that lets us mock the API responses.
- MockHttpTransport transport =
- new MockHttpTransport.Builder()
- .setLowLevelHttpRequest(
- new MockLowLevelHttpRequest() {
- @Override
- public LowLevelHttpResponse execute() throws IOException {
- return response;
- }
- })
- .build();
-
- // A sample BigQuery API client that uses default JsonFactory and RetryHttpInitializer.
- bigquery =
- new Bigquery.Builder(
- transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer())
- .build();
-
- options = PipelineOptionsFactory.create();
- }
-
- @After
- public void tearDown() throws IOException {
- // These three interactions happen for every request in the normal response parsing.
- verify(response, atLeastOnce()).getContentEncoding();
- verify(response, atLeastOnce()).getHeaderCount();
- verify(response, atLeastOnce()).getReasonPhrase();
- verifyNoMoreInteractions(response);
- }
-
- /** A helper to wrap a {@link GenericJson} object in a content stream. */
- private static InputStream toStream(GenericJson content) throws IOException {
- return new ByteArrayInputStream(JacksonFactory.getDefaultInstance().toByteArray(content));
- }
-
- /** A helper that generates the error JSON payload that Google APIs produce. */
- private static GoogleJsonErrorContainer errorWithReasonAndStatus(String reason, int status) {
- ErrorInfo info = new ErrorInfo();
- info.setReason(reason);
- info.setDomain("global");
- // GoogleJsonError contains one or more ErrorInfo objects; our utiities read the first one.
- GoogleJsonError error = new GoogleJsonError();
- error.setErrors(ImmutableList.of(info));
- error.setCode(status);
- // The actual JSON response is an error container.
- GoogleJsonErrorContainer container = new GoogleJsonErrorContainer();
- container.setError(error);
- return container;
- }
-
- /**
- * Tests that {@link BigQueryTableInserter} succeeds on the first try.
- */
- @Test
- public void testCreateTableSucceeds() throws IOException {
- Table testTable = new Table().setDescription("a table");
-
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(200);
- when(response.getContent()).thenReturn(toStream(testTable));
-
- BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
- Table ret =
- inserter.tryCreateTable(
- new Table(),
- "project",
- "dataset",
- new RetryBoundedBackOff(0, BackOff.ZERO_BACKOFF),
- Sleeper.DEFAULT);
- assertEquals(testTable, ret);
- verify(response, times(1)).getStatusCode();
- verify(response, times(1)).getContent();
- verify(response, times(1)).getContentType();
- }
-
- /**
- * Tests that {@link BigQueryTableInserter} succeeds when the table already exists.
- */
- @Test
- public void testCreateTableSucceedsAlreadyExists() throws IOException {
- when(response.getStatusCode()).thenReturn(409); // 409 means already exists
-
- BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
- Table ret =
- inserter.tryCreateTable(
- new Table(),
- "project",
- "dataset",
- new RetryBoundedBackOff(0, BackOff.ZERO_BACKOFF),
- Sleeper.DEFAULT);
-
- assertNull(ret);
- verify(response, times(1)).getStatusCode();
- verify(response, times(1)).getContent();
- verify(response, times(1)).getContentType();
- }
-
- /**
- * Tests that {@link BigQueryTableInserter} retries quota rate limited attempts.
- */
- @Test
- public void testCreateTableRetry() throws IOException {
- TableReference ref =
- new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
- Table testTable = new Table().setTableReference(ref);
-
- // First response is 403 rate limited, second response has valid payload.
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(403).thenReturn(200);
- when(response.getContent())
- .thenReturn(toStream(errorWithReasonAndStatus("rateLimitExceeded", 403)))
- .thenReturn(toStream(testTable));
-
- BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
- Table ret =
- inserter.tryCreateTable(
- testTable,
- "project",
- "dataset",
- new RetryBoundedBackOff(3, BackOff.ZERO_BACKOFF),
- Sleeper.DEFAULT);
- assertEquals(testTable, ret);
- verify(response, times(2)).getStatusCode();
- verify(response, times(2)).getContent();
- verify(response, times(2)).getContentType();
- verifyNotNull(ret.getTableReference());
- expectedLogs.verifyInfo(
- "Quota limit reached when creating table project:dataset.table, "
- + "retrying up to 5.0 minutes");
- }
-
- /**
- * Tests that {@link BigQueryTableInserter} does not retry non-rate-limited attempts.
- */
- @Test
- public void testCreateTableDoesNotRetry() throws IOException {
- Table testTable = new Table().setDescription("a table");
-
- // First response is 403 not-rate-limited, second response has valid payload but should not
- // be invoked.
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(403).thenReturn(200);
- when(response.getContent())
- .thenReturn(toStream(errorWithReasonAndStatus("actually forbidden", 403)))
- .thenReturn(toStream(testTable));
-
- thrown.expect(GoogleJsonResponseException.class);
- thrown.expectMessage("actually forbidden");
-
- BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
- try {
- inserter.tryCreateTable(
- new Table(),
- "project",
- "dataset",
- new RetryBoundedBackOff(3, BackOff.ZERO_BACKOFF),
- Sleeper.DEFAULT);
- fail();
- } catch (IOException e) {
- verify(response, times(1)).getStatusCode();
- verify(response, times(1)).getContent();
- verify(response, times(1)).getContentType();
- throw e;
- }
- }
-
- /**
- * Tests that {@link BigQueryTableInserter#insertAll} retries quota rate limited attempts.
- */
- @Test
- public void testInsertRetry() throws IOException {
- TableReference ref =
- new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
- List<TableRow> rows = new ArrayList<>();
- rows.add(new TableRow());
-
- // First response is 403 rate limited, second response has valid payload.
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(403).thenReturn(200);
- when(response.getContent())
- .thenReturn(toStream(errorWithReasonAndStatus("rateLimitExceeded", 403)))
- .thenReturn(toStream(new TableDataInsertAllResponse()));
-
- BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
-
- inserter.insertAll(ref, rows);
- verify(response, times(2)).getStatusCode();
- verify(response, times(2)).getContent();
- verify(response, times(2)).getContentType();
- expectedLogs.verifyInfo("BigQuery insertAll exceeded rate limit, retrying");
- }
-
- /**
- * Tests that {@link BigQueryTableInserter#insertAll} does not retry non-rate-limited attempts.
- */
- @Test
- public void testInsertDoesNotRetry() throws Throwable {
- TableReference ref =
- new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
- List<TableRow> rows = new ArrayList<>();
- rows.add(new TableRow());
-
- // First response is 403 not-rate-limited, second response has valid payload but should not
- // be invoked.
- when(response.getContentType()).thenReturn(Json.MEDIA_TYPE);
- when(response.getStatusCode()).thenReturn(403).thenReturn(200);
- when(response.getContent())
- .thenReturn(toStream(errorWithReasonAndStatus("actually forbidden", 403)))
- .thenReturn(toStream(new TableDataInsertAllResponse()));
-
- thrown.expect(GoogleJsonResponseException.class);
- thrown.expectMessage("actually forbidden");
-
- BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
-
- try {
- inserter.insertAll(ref, rows);
- fail();
- } catch (RuntimeException e) {
- verify(response, times(1)).getStatusCode();
- verify(response, times(1)).getContent();
- verify(response, times(1)).getContentType();
- throw e.getCause();
- }
- }
-}
[09/10] incubator-beam git commit: BigQueryIO: move to
google-cloud-platform module
Posted by dh...@apache.org.
BigQueryIO: move to google-cloud-platform module
* Move package from io to io.gcp.bigquery
* Move from SDK core into GCP-IO module
* Fixup references and import orders
* Separate AvroUtils into generic AvroUtils and BigQueryAvroUtils
* Rewrite a unit test in sdk core to not depend on BigQueryIO
* Fixup Javadoc in SDK core that need not depend on BigQueryIO
* Make utility classes package-private
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/b240525a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/b240525a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/b240525a
Branch: refs/heads/master
Commit: b240525affb205a83054577233f3a4a508fe1c72
Parents: 436e4a3
Author: Dan Halperin <dh...@google.com>
Authored: Mon Jul 18 12:05:02 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Jul 20 13:02:42 2016 -0700
----------------------------------------------------------------------
.../apache/beam/examples/WindowedWordCount.java | 2 +-
.../beam/examples/complete/AutoComplete.java | 2 +-
.../examples/complete/StreamingWordExtract.java | 2 +-
.../examples/complete/TrafficMaxLaneFlow.java | 2 +-
.../beam/examples/complete/TrafficRoutes.java | 2 +-
.../examples/cookbook/BigQueryTornadoes.java | 2 +-
.../cookbook/CombinePerKeyExamples.java | 2 +-
.../beam/examples/cookbook/FilterExamples.java | 2 +-
.../beam/examples/cookbook/JoinExamples.java | 2 +-
.../examples/cookbook/MaxPerKeyExamples.java | 2 +-
.../beam/examples/cookbook/TriggerExample.java | 2 +-
examples/java8/pom.xml | 5 +
.../complete/game/utils/WriteToBigQuery.java | 6 +-
.../game/utils/WriteWindowedToBigQuery.java | 6 +-
.../beam/runners/dataflow/DataflowRunner.java | 9 -
.../java/org/apache/beam/sdk/io/BigQueryIO.java | 2447 ------------------
.../org/apache/beam/sdk/util/AvroUtils.java | 207 --
.../apache/beam/sdk/util/BigQueryServices.java | 177 --
.../beam/sdk/util/BigQueryServicesImpl.java | 515 ----
.../beam/sdk/util/BigQueryTableInserter.java | 469 ----
.../beam/sdk/util/BigQueryTableRowIterator.java | 472 ----
.../org/apache/beam/sdk/values/PCollection.java | 34 +-
.../org/apache/beam/sdk/io/BigQueryIOTest.java | 1231 ---------
.../org/apache/beam/sdk/util/AvroUtilsTest.java | 78 -
.../beam/sdk/util/BigQueryServicesImplTest.java | 303 ---
.../sdk/util/BigQueryTableInserterTest.java | 311 ---
.../sdk/util/BigQueryTableRowIteratorTest.java | 256 --
.../apache/beam/sdk/util/BigQueryUtilTest.java | 485 ----
.../util/RetryHttpRequestInitializerTest.java | 20 +-
sdks/java/io/google-cloud-platform/pom.xml | 35 +
.../sdk/io/gcp/bigquery/BigQueryAvroUtils.java | 236 ++
.../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2446 +++++++++++++++++
.../sdk/io/gcp/bigquery/BigQueryServices.java | 177 ++
.../io/gcp/bigquery/BigQueryServicesImpl.java | 511 ++++
.../io/gcp/bigquery/BigQueryTableInserter.java | 413 +++
.../gcp/bigquery/BigQueryTableRowIterator.java | 474 ++++
.../beam/sdk/io/gcp/bigquery/package-info.java | 24 +
.../io/gcp/bigquery/BigQueryAvroUtilsTest.java | 143 +
.../sdk/io/gcp/bigquery/BigQueryIOTest.java | 1231 +++++++++
.../gcp/bigquery/BigQueryServicesImplTest.java | 289 +++
.../gcp/bigquery/BigQueryTableInserterTest.java | 313 +++
.../bigquery/BigQueryTableRowIteratorTest.java | 256 ++
.../sdk/io/gcp/bigquery/BigQueryUtilTest.java | 484 ++++
.../main/resources/archetype-resources/pom.xml | 18 +
.../src/main/java/WindowedWordCount.java | 2 +-
45 files changed, 7092 insertions(+), 7013 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
index 882ef7c..7a4b29f 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
@@ -22,8 +22,8 @@ import org.apache.beam.examples.common.ExampleOptions;
import org.apache.beam.examples.common.ExampleUtils;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.io.BigQueryIO;
import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
index 26f6045..c6272e8 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
@@ -27,8 +27,8 @@ import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.coders.AvroCoder;
import org.apache.beam.sdk.coders.DefaultCoder;
-import org.apache.beam.sdk.io.BigQueryIO;
import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.io.gcp.datastore.DatastoreIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
index d4ba8bd..db646a5 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
@@ -22,8 +22,8 @@ import org.apache.beam.examples.common.ExampleOptions;
import org.apache.beam.examples.common.ExampleUtils;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.io.BigQueryIO;
import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
index ae7b8cc..7b1496f 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
@@ -24,8 +24,8 @@ import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.coders.AvroCoder;
import org.apache.beam.sdk.coders.DefaultCoder;
-import org.apache.beam.sdk.io.BigQueryIO;
import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
index 1a3d46d..ebf7b9a 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
@@ -24,8 +24,8 @@ import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.coders.AvroCoder;
import org.apache.beam.sdk.coders.DefaultCoder;
-import org.apache.beam.sdk.io.BigQueryIO;
import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
index a0ac6c5..665be01 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
@@ -18,7 +18,7 @@
package org.apache.beam.examples.cookbook;
import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
index dbaa1f8..252f3cc 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
@@ -18,7 +18,7 @@
package org.apache.beam.examples.cookbook;
import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
index 0804cdb..ea1dcf6 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
@@ -18,7 +18,7 @@
package org.apache.beam.examples.cookbook;
import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
index 3c26123..1b43cc2 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
@@ -18,8 +18,8 @@
package org.apache.beam.examples.cookbook;
import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.BigQueryIO;
import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
index 6630f3d..a37690b 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
@@ -18,7 +18,7 @@
package org.apache.beam.examples.cookbook;
import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
index e44edd7..a0c5181 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
@@ -22,8 +22,8 @@ import org.apache.beam.examples.common.ExampleOptions;
import org.apache.beam.examples.common.ExampleUtils;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.io.BigQueryIO;
import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java8/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml
index cf6b545..36d44c5 100644
--- a/examples/java8/pom.xml
+++ b/examples/java8/pom.xml
@@ -145,6 +145,11 @@
<dependency>
<groupId>org.apache.beam</groupId>
+ <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.beam</groupId>
<artifactId>beam-examples-java</artifactId>
<version>${project.version}</version>
</dependency>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
index 5b472d7..36ed195 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
@@ -19,9 +19,9 @@ package org.apache.beam.examples.complete.game.utils;
import org.apache.beam.examples.complete.game.UserScore;
import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition;
-import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
import org.apache.beam.sdk.options.GcpOptions;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.transforms.DoFn;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
index b1ccaed..b4c9b4a 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
@@ -17,9 +17,9 @@
*/
package org.apache.beam.examples.complete.game.utils;
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition;
-import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess;
import org.apache.beam.sdk.transforms.ParDo;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index fa61587..05ddf45 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -60,7 +60,6 @@ import org.apache.beam.sdk.coders.StandardCoder;
import org.apache.beam.sdk.coders.VarIntCoder;
import org.apache.beam.sdk.coders.VarLongCoder;
import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.BigQueryIO;
import org.apache.beam.sdk.io.BoundedSource;
import org.apache.beam.sdk.io.FileBasedSink;
import org.apache.beam.sdk.io.PubsubIO;
@@ -2823,14 +2822,6 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
* Builds an instance of this class from the overridden transform.
*/
@SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
- public UnsupportedIO(DataflowRunner runner, BigQueryIO.Read.Bound transform) {
- this.transform = transform;
- }
-
- /**
- * Builds an instance of this class from the overridden transform.
- */
- @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
public UnsupportedIO(DataflowRunner runner, TextIO.Read.Bound<?> transform) {
this.transform = transform;
}
[08/10] incubator-beam git commit: BigQueryIO: move to
google-cloud-platform module
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
deleted file mode 100644
index 9141f39..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
+++ /dev/null
@@ -1,2447 +0,0 @@
-/*
- * 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.io;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.Coder.Context;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.StandardCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.TableRowJsonCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition;
-import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition;
-import org.apache.beam.sdk.options.BigQueryOptions;
-import org.apache.beam.sdk.options.GcpOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
-import org.apache.beam.sdk.util.AvroUtils;
-import org.apache.beam.sdk.util.BigQueryServices;
-import org.apache.beam.sdk.util.BigQueryServices.DatasetService;
-import org.apache.beam.sdk.util.BigQueryServices.JobService;
-import org.apache.beam.sdk.util.BigQueryServicesImpl;
-import org.apache.beam.sdk.util.BigQueryTableInserter;
-import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory;
-import org.apache.beam.sdk.util.IOChannelFactory;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.util.MimeTypes;
-import org.apache.beam.sdk.util.PropertyNames;
-import org.apache.beam.sdk.util.Reshuffle;
-import org.apache.beam.sdk.util.SystemDoFnInternal;
-import org.apache.beam.sdk.util.Transport;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-
-import com.google.api.client.json.JsonFactory;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.model.Job;
-import com.google.api.services.bigquery.model.JobConfigurationExtract;
-import com.google.api.services.bigquery.model.JobConfigurationLoad;
-import com.google.api.services.bigquery.model.JobConfigurationQuery;
-import com.google.api.services.bigquery.model.JobReference;
-import com.google.api.services.bigquery.model.JobStatistics;
-import com.google.api.services.bigquery.model.JobStatus;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.hadoop.util.ApiErrorExtractor;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import org.apache.avro.generic.GenericRecord;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.ObjectInputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import javax.annotation.Nullable;
-
-/**
- * {@link PTransform}s for reading and writing
- * <a href="https://developers.google.com/bigquery/">BigQuery</a> tables.
- *
- * <h3>Table References</h3>
- * <p>A fully-qualified BigQuery table name consists of three components:
- * <ul>
- * <li>{@code projectId}: the Cloud project id (defaults to
- * {@link GcpOptions#getProject()}).
- * <li>{@code datasetId}: the BigQuery dataset id, unique within a project.
- * <li>{@code tableId}: a table id, unique within a dataset.
- * </ul>
- *
- * <p>BigQuery table references are stored as a {@link TableReference}, which comes
- * from the <a href="https://cloud.google.com/bigquery/client-libraries">
- * BigQuery Java Client API</a>.
- * Tables can be referred to as Strings, with or without the {@code projectId}.
- * A helper function is provided ({@link BigQueryIO#parseTableSpec(String)})
- * that parses the following string forms into a {@link TableReference}:
- *
- * <ul>
- * <li>[{@code project_id}]:[{@code dataset_id}].[{@code table_id}]
- * <li>[{@code dataset_id}].[{@code table_id}]
- * </ul>
- *
- * <h3>Reading</h3>
- * <p>To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation.
- * This produces a {@link PCollection} of {@link TableRow TableRows} as output:
- * <pre>{@code
- * PCollection<TableRow> shakespeare = pipeline.apply(
- * BigQueryIO.Read.from("clouddataflow-readonly:samples.weather_stations"));
- * }</pre>
- *
- * <p>See {@link TableRow} for more information on the {@link TableRow} object.
- *
- * <p>Users may provide a query to read from rather than reading all of a BigQuery table. If
- * specified, the result obtained by executing the specified query will be used as the data of the
- * input transform.
- *
- * <pre>{@code
- * PCollection<TableRow> shakespeare = pipeline.apply(
- * BigQueryIO.Read.fromQuery("SELECT year, mean_temp FROM samples.weather_stations"));
- * }</pre>
- *
- * <p>When creating a BigQuery input transform, users should provide either a query or a table.
- * Pipeline construction will fail with a validation error if neither or both are specified.
- *
- * <h3>Writing</h3>
- * <p>To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation.
- * This consumes a {@link PCollection} of {@link TableRow TableRows} as input.
- * <pre>{@code
- * PCollection<TableRow> quotes = ...
- *
- * List<TableFieldSchema> fields = new ArrayList<>();
- * fields.add(new TableFieldSchema().setName("source").setType("STRING"));
- * fields.add(new TableFieldSchema().setName("quote").setType("STRING"));
- * TableSchema schema = new TableSchema().setFields(fields);
- *
- * quotes.apply(BigQueryIO.Write
- * .to("my-project:output.output_table")
- * .withSchema(schema)
- * .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
- * }</pre>
- *
- * <p>See {@link BigQueryIO.Write} for details on how to specify if a write should
- * append to an existing table, replace the table, or verify that the table is
- * empty. Note that the dataset being written to must already exist. Unbounded PCollections can only
- * be written using {@link WriteDisposition#WRITE_EMPTY} or {@link WriteDisposition#WRITE_APPEND}.
- *
- * <h3>Sharding BigQuery output tables</h3>
- * <p>A common use case is to dynamically generate BigQuery table names based on
- * the current window. To support this,
- * {@link BigQueryIO.Write#to(SerializableFunction)}
- * accepts a function mapping the current window to a tablespec. For example,
- * here's code that outputs daily tables to BigQuery:
- * <pre>{@code
- * PCollection<TableRow> quotes = ...
- * quotes.apply(Window.<TableRow>into(CalendarWindows.days(1)))
- * .apply(BigQueryIO.Write
- * .withSchema(schema)
- * .to(new SerializableFunction<BoundedWindow, String>() {
- * public String apply(BoundedWindow window) {
- * // The cast below is safe because CalendarWindows.days(1) produces IntervalWindows.
- * String dayString = DateTimeFormat.forPattern("yyyy_MM_dd")
- * .withZone(DateTimeZone.UTC)
- * .print(((IntervalWindow) window).start());
- * return "my-project:output.output_table_" + dayString;
- * }
- * }));
- * }</pre>
- *
- * <p>Per-window tables are not yet supported in batch mode.
- *
- * <h3>Permissions</h3>
- * <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
- * Dataflow job. Please refer to the documentation of corresponding {@link PipelineRunner}s for
- * more details.
- *
- * <p>Please see <a href="https://cloud.google.com/bigquery/access-control">BigQuery Access Control
- * </a> for security and permission related information specific to BigQuery.
- */
-public class BigQueryIO {
- private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class);
-
- /**
- * Singleton instance of the JSON factory used to read and write JSON
- * formatted rows.
- */
- private static final JsonFactory JSON_FACTORY = Transport.getJsonFactory();
-
- /**
- * Project IDs must contain 6-63 lowercase letters, digits, or dashes.
- * IDs must start with a letter and may not end with a dash.
- * This regex isn't exact - this allows for patterns that would be rejected by
- * the service, but this is sufficient for basic parsing of table references.
- */
- private static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]{4,61}[a-z0-9]";
-
- /**
- * Regular expression that matches Dataset IDs.
- */
- private static final String DATASET_REGEXP = "[-\\w.]{1,1024}";
-
- /**
- * Regular expression that matches Table IDs.
- */
- private static final String TABLE_REGEXP = "[-\\w$@]{1,1024}";
-
- /**
- * Matches table specifications in the form {@code "[project_id]:[dataset_id].[table_id]"} or
- * {@code "[dataset_id].[table_id]"}.
- */
- private static final String DATASET_TABLE_REGEXP =
- String.format("((?<PROJECT>%s):)?(?<DATASET>%s)\\.(?<TABLE>%s)", PROJECT_ID_REGEXP,
- DATASET_REGEXP, TABLE_REGEXP);
-
- private static final Pattern TABLE_SPEC = Pattern.compile(DATASET_TABLE_REGEXP);
-
- // TODO: make this private and remove improper access from BigQueryIOTranslator.
- public static final String SET_PROJECT_FROM_OPTIONS_WARNING =
- "No project specified for BigQuery table \"%1$s.%2$s\". Assuming it is in \"%3$s\". If the"
- + " table is in a different project please specify it as a part of the BigQuery table"
- + " definition.";
-
- private static final String RESOURCE_NOT_FOUND_ERROR =
- "BigQuery %1$s not found for table \"%2$s\" . Please create the %1$s before pipeline"
- + " execution. If the %1$s is created by an earlier stage of the pipeline, this"
- + " validation can be disabled using #withoutValidation.";
-
- private static final String UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR =
- "Unable to confirm BigQuery %1$s presence for table \"%2$s\". If the %1$s is created by"
- + " an earlier stage of the pipeline, this validation can be disabled using"
- + " #withoutValidation.";
-
- /**
- * Parse a table specification in the form
- * {@code "[project_id]:[dataset_id].[table_id]"} or {@code "[dataset_id].[table_id]"}.
- *
- * <p>If the project id is omitted, the default project id is used.
- */
- public static TableReference parseTableSpec(String tableSpec) {
- Matcher match = TABLE_SPEC.matcher(tableSpec);
- if (!match.matches()) {
- throw new IllegalArgumentException(
- "Table reference is not in [project_id]:[dataset_id].[table_id] "
- + "format: " + tableSpec);
- }
-
- TableReference ref = new TableReference();
- ref.setProjectId(match.group("PROJECT"));
-
- return ref.setDatasetId(match.group("DATASET")).setTableId(match.group("TABLE"));
- }
-
- /**
- * Returns a canonical string representation of the {@link TableReference}.
- */
- public static String toTableSpec(TableReference ref) {
- StringBuilder sb = new StringBuilder();
- if (ref.getProjectId() != null) {
- sb.append(ref.getProjectId());
- sb.append(":");
- }
-
- sb.append(ref.getDatasetId()).append('.').append(ref.getTableId());
- return sb.toString();
- }
-
- /**
- * A {@link PTransform} that reads from a BigQuery table and returns a
- * {@link PCollection} of {@link TableRow TableRows} containing each of the rows of the table.
- *
- * <p>Each {@link TableRow} contains values indexed by column name. Here is a
- * sample processing function that processes a "line" column from rows:
- * <pre>{@code
- * static class ExtractWordsFn extends DoFn<TableRow, String> {
- * public void processElement(ProcessContext c) {
- * // Get the "line" field of the TableRow object, split it into words, and emit them.
- * TableRow row = c.element();
- * String[] words = row.get("line").toString().split("[^a-zA-Z']+");
- * for (String word : words) {
- * if (!word.isEmpty()) {
- * c.output(word);
- * }
- * }
- * }
- * }}</pre>
- */
- public static class Read {
-
- /**
- * Reads a BigQuery table specified as {@code "[project_id]:[dataset_id].[table_id]"} or
- * {@code "[dataset_id].[table_id]"} for tables within the current project.
- */
- public static Bound from(String tableSpec) {
- return new Bound().from(tableSpec);
- }
-
- /**
- * Reads results received after executing the given query.
- */
- public static Bound fromQuery(String query) {
- return new Bound().fromQuery(query);
- }
-
- /**
- * Reads a BigQuery table specified as a {@link TableReference} object.
- */
- public static Bound from(TableReference table) {
- return new Bound().from(table);
- }
-
- /**
- * Disables BigQuery table validation, which is enabled by default.
- */
- public static Bound withoutValidation() {
- return new Bound().withoutValidation();
- }
-
- /**
- * A {@link PTransform} that reads from a BigQuery table and returns a bounded
- * {@link PCollection} of {@link TableRow TableRows}.
- */
- public static class Bound extends PTransform<PInput, PCollection<TableRow>> {
- @Nullable final String jsonTableRef;
- @Nullable final String query;
-
- /**
- * Disable validation that the table exists or the query succeeds prior to pipeline
- * submission. Basic validation (such as ensuring that a query or table is specified) still
- * occurs.
- */
- final boolean validate;
- @Nullable final Boolean flattenResults;
- @Nullable BigQueryServices bigQueryServices;
-
- private static final String QUERY_VALIDATION_FAILURE_ERROR =
- "Validation of query \"%1$s\" failed. If the query depends on an earlier stage of the"
- + " pipeline, This validation can be disabled using #withoutValidation.";
-
- // The maximum number of retries to poll a BigQuery job in the cleanup phase.
- // We expect the jobs have already DONE, and don't need a high max retires.
- private static final int CLEANUP_JOB_POLL_MAX_RETRIES = 10;
-
- private Bound() {
- this(
- null /* name */,
- null /* query */,
- null /* jsonTableRef */,
- true /* validate */,
- null /* flattenResults */,
- null /* bigQueryServices */);
- }
-
- private Bound(
- String name, @Nullable String query, @Nullable String jsonTableRef, boolean validate,
- @Nullable Boolean flattenResults, @Nullable BigQueryServices bigQueryServices) {
- super(name);
- this.jsonTableRef = jsonTableRef;
- this.query = query;
- this.validate = validate;
- this.flattenResults = flattenResults;
- this.bigQueryServices = bigQueryServices;
- }
-
- /**
- * Returns a copy of this transform that reads from the specified table. Refer to
- * {@link #parseTableSpec(String)} for the specification format.
- *
- * <p>Does not modify this object.
- */
- public Bound from(String tableSpec) {
- return from(parseTableSpec(tableSpec));
- }
-
- /**
- * Returns a copy of this transform that reads from the specified table.
- *
- * <p>Does not modify this object.
- */
- public Bound from(TableReference table) {
- return new Bound(
- name, query, toJsonString(table), validate, flattenResults, bigQueryServices);
- }
-
- /**
- * Returns a copy of this transform that reads the results of the specified query.
- *
- * <p>Does not modify this object.
- *
- * <p>By default, the query results will be flattened -- see
- * "flattenResults" in the <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs">
- * Jobs documentation</a> for more information. To disable flattening, use
- * {@link BigQueryIO.Read.Bound#withoutResultFlattening}.
- */
- public Bound fromQuery(String query) {
- return new Bound(name, query, jsonTableRef, validate,
- MoreObjects.firstNonNull(flattenResults, Boolean.TRUE), bigQueryServices);
- }
-
- /**
- * Disable validation that the table exists or the query succeeds prior to pipeline
- * submission. Basic validation (such as ensuring that a query or table is specified) still
- * occurs.
- */
- public Bound withoutValidation() {
- return new Bound(name, query, jsonTableRef, false, flattenResults, bigQueryServices);
- }
-
- /**
- * Disable <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs">
- * flattening of query results</a>.
- *
- * <p>Only valid when a query is used ({@link #fromQuery}). Setting this option when reading
- * from a table will cause an error during validation.
- */
- public Bound withoutResultFlattening() {
- return new Bound(name, query, jsonTableRef, validate, false, bigQueryServices);
- }
-
- @VisibleForTesting
- Bound withTestServices(BigQueryServices testServices) {
- return new Bound(name, query, jsonTableRef, validate, flattenResults, testServices);
- }
-
- @Override
- public void validate(PInput input) {
- // Even if existence validation is disabled, we need to make sure that the BigQueryIO
- // read is properly specified.
- BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class);
-
- TableReference table = getTableWithDefaultProject(bqOptions);
- if (table == null && query == null) {
- throw new IllegalStateException(
- "Invalid BigQuery read operation, either table reference or query has to be set");
- } else if (table != null && query != null) {
- throw new IllegalStateException("Invalid BigQuery read operation. Specifies both a"
- + " query and a table, only one of these should be provided");
- } else if (table != null && flattenResults != null) {
- throw new IllegalStateException("Invalid BigQuery read operation. Specifies a"
- + " table with a result flattening preference, which is not configurable");
- } else if (query != null && flattenResults == null) {
- throw new IllegalStateException("Invalid BigQuery read operation. Specifies a"
- + " query without a result flattening preference");
- }
-
- if (validate) {
- BigQueryServices bqServices = getBigQueryServices();
- // Check for source table/query presence for early failure notification.
- // Note that a presence check can fail if the table or dataset are created by earlier
- // stages of the pipeline or if a query depends on earlier stages of a pipeline. For these
- // cases the withoutValidation method can be used to disable the check.
- if (table != null) {
- DatasetService datasetService = bqServices.getDatasetService(bqOptions);
- verifyDatasetPresence(datasetService, table);
- verifyTablePresence(datasetService, table);
- }
- if (query != null) {
- JobService jobService = bqServices.getJobService(bqOptions);
- try {
- jobService.dryRunQuery(bqOptions.getProject(), query);
- } catch (Exception e) {
- throw new IllegalArgumentException(
- String.format(QUERY_VALIDATION_FAILURE_ERROR, query), e);
- }
- }
- }
- }
-
- @Override
- public PCollection<TableRow> apply(PInput input) {
- String uuid = randomUUIDString();
- final String jobIdToken = "beam_job_" + uuid;
-
- BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class);
-
- BoundedSource<TableRow> source;
- final BigQueryServices bqServices = getBigQueryServices();
-
- final String extractDestinationDir;
- String tempLocation = bqOptions.getTempLocation();
- try {
- IOChannelFactory factory = IOChannelUtils.getFactory(tempLocation);
- extractDestinationDir = factory.resolve(tempLocation, uuid);
- } catch (IOException e) {
- throw new RuntimeException(
- String.format("Failed to resolve extract destination directory in %s", tempLocation));
- }
-
- final String executingProject = bqOptions.getProject();
- if (!Strings.isNullOrEmpty(query)) {
- String queryTempDatasetId = "temp_dataset_" + uuid;
- String queryTempTableId = "temp_table_" + uuid;
-
- TableReference queryTempTableRef = new TableReference()
- .setProjectId(executingProject)
- .setDatasetId(queryTempDatasetId)
- .setTableId(queryTempTableId);
-
- source = BigQueryQuerySource.create(
- jobIdToken, query, queryTempTableRef, flattenResults,
- extractDestinationDir, bqServices);
- } else {
- TableReference inputTable = getTableWithDefaultProject(bqOptions);
- source = BigQueryTableSource.create(
- jobIdToken, inputTable, extractDestinationDir, bqServices, executingProject);
- }
- PassThroughThenCleanup.CleanupOperation cleanupOperation =
- new PassThroughThenCleanup.CleanupOperation() {
- @Override
- void cleanup(PipelineOptions options) throws Exception {
- BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
-
- JobReference jobRef = new JobReference()
- .setProjectId(executingProject)
- .setJobId(getExtractJobId(jobIdToken));
- Job extractJob = bqServices.getJobService(bqOptions).pollJob(
- jobRef, CLEANUP_JOB_POLL_MAX_RETRIES);
-
- Collection<String> extractFiles = null;
- if (extractJob != null) {
- extractFiles = getExtractFilePaths(extractDestinationDir, extractJob);
- } else {
- IOChannelFactory factory = IOChannelUtils.getFactory(extractDestinationDir);
- Collection<String> dirMatch = factory.match(extractDestinationDir);
- if (!dirMatch.isEmpty()) {
- extractFiles = factory.match(factory.resolve(extractDestinationDir, "*"));
- }
- }
- if (extractFiles != null && !extractFiles.isEmpty()) {
- new GcsUtilFactory().create(options).remove(extractFiles);
- }
- }};
- return input.getPipeline()
- .apply(org.apache.beam.sdk.io.Read.from(source))
- .setCoder(getDefaultOutputCoder())
- .apply(new PassThroughThenCleanup<TableRow>(cleanupOperation));
- }
-
- @Override
- protected Coder<TableRow> getDefaultOutputCoder() {
- return TableRowJsonCoder.of();
- }
-
- @Override
- public void populateDisplayData(DisplayData.Builder builder) {
- super.populateDisplayData(builder);
- TableReference table = getTable();
-
- if (table != null) {
- builder.add(DisplayData.item("table", toTableSpec(table))
- .withLabel("Table"));
- }
-
- builder
- .addIfNotNull(DisplayData.item("query", query)
- .withLabel("Query"))
- .addIfNotNull(DisplayData.item("flattenResults", flattenResults)
- .withLabel("Flatten Query Results"))
- .addIfNotDefault(DisplayData.item("validation", validate)
- .withLabel("Validation Enabled"),
- true);
- }
-
- /**
- * Returns the table to read, or {@code null} if reading from a query instead.
- *
- * <p>If the table's project is not specified, use the executing project.
- */
- @Nullable private TableReference getTableWithDefaultProject(BigQueryOptions bqOptions) {
- TableReference table = getTable();
- if (table != null && Strings.isNullOrEmpty(table.getProjectId())) {
- // If user does not specify a project we assume the table to be located in
- // the default project.
- table.setProjectId(bqOptions.getProject());
- }
- return table;
- }
-
- /**
- * Returns the table to read, or {@code null} if reading from a query instead.
- */
- @Nullable
- public TableReference getTable() {
- return fromJsonString(jsonTableRef, TableReference.class);
- }
-
- /**
- * Returns the query to be read, or {@code null} if reading from a table instead.
- */
- public String getQuery() {
- return query;
- }
-
- /**
- * Returns true if table validation is enabled.
- */
- public boolean getValidate() {
- return validate;
- }
-
- /**
- * Returns true/false if result flattening is enabled/disabled, or null if not applicable.
- */
- public Boolean getFlattenResults() {
- return flattenResults;
- }
-
- private BigQueryServices getBigQueryServices() {
- if (bigQueryServices == null) {
- bigQueryServices = new BigQueryServicesImpl();
- }
- return bigQueryServices;
- }
- }
-
- /** Disallow construction of utility class. */
- private Read() {}
- }
-
- /**
- * A {@link PTransform} that invokes {@link CleanupOperation} after the input {@link PCollection}
- * has been processed.
- */
- @VisibleForTesting
- static class PassThroughThenCleanup<T> extends PTransform<PCollection<T>, PCollection<T>> {
-
- private CleanupOperation cleanupOperation;
-
- PassThroughThenCleanup(CleanupOperation cleanupOperation) {
- this.cleanupOperation = cleanupOperation;
- }
-
- @Override
- public PCollection<T> apply(PCollection<T> input) {
- TupleTag<T> mainOutput = new TupleTag<>();
- TupleTag<Void> cleanupSignal = new TupleTag<>();
- PCollectionTuple outputs = input.apply(ParDo.of(new IdentityFn<T>())
- .withOutputTags(mainOutput, TupleTagList.of(cleanupSignal)));
-
- PCollectionView<Void> cleanupSignalView = outputs.get(cleanupSignal)
- .setCoder(VoidCoder.of())
- .apply(View.<Void>asSingleton().withDefaultValue(null));
-
- input.getPipeline()
- .apply("Create(CleanupOperation)", Create.of(cleanupOperation))
- .apply("Cleanup", ParDo.of(
- new DoFn<CleanupOperation, Void>() {
- @Override
- public void processElement(ProcessContext c)
- throws Exception {
- c.element().cleanup(c.getPipelineOptions());
- }
- }).withSideInputs(cleanupSignalView));
-
- return outputs.get(mainOutput);
- }
-
- private static class IdentityFn<T> extends DoFn<T, T> {
- @Override
- public void processElement(ProcessContext c) {
- c.output(c.element());
- }
- }
-
- abstract static class CleanupOperation implements Serializable {
- abstract void cleanup(PipelineOptions options) throws Exception;
- }
- }
-
- /**
- * A {@link BigQuerySourceBase} for reading BigQuery tables.
- */
- @VisibleForTesting
- static class BigQueryTableSource extends BigQuerySourceBase {
-
- static BigQueryTableSource create(
- String jobIdToken,
- TableReference table,
- String extractDestinationDir,
- BigQueryServices bqServices,
- String executingProject) {
- return new BigQueryTableSource(
- jobIdToken, table, extractDestinationDir, bqServices, executingProject);
- }
-
- private final String jsonTable;
- private final AtomicReference<Long> tableSizeBytes;
-
- private BigQueryTableSource(
- String jobIdToken,
- TableReference table,
- String extractDestinationDir,
- BigQueryServices bqServices,
- String executingProject) {
- super(jobIdToken, extractDestinationDir, bqServices, executingProject);
- checkNotNull(table, "table");
- this.jsonTable = toJsonString(table);
- this.tableSizeBytes = new AtomicReference<>();
- }
-
- @Override
- protected TableReference getTableToExtract(BigQueryOptions bqOptions) throws IOException {
- return JSON_FACTORY.fromString(jsonTable, TableReference.class);
- }
-
- @Override
- public BoundedReader<TableRow> createReader(PipelineOptions options) throws IOException {
- BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
- TableReference tableRef = JSON_FACTORY.fromString(jsonTable, TableReference.class);
- return new BigQueryReader(this, bqServices.getReaderFromTable(bqOptions, tableRef));
- }
-
- @Override
- public synchronized long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
- if (tableSizeBytes.get() == null) {
- TableReference table = JSON_FACTORY.fromString(jsonTable, TableReference.class);
-
- Long numBytes = bqServices.getDatasetService(options.as(BigQueryOptions.class))
- .getTable(table.getProjectId(), table.getDatasetId(), table.getTableId())
- .getNumBytes();
- tableSizeBytes.compareAndSet(null, numBytes);
- }
- return tableSizeBytes.get();
- }
-
- @Override
- protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception {
- // Do nothing.
- }
-
- @Override
- public void populateDisplayData(DisplayData.Builder builder) {
- super.populateDisplayData(builder);
- builder.add(DisplayData.item("table", jsonTable));
- }
- }
-
- /**
- * A {@link BigQuerySourceBase} for querying BigQuery tables.
- */
- @VisibleForTesting
- static class BigQueryQuerySource extends BigQuerySourceBase {
-
- static BigQueryQuerySource create(
- String jobIdToken,
- String query,
- TableReference queryTempTableRef,
- Boolean flattenResults,
- String extractDestinationDir,
- BigQueryServices bqServices) {
- return new BigQueryQuerySource(
- jobIdToken,
- query,
- queryTempTableRef,
- flattenResults,
- extractDestinationDir,
- bqServices);
- }
-
- private final String query;
- private final String jsonQueryTempTable;
- private final Boolean flattenResults;
- private transient AtomicReference<JobStatistics> dryRunJobStats;
-
- private BigQueryQuerySource(
- String jobIdToken,
- String query,
- TableReference queryTempTableRef,
- Boolean flattenResults,
- String extractDestinationDir,
- BigQueryServices bqServices) {
- super(jobIdToken, extractDestinationDir, bqServices,
- checkNotNull(queryTempTableRef, "queryTempTableRef").getProjectId());
- this.query = checkNotNull(query, "query");
- this.jsonQueryTempTable = toJsonString(queryTempTableRef);
- this.flattenResults = checkNotNull(flattenResults, "flattenResults");
- this.dryRunJobStats = new AtomicReference<>();
- }
-
- @Override
- public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
- BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
- return dryRunQueryIfNeeded(bqOptions).getTotalBytesProcessed();
- }
-
- @Override
- public BoundedReader<TableRow> createReader(PipelineOptions options) throws IOException {
- BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
- return new BigQueryReader(this, bqServices.getReaderFromQuery(
- bqOptions, query, executingProject, flattenResults));
- }
-
- @Override
- protected TableReference getTableToExtract(BigQueryOptions bqOptions)
- throws IOException, InterruptedException {
- // 1. Find the location of the query.
- TableReference dryRunTempTable = dryRunQueryIfNeeded(bqOptions)
- .getQuery()
- .getReferencedTables()
- .get(0);
- DatasetService tableService = bqServices.getDatasetService(bqOptions);
- String location = tableService.getTable(
- dryRunTempTable.getProjectId(),
- dryRunTempTable.getDatasetId(),
- dryRunTempTable.getTableId()).getLocation();
-
- // 2. Create the temporary dataset in the query location.
- TableReference tableToExtract =
- JSON_FACTORY.fromString(jsonQueryTempTable, TableReference.class);
- tableService.createDataset(
- tableToExtract.getProjectId(), tableToExtract.getDatasetId(), location, "");
-
- // 3. Execute the query.
- String queryJobId = jobIdToken + "-query";
- executeQuery(
- executingProject,
- queryJobId,
- query,
- tableToExtract,
- flattenResults,
- bqServices.getJobService(bqOptions));
- return tableToExtract;
- }
-
- @Override
- protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception {
- TableReference tableToRemove =
- JSON_FACTORY.fromString(jsonQueryTempTable, TableReference.class);
-
- DatasetService tableService = bqServices.getDatasetService(bqOptions);
- tableService.deleteTable(
- tableToRemove.getProjectId(),
- tableToRemove.getDatasetId(),
- tableToRemove.getTableId());
- tableService.deleteDataset(tableToRemove.getProjectId(), tableToRemove.getDatasetId());
- }
-
- @Override
- public void populateDisplayData(DisplayData.Builder builder) {
- super.populateDisplayData(builder);
- builder.add(DisplayData.item("query", query));
- }
- private synchronized JobStatistics dryRunQueryIfNeeded(BigQueryOptions bqOptions)
- throws InterruptedException, IOException {
- if (dryRunJobStats.get() == null) {
- JobStatistics jobStats =
- bqServices.getJobService(bqOptions).dryRunQuery(executingProject, query);
- dryRunJobStats.compareAndSet(null, jobStats);
- }
- return dryRunJobStats.get();
- }
-
- private static void executeQuery(
- String executingProject,
- String jobId,
- String query,
- TableReference destinationTable,
- boolean flattenResults,
- JobService jobService) throws IOException, InterruptedException {
- JobReference jobRef = new JobReference()
- .setProjectId(executingProject)
- .setJobId(jobId);
- JobConfigurationQuery queryConfig = new JobConfigurationQuery();
- queryConfig
- .setQuery(query)
- .setAllowLargeResults(true)
- .setCreateDisposition("CREATE_IF_NEEDED")
- .setDestinationTable(destinationTable)
- .setFlattenResults(flattenResults)
- .setPriority("BATCH")
- .setWriteDisposition("WRITE_EMPTY");
- jobService.startQueryJob(jobRef, queryConfig);
- Job job = jobService.pollJob(jobRef, JOB_POLL_MAX_RETRIES);
- if (parseStatus(job) != Status.SUCCEEDED) {
- throw new IOException("Query job failed: " + jobId);
- }
- return;
- }
-
- private void readObject(ObjectInputStream in) throws ClassNotFoundException, IOException {
- in.defaultReadObject();
- dryRunJobStats = new AtomicReference<>();
- }
- }
-
- /**
- * An abstract {@link BoundedSource} to read a table from BigQuery.
- *
- * <p>This source uses a BigQuery export job to take a snapshot of the table on GCS, and then
- * reads in parallel from each produced file. It is implemented by {@link BigQueryTableSource},
- * and {@link BigQueryQuerySource}, depending on the configuration of the read.
- * Specifically,
- * <ul>
- * <li>{@link BigQueryTableSource} is for reading BigQuery tables</li>
- * <li>{@link BigQueryQuerySource} is for querying BigQuery tables</li>
- * </ul>
- * ...
- */
- private abstract static class BigQuerySourceBase extends BoundedSource<TableRow> {
- // The maximum number of attempts to verify temp files.
- private static final int MAX_FILES_VERIFY_ATTEMPTS = 10;
-
- // The maximum number of retries to poll a BigQuery job.
- protected static final int JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE;
-
- // The initial backoff for verifying temp files.
- private static final long INITIAL_FILES_VERIFY_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1);
-
- protected final String jobIdToken;
- protected final String extractDestinationDir;
- protected final BigQueryServices bqServices;
- protected final String executingProject;
-
- private BigQuerySourceBase(
- String jobIdToken,
- String extractDestinationDir,
- BigQueryServices bqServices,
- String executingProject) {
- this.jobIdToken = checkNotNull(jobIdToken, "jobIdToken");
- this.extractDestinationDir = checkNotNull(extractDestinationDir, "extractDestinationDir");
- this.bqServices = checkNotNull(bqServices, "bqServices");
- this.executingProject = checkNotNull(executingProject, "executingProject");
- }
-
- @Override
- public List<BoundedSource<TableRow>> splitIntoBundles(
- long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
- BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
- TableReference tableToExtract = getTableToExtract(bqOptions);
- JobService jobService = bqServices.getJobService(bqOptions);
- String extractJobId = getExtractJobId(jobIdToken);
- List<String> tempFiles = executeExtract(extractJobId, tableToExtract, jobService);
-
- TableSchema tableSchema = bqServices.getDatasetService(bqOptions).getTable(
- tableToExtract.getProjectId(),
- tableToExtract.getDatasetId(),
- tableToExtract.getTableId()).getSchema();
-
- cleanupTempResource(bqOptions);
- return createSources(tempFiles, tableSchema);
- }
-
- protected abstract TableReference getTableToExtract(BigQueryOptions bqOptions) throws Exception;
-
- protected abstract void cleanupTempResource(BigQueryOptions bqOptions) throws Exception;
-
- @Override
- public boolean producesSortedKeys(PipelineOptions options) throws Exception {
- return false;
- }
-
- @Override
- public void validate() {
- // Do nothing, validation is done in BigQuery.Read.
- }
-
- @Override
- public Coder<TableRow> getDefaultOutputCoder() {
- return TableRowJsonCoder.of();
- }
-
- private List<String> executeExtract(
- String jobId, TableReference table, JobService jobService)
- throws InterruptedException, IOException {
- JobReference jobRef = new JobReference()
- .setProjectId(executingProject)
- .setJobId(jobId);
-
- String destinationUri = getExtractDestinationUri(extractDestinationDir);
- JobConfigurationExtract extract = new JobConfigurationExtract()
- .setSourceTable(table)
- .setDestinationFormat("AVRO")
- .setDestinationUris(ImmutableList.of(destinationUri));
-
- LOG.info("Starting BigQuery extract job: {}", jobId);
- jobService.startExtractJob(jobRef, extract);
- Job extractJob =
- jobService.pollJob(jobRef, JOB_POLL_MAX_RETRIES);
- if (parseStatus(extractJob) != Status.SUCCEEDED) {
- throw new IOException(String.format(
- "Extract job %s failed, status: %s",
- extractJob.getJobReference().getJobId(), extractJob.getStatus()));
- }
-
- List<String> tempFiles = getExtractFilePaths(extractDestinationDir, extractJob);
- return ImmutableList.copyOf(tempFiles);
- }
-
- private List<BoundedSource<TableRow>> createSources(
- List<String> files, TableSchema tableSchema) throws IOException, InterruptedException {
- final String jsonSchema = JSON_FACTORY.toString(tableSchema);
-
- SerializableFunction<GenericRecord, TableRow> function =
- new SerializableFunction<GenericRecord, TableRow>() {
- @Override
- public TableRow apply(GenericRecord input) {
- return AvroUtils.convertGenericRecordToTableRow(
- input, fromJsonString(jsonSchema, TableSchema.class));
- }};
-
- List<BoundedSource<TableRow>> avroSources = Lists.newArrayList();
- BackOff backoff = new AttemptBoundedExponentialBackOff(
- MAX_FILES_VERIFY_ATTEMPTS, INITIAL_FILES_VERIFY_BACKOFF_MILLIS);
- for (String fileName : files) {
- while (BackOffUtils.next(Sleeper.DEFAULT, backoff)) {
- if (IOChannelUtils.getFactory(fileName).getSizeBytes(fileName) != -1) {
- break;
- }
- }
- avroSources.add(new TransformingSource<>(
- AvroSource.from(fileName), function, getDefaultOutputCoder()));
- }
- return ImmutableList.copyOf(avroSources);
- }
-
- protected static class BigQueryReader extends BoundedSource.BoundedReader<TableRow> {
- private final BigQuerySourceBase source;
- private final BigQueryServices.BigQueryJsonReader reader;
-
- private BigQueryReader(
- BigQuerySourceBase source, BigQueryServices.BigQueryJsonReader reader) {
- this.source = source;
- this.reader = reader;
- }
-
- @Override
- public BoundedSource<TableRow> getCurrentSource() {
- return source;
- }
-
- @Override
- public boolean start() throws IOException {
- return reader.start();
- }
-
- @Override
- public boolean advance() throws IOException {
- return reader.advance();
- }
-
- @Override
- public TableRow getCurrent() throws NoSuchElementException {
- return reader.getCurrent();
- }
-
- @Override
- public void close() throws IOException {
- reader.close();
- }
- }
- }
-
- /**
- * A {@link BoundedSource} that reads from {@code BoundedSource<T>}
- * and transforms elements to type {@code V}.
- */
- @VisibleForTesting
- static class TransformingSource<T, V> extends BoundedSource<V> {
- private final BoundedSource<T> boundedSource;
- private final SerializableFunction<T, V> function;
- private final Coder<V> outputCoder;
-
- TransformingSource(
- BoundedSource<T> boundedSource,
- SerializableFunction<T, V> function,
- Coder<V> outputCoder) {
- this.boundedSource = boundedSource;
- this.function = function;
- this.outputCoder = outputCoder;
- }
-
- @Override
- public List<? extends BoundedSource<V>> splitIntoBundles(
- long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
- return Lists.transform(
- boundedSource.splitIntoBundles(desiredBundleSizeBytes, options),
- new Function<BoundedSource<T>, BoundedSource<V>>() {
- @Override
- public BoundedSource<V> apply(BoundedSource<T> input) {
- return new TransformingSource<>(input, function, outputCoder);
- }
- });
- }
-
- @Override
- public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
- return boundedSource.getEstimatedSizeBytes(options);
- }
-
- @Override
- public boolean producesSortedKeys(PipelineOptions options) throws Exception {
- return boundedSource.producesSortedKeys(options);
- }
-
- @Override
- public BoundedReader<V> createReader(PipelineOptions options) throws IOException {
- return new TransformingReader(boundedSource.createReader(options));
- }
-
- @Override
- public void validate() {
- boundedSource.validate();
- }
-
- @Override
- public Coder<V> getDefaultOutputCoder() {
- return outputCoder;
- }
-
- private class TransformingReader extends BoundedReader<V> {
- private final BoundedReader<T> boundedReader;
-
- private TransformingReader(BoundedReader<T> boundedReader) {
- this.boundedReader = boundedReader;
- }
-
- @Override
- public synchronized BoundedSource<V> getCurrentSource() {
- return new TransformingSource<>(boundedReader.getCurrentSource(), function, outputCoder);
- }
-
- @Override
- public boolean start() throws IOException {
- return boundedReader.start();
- }
-
- @Override
- public boolean advance() throws IOException {
- return boundedReader.advance();
- }
-
- @Override
- public V getCurrent() throws NoSuchElementException {
- T current = boundedReader.getCurrent();
- return function.apply(current);
- }
-
- @Override
- public void close() throws IOException {
- boundedReader.close();
- }
-
- @Override
- public synchronized BoundedSource<V> splitAtFraction(double fraction) {
- return new TransformingSource<>(
- boundedReader.splitAtFraction(fraction), function, outputCoder);
- }
-
- @Override
- public Double getFractionConsumed() {
- return boundedReader.getFractionConsumed();
- }
-
- @Override
- public Instant getCurrentTimestamp() throws NoSuchElementException {
- return boundedReader.getCurrentTimestamp();
- }
- }
- }
-
- private static String getExtractJobId(String jobIdToken) {
- return jobIdToken + "-extract";
- }
-
- private static String getExtractDestinationUri(String extractDestinationDir) {
- return String.format("%s/%s", extractDestinationDir, "*.avro");
- }
-
- private static List<String> getExtractFilePaths(String extractDestinationDir, Job extractJob)
- throws IOException {
- JobStatistics jobStats = extractJob.getStatistics();
- List<Long> counts = jobStats.getExtract().getDestinationUriFileCounts();
- if (counts.size() != 1) {
- String errorMessage = (counts.size() == 0
- ? "No destination uri file count received."
- : String.format("More than one destination uri file count received. First two are %s, %s",
- counts.get(0), counts.get(1)));
- throw new RuntimeException(errorMessage);
- }
- long filesCount = counts.get(0);
-
- ImmutableList.Builder<String> paths = ImmutableList.builder();
- IOChannelFactory factory = IOChannelUtils.getFactory(extractDestinationDir);
- for (long i = 0; i < filesCount; ++i) {
- String filePath =
- factory.resolve(extractDestinationDir, String.format("%012d%s", i, ".avro"));
- paths.add(filePath);
- }
- return paths.build();
- }
-
- /////////////////////////////////////////////////////////////////////////////
-
- /**
- * A {@link PTransform} that writes a {@link PCollection} containing {@link TableRow TableRows}
- * to a BigQuery table.
- *
- * <p>In BigQuery, each table has an encosing dataset. The dataset being written must already
- * exist.
- *
- * <p>By default, tables will be created if they do not exist, which corresponds to a
- * {@link CreateDisposition#CREATE_IF_NEEDED} disposition that matches the default of BigQuery's
- * Jobs API. A schema must be provided (via {@link BigQueryIO.Write#withSchema(TableSchema)}),
- * or else the transform may fail at runtime with an {@link IllegalArgumentException}.
- *
- * <p>By default, writes require an empty table, which corresponds to
- * a {@link WriteDisposition#WRITE_EMPTY} disposition that matches the
- * default of BigQuery's Jobs API.
- *
- * <p>Here is a sample transform that produces TableRow values containing
- * "word" and "count" columns:
- * <pre>{@code
- * static class FormatCountsFn extends DoFn<KV<String, Long>, TableRow> {
- * public void processElement(ProcessContext c) {
- * TableRow row = new TableRow()
- * .set("word", c.element().getKey())
- * .set("count", c.element().getValue().intValue());
- * c.output(row);
- * }
- * }}</pre>
- */
- public static class Write {
- /**
- * An enumeration type for the BigQuery create disposition strings.
- *
- * @see <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.query.createDisposition">
- * <code>configuration.query.createDisposition</code> in the BigQuery Jobs API</a>
- */
- public enum CreateDisposition {
- /**
- * Specifics that tables should not be created.
- *
- * <p>If the output table does not exist, the write fails.
- */
- CREATE_NEVER,
-
- /**
- * Specifies that tables should be created if needed. This is the default
- * behavior.
- *
- * <p>Requires that a table schema is provided via {@link BigQueryIO.Write#withSchema}.
- * This precondition is checked before starting a job. The schema is
- * not required to match an existing table's schema.
- *
- * <p>When this transformation is executed, if the output table does not
- * exist, the table is created from the provided schema. Note that even if
- * the table exists, it may be recreated if necessary when paired with a
- * {@link WriteDisposition#WRITE_TRUNCATE}.
- */
- CREATE_IF_NEEDED
- }
-
- /**
- * An enumeration type for the BigQuery write disposition strings.
- *
- * @see <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.query.writeDisposition">
- * <code>configuration.query.writeDisposition</code> in the BigQuery Jobs API</a>
- */
- public enum WriteDisposition {
- /**
- * Specifies that write should replace a table.
- *
- * <p>The replacement may occur in multiple steps - for instance by first
- * removing the existing table, then creating a replacement, then filling
- * it in. This is not an atomic operation, and external programs may
- * see the table in any of these intermediate steps.
- */
- WRITE_TRUNCATE,
-
- /**
- * Specifies that rows may be appended to an existing table.
- */
- WRITE_APPEND,
-
- /**
- * Specifies that the output table must be empty. This is the default
- * behavior.
- *
- * <p>If the output table is not empty, the write fails at runtime.
- *
- * <p>This check may occur long before data is written, and does not
- * guarantee exclusive access to the table. If two programs are run
- * concurrently, each specifying the same output table and
- * a {@link WriteDisposition} of {@link WriteDisposition#WRITE_EMPTY}, it is possible
- * for both to succeed.
- */
- WRITE_EMPTY
- }
-
- /**
- * Creates a write transformation for the given table specification.
- *
- * <p>Refer to {@link #parseTableSpec(String)} for the specification format.
- */
- public static Bound to(String tableSpec) {
- return new Bound().to(tableSpec);
- }
-
- /** Creates a write transformation for the given table. */
- public static Bound to(TableReference table) {
- return new Bound().to(table);
- }
-
- /**
- * Creates a write transformation from a function that maps windows to table specifications.
- * Each time a new window is encountered, this function will be called and the resulting table
- * will be created. Records within that window will be written to the associated table.
- *
- * <p>See {@link #parseTableSpec(String)} for the format that {@code tableSpecFunction} should
- * return.
- *
- * <p>{@code tableSpecFunction} should be deterministic. When given the same window, it should
- * always return the same table specification.
- */
- public static Bound to(SerializableFunction<BoundedWindow, String> tableSpecFunction) {
- return new Bound().to(tableSpecFunction);
- }
-
- /**
- * Creates a write transformation from a function that maps windows to {@link TableReference}
- * objects.
- *
- * <p>{@code tableRefFunction} should be deterministic. When given the same window, it should
- * always return the same table reference.
- */
- public static Bound toTableReference(
- SerializableFunction<BoundedWindow, TableReference> tableRefFunction) {
- return new Bound().toTableReference(tableRefFunction);
- }
-
- /**
- * Creates a write transformation with the specified schema to use in table creation.
- *
- * <p>The schema is <i>required</i> only if writing to a table that does not already
- * exist, and {@link CreateDisposition} is set to
- * {@link CreateDisposition#CREATE_IF_NEEDED}.
- */
- public static Bound withSchema(TableSchema schema) {
- return new Bound().withSchema(schema);
- }
-
- /** Creates a write transformation with the specified options for creating the table. */
- public static Bound withCreateDisposition(CreateDisposition disposition) {
- return new Bound().withCreateDisposition(disposition);
- }
-
- /** Creates a write transformation with the specified options for writing to the table. */
- public static Bound withWriteDisposition(WriteDisposition disposition) {
- return new Bound().withWriteDisposition(disposition);
- }
-
- /**
- * Creates a write transformation with BigQuery table validation disabled.
- */
- public static Bound withoutValidation() {
- return new Bound().withoutValidation();
- }
-
- /**
- * A {@link PTransform} that can write either a bounded or unbounded
- * {@link PCollection} of {@link TableRow TableRows} to a BigQuery table.
- */
- public static class Bound extends PTransform<PCollection<TableRow>, PDone> {
- @Nullable final String jsonTableRef;
-
- @Nullable final SerializableFunction<BoundedWindow, TableReference> tableRefFunction;
-
- // Table schema. The schema is required only if the table does not exist.
- @Nullable final String jsonSchema;
-
- // Options for creating the table. Valid values are CREATE_IF_NEEDED and
- // CREATE_NEVER.
- final CreateDisposition createDisposition;
-
- // Options for writing to the table. Valid values are WRITE_TRUNCATE,
- // WRITE_APPEND and WRITE_EMPTY.
- final WriteDisposition writeDisposition;
-
- // An option to indicate if table validation is desired. Default is true.
- final boolean validate;
-
- @Nullable private BigQueryServices bigQueryServices;
-
- private static class TranslateTableSpecFunction implements
- SerializableFunction<BoundedWindow, TableReference> {
- private SerializableFunction<BoundedWindow, String> tableSpecFunction;
-
- TranslateTableSpecFunction(SerializableFunction<BoundedWindow, String> tableSpecFunction) {
- this.tableSpecFunction = tableSpecFunction;
- }
-
- @Override
- public TableReference apply(BoundedWindow value) {
- return parseTableSpec(tableSpecFunction.apply(value));
- }
- }
-
- /**
- * @deprecated Should be private. Instead, use one of the factory methods in
- * {@link BigQueryIO.Write}, such as {@link BigQueryIO.Write#to(String)}, to create an
- * instance of this class.
- */
- @Deprecated
- public Bound() {
- this(
- null /* name */,
- null /* jsonTableRef */,
- null /* tableRefFunction */,
- null /* jsonSchema */,
- CreateDisposition.CREATE_IF_NEEDED,
- WriteDisposition.WRITE_EMPTY,
- true /* validate */,
- null /* bigQueryServices */);
- }
-
- private Bound(String name, @Nullable String jsonTableRef,
- @Nullable SerializableFunction<BoundedWindow, TableReference> tableRefFunction,
- @Nullable String jsonSchema,
- CreateDisposition createDisposition, WriteDisposition writeDisposition, boolean validate,
- @Nullable BigQueryServices bigQueryServices) {
- super(name);
- this.jsonTableRef = jsonTableRef;
- this.tableRefFunction = tableRefFunction;
- this.jsonSchema = jsonSchema;
- this.createDisposition = checkNotNull(createDisposition, "createDisposition");
- this.writeDisposition = checkNotNull(writeDisposition, "writeDisposition");
- this.validate = validate;
- this.bigQueryServices = bigQueryServices;
- }
-
- /**
- * Returns a copy of this write transformation, but writing to the specified table. Refer to
- * {@link #parseTableSpec(String)} for the specification format.
- *
- * <p>Does not modify this object.
- */
- public Bound to(String tableSpec) {
- return to(parseTableSpec(tableSpec));
- }
-
- /**
- * Returns a copy of this write transformation, but writing to the specified table.
- *
- * <p>Does not modify this object.
- */
- public Bound to(TableReference table) {
- return new Bound(name, toJsonString(table), tableRefFunction, jsonSchema, createDisposition,
- writeDisposition, validate, bigQueryServices);
- }
-
- /**
- * Returns a copy of this write transformation, but using the specified function to determine
- * which table to write to for each window.
- *
- * <p>Does not modify this object.
- *
- * <p>{@code tableSpecFunction} should be deterministic. When given the same window, it
- * should always return the same table specification.
- */
- public Bound to(
- SerializableFunction<BoundedWindow, String> tableSpecFunction) {
- return toTableReference(new TranslateTableSpecFunction(tableSpecFunction));
- }
-
- /**
- * Returns a copy of this write transformation, but using the specified function to determine
- * which table to write to for each window.
- *
- * <p>Does not modify this object.
- *
- * <p>{@code tableRefFunction} should be deterministic. When given the same window, it should
- * always return the same table reference.
- */
- public Bound toTableReference(
- SerializableFunction<BoundedWindow, TableReference> tableRefFunction) {
- return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
- writeDisposition, validate, bigQueryServices);
- }
-
- /**
- * Returns a copy of this write transformation, but using the specified schema for rows
- * to be written.
- *
- * <p>Does not modify this object.
- */
- public Bound withSchema(TableSchema schema) {
- return new Bound(name, jsonTableRef, tableRefFunction, toJsonString(schema),
- createDisposition, writeDisposition, validate, bigQueryServices);
- }
-
- /**
- * Returns a copy of this write transformation, but using the specified create disposition.
- *
- * <p>Does not modify this object.
- */
- public Bound withCreateDisposition(CreateDisposition createDisposition) {
- return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
- writeDisposition, validate, bigQueryServices);
- }
-
- /**
- * Returns a copy of this write transformation, but using the specified write disposition.
- *
- * <p>Does not modify this object.
- */
- public Bound withWriteDisposition(WriteDisposition writeDisposition) {
- return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
- writeDisposition, validate, bigQueryServices);
- }
-
- /**
- * Returns a copy of this write transformation, but without BigQuery table validation.
- *
- * <p>Does not modify this object.
- */
- public Bound withoutValidation() {
- return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
- writeDisposition, false, bigQueryServices);
- }
-
- @VisibleForTesting
- Bound withTestServices(BigQueryServices testServices) {
- return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
- writeDisposition, validate, testServices);
- }
-
- private static void verifyTableEmpty(
- DatasetService datasetService,
- TableReference table) {
- try {
- boolean isEmpty = datasetService.isTableEmpty(
- table.getProjectId(), table.getDatasetId(), table.getTableId());
- if (!isEmpty) {
- throw new IllegalArgumentException(
- "BigQuery table is not empty: " + BigQueryIO.toTableSpec(table));
- }
- } catch (IOException | InterruptedException e) {
- ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
- if (e instanceof IOException && errorExtractor.itemNotFound((IOException) e)) {
- // Nothing to do. If the table does not exist, it is considered empty.
- } else {
- throw new RuntimeException(
- "unable to confirm BigQuery table emptiness for table "
- + BigQueryIO.toTableSpec(table), e);
- }
- }
- }
-
- @Override
- public void validate(PCollection<TableRow> input) {
- BigQueryOptions options = input.getPipeline().getOptions().as(BigQueryOptions.class);
-
- // Exactly one of the table and table reference can be configured.
- checkState(
- jsonTableRef != null || tableRefFunction != null,
- "must set the table reference of a BigQueryIO.Write transform");
- checkState(
- jsonTableRef == null || tableRefFunction == null,
- "Cannot set both a table reference and a table function for a BigQueryIO.Write"
- + " transform");
-
- // Require a schema if creating one or more tables.
- checkArgument(
- createDisposition != CreateDisposition.CREATE_IF_NEEDED || jsonSchema != null,
- "CreateDisposition is CREATE_IF_NEEDED, however no schema was provided.");
-
- // The user specified a table.
- if (jsonTableRef != null && validate) {
- TableReference table = getTableWithDefaultProject(options);
-
- DatasetService datasetService = getBigQueryServices().getDatasetService(options);
- // Check for destination table presence and emptiness for early failure notification.
- // Note that a presence check can fail when the table or dataset is created by an earlier
- // stage of the pipeline. For these cases the #withoutValidation method can be used to
- // disable the check.
- verifyDatasetPresence(datasetService, table);
- if (getCreateDisposition() == BigQueryIO.Write.CreateDisposition.CREATE_NEVER) {
- verifyTablePresence(datasetService, table);
- }
- if (getWriteDisposition() == BigQueryIO.Write.WriteDisposition.WRITE_EMPTY) {
- verifyTableEmpty(datasetService, table);
- }
- }
-
- if (input.isBounded() == PCollection.IsBounded.UNBOUNDED || tableRefFunction != null) {
- // We will use BigQuery's streaming write API -- validate supported dispositions.
- checkArgument(
- createDisposition != CreateDisposition.CREATE_NEVER,
- "CreateDisposition.CREATE_NEVER is not supported for an unbounded PCollection or when"
- + " using a tablespec function.");
-
- checkArgument(
- writeDisposition != WriteDisposition.WRITE_TRUNCATE,
- "WriteDisposition.WRITE_TRUNCATE is not supported for an unbounded PCollection or"
- + " when using a tablespec function.");
- } else {
- // We will use a BigQuery load job -- validate the temp location.
- String tempLocation = options.getTempLocation();
- checkArgument(
- !Strings.isNullOrEmpty(tempLocation),
- "BigQueryIO.Write needs a GCS temp location to store temp files.");
- if (bigQueryServices == null) {
- try {
- GcsPath.fromUri(tempLocation);
- } catch (IllegalArgumentException e) {
- throw new IllegalArgumentException(
- String.format(
- "BigQuery temp location expected a valid 'gs://' path, but was given '%s'",
- tempLocation),
- e);
- }
- }
- }
- }
-
- @Override
- public PDone apply(PCollection<TableRow> input) {
- BigQueryOptions options = input.getPipeline().getOptions().as(BigQueryOptions.class);
-
- // In a streaming job, or when a tablespec function is defined, we use StreamWithDeDup
- // and BigQuery's streaming import API.
- if (options.isStreaming() || tableRefFunction != null) {
- return input.apply(new StreamWithDeDup(getTable(), tableRefFunction, getSchema()));
- }
-
- TableReference table = fromJsonString(jsonTableRef, TableReference.class);
- if (Strings.isNullOrEmpty(table.getProjectId())) {
- table.setProjectId(options.getProject());
- }
- String jobIdToken = randomUUIDString();
- String tempLocation = options.getTempLocation();
- String tempFilePrefix;
- try {
- IOChannelFactory factory = IOChannelUtils.getFactory(tempLocation);
- tempFilePrefix = factory.resolve(
- factory.resolve(tempLocation, "BigQuerySinkTemp"),
- jobIdToken);
- } catch (IOException e) {
- throw new RuntimeException(
- String.format("Failed to resolve BigQuery temp location in %s", tempLocation),
- e);
- }
-
- BigQueryServices bqServices = getBigQueryServices();
- return input.apply("Write", org.apache.beam.sdk.io.Write.to(
- new BigQuerySink(
- jobIdToken,
- table,
- jsonSchema,
- getWriteDisposition(),
- getCreateDisposition(),
- tempFilePrefix,
- input.getCoder(),
- bqServices)));
- }
-
- @Override
- protected Coder<Void> getDefaultOutputCoder() {
- return VoidCoder.of();
- }
-
- @Override
- public void populateDisplayData(DisplayData.Builder builder) {
- super.populateDisplayData(builder);
-
- builder
- .addIfNotNull(DisplayData.item("table", jsonTableRef)
- .withLabel("Table Reference"))
- .addIfNotNull(DisplayData.item("schema", jsonSchema)
- .withLabel("Table Schema"));
-
- if (tableRefFunction != null) {
- builder.add(DisplayData.item("tableFn", tableRefFunction.getClass())
- .withLabel("Table Reference Function"));
- }
-
- builder
- .add(DisplayData.item("createDisposition", createDisposition.toString())
- .withLabel("Table CreateDisposition"))
- .add(DisplayData.item("writeDisposition", writeDisposition.toString())
- .withLabel("Table WriteDisposition"))
- .addIfNotDefault(DisplayData.item("validation", validate)
- .withLabel("Validation Enabled"), true);
- }
-
- /** Returns the create disposition. */
- public CreateDisposition getCreateDisposition() {
- return createDisposition;
- }
-
- /** Returns the write disposition. */
- public WriteDisposition getWriteDisposition() {
- return writeDisposition;
- }
-
- /** Returns the table schema. */
- public TableSchema getSchema() {
- return fromJsonString(jsonSchema, TableSchema.class);
- }
-
- /**
- * Returns the table to write, or {@code null} if writing with {@code tableRefFunction}.
- *
- * <p>If the table's project is not specified, use the executing project.
- */
- @Nullable private TableReference getTableWithDefaultProject(BigQueryOptions bqOptions) {
- TableReference table = getTable();
- if (table != null && Strings.isNullOrEmpty(table.getProjectId())) {
- // If user does not specify a project we assume the table to be located in
- // the default project.
- table.setProjectId(bqOptions.getProject());
- }
- return table;
- }
-
- /** Returns the table reference, or {@code null}. */
- @Nullable
- public TableReference getTable() {
- return fromJsonString(jsonTableRef, TableReference.class);
- }
-
- /** Returns {@code true} if table validation is enabled. */
- public boolean getValidate() {
- return validate;
- }
-
- private BigQueryServices getBigQueryServices() {
- if (bigQueryServices == null) {
- bigQueryServices = new BigQueryServicesImpl();
- }
- return bigQueryServices;
- }
- }
-
- /** Disallow construction of utility class. */
- private Write() {}
- }
-
- /**
- * {@link BigQuerySink} is implemented as a {@link FileBasedSink}.
- *
- * <p>It uses BigQuery load job to import files into BigQuery.
- */
- static class BigQuerySink extends FileBasedSink<TableRow> {
- private final String jobIdToken;
- @Nullable private final String jsonTable;
- @Nullable private final String jsonSchema;
- private final WriteDisposition writeDisposition;
- private final CreateDisposition createDisposition;
- private final Coder<TableRow> coder;
- private final BigQueryServices bqServices;
-
- public BigQuerySink(
- String jobIdToken,
- @Nullable TableReference table,
- @Nullable String jsonSchema,
- WriteDisposition writeDisposition,
- CreateDisposition createDisposition,
- String tempFile,
- Coder<TableRow> coder,
- BigQueryServices bqServices) {
- super(tempFile, ".json");
- this.jobIdToken = checkNotNull(jobIdToken, "jobIdToken");
- if (table == null) {
- this.jsonTable = null;
- } else {
- checkArgument(!Strings.isNullOrEmpty(table.getProjectId()),
- "Table %s should have a project specified", table);
- this.jsonTable = toJsonString(table);
- }
- this.jsonSchema = jsonSchema;
- this.writeDisposition = checkNotNull(writeDisposition, "writeDisposition");
- this.createDisposition = checkNotNull(createDisposition, "createDisposition");
- this.coder = checkNotNull(coder, "coder");
- this.bqServices = checkNotNull(bqServices, "bqServices");
- }
-
- @Override
- public FileBasedSink.FileBasedWriteOperation<TableRow> createWriteOperation(
- PipelineOptions options) {
- return new BigQueryWriteOperation(this);
- }
-
- @Override
- public void populateDisplayData(DisplayData.Builder builder) {
- super.populateDisplayData(builder);
-
- builder
- .addIfNotNull(DisplayData.item("schema", jsonSchema)
- .withLabel("Table Schema"))
- .addIfNotNull(DisplayData.item("tableSpec", jsonTable)
- .withLabel("Table Specification"));
- }
-
- private static class BigQueryWriteOperation extends FileBasedWriteOperation<TableRow> {
- // The maximum number of retry load jobs.
- private static final int MAX_RETRY_LOAD_JOBS = 3;
-
- // The maximum number of retries to poll the status of a load job.
- // It sets to {@code Integer.MAX_VALUE} to block until the BigQuery job finishes.
- private static final int LOAD_JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE;
-
- private final BigQuerySink bigQuerySink;
-
- private BigQueryWriteOperation(BigQuerySink sink) {
- super(checkNotNull(sink, "sink"));
- this.bigQuerySink = sink;
- }
-
- @Override
- public FileBasedWriter<TableRow> createWriter(PipelineOptions options) throws Exception {
- return new TableRowWriter(this, bigQuerySink.coder);
- }
-
- @Override
- public void finalize(Iterable<FileResult> writerResults, PipelineOptions options)
- throws IOException, InterruptedException {
- try {
- BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
- List<String> tempFiles = Lists.newArrayList();
- for (FileResult result : writerResults) {
- tempFiles.add(result.getFilename());
- }
- if (!tempFiles.isEmpty()) {
- load(
- bigQuerySink.bqServices.getJobService(bqOptions),
- bigQuerySink.jobIdToken,
- fromJsonString(bigQuerySink.jsonTable, TableReference.class),
- tempFiles,
- fromJsonString(bigQuerySink.jsonSchema, TableSchema.class),
- bigQuerySink.writeDisposition,
- bigQuerySink.createDisposition);
- }
- } finally {
- removeTemporaryFiles(options);
- }
- }
-
- /**
- * Import files into BigQuery with load jobs.
- *
- * <p>Returns if files are successfully loaded into BigQuery.
- * Throws a RuntimeException if:
- * 1. The status of one load job is UNKNOWN. This is to avoid duplicating data.
- * 2. It exceeds {@code MAX_RETRY_LOAD_JOBS}.
- *
- * <p>If a load job failed, it will try another load job with a different job id.
- */
- private void load(
- JobService jobService,
- String jobIdPrefix,
- TableReference ref,
- List<String> gcsUris,
- @Nullable TableSchema schema,
- WriteDisposition writeDisposition,
- CreateDisposition createDisposition) throws InterruptedException, IOException {
- JobConfigurationLoad loadConfig = new JobConfigurationLoad()
- .setSourceUris(gcsUris)
- .setDestinationTable(ref)
- .setSchema(schema)
- .setWriteDisposition(writeDisposition.name())
- .setCreateDisposition(createDisposition.name())
- .setSourceFormat("NEWLINE_DELIMITED_JSON");
-
- boolean retrying = false;
- String projectId = ref.getProjectId();
- for (int i = 0; i < MAX_RETRY_LOAD_JOBS; ++i) {
- String jobId = jobIdPrefix + "-" + i;
- if (retrying) {
- LOG.info("Previous load jobs failed, retrying.");
- }
- LOG.info("Starting BigQuery load job: {}", jobId);
- JobReference jobRef = new JobReference()
- .setProjectId(projectId)
- .setJobId(jobId);
- jobService.startLoadJob(jobRef, loadConfig);
- Status jobStatus =
- parseStatus(jobService.pollJob(jobRef, LOAD_JOB_POLL_MAX_RETRIES));
- switch (jobStatus) {
- case SUCCEEDED:
- return;
- case UNKNOWN:
- throw new RuntimeException("Failed to poll the load job status.");
- case FAILED:
- LOG.info("BigQuery load job failed: {}", jobId);
- retrying = true;
- continue;
- default:
- throw new IllegalStateException("Unexpected job status: " + jobStatus);
- }
- }
- throw new RuntimeException(
- "Failed to create the load job, reached max retries: " + MAX_RETRY_LOAD_JOBS);
- }
- }
-
- private static class TableRowWriter extends FileBasedWriter<TableRow> {
- private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8);
- private final Coder<TableRow> coder;
- private OutputStream out;
-
- public TableRowWriter(
- FileBasedWriteOperation<TableRow> writeOperation, Coder<TableRow> coder) {
- super(writeOperation);
- this.mimeType = MimeTypes.TEXT;
- this.coder = coder;
- }
-
- @Override
- protected void prepareWrite(WritableByteChannel channel) throws Exception {
- out = Channels.newOutputStream(channel);
- }
-
- @Override
- public void write(TableRow value) throws Exception {
- // Use Context.OUTER to encode and NEWLINE as the delimeter.
- coder.encode(value, out, Context.OUTER);
- out.write(NEWLINE);
- }
- }
- }
-
- private static void verifyDatasetPresence(DatasetService datasetService, TableReference table) {
- try {
- datasetService.getDataset(table.getProjectId(), table.getDatasetId());
- } catch (Exception e) {
- ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
- if ((e instanceof IOException) && errorExtractor.itemNotFound((IOException) e)) {
- throw new IllegalArgumentException(
- String.format(RESOURCE_NOT_FOUND_ERROR, "dataset", BigQueryIO.toTableSpec(table)),
- e);
- } else {
- throw new RuntimeException(
- String.format(UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR, "dataset",
- BigQueryIO.toTableSpec(table)),
- e);
- }
- }
- }
-
- private static void verifyTablePresence(DatasetService datasetService, TableReference table) {
- try {
- datasetService.getTable(table.getProjectId(), table.getDatasetId(), table.getTableId());
- } catch (Exception e) {
- ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
- if ((e instanceof IOException) && errorExtractor.itemNotFound((IOException) e)) {
- throw new IllegalArgumentException(
- String.format(RESOURCE_NOT_FOUND_ERROR, "table", BigQueryIO.toTableSpec(table)), e);
- } else {
- throw new RuntimeException(
- String.format(UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR, "table",
- BigQueryIO.toTableSpec(table)),
- e);
- }
- }
- }
-
- /////////////////////////////////////////////////////////////////////////////
-
- /**
- * Implementation of DoFn to perform streaming BigQuery write.
- */
- @SystemDoFnInternal
- private static class StreamingWriteFn
- extends DoFn<KV<ShardedKey<String>, TableRowInfo>, Void> {
- /** TableSchema in JSON. Use String to make the class Serializable. */
- private final String jsonTableSchema;
-
- /** JsonTableRows to accumulate BigQuery rows in order to batch writes. */
- private transient Map<String, List<TableRow>> tableRows;
-
- /** The list of unique ids for each BigQuery table row. */
- private transient Map<String, List<String>> uniqueIdsForTableRows;
-
- /** The list of tables created so far, so we don't try the creation
- each time. */
- private static Set<String> createdTables =
- Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
-
- /** Tracks bytes written, exposed as "ByteCount" Counter. */
- private Aggregator<Long, Long> byteCountAggregator =
- createAggregator("ByteCount", new Sum.SumLongFn());
-
- /** Constructor. */
- StreamingWriteFn(TableSchema schema) {
- jsonTableSchema = toJsonString(schema);
- }
-
- /** Prepares a target BigQuery table. */
- @Override
- public void startBundle(Context context) {
- tableRows = new HashMap<>();
- uniqueIdsForTableRows = new HashMap<>();
- }
-
- /** Accumulates the input into JsonTableRows and uniqueIdsForTableRows. */
- @Override
- public void processElement(ProcessContext context) {
- String tableSpec = context.element().getKey().getKey();
- List<TableRow> rows = getOrCreateMapListValue(tableRows, tableSpec);
- List<String> uniqueIds = getOrCreateMapListValue(uniqueIdsForTableRows, tableSpec);
-
- rows.add(context.element().getValue().tableRow);
- uniqueIds.add(context.element().getValue().uniqueId);
- }
-
- /** Writes the accumulated rows into BigQuery with streaming API. */
- @Override
- public void finishBundle(Context context) throws Exception {
- BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class);
- Bigquery client = Transport.newBigQueryClient(options).build();
-
- for (Map.Entry<String, List<TableRow>> entry : tableRows.entrySet()) {
- TableReference tableReference = getOrCreateTable(options, entry.getKey());
- flushRows(client, tableReference, entry.getValue(),
- uniqueIdsForTableRows.get(entry.getKey()), options);
- }
- tableRows.clear();
- uniqueIdsForTableRows.clear();
- }
-
- @Override
- public void populateDisplayData(DisplayData.Builder builder) {
- super.populateDisplayData(builder);
-
- builder.addIfNotNull(DisplayData.item("schema", jsonTableSchema)
- .withLabel("Table Schema"));
- }
-
- public TableReference getOrCreateTable(BigQueryOptions options, String tableSpec)
- throws IOException {
- TableReference tableReference = parseTableSpec(tableSpec);
- if (!createdTables.contains(tableSpec)) {
- synchronized (createdTables) {
- // Another thread may have succeeded in creating the table in the meanwhile, so
- // check again. This check isn't needed for correctness, but we add it to prevent
- // every thread from attempting a create and overwhelming our BigQuery quota.
- if (!createdTables.contains(tableSpec)) {
- TableSchema tableSchema = JSON_FACTORY.fromString(jsonTableSchema, TableSchema.class);
- Bigquery client = Transport.newBigQueryClient(options).build();
- BigQueryTableInserter inserter = new BigQueryTableInserter(client, options);
- inserter.getOrCreateTable(tableReference, WriteDisposition.WRITE_APPEND,
- CreateDisposition.CREATE_IF_NEEDED, tableSchema);
- createdTables.add(tableSpec);
- }
- }
- }
- return tableReference;
- }
-
- /** Writes the accumulated rows into BigQuery with streaming API. */
- private void flushRows(Bigquery client, TableReference tableReference,
- List<TableRow> tableRows, List<String> uniqueIds, BigQueryOptions options) {
- if (!tableRows.isEmpty()) {
- try {
- BigQueryTableInserter inserter = new BigQueryTableInserter(client, options);
- inserter.insertAll(tableReference, tableRows, uniqueIds, byteCountAggregator);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
- }
- }
-
- private static class ShardedKey<K> {
- private final K key;
- private final int shardNumber;
-
- public static <K> ShardedKey<K> of(K key, int shardNumber) {
- return new ShardedKey<K>(key, shardNumber);
- }
-
- private ShardedKey(K key, int shardNumber) {
- this.key = key;
- this.shardNumber = shardNumber;
- }
-
- public K getKey() {
- return key;
- }
-
- public int getShardNumber() {
- return shardNumber;
- }
- }
-
- /**
- * A {@link Coder} for {@link ShardedKey}, using a wrapped key {@link Coder}.
- */
- private static class ShardedKeyCoder<KeyT>
- extends StandardCoder<ShardedKey<KeyT>> {
- public static <KeyT> ShardedKeyCoder<KeyT> of(Coder<KeyT> keyCoder) {
- return new ShardedKeyCoder<>(keyCoder);
- }
-
- @JsonCreator
- public static <KeyT> ShardedKeyCoder<KeyT> of(
- @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
- List<Coder<KeyT>> components) {
- checkArgument(components.size() == 1, "Expecting 1 component, got %s", components.size());
- return of(components.get(0));
- }
-
- protected ShardedKeyCoder(Coder<KeyT> keyCoder) {
- this.keyCoder = keyCoder;
- this.shardNumberCoder = VarIntCoder.of();
- }
-
- @Override
- public List<? extends Coder<?>> getCoderArguments() {
- return Arrays.asList(keyCoder);
- }
-
- @Override
- public void encode(ShardedKey<KeyT> key, OutputStream outStream, Context context)
- throws IOException {
- keyCoder.encode(key.getKey(), outStream, context.nested());
- shardNumberCoder.encode(key.getShardNumber(), outStream, context);
- }
-
- @Override
- public ShardedKey<KeyT> decode(InputStream inStream, Context context)
- throws IOException {
- return new ShardedKey<KeyT>(
- keyCoder.decode(inStream, context.nested()),
- shardNumberCoder.decode(inStream, context));
- }
-
- @Override
- public void verifyDeterministic() throws NonDeterministicException {
- keyCoder.verifyDeterministic();
- }
-
- Coder<KeyT> keyCoder;
- VarIntCoder shardNumberCoder;
- }
-
- private static class TableRowInfoCoder extends AtomicCoder<TableRowInfo> {
- private static final TableRowInfoCoder INSTANCE = new TableRowInfoCoder();
-
- @JsonCreator
- public static TableRowInfoCoder of() {
- return INSTANCE;
- }
-
- @Override
- public void encode(TableRowInfo value, OutputStream outStream, Context context)
- throws IOException {
- if (value == null) {
- throw new CoderException("cannot encode a null value");
- }
- tableRowCoder.encode(value.tableRow, outStream, context.nested());
- idCoder.encode(value.uniqueId, outStream, context.nested());
- }
-
- @Override
- public TableRowInfo decode(InputStream inStream, Context context)
- throws IOException {
- return new TableRowInfo(
- tableRowCoder.decode(inStream, context.nested()),
- idCoder.decode(inStream, context.nested()));
- }
-
- @Override
- public void verifyDeterministic() throws NonDeterministicException {
- throw new NonDeterministicException(this, "TableRows are not deterministic.");
- }
-
- TableRowJsonCoder tableRowCoder = TableRowJsonCoder.of();
- StringUtf8Coder idCoder = StringUtf8Coder.of();
- }
-
- private static class TableRowInfo {
- TableRowInfo(TableRo
<TRUNCATED>
[03/10] incubator-beam git commit: BigQueryIO: move to
google-cloud-platform module
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java
new file mode 100644
index 0000000..3e77362
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java
@@ -0,0 +1,177 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import org.apache.beam.sdk.options.BigQueryOptions;
+
+import com.google.api.services.bigquery.model.Dataset;
+import com.google.api.services.bigquery.model.Job;
+import com.google.api.services.bigquery.model.JobConfigurationExtract;
+import com.google.api.services.bigquery.model.JobConfigurationLoad;
+import com.google.api.services.bigquery.model.JobConfigurationQuery;
+import com.google.api.services.bigquery.model.JobReference;
+import com.google.api.services.bigquery.model.JobStatistics;
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.NoSuchElementException;
+
+import javax.annotation.Nullable;
+
+/**
+ * An interface for real, mock, or fake implementations of Cloud BigQuery services.
+ */
+interface BigQueryServices extends Serializable {
+
+ /**
+ * Returns a real, mock, or fake {@link JobService}.
+ */
+ JobService getJobService(BigQueryOptions bqOptions);
+
+ /**
+ * Returns a real, mock, or fake {@link DatasetService}.
+ */
+ DatasetService getDatasetService(BigQueryOptions bqOptions);
+
+ /**
+ * Returns a real, mock, or fake {@link BigQueryJsonReader} to read tables.
+ */
+ BigQueryJsonReader getReaderFromTable(BigQueryOptions bqOptions, TableReference tableRef);
+
+ /**
+ * Returns a real, mock, or fake {@link BigQueryJsonReader} to query tables.
+ */
+ BigQueryJsonReader getReaderFromQuery(
+ BigQueryOptions bqOptions, String query, String projectId, @Nullable Boolean flatten);
+
+ /**
+ * An interface for the Cloud BigQuery load service.
+ */
+ interface JobService {
+ /**
+ * Start a BigQuery load job.
+ */
+ void startLoadJob(JobReference jobRef, JobConfigurationLoad loadConfig)
+ throws InterruptedException, IOException;
+ /**
+ * Start a BigQuery extract job.
+ */
+ void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig)
+ throws InterruptedException, IOException;
+
+ /**
+ * Start a BigQuery query job.
+ */
+ void startQueryJob(JobReference jobRef, JobConfigurationQuery query)
+ throws IOException, InterruptedException;
+
+ /**
+ * Waits for the job is Done, and returns the job.
+ *
+ * <p>Returns null if the {@code maxAttempts} retries reached.
+ */
+ Job pollJob(JobReference jobRef, int maxAttempts)
+ throws InterruptedException, IOException;
+
+ /**
+ * Dry runs the query in the given project.
+ */
+ JobStatistics dryRunQuery(String projectId, String query)
+ throws InterruptedException, IOException;
+ }
+
+ /**
+ * An interface to get, create and delete Cloud BigQuery datasets and tables.
+ */
+ interface DatasetService {
+ /**
+ * Gets the specified {@link Table} resource by table ID.
+ */
+ Table getTable(String projectId, String datasetId, String tableId)
+ throws InterruptedException, IOException;
+
+ /**
+ * Deletes the table specified by tableId from the dataset.
+ * If the table contains data, all the data will be deleted.
+ */
+ void deleteTable(String projectId, String datasetId, String tableId)
+ throws IOException, InterruptedException;
+
+ /**
+ * Returns true if the table is empty.
+ */
+ boolean isTableEmpty(String projectId, String datasetId, String tableId)
+ throws IOException, InterruptedException;
+
+ /**
+ * Gets the specified {@link Dataset} resource by dataset ID.
+ */
+ Dataset getDataset(String projectId, String datasetId)
+ throws IOException, InterruptedException;
+
+ /**
+ * Create a {@link Dataset} with the given {@code location} and {@code description}.
+ */
+ void createDataset(String projectId, String datasetId, String location, String description)
+ throws IOException, InterruptedException;
+
+ /**
+ * Deletes the dataset specified by the datasetId value.
+ *
+ * <p>Before you can delete a dataset, you must delete all its tables.
+ */
+ void deleteDataset(String projectId, String datasetId)
+ throws IOException, InterruptedException;
+ }
+
+ /**
+ * An interface to read the Cloud BigQuery directly.
+ */
+ interface BigQueryJsonReader {
+ /**
+ * Initializes the reader and advances the reader to the first record.
+ */
+ boolean start() throws IOException;
+
+ /**
+ * Advances the reader to the next valid record.
+ */
+ boolean advance() throws IOException;
+
+ /**
+ * Returns the value of the data item that was read by the last {@link #start} or
+ * {@link #advance} call. The returned value must be effectively immutable and remain valid
+ * indefinitely.
+ *
+ * <p>Multiple calls to this method without an intervening call to {@link #advance} should
+ * return the same result.
+ *
+ * @throws java.util.NoSuchElementException if {@link #start} was never called, or if
+ * the last {@link #start} or {@link #advance} returned {@code false}.
+ */
+ TableRow getCurrent() throws NoSuchElementException;
+
+ /**
+ * Closes the reader. The reader cannot be used after this method is called.
+ */
+ void close() throws IOException;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
new file mode 100644
index 0000000..414baae
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
@@ -0,0 +1,511 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.Transport;
+
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.model.Dataset;
+import com.google.api.services.bigquery.model.DatasetReference;
+import com.google.api.services.bigquery.model.Job;
+import com.google.api.services.bigquery.model.JobConfiguration;
+import com.google.api.services.bigquery.model.JobConfigurationExtract;
+import com.google.api.services.bigquery.model.JobConfigurationLoad;
+import com.google.api.services.bigquery.model.JobConfigurationQuery;
+import com.google.api.services.bigquery.model.JobReference;
+import com.google.api.services.bigquery.model.JobStatistics;
+import com.google.api.services.bigquery.model.JobStatus;
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableDataList;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.hadoop.util.ApiErrorExtractor;
+import com.google.common.annotations.VisibleForTesting;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.Nullable;
+
+/**
+ * An implementation of {@link BigQueryServices} that actually communicates with the cloud BigQuery
+ * service.
+ */
+class BigQueryServicesImpl implements BigQueryServices {
+
+ private static final Logger LOG = LoggerFactory.getLogger(BigQueryServicesImpl.class);
+
+ // The maximum number of attempts to execute a BigQuery RPC.
+ private static final int MAX_RPC_ATTEMPTS = 10;
+
+ // The initial backoff for executing a BigQuery RPC.
+ private static final long INITIAL_RPC_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1);
+
+ // The initial backoff for polling the status of a BigQuery job.
+ private static final long INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1);
+
+ @Override
+ public JobService getJobService(BigQueryOptions options) {
+ return new JobServiceImpl(options);
+ }
+
+ @Override
+ public DatasetService getDatasetService(BigQueryOptions options) {
+ return new DatasetServiceImpl(options);
+ }
+
+ @Override
+ public BigQueryJsonReader getReaderFromTable(BigQueryOptions bqOptions, TableReference tableRef) {
+ return BigQueryJsonReaderImpl.fromTable(bqOptions, tableRef);
+ }
+
+ @Override
+ public BigQueryJsonReader getReaderFromQuery(
+ BigQueryOptions bqOptions, String query, String projectId, @Nullable Boolean flatten) {
+ return BigQueryJsonReaderImpl.fromQuery(bqOptions, query, projectId, flatten);
+ }
+
+ @VisibleForTesting
+ static class JobServiceImpl implements BigQueryServices.JobService {
+ private final ApiErrorExtractor errorExtractor;
+ private final Bigquery client;
+
+ @VisibleForTesting
+ JobServiceImpl(Bigquery client) {
+ this.errorExtractor = new ApiErrorExtractor();
+ this.client = client;
+ }
+
+ private JobServiceImpl(BigQueryOptions options) {
+ this.errorExtractor = new ApiErrorExtractor();
+ this.client = Transport.newBigQueryClient(options).build();
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+ *
+ * @throws IOException if it exceeds max RPC .
+ */
+ @Override
+ public void startLoadJob(
+ JobReference jobRef,
+ JobConfigurationLoad loadConfig) throws InterruptedException, IOException {
+ Job job = new Job()
+ .setJobReference(jobRef)
+ .setConfiguration(new JobConfiguration().setLoad(loadConfig));
+
+ startJob(job, errorExtractor, client);
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+ *
+ * @throws IOException if it exceeds max RPC .
+ */
+ @Override
+ public void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig)
+ throws InterruptedException, IOException {
+ Job job = new Job()
+ .setJobReference(jobRef)
+ .setConfiguration(
+ new JobConfiguration().setExtract(extractConfig));
+
+ startJob(job, errorExtractor, client);
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+ *
+ * @throws IOException if it exceeds max RPC .
+ */
+ @Override
+ public void startQueryJob(JobReference jobRef, JobConfigurationQuery queryConfig)
+ throws IOException, InterruptedException {
+ Job job = new Job()
+ .setJobReference(jobRef)
+ .setConfiguration(
+ new JobConfiguration().setQuery(queryConfig));
+
+ startJob(job, errorExtractor, client);
+ }
+
+ private static void startJob(Job job,
+ ApiErrorExtractor errorExtractor,
+ Bigquery client) throws IOException, InterruptedException {
+ BackOff backoff =
+ new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+ startJob(job, errorExtractor, client, Sleeper.DEFAULT, backoff);
+ }
+
+ @VisibleForTesting
+ static void startJob(
+ Job job,
+ ApiErrorExtractor errorExtractor,
+ Bigquery client,
+ Sleeper sleeper,
+ BackOff backoff) throws IOException, InterruptedException {
+ JobReference jobRef = job.getJobReference();
+ Exception lastException = null;
+ do {
+ try {
+ client.jobs().insert(jobRef.getProjectId(), job).execute();
+ return; // SUCCEEDED
+ } catch (GoogleJsonResponseException e) {
+ if (errorExtractor.itemAlreadyExists(e)) {
+ return; // SUCCEEDED
+ }
+ // ignore and retry
+ LOG.warn("Ignore the error and retry inserting the job.", e);
+ lastException = e;
+ } catch (IOException e) {
+ // ignore and retry
+ LOG.warn("Ignore the error and retry inserting the job.", e);
+ lastException = e;
+ }
+ } while (nextBackOff(sleeper, backoff));
+ throw new IOException(
+ String.format(
+ "Unable to insert job: %s, aborting after %d .",
+ jobRef.getJobId(), MAX_RPC_ATTEMPTS),
+ lastException);
+ }
+
+ @Override
+ public Job pollJob(JobReference jobRef, int maxAttempts)
+ throws InterruptedException {
+ BackOff backoff = new AttemptBoundedExponentialBackOff(
+ maxAttempts, INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS);
+ return pollJob(jobRef, Sleeper.DEFAULT, backoff);
+ }
+
+ @VisibleForTesting
+ Job pollJob(
+ JobReference jobRef,
+ Sleeper sleeper,
+ BackOff backoff) throws InterruptedException {
+ do {
+ try {
+ Job job = client.jobs().get(jobRef.getProjectId(), jobRef.getJobId()).execute();
+ JobStatus status = job.getStatus();
+ if (status != null && status.getState() != null && status.getState().equals("DONE")) {
+ return job;
+ }
+ // The job is not DONE, wait longer and retry.
+ } catch (IOException e) {
+ // ignore and retry
+ LOG.warn("Ignore the error and retry polling job status.", e);
+ }
+ } while (nextBackOff(sleeper, backoff));
+ LOG.warn("Unable to poll job status: {}, aborting after reached max .", jobRef.getJobId());
+ return null;
+ }
+
+ @Override
+ public JobStatistics dryRunQuery(String projectId, String query)
+ throws InterruptedException, IOException {
+ Job job = new Job()
+ .setConfiguration(new JobConfiguration()
+ .setQuery(new JobConfigurationQuery()
+ .setQuery(query))
+ .setDryRun(true));
+ BackOff backoff =
+ new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+ return executeWithRetries(
+ client.jobs().insert(projectId, job),
+ String.format(
+ "Unable to dry run query: %s, aborting after %d retries.",
+ query, MAX_RPC_ATTEMPTS),
+ Sleeper.DEFAULT,
+ backoff).getStatistics();
+ }
+ }
+
+ @VisibleForTesting
+ static class DatasetServiceImpl implements DatasetService {
+ private final ApiErrorExtractor errorExtractor;
+ private final Bigquery client;
+
+ private DatasetServiceImpl(BigQueryOptions bqOptions) {
+ this.errorExtractor = new ApiErrorExtractor();
+ this.client = Transport.newBigQueryClient(bqOptions).build();
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+ *
+ * @throws IOException if it exceeds max RPC .
+ */
+ @Override
+ public Table getTable(String projectId, String datasetId, String tableId)
+ throws IOException, InterruptedException {
+ BackOff backoff =
+ new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+ return executeWithRetries(
+ client.tables().get(projectId, datasetId, tableId),
+ String.format(
+ "Unable to get table: %s, aborting after %d retries.",
+ tableId, MAX_RPC_ATTEMPTS),
+ Sleeper.DEFAULT,
+ backoff);
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+ *
+ * @throws IOException if it exceeds max RPC .
+ */
+ @Override
+ public void deleteTable(String projectId, String datasetId, String tableId)
+ throws IOException, InterruptedException {
+ BackOff backoff =
+ new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+ executeWithRetries(
+ client.tables().delete(projectId, datasetId, tableId),
+ String.format(
+ "Unable to delete table: %s, aborting after %d retries.",
+ tableId, MAX_RPC_ATTEMPTS),
+ Sleeper.DEFAULT,
+ backoff);
+ }
+
+ @Override
+ public boolean isTableEmpty(String projectId, String datasetId, String tableId)
+ throws IOException, InterruptedException {
+ BackOff backoff =
+ new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+ TableDataList dataList = executeWithRetries(
+ client.tabledata().list(projectId, datasetId, tableId),
+ String.format(
+ "Unable to list table data: %s, aborting after %d retries.",
+ tableId, MAX_RPC_ATTEMPTS),
+ Sleeper.DEFAULT,
+ backoff);
+ return dataList.getRows() == null || dataList.getRows().isEmpty();
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+ *
+ * @throws IOException if it exceeds max RPC .
+ */
+ @Override
+ public Dataset getDataset(String projectId, String datasetId)
+ throws IOException, InterruptedException {
+ BackOff backoff =
+ new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+ return executeWithRetries(
+ client.datasets().get(projectId, datasetId),
+ String.format(
+ "Unable to get dataset: %s, aborting after %d retries.",
+ datasetId, MAX_RPC_ATTEMPTS),
+ Sleeper.DEFAULT,
+ backoff);
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+ *
+ * @throws IOException if it exceeds max RPC .
+ */
+ @Override
+ public void createDataset(
+ String projectId, String datasetId, String location, String description)
+ throws IOException, InterruptedException {
+ BackOff backoff =
+ new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+ createDataset(projectId, datasetId, location, description, Sleeper.DEFAULT, backoff);
+ }
+
+ @VisibleForTesting
+ void createDataset(
+ String projectId,
+ String datasetId,
+ String location,
+ String description,
+ Sleeper sleeper,
+ BackOff backoff) throws IOException, InterruptedException {
+ DatasetReference datasetRef = new DatasetReference()
+ .setProjectId(projectId)
+ .setDatasetId(datasetId);
+
+ Dataset dataset = new Dataset()
+ .setDatasetReference(datasetRef)
+ .setLocation(location)
+ .setFriendlyName(location)
+ .setDescription(description);
+
+ Exception lastException;
+ do {
+ try {
+ client.datasets().insert(projectId, dataset).execute();
+ return; // SUCCEEDED
+ } catch (GoogleJsonResponseException e) {
+ if (errorExtractor.itemAlreadyExists(e)) {
+ return; // SUCCEEDED
+ }
+ // ignore and retry
+ LOG.warn("Ignore the error and retry creating the dataset.", e);
+ lastException = e;
+ } catch (IOException e) {
+ LOG.warn("Ignore the error and retry creating the dataset.", e);
+ lastException = e;
+ }
+ } while (nextBackOff(sleeper, backoff));
+ throw new IOException(
+ String.format(
+ "Unable to create dataset: %s, aborting after %d .",
+ datasetId, MAX_RPC_ATTEMPTS),
+ lastException);
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+ *
+ * @throws IOException if it exceeds max RPC .
+ */
+ @Override
+ public void deleteDataset(String projectId, String datasetId)
+ throws IOException, InterruptedException {
+ BackOff backoff =
+ new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+ executeWithRetries(
+ client.datasets().delete(projectId, datasetId),
+ String.format(
+ "Unable to delete table: %s, aborting after %d retries.",
+ datasetId, MAX_RPC_ATTEMPTS),
+ Sleeper.DEFAULT,
+ backoff);
+ }
+ }
+
+ private static class BigQueryJsonReaderImpl implements BigQueryJsonReader {
+ BigQueryTableRowIterator iterator;
+
+ private BigQueryJsonReaderImpl(BigQueryTableRowIterator iterator) {
+ this.iterator = iterator;
+ }
+
+ private static BigQueryJsonReader fromQuery(
+ BigQueryOptions bqOptions,
+ String query,
+ String projectId,
+ @Nullable Boolean flattenResults) {
+ return new BigQueryJsonReaderImpl(
+ BigQueryTableRowIterator.fromQuery(
+ query, projectId, Transport.newBigQueryClient(bqOptions).build(), flattenResults));
+ }
+
+ private static BigQueryJsonReader fromTable(
+ BigQueryOptions bqOptions,
+ TableReference tableRef) {
+ return new BigQueryJsonReaderImpl(BigQueryTableRowIterator.fromTable(
+ tableRef, Transport.newBigQueryClient(bqOptions).build()));
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ try {
+ iterator.open();
+ return iterator.advance();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException("Interrupted during start() operation", e);
+ }
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ try {
+ return iterator.advance();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException("Interrupted during advance() operation", e);
+ }
+ }
+
+ @Override
+ public TableRow getCurrent() throws NoSuchElementException {
+ return iterator.getCurrent();
+ }
+
+ @Override
+ public void close() throws IOException {
+ iterator.close();
+ }
+ }
+
+ @VisibleForTesting
+ static <T> T executeWithRetries(
+ AbstractGoogleClientRequest<T> request,
+ String errorMessage,
+ Sleeper sleeper,
+ BackOff backoff)
+ throws IOException, InterruptedException {
+ Exception lastException = null;
+ do {
+ try {
+ return request.execute();
+ } catch (IOException e) {
+ LOG.warn("Ignore the error and retry the request.", e);
+ lastException = e;
+ }
+ } while (nextBackOff(sleeper, backoff));
+ throw new IOException(
+ errorMessage,
+ lastException);
+ }
+
+ /**
+ * Identical to {@link BackOffUtils#next} but without checked IOException.
+ * @throws InterruptedException
+ */
+ private static boolean nextBackOff(Sleeper sleeper, BackOff backoff) throws InterruptedException {
+ try {
+ return BackOffUtils.next(sleeper, backoff);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java
new file mode 100644
index 0000000..00a4fa3
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java
@@ -0,0 +1,413 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff;
+
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.ExponentialBackOff;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableDataInsertAllRequest;
+import com.google.api.services.bigquery.model.TableDataInsertAllResponse;
+import com.google.api.services.bigquery.model.TableDataList;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.hadoop.util.ApiErrorExtractor;
+import com.google.common.annotations.VisibleForTesting;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.Nullable;
+
+/**
+ * Inserts rows into BigQuery.
+ */
+class BigQueryTableInserter {
+ private static final Logger LOG = LoggerFactory.getLogger(BigQueryTableInserter.class);
+
+ // Approximate amount of table data to upload per InsertAll request.
+ private static final long UPLOAD_BATCH_SIZE_BYTES = 64 * 1024;
+
+ // The maximum number of rows to upload per InsertAll request.
+ private static final long MAX_ROWS_PER_BATCH = 500;
+
+ // The maximum number of times to retry inserting rows into BigQuery.
+ private static final int MAX_INSERT_ATTEMPTS = 5;
+
+ // The initial backoff after a failure inserting rows into BigQuery.
+ private static final long INITIAL_INSERT_BACKOFF_INTERVAL_MS = 200L;
+
+ // Backoff time bounds for rate limit exceeded errors.
+ private static final long INITIAL_RATE_LIMIT_EXCEEDED_BACKOFF_MS = TimeUnit.SECONDS.toMillis(1);
+ private static final long MAX_RATE_LIMIT_EXCEEDED_BACKOFF_MS = TimeUnit.MINUTES.toMillis(2);
+
+ private final Bigquery client;
+ private final long maxRowsPerBatch;
+
+ private ExecutorService executor;
+
+ /**
+ * Constructs a new row inserter.
+ *
+ * @param client a BigQuery client
+ * @param options a PipelineOptions object
+ */
+ BigQueryTableInserter(Bigquery client, PipelineOptions options) {
+ this.client = client;
+ this.maxRowsPerBatch = MAX_ROWS_PER_BATCH;
+ this.executor = options.as(GcsOptions.class).getExecutorService();
+ }
+
+ /**
+ * Constructs a new row inserter.
+ *
+ * @param client a BigQuery client
+ * @param options a PipelineOptions object
+ * @param maxRowsPerBatch maximum number of rows to insert per call to BigQuery
+ */
+ BigQueryTableInserter(Bigquery client, PipelineOptions options,
+ int maxRowsPerBatch) {
+ this.client = client;
+ this.maxRowsPerBatch = maxRowsPerBatch;
+ this.executor = options.as(GcsOptions.class).getExecutorService();
+ }
+
+ /**
+ * Insert all rows from the given list.
+ */
+ void insertAll(TableReference ref, List<TableRow> rowList) throws IOException {
+ insertAll(ref, rowList, null, null);
+ }
+
+ /**
+ * Insert all rows from the given list using specified insertIds if not null. Track count of
+ * bytes written with the Aggregator.
+ */
+ void insertAll(TableReference ref, List<TableRow> rowList,
+ @Nullable List<String> insertIdList, @Nullable Aggregator<Long, Long> byteCountAggregator)
+ throws IOException {
+ checkNotNull(ref, "ref");
+ if (insertIdList != null && rowList.size() != insertIdList.size()) {
+ throw new AssertionError("If insertIdList is not null it needs to have at least "
+ + "as many elements as rowList");
+ }
+
+ AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff(
+ MAX_INSERT_ATTEMPTS,
+ INITIAL_INSERT_BACKOFF_INTERVAL_MS);
+
+ List<TableDataInsertAllResponse.InsertErrors> allErrors = new ArrayList<>();
+ // These lists contain the rows to publish. Initially the contain the entire list. If there are
+ // failures, they will contain only the failed rows to be retried.
+ List<TableRow> rowsToPublish = rowList;
+ List<String> idsToPublish = insertIdList;
+ while (true) {
+ List<TableRow> retryRows = new ArrayList<>();
+ List<String> retryIds = (idsToPublish != null) ? new ArrayList<String>() : null;
+
+ int strideIndex = 0;
+ // Upload in batches.
+ List<TableDataInsertAllRequest.Rows> rows = new LinkedList<>();
+ int dataSize = 0;
+
+ List<Future<List<TableDataInsertAllResponse.InsertErrors>>> futures = new ArrayList<>();
+ List<Integer> strideIndices = new ArrayList<>();
+
+ for (int i = 0; i < rowsToPublish.size(); ++i) {
+ TableRow row = rowsToPublish.get(i);
+ TableDataInsertAllRequest.Rows out = new TableDataInsertAllRequest.Rows();
+ if (idsToPublish != null) {
+ out.setInsertId(idsToPublish.get(i));
+ }
+ out.setJson(row.getUnknownKeys());
+ rows.add(out);
+
+ dataSize += row.toString().length();
+ if (dataSize >= UPLOAD_BATCH_SIZE_BYTES || rows.size() >= maxRowsPerBatch
+ || i == rowsToPublish.size() - 1) {
+ TableDataInsertAllRequest content = new TableDataInsertAllRequest();
+ content.setRows(rows);
+
+ final Bigquery.Tabledata.InsertAll insert = client.tabledata()
+ .insertAll(ref.getProjectId(), ref.getDatasetId(), ref.getTableId(),
+ content);
+
+ futures.add(
+ executor.submit(new Callable<List<TableDataInsertAllResponse.InsertErrors>>() {
+ @Override
+ public List<TableDataInsertAllResponse.InsertErrors> call() throws IOException {
+ BackOff backoff = new IntervalBoundedExponentialBackOff(
+ MAX_RATE_LIMIT_EXCEEDED_BACKOFF_MS, INITIAL_RATE_LIMIT_EXCEEDED_BACKOFF_MS);
+ while (true) {
+ try {
+ return insert.execute().getInsertErrors();
+ } catch (IOException e) {
+ if (new ApiErrorExtractor().rateLimited(e)) {
+ LOG.info("BigQuery insertAll exceeded rate limit, retrying");
+ try {
+ Thread.sleep(backoff.nextBackOffMillis());
+ } catch (InterruptedException interrupted) {
+ throw new IOException(
+ "Interrupted while waiting before retrying insertAll");
+ }
+ } else {
+ throw e;
+ }
+ }
+ }
+ }
+ }));
+ strideIndices.add(strideIndex);
+
+ if (byteCountAggregator != null) {
+ byteCountAggregator.addValue((long) dataSize);
+ }
+ dataSize = 0;
+ strideIndex = i + 1;
+ rows = new LinkedList<>();
+ }
+ }
+
+ try {
+ for (int i = 0; i < futures.size(); i++) {
+ List<TableDataInsertAllResponse.InsertErrors> errors = futures.get(i).get();
+ if (errors != null) {
+ for (TableDataInsertAllResponse.InsertErrors error : errors) {
+ allErrors.add(error);
+ if (error.getIndex() == null) {
+ throw new IOException("Insert failed: " + allErrors);
+ }
+
+ int errorIndex = error.getIndex().intValue() + strideIndices.get(i);
+ retryRows.add(rowsToPublish.get(errorIndex));
+ if (retryIds != null) {
+ retryIds.add(idsToPublish.get(errorIndex));
+ }
+ }
+ }
+ }
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new IOException("Interrupted while inserting " + rowsToPublish);
+ } catch (ExecutionException e) {
+ throw new RuntimeException(e.getCause());
+ }
+
+ if (!allErrors.isEmpty() && !backoff.atMaxAttempts()) {
+ try {
+ Thread.sleep(backoff.nextBackOffMillis());
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new IOException("Interrupted while waiting before retrying insert of " + retryRows);
+ }
+ LOG.info("Retrying failed inserts to BigQuery");
+ rowsToPublish = retryRows;
+ idsToPublish = retryIds;
+ allErrors.clear();
+ } else {
+ break;
+ }
+ }
+ if (!allErrors.isEmpty()) {
+ throw new IOException("Insert failed: " + allErrors);
+ }
+ }
+
+ /**
+ * Retrieves or creates the table.
+ *
+ * <p>The table is checked to conform to insertion requirements as specified
+ * by WriteDisposition and CreateDisposition.
+ *
+ * <p>If table truncation is requested (WriteDisposition.WRITE_TRUNCATE), then
+ * this will re-create the table if necessary to ensure it is empty.
+ *
+ * <p>If an empty table is required (WriteDisposition.WRITE_EMPTY), then this
+ * will fail if the table exists and is not empty.
+ *
+ * <p>When constructing a table, a {@code TableSchema} must be available. If a
+ * schema is provided, then it will be used. If no schema is provided, but
+ * an existing table is being cleared (WRITE_TRUNCATE option above), then
+ * the existing schema will be re-used. If no schema is available, then an
+ * {@code IOException} is thrown.
+ */
+ Table getOrCreateTable(
+ TableReference ref,
+ WriteDisposition writeDisposition,
+ CreateDisposition createDisposition,
+ @Nullable TableSchema schema) throws IOException {
+ // Check if table already exists.
+ Bigquery.Tables.Get get = client.tables()
+ .get(ref.getProjectId(), ref.getDatasetId(), ref.getTableId());
+ Table table = null;
+ try {
+ table = get.execute();
+ } catch (IOException e) {
+ ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
+ if (!errorExtractor.itemNotFound(e)
+ || createDisposition != CreateDisposition.CREATE_IF_NEEDED) {
+ // Rethrow.
+ throw e;
+ }
+ }
+
+ // If we want an empty table, and it isn't, then delete it first.
+ if (table != null) {
+ if (writeDisposition == WriteDisposition.WRITE_APPEND) {
+ return table;
+ }
+
+ boolean empty = isEmpty(ref);
+ if (empty) {
+ if (writeDisposition == WriteDisposition.WRITE_TRUNCATE) {
+ LOG.info("Empty table found, not removing {}", BigQueryIO.toTableSpec(ref));
+ }
+ return table;
+
+ } else if (writeDisposition == WriteDisposition.WRITE_EMPTY) {
+ throw new IOException("WriteDisposition is WRITE_EMPTY, "
+ + "but table is not empty");
+ }
+
+ // Reuse the existing schema if none was provided.
+ if (schema == null) {
+ schema = table.getSchema();
+ }
+
+ // Delete table and fall through to re-creating it below.
+ LOG.info("Deleting table {}", BigQueryIO.toTableSpec(ref));
+ Bigquery.Tables.Delete delete = client.tables()
+ .delete(ref.getProjectId(), ref.getDatasetId(), ref.getTableId());
+ delete.execute();
+ }
+
+ if (schema == null) {
+ throw new IllegalArgumentException(
+ "Table schema required for new table.");
+ }
+
+ // Create the table.
+ return tryCreateTable(ref, schema);
+ }
+
+ /**
+ * Checks if a table is empty.
+ */
+ private boolean isEmpty(TableReference ref) throws IOException {
+ Bigquery.Tabledata.List list = client.tabledata()
+ .list(ref.getProjectId(), ref.getDatasetId(), ref.getTableId());
+ list.setMaxResults(1L);
+ TableDataList dataList = list.execute();
+
+ return dataList.getRows() == null || dataList.getRows().isEmpty();
+ }
+
+ /**
+ * Retry table creation up to 5 minutes (with exponential backoff) when this user is near the
+ * quota for table creation. This relatively innocuous behavior can happen when BigQueryIO is
+ * configured with a table spec function to use different tables for each window.
+ */
+ private static final int RETRY_CREATE_TABLE_DURATION_MILLIS = (int) TimeUnit.MINUTES.toMillis(5);
+
+ /**
+ * Tries to create the BigQuery table.
+ * If a table with the same name already exists in the dataset, the table
+ * creation fails, and the function returns null. In such a case,
+ * the existing table doesn't necessarily have the same schema as specified
+ * by the parameter.
+ *
+ * @param schema Schema of the new BigQuery table.
+ * @return The newly created BigQuery table information, or null if the table
+ * with the same name already exists.
+ * @throws IOException if other error than already existing table occurs.
+ */
+ @Nullable
+ private Table tryCreateTable(TableReference ref, TableSchema schema) throws IOException {
+ LOG.info("Trying to create BigQuery table: {}", BigQueryIO.toTableSpec(ref));
+ BackOff backoff =
+ new ExponentialBackOff.Builder()
+ .setMaxElapsedTimeMillis(RETRY_CREATE_TABLE_DURATION_MILLIS)
+ .build();
+
+ Table table = new Table().setTableReference(ref).setSchema(schema);
+ return tryCreateTable(table, ref.getProjectId(), ref.getDatasetId(), backoff, Sleeper.DEFAULT);
+ }
+
+ @VisibleForTesting
+ @Nullable
+ Table tryCreateTable(
+ Table table, String projectId, String datasetId, BackOff backoff, Sleeper sleeper)
+ throws IOException {
+ boolean retry = false;
+ while (true) {
+ try {
+ return client.tables().insert(projectId, datasetId, table).execute();
+ } catch (IOException e) {
+ ApiErrorExtractor extractor = new ApiErrorExtractor();
+ if (extractor.itemAlreadyExists(e)) {
+ // The table already exists, nothing to return.
+ return null;
+ } else if (extractor.rateLimited(e)) {
+ // The request failed because we hit a temporary quota. Back off and try again.
+ try {
+ if (BackOffUtils.next(sleeper, backoff)) {
+ if (!retry) {
+ LOG.info(
+ "Quota limit reached when creating table {}:{}.{}, retrying up to {} minutes",
+ projectId,
+ datasetId,
+ table.getTableReference().getTableId(),
+ TimeUnit.MILLISECONDS.toSeconds(RETRY_CREATE_TABLE_DURATION_MILLIS) / 60.0);
+ retry = true;
+ }
+ continue;
+ }
+ } catch (InterruptedException e1) {
+ // Restore interrupted state and throw the last failure.
+ Thread.currentThread().interrupt();
+ throw e;
+ }
+ }
+ throw e;
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java
new file mode 100644
index 0000000..3afdffa
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java
@@ -0,0 +1,474 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+
+import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.ClassInfo;
+import com.google.api.client.util.Data;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.Bigquery.Jobs.Insert;
+import com.google.api.services.bigquery.model.Dataset;
+import com.google.api.services.bigquery.model.DatasetReference;
+import com.google.api.services.bigquery.model.ErrorProto;
+import com.google.api.services.bigquery.model.Job;
+import com.google.api.services.bigquery.model.JobConfiguration;
+import com.google.api.services.bigquery.model.JobConfigurationQuery;
+import com.google.api.services.bigquery.model.JobReference;
+import com.google.api.services.bigquery.model.JobStatus;
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableCell;
+import com.google.api.services.bigquery.model.TableDataList;
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.Uninterruptibles;
+
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.Nullable;
+
+/**
+ * Iterates over all rows in a table.
+ */
+class BigQueryTableRowIterator implements AutoCloseable {
+ private static final Logger LOG = LoggerFactory.getLogger(BigQueryTableRowIterator.class);
+
+ @Nullable private TableReference ref;
+ @Nullable private final String projectId;
+ @Nullable private TableSchema schema;
+ private final Bigquery client;
+ private String pageToken;
+ private Iterator<TableRow> iteratorOverCurrentBatch;
+ private TableRow current;
+ // Set true when the final page is seen from the service.
+ private boolean lastPage = false;
+
+ // The maximum number of times a BigQuery request will be retried
+ private static final int MAX_RETRIES = 3;
+ // Initial wait time for the backoff implementation
+ private static final Duration INITIAL_BACKOFF_TIME = Duration.standardSeconds(1);
+
+ // After sending a query to BQ service we will be polling the BQ service to check the status with
+ // following interval to check the status of query execution job
+ private static final Duration QUERY_COMPLETION_POLL_TIME = Duration.standardSeconds(1);
+
+ private final String query;
+ // Whether to flatten query results.
+ private final boolean flattenResults;
+ // Temporary dataset used to store query results.
+ private String temporaryDatasetId = null;
+ // Temporary table used to store query results.
+ private String temporaryTableId = null;
+
+ private BigQueryTableRowIterator(
+ @Nullable TableReference ref, @Nullable String query, @Nullable String projectId,
+ Bigquery client, boolean flattenResults) {
+ this.ref = ref;
+ this.query = query;
+ this.projectId = projectId;
+ this.client = checkNotNull(client, "client");
+ this.flattenResults = flattenResults;
+ }
+
+ /**
+ * Constructs a {@code BigQueryTableRowIterator} that reads from the specified table.
+ */
+ public static BigQueryTableRowIterator fromTable(TableReference ref, Bigquery client) {
+ checkNotNull(ref, "ref");
+ checkNotNull(client, "client");
+ return new BigQueryTableRowIterator(ref, null, ref.getProjectId(), client, true);
+ }
+
+ /**
+ * Constructs a {@code BigQueryTableRowIterator} that reads from the results of executing the
+ * specified query in the specified project.
+ */
+ public static BigQueryTableRowIterator fromQuery(
+ String query, String projectId, Bigquery client, @Nullable Boolean flattenResults) {
+ checkNotNull(query, "query");
+ checkNotNull(projectId, "projectId");
+ checkNotNull(client, "client");
+ return new BigQueryTableRowIterator(null, query, projectId, client,
+ MoreObjects.firstNonNull(flattenResults, Boolean.TRUE));
+ }
+
+ /**
+ * Opens the table for read.
+ * @throws IOException on failure
+ */
+ public void open() throws IOException, InterruptedException {
+ if (query != null) {
+ ref = executeQueryAndWaitForCompletion();
+ }
+ // Get table schema.
+ Bigquery.Tables.Get get =
+ client.tables().get(ref.getProjectId(), ref.getDatasetId(), ref.getTableId());
+
+ Table table =
+ executeWithBackOff(
+ get,
+ "Error opening BigQuery table %s of dataset %s : {}",
+ ref.getTableId(),
+ ref.getDatasetId());
+ schema = table.getSchema();
+ }
+
+ public boolean advance() throws IOException, InterruptedException {
+ while (true) {
+ if (iteratorOverCurrentBatch != null && iteratorOverCurrentBatch.hasNext()) {
+ // Embed schema information into the raw row, so that values have an
+ // associated key. This matches how rows are read when using the
+ // DataflowRunner.
+ current = getTypedTableRow(schema.getFields(), iteratorOverCurrentBatch.next());
+ return true;
+ }
+ if (lastPage) {
+ return false;
+ }
+
+ Bigquery.Tabledata.List list =
+ client.tabledata().list(ref.getProjectId(), ref.getDatasetId(), ref.getTableId());
+ if (pageToken != null) {
+ list.setPageToken(pageToken);
+ }
+
+ TableDataList result =
+ executeWithBackOff(
+ list,
+ "Error reading from BigQuery table %s of dataset %s : {}",
+ ref.getTableId(),
+ ref.getDatasetId());
+
+ pageToken = result.getPageToken();
+ iteratorOverCurrentBatch =
+ result.getRows() != null
+ ? result.getRows().iterator()
+ : Collections.<TableRow>emptyIterator();
+
+ // The server may return a page token indefinitely on a zero-length table.
+ if (pageToken == null || result.getTotalRows() != null && result.getTotalRows() == 0) {
+ lastPage = true;
+ }
+ }
+ }
+
+ public TableRow getCurrent() {
+ if (current == null) {
+ throw new NoSuchElementException();
+ }
+ return current;
+ }
+
+ /**
+ * Adjusts a field returned from the BigQuery API to match what we will receive when running
+ * BigQuery's export-to-GCS and parallel read, which is the efficient parallel implementation
+ * used for batch jobs executed on the Cloud Dataflow service.
+ *
+ * <p>The following is the relationship between BigQuery schema and Java types:
+ *
+ * <ul>
+ * <li>Nulls are {@code null}.
+ * <li>Repeated fields are {@code List} of objects.
+ * <li>Record columns are {@link TableRow} objects.
+ * <li>{@code BOOLEAN} columns are JSON booleans, hence Java {@code Boolean} objects.
+ * <li>{@code FLOAT} columns are JSON floats, hence Java {@code Double} objects.
+ * <li>{@code TIMESTAMP} columns are {@code String} objects that are of the format
+ * {@code yyyy-MM-dd HH:mm:ss[.SSSSSS] UTC}, where the {@code .SSSSSS} has no trailing
+ * zeros and can be 1 to 6 digits long.
+ * <li>Every other atomic type is a {@code String}.
+ * </ul>
+ *
+ * <p>Note that integers are encoded as strings to match BigQuery's exported JSON format.
+ *
+ * <p>Finally, values are stored in the {@link TableRow} as {"field name": value} pairs
+ * and are not accessible through the {@link TableRow#getF} function.
+ */
+ @Nullable private Object getTypedCellValue(TableFieldSchema fieldSchema, Object v) {
+ if (Data.isNull(v)) {
+ return null;
+ }
+
+ if (Objects.equals(fieldSchema.getMode(), "REPEATED")) {
+ TableFieldSchema elementSchema = fieldSchema.clone().setMode("REQUIRED");
+ @SuppressWarnings("unchecked")
+ List<Map<String, Object>> rawCells = (List<Map<String, Object>>) v;
+ ImmutableList.Builder<Object> values = ImmutableList.builder();
+ for (Map<String, Object> element : rawCells) {
+ values.add(getTypedCellValue(elementSchema, element.get("v")));
+ }
+ return values.build();
+ }
+
+ if (fieldSchema.getType().equals("RECORD")) {
+ @SuppressWarnings("unchecked")
+ Map<String, Object> typedV = (Map<String, Object>) v;
+ return getTypedTableRow(fieldSchema.getFields(), typedV);
+ }
+
+ if (fieldSchema.getType().equals("FLOAT")) {
+ return Double.parseDouble((String) v);
+ }
+
+ if (fieldSchema.getType().equals("BOOLEAN")) {
+ return Boolean.parseBoolean((String) v);
+ }
+
+ if (fieldSchema.getType().equals("TIMESTAMP")) {
+ return BigQueryAvroUtils.formatTimestamp((String) v);
+ }
+
+ return v;
+ }
+
+ /**
+ * A list of the field names that cannot be used in BigQuery tables processed by Dataflow,
+ * because they are reserved keywords in {@link TableRow}.
+ */
+ // TODO: This limitation is unfortunate. We need to give users a way to use BigQueryIO that does
+ // not indirect through our broken use of {@link TableRow}.
+ // See discussion: https://github.com/GoogleCloudPlatform/DataflowJavaSDK/pull/41
+ private static final Collection<String> RESERVED_FIELD_NAMES =
+ ClassInfo.of(TableRow.class).getNames();
+
+ /**
+ * Converts a row returned from the BigQuery JSON API as a {@code Map<String, Object>} into a
+ * Java {@link TableRow} with nested {@link TableCell TableCells}. The {@code Object} values in
+ * the cells are converted to Java types according to the provided field schemas.
+ *
+ * <p>See {@link #getTypedCellValue(TableFieldSchema, Object)} for details on how BigQuery
+ * types are mapped to Java types.
+ */
+ private TableRow getTypedTableRow(List<TableFieldSchema> fields, Map<String, Object> rawRow) {
+ // If rawRow is a TableRow, use it. If not, create a new one.
+ TableRow row;
+ List<? extends Map<String, Object>> cells;
+ if (rawRow instanceof TableRow) {
+ // Since rawRow is a TableRow it already has TableCell objects in setF. We do not need to do
+ // any type conversion, but extract the cells for cell-wise processing below.
+ row = (TableRow) rawRow;
+ cells = row.getF();
+ // Clear the cells from the row, so that row.getF() will return null. This matches the
+ // behavior of rows produced by the BigQuery export API used on the service.
+ row.setF(null);
+ } else {
+ row = new TableRow();
+
+ // Since rawRow is a Map<String, Object> we use Map.get("f") instead of TableRow.getF() to
+ // get its cells. Similarly, when rawCell is a Map<String, Object> instead of a TableCell,
+ // we will use Map.get("v") instead of TableCell.getV() get its value.
+ @SuppressWarnings("unchecked")
+ List<? extends Map<String, Object>> rawCells =
+ (List<? extends Map<String, Object>>) rawRow.get("f");
+ cells = rawCells;
+ }
+
+ checkState(cells.size() == fields.size(),
+ "Expected that the row has the same number of cells %s as fields in the schema %s",
+ cells.size(), fields.size());
+
+ // Loop through all the fields in the row, normalizing their types with the TableFieldSchema
+ // and storing the normalized values by field name in the Map<String, Object> that
+ // underlies the TableRow.
+ Iterator<? extends Map<String, Object>> cellIt = cells.iterator();
+ Iterator<TableFieldSchema> fieldIt = fields.iterator();
+ while (cellIt.hasNext()) {
+ Map<String, Object> cell = cellIt.next();
+ TableFieldSchema fieldSchema = fieldIt.next();
+
+ // Convert the object in this cell to the Java type corresponding to its type in the schema.
+ Object convertedValue = getTypedCellValue(fieldSchema, cell.get("v"));
+
+ String fieldName = fieldSchema.getName();
+ checkArgument(!RESERVED_FIELD_NAMES.contains(fieldName),
+ "BigQueryIO does not support records with columns named %s", fieldName);
+
+ if (convertedValue == null) {
+ // BigQuery does not include null values when the export operation (to JSON) is used.
+ // To match that behavior, BigQueryTableRowiterator, and the DirectRunner,
+ // intentionally omits columns with null values.
+ continue;
+ }
+
+ row.set(fieldName, convertedValue);
+ }
+ return row;
+ }
+
+ // Create a new BigQuery dataset
+ private void createDataset(String datasetId) throws IOException, InterruptedException {
+ Dataset dataset = new Dataset();
+ DatasetReference reference = new DatasetReference();
+ reference.setProjectId(projectId);
+ reference.setDatasetId(datasetId);
+ dataset.setDatasetReference(reference);
+
+ String createDatasetError =
+ "Error when trying to create the temporary dataset " + datasetId + " in project "
+ + projectId;
+ executeWithBackOff(
+ client.datasets().insert(projectId, dataset), createDatasetError + " :{}");
+ }
+
+ // Delete the given table that is available in the given dataset.
+ private void deleteTable(String datasetId, String tableId)
+ throws IOException, InterruptedException {
+ executeWithBackOff(
+ client.tables().delete(projectId, datasetId, tableId),
+ "Error when trying to delete the temporary table " + datasetId + " in dataset " + datasetId
+ + " of project " + projectId + ". Manual deletion may be required. Error message : {}");
+ }
+
+ // Delete the given dataset. This will fail if the given dataset has any tables.
+ private void deleteDataset(String datasetId) throws IOException, InterruptedException {
+ executeWithBackOff(
+ client.datasets().delete(projectId, datasetId),
+ "Error when trying to delete the temporary dataset " + datasetId + " in project "
+ + projectId + ". Manual deletion may be required. Error message : {}");
+ }
+
+ /**
+ * Executes the specified query and returns a reference to the temporary BigQuery table created
+ * to hold the results.
+ *
+ * @throws IOException if the query fails.
+ */
+ private TableReference executeQueryAndWaitForCompletion()
+ throws IOException, InterruptedException {
+ // Create a temporary dataset to store results.
+ // Starting dataset name with an "_" so that it is hidden.
+ Random rnd = new Random(System.currentTimeMillis());
+ temporaryDatasetId = "_dataflow_temporary_dataset_" + rnd.nextInt(1000000);
+ temporaryTableId = "dataflow_temporary_table_" + rnd.nextInt(1000000);
+
+ createDataset(temporaryDatasetId);
+ Job job = new Job();
+ JobConfiguration config = new JobConfiguration();
+ JobConfigurationQuery queryConfig = new JobConfigurationQuery();
+ config.setQuery(queryConfig);
+ job.setConfiguration(config);
+ queryConfig.setQuery(query);
+ queryConfig.setAllowLargeResults(true);
+ queryConfig.setFlattenResults(flattenResults);
+
+ TableReference destinationTable = new TableReference();
+ destinationTable.setProjectId(projectId);
+ destinationTable.setDatasetId(temporaryDatasetId);
+ destinationTable.setTableId(temporaryTableId);
+ queryConfig.setDestinationTable(destinationTable);
+
+ Insert insert = client.jobs().insert(projectId, job);
+ Job queryJob = executeWithBackOff(
+ insert, "Error when trying to execute the job for query " + query + " :{}");
+ JobReference jobId = queryJob.getJobReference();
+
+ while (true) {
+ Job pollJob = executeWithBackOff(
+ client.jobs().get(projectId, jobId.getJobId()),
+ "Error when trying to get status of the job for query " + query + " :{}");
+ JobStatus status = pollJob.getStatus();
+ if (status.getState().equals("DONE")) {
+ // Job is DONE, but did not necessarily succeed.
+ ErrorProto error = status.getErrorResult();
+ if (error == null) {
+ return pollJob.getConfiguration().getQuery().getDestinationTable();
+ } else {
+ // There will be no temporary table to delete, so null out the reference.
+ temporaryTableId = null;
+ throw new IOException("Executing query " + query + " failed: " + error.getMessage());
+ }
+ }
+ Uninterruptibles.sleepUninterruptibly(
+ QUERY_COMPLETION_POLL_TIME.getMillis(), TimeUnit.MILLISECONDS);
+ }
+ }
+
+ // Execute a BQ request with exponential backoff and return the result.
+ // client - BQ request to be executed
+ // error - Formatted message to log if when a request fails. Takes exception message as a
+ // formatter parameter.
+ public static <T> T executeWithBackOff(AbstractGoogleClientRequest<T> client, String error,
+ Object... errorArgs) throws IOException, InterruptedException {
+ Sleeper sleeper = Sleeper.DEFAULT;
+ BackOff backOff =
+ new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_TIME.getMillis());
+
+ T result = null;
+ while (true) {
+ try {
+ result = client.execute();
+ break;
+ } catch (IOException e) {
+ LOG.error(String.format(error, errorArgs), e.getMessage());
+ if (!BackOffUtils.next(sleeper, backOff)) {
+ LOG.error(
+ String.format(error, errorArgs), "Failing after retrying " + MAX_RETRIES + " times.");
+ throw e;
+ }
+ }
+ }
+
+ return result;
+ }
+
+ @Override
+ public void close() {
+ // Prevent any further requests.
+ lastPage = true;
+
+ try {
+ // Deleting temporary table and dataset that gets generated when executing a query.
+ if (temporaryDatasetId != null) {
+ if (temporaryTableId != null) {
+ deleteTable(temporaryDatasetId, temporaryTableId);
+ }
+ deleteDataset(temporaryDatasetId);
+ }
+ } catch (IOException | InterruptedException e) {
+ if (e instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
+ throw new RuntimeException(e);
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/package-info.java
new file mode 100644
index 0000000..a6fc693
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/**
+ * Defines transforms for reading and writing from Google BigQuery.
+ *
+ * @see org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO
+ */
+package org.apache.beam.sdk.io.gcp.bigquery;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..316392f
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.io.gcp.bigquery;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.DefaultCoder;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.common.collect.Lists;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.reflect.Nullable;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link BigQueryAvroUtils}.
+ */
+@RunWith(JUnit4.class)
+public class BigQueryAvroUtilsTest {
+ @Test
+ public void testConvertGenericRecordToTableRow() throws Exception {
+ TableSchema tableSchema = new TableSchema();
+ List<TableFieldSchema> subFields = Lists.<TableFieldSchema>newArrayList(
+ new TableFieldSchema().setName("species").setType("STRING").setMode("NULLABLE"));
+ /*
+ * Note that the quality and quantity fields do not have their mode set, so they should default
+ * to NULLABLE. This is an important test of BigQuery semantics.
+ *
+ * All the other fields we set in this function are required on the Schema response.
+ *
+ * See https://cloud.google.com/bigquery/docs/reference/v2/tables#schema
+ */
+ List<TableFieldSchema> fields =
+ Lists.<TableFieldSchema>newArrayList(
+ new TableFieldSchema().setName("number").setType("INTEGER").setMode("REQUIRED"),
+ new TableFieldSchema().setName("species").setType("STRING").setMode("NULLABLE"),
+ 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("flighted").setType("BOOLEAN").setMode("NULLABLE"),
+ new TableFieldSchema().setName("scion").setType("RECORD").setMode("NULLABLE")
+ .setFields(subFields),
+ new TableFieldSchema().setName("associates").setType("RECORD").setMode("REPEATED")
+ .setFields(subFields));
+ tableSchema.setFields(fields);
+ Schema avroSchema = AvroCoder.of(Bird.class).getSchema();
+
+ {
+ // Test nullable fields.
+ GenericRecord record = new GenericData.Record(avroSchema);
+ record.put("number", 5L);
+ TableRow convertedRow = BigQueryAvroUtils.convertGenericRecordToTableRow(record, tableSchema);
+ TableRow row = new TableRow()
+ .set("number", "5")
+ .set("associates", new ArrayList<TableRow>());
+ assertEquals(row, convertedRow);
+ }
+ {
+ // Test type conversion for TIMESTAMP, INTEGER, BOOLEAN, and FLOAT.
+ GenericRecord record = new GenericData.Record(avroSchema);
+ record.put("number", 5L);
+ record.put("quality", 5.0);
+ record.put("birthday", 5L);
+ record.put("flighted", Boolean.TRUE);
+ TableRow convertedRow = BigQueryAvroUtils.convertGenericRecordToTableRow(record, tableSchema);
+ TableRow row = new TableRow()
+ .set("number", "5")
+ .set("birthday", "1970-01-01 00:00:00.000005 UTC")
+ .set("quality", 5.0)
+ .set("associates", new ArrayList<TableRow>())
+ .set("flighted", Boolean.TRUE);
+ assertEquals(row, convertedRow);
+ }
+ {
+ // Test repeated fields.
+ Schema subBirdSchema = AvroCoder.of(Bird.SubBird.class).getSchema();
+ GenericRecord nestedRecord = new GenericData.Record(subBirdSchema);
+ nestedRecord.put("species", "other");
+ GenericRecord record = new GenericData.Record(avroSchema);
+ record.put("number", 5L);
+ record.put("associates", Lists.<GenericRecord>newArrayList(nestedRecord));
+ TableRow convertedRow = BigQueryAvroUtils.convertGenericRecordToTableRow(record, tableSchema);
+ TableRow row = new TableRow()
+ .set("associates", Lists.<TableRow>newArrayList(
+ new TableRow().set("species", "other")))
+ .set("number", "5");
+ assertEquals(row, convertedRow);
+ }
+ }
+
+ /**
+ * Pojo class used as the record type in tests.
+ */
+ @DefaultCoder(AvroCoder.class)
+ @SuppressWarnings("unused") // Used by Avro reflection.
+ static class Bird {
+ long number;
+ @Nullable String species;
+ @Nullable Double quality;
+ @Nullable Long quantity;
+ @Nullable Long birthday; // Exercises TIMESTAMP.
+ @Nullable Boolean flighted;
+ @Nullable SubBird scion;
+ SubBird[] associates;
+
+ static class SubBird {
+ @Nullable String species;
+
+ public SubBird() {}
+ }
+
+ public Bird() {
+ associates = new SubBird[1];
+ associates[0] = new SubBird();
+ }
+ }
+}
[07/10] incubator-beam git commit: BigQueryIO: move to
google-cloud-platform module
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java
index df2308d..16fc6fa 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java
@@ -17,34 +17,15 @@
*/
package org.apache.beam.sdk.util;
-import static com.google.common.base.MoreObjects.firstNonNull;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Verify.verify;
-
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-import org.apache.avro.Schema;
-import org.apache.avro.Schema.Field;
-import org.apache.avro.Schema.Type;
import org.apache.avro.file.DataFileConstants;
-import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.BinaryDecoder;
import org.apache.avro.io.DecoderFactory;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.util.Arrays;
-import java.util.List;
-
-import javax.annotation.Nullable;
/**
* A set of utilities for working with Avro files.
@@ -154,192 +135,4 @@ public class AvroUtils {
}
return new AvroMetadata(syncMarker, codec, schemaString);
}
-
- /**
- * Formats BigQuery seconds-since-epoch into String matching JSON export. Thread-safe and
- * immutable.
- */
- private static final DateTimeFormatter DATE_AND_SECONDS_FORMATTER =
- DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").withZoneUTC();
- // Package private for BigQueryTableRowIterator to use.
- static String formatTimestamp(String timestamp) {
- // timestamp is in "seconds since epoch" format, with scientific notation.
- // e.g., "1.45206229112345E9" to mean "2016-01-06 06:38:11.123456 UTC".
- // Separate into seconds and microseconds.
- double timestampDoubleMicros = Double.parseDouble(timestamp) * 1000000;
- long timestampMicros = (long) timestampDoubleMicros;
- long seconds = timestampMicros / 1000000;
- int micros = (int) (timestampMicros % 1000000);
- String dayAndTime = DATE_AND_SECONDS_FORMATTER.print(seconds * 1000);
-
- // No sub-second component.
- if (micros == 0) {
- return String.format("%s UTC", dayAndTime);
- }
-
- // Sub-second component.
- int digits = 6;
- int subsecond = micros;
- while (subsecond % 10 == 0) {
- digits--;
- subsecond /= 10;
- }
- String formatString = String.format("%%0%dd", digits);
- String fractionalSeconds = String.format(formatString, subsecond);
- return String.format("%s.%s UTC", dayAndTime, fractionalSeconds);
- }
-
- /**
- * Utility function to convert from an Avro {@link GenericRecord} to a BigQuery {@link TableRow}.
- *
- * See <a href="https://cloud.google.com/bigquery/exporting-data-from-bigquery#config">
- * "Avro format"</a> for more information.
- */
- public static TableRow convertGenericRecordToTableRow(GenericRecord record, TableSchema schema) {
- return convertGenericRecordToTableRow(record, schema.getFields());
- }
-
- private static TableRow convertGenericRecordToTableRow(
- GenericRecord record, List<TableFieldSchema> fields) {
- TableRow row = new TableRow();
- for (TableFieldSchema subSchema : fields) {
- // Per https://cloud.google.com/bigquery/docs/reference/v2/tables#schema, the name field
- // is required, so it may not be null.
- Field field = record.getSchema().getField(subSchema.getName());
- Object convertedValue =
- getTypedCellValue(field.schema(), subSchema, record.get(field.name()));
- if (convertedValue != null) {
- // To match the JSON files exported by BigQuery, do not include null values in the output.
- row.set(field.name(), convertedValue);
- }
- }
- return row;
- }
-
- @Nullable
- private static Object getTypedCellValue(Schema schema, TableFieldSchema fieldSchema, Object v) {
- // Per https://cloud.google.com/bigquery/docs/reference/v2/tables#schema, the mode field
- // is optional (and so it may be null), but defaults to "NULLABLE".
- String mode = firstNonNull(fieldSchema.getMode(), "NULLABLE");
- switch (mode) {
- case "REQUIRED":
- return convertRequiredField(schema.getType(), fieldSchema, v);
- case "REPEATED":
- return convertRepeatedField(schema, fieldSchema, v);
- case "NULLABLE":
- return convertNullableField(schema, fieldSchema, v);
- default:
- throw new UnsupportedOperationException(
- "Parsing a field with BigQuery field schema mode " + fieldSchema.getMode());
- }
- }
-
- private static List<Object> convertRepeatedField(
- Schema schema, TableFieldSchema fieldSchema, Object v) {
- Type arrayType = schema.getType();
- verify(
- arrayType == Type.ARRAY,
- "BigQuery REPEATED field %s should be Avro ARRAY, not %s",
- fieldSchema.getName(),
- arrayType);
- // REPEATED fields are represented as Avro arrays.
- if (v == null) {
- // Handle the case of an empty repeated field.
- return ImmutableList.of();
- }
- @SuppressWarnings("unchecked")
- List<Object> elements = (List<Object>) v;
- ImmutableList.Builder<Object> values = ImmutableList.builder();
- Type elementType = schema.getElementType().getType();
- for (Object element : elements) {
- values.add(convertRequiredField(elementType, fieldSchema, element));
- }
- return values.build();
- }
-
- private static Object convertRequiredField(
- Type avroType, TableFieldSchema fieldSchema, Object v) {
- // REQUIRED fields are represented as the corresponding Avro types. For example, a BigQuery
- // INTEGER type maps to an Avro LONG type.
- checkNotNull(v, "REQUIRED field %s should not be null", fieldSchema.getName());
- ImmutableMap<String, Type> fieldMap =
- ImmutableMap.<String, Type>builder()
- .put("STRING", Type.STRING)
- .put("INTEGER", Type.LONG)
- .put("FLOAT", Type.DOUBLE)
- .put("BOOLEAN", Type.BOOLEAN)
- .put("TIMESTAMP", Type.LONG)
- .put("RECORD", Type.RECORD)
- .build();
- // Per https://cloud.google.com/bigquery/docs/reference/v2/tables#schema, the type field
- // is required, so it may not be null.
- String bqType = fieldSchema.getType();
- Type expectedAvroType = fieldMap.get(bqType);
- verify(
- avroType == expectedAvroType,
- "Expected Avro schema type %s, not %s, for BigQuery %s field %s",
- expectedAvroType,
- avroType,
- bqType,
- fieldSchema.getName());
- switch (fieldSchema.getType()) {
- case "STRING":
- // Avro will use a CharSequence to represent String objects, but it may not always use
- // java.lang.String; for example, it may prefer org.apache.avro.util.Utf8.
- verify(v instanceof CharSequence, "Expected CharSequence (String), got %s", v.getClass());
- return v.toString();
- case "INTEGER":
- verify(v instanceof Long, "Expected Long, got %s", v.getClass());
- return ((Long) v).toString();
- case "FLOAT":
- verify(v instanceof Double, "Expected Double, got %s", v.getClass());
- return v;
- case "BOOLEAN":
- verify(v instanceof Boolean, "Expected Boolean, got %s", v.getClass());
- return v;
- case "TIMESTAMP":
- // TIMESTAMP data types are represented as Avro LONG types. They are converted back to
- // Strings with variable-precision (up to six digits) to match the JSON files export
- // by BigQuery.
- verify(v instanceof Long, "Expected Long, got %s", v.getClass());
- Double doubleValue = ((Long) v) / 1000000.0;
- return formatTimestamp(doubleValue.toString());
- case "RECORD":
- verify(v instanceof GenericRecord, "Expected GenericRecord, got %s", v.getClass());
- return convertGenericRecordToTableRow((GenericRecord) v, fieldSchema.getFields());
- default:
- throw new UnsupportedOperationException(
- String.format(
- "Unexpected BigQuery field schema type %s for field named %s",
- fieldSchema.getType(),
- fieldSchema.getName()));
- }
- }
-
- @Nullable
- private static Object convertNullableField(
- Schema avroSchema, TableFieldSchema fieldSchema, Object v) {
- // NULLABLE fields are represented as an Avro Union of the corresponding type and "null".
- verify(
- avroSchema.getType() == Type.UNION,
- "Expected Avro schema type UNION, not %s, for BigQuery NULLABLE field %s",
- avroSchema.getType(),
- fieldSchema.getName());
- List<Schema> unionTypes = avroSchema.getTypes();
- verify(
- unionTypes.size() == 2,
- "BigQuery NULLABLE field %s should be an Avro UNION of NULL and another type, not %s",
- fieldSchema.getName(),
- unionTypes);
-
- if (v == null) {
- return null;
- }
-
- Type firstType = unionTypes.get(0).getType();
- if (!firstType.equals(Type.NULL)) {
- return convertRequiredField(firstType, fieldSchema, v);
- }
- return convertRequiredField(unionTypes.get(1).getType(), fieldSchema, v);
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java
deleted file mode 100644
index 514e005..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.beam.sdk.options.BigQueryOptions;
-
-import com.google.api.services.bigquery.model.Dataset;
-import com.google.api.services.bigquery.model.Job;
-import com.google.api.services.bigquery.model.JobConfigurationExtract;
-import com.google.api.services.bigquery.model.JobConfigurationLoad;
-import com.google.api.services.bigquery.model.JobConfigurationQuery;
-import com.google.api.services.bigquery.model.JobReference;
-import com.google.api.services.bigquery.model.JobStatistics;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.NoSuchElementException;
-
-import javax.annotation.Nullable;
-
-/**
- * An interface for real, mock, or fake implementations of Cloud BigQuery services.
- */
-public interface BigQueryServices extends Serializable {
-
- /**
- * Returns a real, mock, or fake {@link JobService}.
- */
- public JobService getJobService(BigQueryOptions bqOptions);
-
- /**
- * Returns a real, mock, or fake {@link DatasetService}.
- */
- public DatasetService getDatasetService(BigQueryOptions bqOptions);
-
- /**
- * Returns a real, mock, or fake {@link BigQueryJsonReader} to read tables.
- */
- public BigQueryJsonReader getReaderFromTable(BigQueryOptions bqOptions, TableReference tableRef);
-
- /**
- * Returns a real, mock, or fake {@link BigQueryJsonReader} to query tables.
- */
- public BigQueryJsonReader getReaderFromQuery(
- BigQueryOptions bqOptions, String query, String projectId, @Nullable Boolean flatten);
-
- /**
- * An interface for the Cloud BigQuery load service.
- */
- public interface JobService {
- /**
- * Start a BigQuery load job.
- */
- void startLoadJob(JobReference jobRef, JobConfigurationLoad loadConfig)
- throws InterruptedException, IOException;
- /**
- * Start a BigQuery extract job.
- */
- void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig)
- throws InterruptedException, IOException;
-
- /**
- * Start a BigQuery query job.
- */
- void startQueryJob(JobReference jobRef, JobConfigurationQuery query)
- throws IOException, InterruptedException;
-
- /**
- * Waits for the job is Done, and returns the job.
- *
- * <p>Returns null if the {@code maxAttempts} retries reached.
- */
- Job pollJob(JobReference jobRef, int maxAttempts)
- throws InterruptedException, IOException;
-
- /**
- * Dry runs the query in the given project.
- */
- JobStatistics dryRunQuery(String projectId, String query)
- throws InterruptedException, IOException;
- }
-
- /**
- * An interface to get, create and delete Cloud BigQuery datasets and tables.
- */
- public interface DatasetService {
- /**
- * Gets the specified {@link Table} resource by table ID.
- */
- Table getTable(String projectId, String datasetId, String tableId)
- throws InterruptedException, IOException;
-
- /**
- * Deletes the table specified by tableId from the dataset.
- * If the table contains data, all the data will be deleted.
- */
- void deleteTable(String projectId, String datasetId, String tableId)
- throws IOException, InterruptedException;
-
- /**
- * Returns true if the table is empty.
- */
- boolean isTableEmpty(String projectId, String datasetId, String tableId)
- throws IOException, InterruptedException;
-
- /**
- * Gets the specified {@link Dataset} resource by dataset ID.
- */
- Dataset getDataset(String projectId, String datasetId)
- throws IOException, InterruptedException;
-
- /**
- * Create a {@link Dataset} with the given {@code location} and {@code description}.
- */
- void createDataset(String projectId, String datasetId, String location, String description)
- throws IOException, InterruptedException;
-
- /**
- * Deletes the dataset specified by the datasetId value.
- *
- * <p>Before you can delete a dataset, you must delete all its tables.
- */
- void deleteDataset(String projectId, String datasetId)
- throws IOException, InterruptedException;
- }
-
- /**
- * An interface to read the Cloud BigQuery directly.
- */
- public interface BigQueryJsonReader {
- /**
- * Initializes the reader and advances the reader to the first record.
- */
- boolean start() throws IOException;
-
- /**
- * Advances the reader to the next valid record.
- */
- boolean advance() throws IOException;
-
- /**
- * Returns the value of the data item that was read by the last {@link #start} or
- * {@link #advance} call. The returned value must be effectively immutable and remain valid
- * indefinitely.
- *
- * <p>Multiple calls to this method without an intervening call to {@link #advance} should
- * return the same result.
- *
- * @throws java.util.NoSuchElementException if {@link #start} was never called, or if
- * the last {@link #start} or {@link #advance} returned {@code false}.
- */
- TableRow getCurrent() throws NoSuchElementException;
-
- /**
- * Closes the reader. The reader cannot be used after this method is called.
- */
- void close() throws IOException;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java
deleted file mode 100644
index 1aadeb2..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java
+++ /dev/null
@@ -1,515 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.beam.sdk.options.BigQueryOptions;
-
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.model.Dataset;
-import com.google.api.services.bigquery.model.DatasetReference;
-import com.google.api.services.bigquery.model.Job;
-import com.google.api.services.bigquery.model.JobConfiguration;
-import com.google.api.services.bigquery.model.JobConfigurationExtract;
-import com.google.api.services.bigquery.model.JobConfigurationLoad;
-import com.google.api.services.bigquery.model.JobConfigurationQuery;
-import com.google.api.services.bigquery.model.JobReference;
-import com.google.api.services.bigquery.model.JobStatistics;
-import com.google.api.services.bigquery.model.JobStatus;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableDataList;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.hadoop.util.ApiErrorExtractor;
-import com.google.common.annotations.VisibleForTesting;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.NoSuchElementException;
-import java.util.concurrent.TimeUnit;
-
-import javax.annotation.Nullable;
-
-/**
- * An implementation of {@link BigQueryServices} that actually communicates with the cloud BigQuery
- * service.
- */
-public class BigQueryServicesImpl implements BigQueryServices {
-
- private static final Logger LOG = LoggerFactory.getLogger(BigQueryServicesImpl.class);
-
- // The maximum number of attempts to execute a BigQuery RPC.
- private static final int MAX_RPC_ATTEMPTS = 10;
-
- // The initial backoff for executing a BigQuery RPC.
- private static final long INITIAL_RPC_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1);
-
- // The initial backoff for polling the status of a BigQuery job.
- private static final long INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1);
-
- @Override
- public JobService getJobService(BigQueryOptions options) {
- return new JobServiceImpl(options);
- }
-
- @Override
- public DatasetService getDatasetService(BigQueryOptions options) {
- return new DatasetServiceImpl(options);
- }
-
- @Override
- public BigQueryJsonReader getReaderFromTable(BigQueryOptions bqOptions, TableReference tableRef) {
- return BigQueryJsonReaderImpl.fromTable(bqOptions, tableRef);
- }
-
- @Override
- public BigQueryJsonReader getReaderFromQuery(
- BigQueryOptions bqOptions, String query, String projectId, @Nullable Boolean flatten) {
- return BigQueryJsonReaderImpl.fromQuery(bqOptions, query, projectId, flatten);
- }
-
- @VisibleForTesting
- static class JobServiceImpl implements BigQueryServices.JobService {
- private final ApiErrorExtractor errorExtractor;
- private final Bigquery client;
-
- @VisibleForTesting
- JobServiceImpl(Bigquery client) {
- this.errorExtractor = new ApiErrorExtractor();
- this.client = client;
- }
-
- private JobServiceImpl(BigQueryOptions options) {
- this.errorExtractor = new ApiErrorExtractor();
- this.client = Transport.newBigQueryClient(options).build();
- }
-
- /**
- * {@inheritDoc}
- *
- * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
- *
- * @throws IOException if it exceeds max RPC .
- */
- @Override
- public void startLoadJob(
- JobReference jobRef,
- JobConfigurationLoad loadConfig) throws InterruptedException, IOException {
- Job job = new Job()
- .setJobReference(jobRef)
- .setConfiguration(new JobConfiguration().setLoad(loadConfig));
-
- startJob(job, errorExtractor, client);
- }
-
- /**
- * {@inheritDoc}
- *
- * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
- *
- * @throws IOException if it exceeds max RPC .
- */
- @Override
- public void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig)
- throws InterruptedException, IOException {
- Job job = new Job()
- .setJobReference(jobRef)
- .setConfiguration(
- new JobConfiguration().setExtract(extractConfig));
-
- startJob(job, errorExtractor, client);
- }
-
- /**
- * {@inheritDoc}
- *
- * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
- *
- * @throws IOException if it exceeds max RPC .
- */
- @Override
- public void startQueryJob(JobReference jobRef, JobConfigurationQuery queryConfig)
- throws IOException, InterruptedException {
- Job job = new Job()
- .setJobReference(jobRef)
- .setConfiguration(
- new JobConfiguration().setQuery(queryConfig));
-
- startJob(job, errorExtractor, client);
- }
-
- private static void startJob(Job job,
- ApiErrorExtractor errorExtractor,
- Bigquery client) throws IOException, InterruptedException {
- BackOff backoff =
- new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
- startJob(job, errorExtractor, client, Sleeper.DEFAULT, backoff);
- }
-
- @VisibleForTesting
- static void startJob(
- Job job,
- ApiErrorExtractor errorExtractor,
- Bigquery client,
- Sleeper sleeper,
- BackOff backoff) throws IOException, InterruptedException {
- JobReference jobRef = job.getJobReference();
- Exception lastException = null;
- do {
- try {
- client.jobs().insert(jobRef.getProjectId(), job).execute();
- return; // SUCCEEDED
- } catch (GoogleJsonResponseException e) {
- if (errorExtractor.itemAlreadyExists(e)) {
- return; // SUCCEEDED
- }
- // ignore and retry
- LOG.warn("Ignore the error and retry inserting the job.", e);
- lastException = e;
- } catch (IOException e) {
- // ignore and retry
- LOG.warn("Ignore the error and retry inserting the job.", e);
- lastException = e;
- }
- } while (nextBackOff(sleeper, backoff));
- throw new IOException(
- String.format(
- "Unable to insert job: %s, aborting after %d .",
- jobRef.getJobId(), MAX_RPC_ATTEMPTS),
- lastException);
- }
-
- @Override
- public Job pollJob(JobReference jobRef, int maxAttempts)
- throws InterruptedException {
- BackOff backoff = new AttemptBoundedExponentialBackOff(
- maxAttempts, INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS);
- return pollJob(jobRef, Sleeper.DEFAULT, backoff);
- }
-
- @VisibleForTesting
- Job pollJob(
- JobReference jobRef,
- Sleeper sleeper,
- BackOff backoff) throws InterruptedException {
- do {
- try {
- Job job = client.jobs().get(jobRef.getProjectId(), jobRef.getJobId()).execute();
- JobStatus status = job.getStatus();
- if (status != null && status.getState() != null && status.getState().equals("DONE")) {
- return job;
- }
- // The job is not DONE, wait longer and retry.
- } catch (IOException e) {
- // ignore and retry
- LOG.warn("Ignore the error and retry polling job status.", e);
- }
- } while (nextBackOff(sleeper, backoff));
- LOG.warn("Unable to poll job status: {}, aborting after reached max .", jobRef.getJobId());
- return null;
- }
-
- @Override
- public JobStatistics dryRunQuery(String projectId, String query)
- throws InterruptedException, IOException {
- Job job = new Job()
- .setConfiguration(new JobConfiguration()
- .setQuery(new JobConfigurationQuery()
- .setQuery(query))
- .setDryRun(true));
- BackOff backoff =
- new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
- return executeWithRetries(
- client.jobs().insert(projectId, job),
- String.format(
- "Unable to dry run query: %s, aborting after %d retries.",
- query, MAX_RPC_ATTEMPTS),
- Sleeper.DEFAULT,
- backoff).getStatistics();
- }
- }
-
- @VisibleForTesting
- static class DatasetServiceImpl implements DatasetService {
- private final ApiErrorExtractor errorExtractor;
- private final Bigquery client;
-
- @VisibleForTesting
- DatasetServiceImpl(Bigquery client) {
- this.errorExtractor = new ApiErrorExtractor();
- this.client = client;
- }
-
- private DatasetServiceImpl(BigQueryOptions bqOptions) {
- this.errorExtractor = new ApiErrorExtractor();
- this.client = Transport.newBigQueryClient(bqOptions).build();
- }
-
- /**
- * {@inheritDoc}
- *
- * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
- *
- * @throws IOException if it exceeds max RPC .
- */
- @Override
- public Table getTable(String projectId, String datasetId, String tableId)
- throws IOException, InterruptedException {
- BackOff backoff =
- new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
- return executeWithRetries(
- client.tables().get(projectId, datasetId, tableId),
- String.format(
- "Unable to get table: %s, aborting after %d retries.",
- tableId, MAX_RPC_ATTEMPTS),
- Sleeper.DEFAULT,
- backoff);
- }
-
- /**
- * {@inheritDoc}
- *
- * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
- *
- * @throws IOException if it exceeds max RPC .
- */
- @Override
- public void deleteTable(String projectId, String datasetId, String tableId)
- throws IOException, InterruptedException {
- BackOff backoff =
- new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
- executeWithRetries(
- client.tables().delete(projectId, datasetId, tableId),
- String.format(
- "Unable to delete table: %s, aborting after %d retries.",
- tableId, MAX_RPC_ATTEMPTS),
- Sleeper.DEFAULT,
- backoff);
- }
-
- @Override
- public boolean isTableEmpty(String projectId, String datasetId, String tableId)
- throws IOException, InterruptedException {
- BackOff backoff =
- new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
- TableDataList dataList = executeWithRetries(
- client.tabledata().list(projectId, datasetId, tableId),
- String.format(
- "Unable to list table data: %s, aborting after %d retries.",
- tableId, MAX_RPC_ATTEMPTS),
- Sleeper.DEFAULT,
- backoff);
- return dataList.getRows() == null || dataList.getRows().isEmpty();
- }
-
- /**
- * {@inheritDoc}
- *
- * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
- *
- * @throws IOException if it exceeds max RPC .
- */
- @Override
- public Dataset getDataset(String projectId, String datasetId)
- throws IOException, InterruptedException {
- BackOff backoff =
- new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
- return executeWithRetries(
- client.datasets().get(projectId, datasetId),
- String.format(
- "Unable to get dataset: %s, aborting after %d retries.",
- datasetId, MAX_RPC_ATTEMPTS),
- Sleeper.DEFAULT,
- backoff);
- }
-
- /**
- * {@inheritDoc}
- *
- * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
- *
- * @throws IOException if it exceeds max RPC .
- */
- @Override
- public void createDataset(
- String projectId, String datasetId, String location, String description)
- throws IOException, InterruptedException {
- BackOff backoff =
- new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
- createDataset(projectId, datasetId, location, description, Sleeper.DEFAULT, backoff);
- }
-
- @VisibleForTesting
- void createDataset(
- String projectId,
- String datasetId,
- String location,
- String description,
- Sleeper sleeper,
- BackOff backoff) throws IOException, InterruptedException {
- DatasetReference datasetRef = new DatasetReference()
- .setProjectId(projectId)
- .setDatasetId(datasetId);
-
- Dataset dataset = new Dataset()
- .setDatasetReference(datasetRef)
- .setLocation(location)
- .setFriendlyName(location)
- .setDescription(description);
-
- Exception lastException;
- do {
- try {
- client.datasets().insert(projectId, dataset).execute();
- return; // SUCCEEDED
- } catch (GoogleJsonResponseException e) {
- if (errorExtractor.itemAlreadyExists(e)) {
- return; // SUCCEEDED
- }
- // ignore and retry
- LOG.warn("Ignore the error and retry creating the dataset.", e);
- lastException = e;
- } catch (IOException e) {
- LOG.warn("Ignore the error and retry creating the dataset.", e);
- lastException = e;
- }
- } while (nextBackOff(sleeper, backoff));
- throw new IOException(
- String.format(
- "Unable to create dataset: %s, aborting after %d .",
- datasetId, MAX_RPC_ATTEMPTS),
- lastException);
- }
-
- /**
- * {@inheritDoc}
- *
- * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
- *
- * @throws IOException if it exceeds max RPC .
- */
- @Override
- public void deleteDataset(String projectId, String datasetId)
- throws IOException, InterruptedException {
- BackOff backoff =
- new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
- executeWithRetries(
- client.datasets().delete(projectId, datasetId),
- String.format(
- "Unable to delete table: %s, aborting after %d retries.",
- datasetId, MAX_RPC_ATTEMPTS),
- Sleeper.DEFAULT,
- backoff);
- }
- }
-
- private static class BigQueryJsonReaderImpl implements BigQueryJsonReader {
- BigQueryTableRowIterator iterator;
-
- private BigQueryJsonReaderImpl(BigQueryTableRowIterator iterator) {
- this.iterator = iterator;
- }
-
- private static BigQueryJsonReader fromQuery(
- BigQueryOptions bqOptions,
- String query,
- String projectId,
- @Nullable Boolean flattenResults) {
- return new BigQueryJsonReaderImpl(
- BigQueryTableRowIterator.fromQuery(
- query, projectId, Transport.newBigQueryClient(bqOptions).build(), flattenResults));
- }
-
- private static BigQueryJsonReader fromTable(
- BigQueryOptions bqOptions,
- TableReference tableRef) {
- return new BigQueryJsonReaderImpl(BigQueryTableRowIterator.fromTable(
- tableRef, Transport.newBigQueryClient(bqOptions).build()));
- }
-
- @Override
- public boolean start() throws IOException {
- try {
- iterator.open();
- return iterator.advance();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new RuntimeException("Interrupted during start() operation", e);
- }
- }
-
- @Override
- public boolean advance() throws IOException {
- try {
- return iterator.advance();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new RuntimeException("Interrupted during advance() operation", e);
- }
- }
-
- @Override
- public TableRow getCurrent() throws NoSuchElementException {
- return iterator.getCurrent();
- }
-
- @Override
- public void close() throws IOException {
- iterator.close();
- }
- }
-
- @VisibleForTesting
- static <T> T executeWithRetries(
- AbstractGoogleClientRequest<T> request,
- String errorMessage,
- Sleeper sleeper,
- BackOff backoff)
- throws IOException, InterruptedException {
- Exception lastException = null;
- do {
- try {
- return request.execute();
- } catch (IOException e) {
- LOG.warn("Ignore the error and retry the request.", e);
- lastException = e;
- }
- } while (nextBackOff(sleeper, backoff));
- throw new IOException(
- errorMessage,
- lastException);
- }
-
- /**
- * Identical to {@link BackOffUtils#next} but without checked IOException.
- * @throws InterruptedException
- */
- private static boolean nextBackOff(Sleeper sleeper, BackOff backoff) throws InterruptedException {
- try {
- return BackOffUtils.next(sleeper, backoff);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
deleted file mode 100644
index 84004a7..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
+++ /dev/null
@@ -1,469 +0,0 @@
-/*
- * 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.util;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition;
-import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition;
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator;
-
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.ExponentialBackOff;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableDataInsertAllRequest;
-import com.google.api.services.bigquery.model.TableDataInsertAllResponse;
-import com.google.api.services.bigquery.model.TableDataList;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.hadoop.util.ApiErrorExtractor;
-import com.google.common.annotations.VisibleForTesting;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-import javax.annotation.Nullable;
-
-/**
- * Inserts rows into BigQuery.
- */
-public class BigQueryTableInserter {
- private static final Logger LOG = LoggerFactory.getLogger(BigQueryTableInserter.class);
-
- // Approximate amount of table data to upload per InsertAll request.
- private static final long UPLOAD_BATCH_SIZE_BYTES = 64 * 1024;
-
- // The maximum number of rows to upload per InsertAll request.
- private static final long MAX_ROWS_PER_BATCH = 500;
-
- // The maximum number of times to retry inserting rows into BigQuery.
- private static final int MAX_INSERT_ATTEMPTS = 5;
-
- // The initial backoff after a failure inserting rows into BigQuery.
- private static final long INITIAL_INSERT_BACKOFF_INTERVAL_MS = 200L;
-
- // Backoff time bounds for rate limit exceeded errors.
- private static final long INITIAL_RATE_LIMIT_EXCEEDED_BACKOFF_MS = TimeUnit.SECONDS.toMillis(1);
- private static final long MAX_RATE_LIMIT_EXCEEDED_BACKOFF_MS = TimeUnit.MINUTES.toMillis(2);
-
- private final Bigquery client;
- private final TableReference defaultRef;
- private final long maxRowsPerBatch;
-
- private ExecutorService executor;
-
- /**
- * Constructs a new row inserter.
- *
- * @param client a BigQuery client
- * @param options a PipelineOptions object
- */
- public BigQueryTableInserter(Bigquery client, PipelineOptions options) {
- this.client = client;
- this.defaultRef = null;
- this.maxRowsPerBatch = MAX_ROWS_PER_BATCH;
- this.executor = options.as(GcsOptions.class).getExecutorService();
- }
-
- /**
- * Constructs a new row inserter.
- *
- * @param client a BigQuery client
- * @param options a PipelineOptions object
- * @param defaultRef identifies the table to insert into
- * @deprecated replaced by {@link #BigQueryTableInserter(Bigquery, PipelineOptions)}
- */
- @Deprecated
- public BigQueryTableInserter(Bigquery client, PipelineOptions options,
- TableReference defaultRef) {
- this.client = client;
- this.defaultRef = defaultRef;
- this.maxRowsPerBatch = MAX_ROWS_PER_BATCH;
- this.executor = options.as(GcsOptions.class).getExecutorService();
- }
-
- /**
- * Constructs a new row inserter.
- *
- * @param client a BigQuery client
- * @param options a PipelineOptions object
- * @param maxRowsPerBatch maximum number of rows to insert per call to BigQuery
- */
- public BigQueryTableInserter(Bigquery client, PipelineOptions options,
- int maxRowsPerBatch) {
- this.client = client;
- this.defaultRef = null;
- this.maxRowsPerBatch = maxRowsPerBatch;
- this.executor = options.as(GcsOptions.class).getExecutorService();
- }
-
- /**
- * Constructs a new row inserter.
- *
- * @param client a BigQuery client
- * @param defaultRef identifies the default table to insert into
- * @deprecated replaced by {@link #BigQueryTableInserter(Bigquery, PipelineOptions, int)}
- */
- @Deprecated
- public BigQueryTableInserter(Bigquery client, PipelineOptions options,
- TableReference defaultRef, int maxRowsPerBatch) {
- this.client = client;
- this.defaultRef = defaultRef;
- this.maxRowsPerBatch = maxRowsPerBatch;
- this.executor = options.as(GcsOptions.class).getExecutorService();
- }
-
- /**
- * Insert all rows from the given list.
- *
- * @deprecated replaced by {@link #insertAll(TableReference, List)}
- */
- @Deprecated
- public void insertAll(List<TableRow> rowList) throws IOException {
- insertAll(defaultRef, rowList, null, null);
- }
-
- /**
- * Insert all rows from the given list using specified insertIds if not null.
- *
- * @deprecated replaced by {@link #insertAll(TableReference, List, List)}
- */
- @Deprecated
- public void insertAll(List<TableRow> rowList,
- @Nullable List<String> insertIdList) throws IOException {
- insertAll(defaultRef, rowList, insertIdList, null);
- }
-
- /**
- * Insert all rows from the given list.
- */
- public void insertAll(TableReference ref, List<TableRow> rowList) throws IOException {
- insertAll(ref, rowList, null, null);
- }
-
- /**
- * Insert all rows from the given list using specified insertIds if not null. Track count of
- * bytes written with the Aggregator.
- */
- public void insertAll(TableReference ref, List<TableRow> rowList,
- @Nullable List<String> insertIdList, Aggregator<Long, Long> byteCountAggregator)
- throws IOException {
- checkNotNull(ref, "ref");
- if (insertIdList != null && rowList.size() != insertIdList.size()) {
- throw new AssertionError("If insertIdList is not null it needs to have at least "
- + "as many elements as rowList");
- }
-
- AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff(
- MAX_INSERT_ATTEMPTS,
- INITIAL_INSERT_BACKOFF_INTERVAL_MS);
-
- List<TableDataInsertAllResponse.InsertErrors> allErrors = new ArrayList<>();
- // These lists contain the rows to publish. Initially the contain the entire list. If there are
- // failures, they will contain only the failed rows to be retried.
- List<TableRow> rowsToPublish = rowList;
- List<String> idsToPublish = insertIdList;
- while (true) {
- List<TableRow> retryRows = new ArrayList<>();
- List<String> retryIds = (idsToPublish != null) ? new ArrayList<String>() : null;
-
- int strideIndex = 0;
- // Upload in batches.
- List<TableDataInsertAllRequest.Rows> rows = new LinkedList<>();
- int dataSize = 0;
-
- List<Future<List<TableDataInsertAllResponse.InsertErrors>>> futures = new ArrayList<>();
- List<Integer> strideIndices = new ArrayList<>();
-
- for (int i = 0; i < rowsToPublish.size(); ++i) {
- TableRow row = rowsToPublish.get(i);
- TableDataInsertAllRequest.Rows out = new TableDataInsertAllRequest.Rows();
- if (idsToPublish != null) {
- out.setInsertId(idsToPublish.get(i));
- }
- out.setJson(row.getUnknownKeys());
- rows.add(out);
-
- dataSize += row.toString().length();
- if (dataSize >= UPLOAD_BATCH_SIZE_BYTES || rows.size() >= maxRowsPerBatch
- || i == rowsToPublish.size() - 1) {
- TableDataInsertAllRequest content = new TableDataInsertAllRequest();
- content.setRows(rows);
-
- final Bigquery.Tabledata.InsertAll insert = client.tabledata()
- .insertAll(ref.getProjectId(), ref.getDatasetId(), ref.getTableId(),
- content);
-
- futures.add(
- executor.submit(new Callable<List<TableDataInsertAllResponse.InsertErrors>>() {
- @Override
- public List<TableDataInsertAllResponse.InsertErrors> call() throws IOException {
- BackOff backoff = new IntervalBoundedExponentialBackOff(
- MAX_RATE_LIMIT_EXCEEDED_BACKOFF_MS, INITIAL_RATE_LIMIT_EXCEEDED_BACKOFF_MS);
- while (true) {
- try {
- return insert.execute().getInsertErrors();
- } catch (IOException e) {
- if (new ApiErrorExtractor().rateLimited(e)) {
- LOG.info("BigQuery insertAll exceeded rate limit, retrying");
- try {
- Thread.sleep(backoff.nextBackOffMillis());
- } catch (InterruptedException interrupted) {
- throw new IOException(
- "Interrupted while waiting before retrying insertAll");
- }
- } else {
- throw e;
- }
- }
- }
- }
- }));
- strideIndices.add(strideIndex);
-
- if (byteCountAggregator != null) {
- byteCountAggregator.addValue(Long.valueOf(dataSize));
- }
- dataSize = 0;
- strideIndex = i + 1;
- rows = new LinkedList<>();
- }
- }
-
- try {
- for (int i = 0; i < futures.size(); i++) {
- List<TableDataInsertAllResponse.InsertErrors> errors = futures.get(i).get();
- if (errors != null) {
- for (TableDataInsertAllResponse.InsertErrors error : errors) {
- allErrors.add(error);
- if (error.getIndex() == null) {
- throw new IOException("Insert failed: " + allErrors);
- }
-
- int errorIndex = error.getIndex().intValue() + strideIndices.get(i);
- retryRows.add(rowsToPublish.get(errorIndex));
- if (retryIds != null) {
- retryIds.add(idsToPublish.get(errorIndex));
- }
- }
- }
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new IOException("Interrupted while inserting " + rowsToPublish);
- } catch (ExecutionException e) {
- throw new RuntimeException(e.getCause());
- }
-
- if (!allErrors.isEmpty() && !backoff.atMaxAttempts()) {
- try {
- Thread.sleep(backoff.nextBackOffMillis());
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new IOException("Interrupted while waiting before retrying insert of " + retryRows);
- }
- LOG.info("Retrying failed inserts to BigQuery");
- rowsToPublish = retryRows;
- idsToPublish = retryIds;
- allErrors.clear();
- } else {
- break;
- }
- }
- if (!allErrors.isEmpty()) {
- throw new IOException("Insert failed: " + allErrors);
- }
- }
-
- /**
- * Retrieves or creates the table.
- *
- * <p>The table is checked to conform to insertion requirements as specified
- * by WriteDisposition and CreateDisposition.
- *
- * <p>If table truncation is requested (WriteDisposition.WRITE_TRUNCATE), then
- * this will re-create the table if necessary to ensure it is empty.
- *
- * <p>If an empty table is required (WriteDisposition.WRITE_EMPTY), then this
- * will fail if the table exists and is not empty.
- *
- * <p>When constructing a table, a {@code TableSchema} must be available. If a
- * schema is provided, then it will be used. If no schema is provided, but
- * an existing table is being cleared (WRITE_TRUNCATE option above), then
- * the existing schema will be re-used. If no schema is available, then an
- * {@code IOException} is thrown.
- */
- public Table getOrCreateTable(
- TableReference ref,
- WriteDisposition writeDisposition,
- CreateDisposition createDisposition,
- @Nullable TableSchema schema) throws IOException {
- // Check if table already exists.
- Bigquery.Tables.Get get = client.tables()
- .get(ref.getProjectId(), ref.getDatasetId(), ref.getTableId());
- Table table = null;
- try {
- table = get.execute();
- } catch (IOException e) {
- ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
- if (!errorExtractor.itemNotFound(e)
- || createDisposition != CreateDisposition.CREATE_IF_NEEDED) {
- // Rethrow.
- throw e;
- }
- }
-
- // If we want an empty table, and it isn't, then delete it first.
- if (table != null) {
- if (writeDisposition == WriteDisposition.WRITE_APPEND) {
- return table;
- }
-
- boolean empty = isEmpty(ref);
- if (empty) {
- if (writeDisposition == WriteDisposition.WRITE_TRUNCATE) {
- LOG.info("Empty table found, not removing {}", BigQueryIO.toTableSpec(ref));
- }
- return table;
-
- } else if (writeDisposition == WriteDisposition.WRITE_EMPTY) {
- throw new IOException("WriteDisposition is WRITE_EMPTY, "
- + "but table is not empty");
- }
-
- // Reuse the existing schema if none was provided.
- if (schema == null) {
- schema = table.getSchema();
- }
-
- // Delete table and fall through to re-creating it below.
- LOG.info("Deleting table {}", BigQueryIO.toTableSpec(ref));
- Bigquery.Tables.Delete delete = client.tables()
- .delete(ref.getProjectId(), ref.getDatasetId(), ref.getTableId());
- delete.execute();
- }
-
- if (schema == null) {
- throw new IllegalArgumentException(
- "Table schema required for new table.");
- }
-
- // Create the table.
- return tryCreateTable(ref, schema);
- }
-
- /**
- * Checks if a table is empty.
- */
- public boolean isEmpty(TableReference ref) throws IOException {
- Bigquery.Tabledata.List list = client.tabledata()
- .list(ref.getProjectId(), ref.getDatasetId(), ref.getTableId());
- list.setMaxResults(1L);
- TableDataList dataList = list.execute();
-
- return dataList.getRows() == null || dataList.getRows().isEmpty();
- }
-
- /**
- * Retry table creation up to 5 minutes (with exponential backoff) when this user is near the
- * quota for table creation. This relatively innocuous behavior can happen when BigQueryIO is
- * configured with a table spec function to use different tables for each window.
- */
- private static final int RETRY_CREATE_TABLE_DURATION_MILLIS = (int) TimeUnit.MINUTES.toMillis(5);
-
- /**
- * Tries to create the BigQuery table.
- * If a table with the same name already exists in the dataset, the table
- * creation fails, and the function returns null. In such a case,
- * the existing table doesn't necessarily have the same schema as specified
- * by the parameter.
- *
- * @param schema Schema of the new BigQuery table.
- * @return The newly created BigQuery table information, or null if the table
- * with the same name already exists.
- * @throws IOException if other error than already existing table occurs.
- */
- @Nullable
- public Table tryCreateTable(TableReference ref, TableSchema schema) throws IOException {
- LOG.info("Trying to create BigQuery table: {}", BigQueryIO.toTableSpec(ref));
- BackOff backoff =
- new ExponentialBackOff.Builder()
- .setMaxElapsedTimeMillis(RETRY_CREATE_TABLE_DURATION_MILLIS)
- .build();
-
- Table table = new Table().setTableReference(ref).setSchema(schema);
- return tryCreateTable(table, ref.getProjectId(), ref.getDatasetId(), backoff, Sleeper.DEFAULT);
- }
-
- @VisibleForTesting
- @Nullable
- Table tryCreateTable(
- Table table, String projectId, String datasetId, BackOff backoff, Sleeper sleeper)
- throws IOException {
- boolean retry = false;
- while (true) {
- try {
- return client.tables().insert(projectId, datasetId, table).execute();
- } catch (IOException e) {
- ApiErrorExtractor extractor = new ApiErrorExtractor();
- if (extractor.itemAlreadyExists(e)) {
- // The table already exists, nothing to return.
- return null;
- } else if (extractor.rateLimited(e)) {
- // The request failed because we hit a temporary quota. Back off and try again.
- try {
- if (BackOffUtils.next(sleeper, backoff)) {
- if (!retry) {
- LOG.info(
- "Quota limit reached when creating table {}:{}.{}, retrying up to {} minutes",
- projectId,
- datasetId,
- table.getTableReference().getTableId(),
- TimeUnit.MILLISECONDS.toSeconds(RETRY_CREATE_TABLE_DURATION_MILLIS) / 60.0);
- retry = true;
- }
- continue;
- }
- } catch (InterruptedException e1) {
- // Restore interrupted state and throw the last failure.
- Thread.currentThread().interrupt();
- throw e;
- }
- }
- throw e;
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
deleted file mode 100644
index 3865654..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
+++ /dev/null
@@ -1,472 +0,0 @@
-/*
- * 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.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.ClassInfo;
-import com.google.api.client.util.Data;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.Bigquery.Jobs.Insert;
-import com.google.api.services.bigquery.model.Dataset;
-import com.google.api.services.bigquery.model.DatasetReference;
-import com.google.api.services.bigquery.model.ErrorProto;
-import com.google.api.services.bigquery.model.Job;
-import com.google.api.services.bigquery.model.JobConfiguration;
-import com.google.api.services.bigquery.model.JobConfigurationQuery;
-import com.google.api.services.bigquery.model.JobReference;
-import com.google.api.services.bigquery.model.JobStatus;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableCell;
-import com.google.api.services.bigquery.model.TableDataList;
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-import com.google.common.util.concurrent.Uninterruptibles;
-
-import org.joda.time.Duration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Objects;
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-
-import javax.annotation.Nullable;
-
-/**
- * Iterates over all rows in a table.
- */
-public class BigQueryTableRowIterator implements AutoCloseable {
- private static final Logger LOG = LoggerFactory.getLogger(BigQueryTableRowIterator.class);
-
- @Nullable private TableReference ref;
- @Nullable private final String projectId;
- @Nullable private TableSchema schema;
- private final Bigquery client;
- private String pageToken;
- private Iterator<TableRow> iteratorOverCurrentBatch;
- private TableRow current;
- // Set true when the final page is seen from the service.
- private boolean lastPage = false;
-
- // The maximum number of times a BigQuery request will be retried
- private static final int MAX_RETRIES = 3;
- // Initial wait time for the backoff implementation
- private static final Duration INITIAL_BACKOFF_TIME = Duration.standardSeconds(1);
-
- // After sending a query to BQ service we will be polling the BQ service to check the status with
- // following interval to check the status of query execution job
- private static final Duration QUERY_COMPLETION_POLL_TIME = Duration.standardSeconds(1);
-
- private final String query;
- // Whether to flatten query results.
- private final boolean flattenResults;
- // Temporary dataset used to store query results.
- private String temporaryDatasetId = null;
- // Temporary table used to store query results.
- private String temporaryTableId = null;
-
- private BigQueryTableRowIterator(
- @Nullable TableReference ref, @Nullable String query, @Nullable String projectId,
- Bigquery client, boolean flattenResults) {
- this.ref = ref;
- this.query = query;
- this.projectId = projectId;
- this.client = checkNotNull(client, "client");
- this.flattenResults = flattenResults;
- }
-
- /**
- * Constructs a {@code BigQueryTableRowIterator} that reads from the specified table.
- */
- public static BigQueryTableRowIterator fromTable(TableReference ref, Bigquery client) {
- checkNotNull(ref, "ref");
- checkNotNull(client, "client");
- return new BigQueryTableRowIterator(ref, null, ref.getProjectId(), client, true);
- }
-
- /**
- * Constructs a {@code BigQueryTableRowIterator} that reads from the results of executing the
- * specified query in the specified project.
- */
- public static BigQueryTableRowIterator fromQuery(
- String query, String projectId, Bigquery client, @Nullable Boolean flattenResults) {
- checkNotNull(query, "query");
- checkNotNull(projectId, "projectId");
- checkNotNull(client, "client");
- return new BigQueryTableRowIterator(null, query, projectId, client,
- MoreObjects.firstNonNull(flattenResults, Boolean.TRUE));
- }
-
- /**
- * Opens the table for read.
- * @throws IOException on failure
- */
- public void open() throws IOException, InterruptedException {
- if (query != null) {
- ref = executeQueryAndWaitForCompletion();
- }
- // Get table schema.
- Bigquery.Tables.Get get =
- client.tables().get(ref.getProjectId(), ref.getDatasetId(), ref.getTableId());
-
- Table table =
- executeWithBackOff(
- get,
- "Error opening BigQuery table %s of dataset %s : {}",
- ref.getTableId(),
- ref.getDatasetId());
- schema = table.getSchema();
- }
-
- public boolean advance() throws IOException, InterruptedException {
- while (true) {
- if (iteratorOverCurrentBatch != null && iteratorOverCurrentBatch.hasNext()) {
- // Embed schema information into the raw row, so that values have an
- // associated key. This matches how rows are read when using the
- // DataflowRunner.
- current = getTypedTableRow(schema.getFields(), iteratorOverCurrentBatch.next());
- return true;
- }
- if (lastPage) {
- return false;
- }
-
- Bigquery.Tabledata.List list =
- client.tabledata().list(ref.getProjectId(), ref.getDatasetId(), ref.getTableId());
- if (pageToken != null) {
- list.setPageToken(pageToken);
- }
-
- TableDataList result =
- executeWithBackOff(
- list,
- "Error reading from BigQuery table %s of dataset %s : {}",
- ref.getTableId(),
- ref.getDatasetId());
-
- pageToken = result.getPageToken();
- iteratorOverCurrentBatch =
- result.getRows() != null
- ? result.getRows().iterator()
- : Collections.<TableRow>emptyIterator();
-
- // The server may return a page token indefinitely on a zero-length table.
- if (pageToken == null || result.getTotalRows() != null && result.getTotalRows() == 0) {
- lastPage = true;
- }
- }
- }
-
- public TableRow getCurrent() {
- if (current == null) {
- throw new NoSuchElementException();
- }
- return current;
- }
-
- /**
- * Adjusts a field returned from the BigQuery API to match what we will receive when running
- * BigQuery's export-to-GCS and parallel read, which is the efficient parallel implementation
- * used for batch jobs executed on the Cloud Dataflow service.
- *
- * <p>The following is the relationship between BigQuery schema and Java types:
- *
- * <ul>
- * <li>Nulls are {@code null}.
- * <li>Repeated fields are {@code List} of objects.
- * <li>Record columns are {@link TableRow} objects.
- * <li>{@code BOOLEAN} columns are JSON booleans, hence Java {@code Boolean} objects.
- * <li>{@code FLOAT} columns are JSON floats, hence Java {@code Double} objects.
- * <li>{@code TIMESTAMP} columns are {@code String} objects that are of the format
- * {@code yyyy-MM-dd HH:mm:ss[.SSSSSS] UTC}, where the {@code .SSSSSS} has no trailing
- * zeros and can be 1 to 6 digits long.
- * <li>Every other atomic type is a {@code String}.
- * </ul>
- *
- * <p>Note that integers are encoded as strings to match BigQuery's exported JSON format.
- *
- * <p>Finally, values are stored in the {@link TableRow} as {"field name": value} pairs
- * and are not accessible through the {@link TableRow#getF} function.
- */
- @Nullable private Object getTypedCellValue(TableFieldSchema fieldSchema, Object v) {
- if (Data.isNull(v)) {
- return null;
- }
-
- if (Objects.equals(fieldSchema.getMode(), "REPEATED")) {
- TableFieldSchema elementSchema = fieldSchema.clone().setMode("REQUIRED");
- @SuppressWarnings("unchecked")
- List<Map<String, Object>> rawCells = (List<Map<String, Object>>) v;
- ImmutableList.Builder<Object> values = ImmutableList.builder();
- for (Map<String, Object> element : rawCells) {
- values.add(getTypedCellValue(elementSchema, element.get("v")));
- }
- return values.build();
- }
-
- if (fieldSchema.getType().equals("RECORD")) {
- @SuppressWarnings("unchecked")
- Map<String, Object> typedV = (Map<String, Object>) v;
- return getTypedTableRow(fieldSchema.getFields(), typedV);
- }
-
- if (fieldSchema.getType().equals("FLOAT")) {
- return Double.parseDouble((String) v);
- }
-
- if (fieldSchema.getType().equals("BOOLEAN")) {
- return Boolean.parseBoolean((String) v);
- }
-
- if (fieldSchema.getType().equals("TIMESTAMP")) {
- return AvroUtils.formatTimestamp((String) v);
- }
-
- return v;
- }
-
- /**
- * A list of the field names that cannot be used in BigQuery tables processed by Dataflow,
- * because they are reserved keywords in {@link TableRow}.
- */
- // TODO: This limitation is unfortunate. We need to give users a way to use BigQueryIO that does
- // not indirect through our broken use of {@link TableRow}.
- // See discussion: https://github.com/GoogleCloudPlatform/DataflowJavaSDK/pull/41
- private static final Collection<String> RESERVED_FIELD_NAMES =
- ClassInfo.of(TableRow.class).getNames();
-
- /**
- * Converts a row returned from the BigQuery JSON API as a {@code Map<String, Object>} into a
- * Java {@link TableRow} with nested {@link TableCell TableCells}. The {@code Object} values in
- * the cells are converted to Java types according to the provided field schemas.
- *
- * <p>See {@link #getTypedCellValue(TableFieldSchema, Object)} for details on how BigQuery
- * types are mapped to Java types.
- */
- private TableRow getTypedTableRow(List<TableFieldSchema> fields, Map<String, Object> rawRow) {
- // If rawRow is a TableRow, use it. If not, create a new one.
- TableRow row;
- List<? extends Map<String, Object>> cells;
- if (rawRow instanceof TableRow) {
- // Since rawRow is a TableRow it already has TableCell objects in setF. We do not need to do
- // any type conversion, but extract the cells for cell-wise processing below.
- row = (TableRow) rawRow;
- cells = row.getF();
- // Clear the cells from the row, so that row.getF() will return null. This matches the
- // behavior of rows produced by the BigQuery export API used on the service.
- row.setF(null);
- } else {
- row = new TableRow();
-
- // Since rawRow is a Map<String, Object> we use Map.get("f") instead of TableRow.getF() to
- // get its cells. Similarly, when rawCell is a Map<String, Object> instead of a TableCell,
- // we will use Map.get("v") instead of TableCell.getV() get its value.
- @SuppressWarnings("unchecked")
- List<? extends Map<String, Object>> rawCells =
- (List<? extends Map<String, Object>>) rawRow.get("f");
- cells = rawCells;
- }
-
- checkState(cells.size() == fields.size(),
- "Expected that the row has the same number of cells %s as fields in the schema %s",
- cells.size(), fields.size());
-
- // Loop through all the fields in the row, normalizing their types with the TableFieldSchema
- // and storing the normalized values by field name in the Map<String, Object> that
- // underlies the TableRow.
- Iterator<? extends Map<String, Object>> cellIt = cells.iterator();
- Iterator<TableFieldSchema> fieldIt = fields.iterator();
- while (cellIt.hasNext()) {
- Map<String, Object> cell = cellIt.next();
- TableFieldSchema fieldSchema = fieldIt.next();
-
- // Convert the object in this cell to the Java type corresponding to its type in the schema.
- Object convertedValue = getTypedCellValue(fieldSchema, cell.get("v"));
-
- String fieldName = fieldSchema.getName();
- checkArgument(!RESERVED_FIELD_NAMES.contains(fieldName),
- "BigQueryIO does not support records with columns named %s", fieldName);
-
- if (convertedValue == null) {
- // BigQuery does not include null values when the export operation (to JSON) is used.
- // To match that behavior, BigQueryTableRowiterator, and the DirectRunner,
- // intentionally omits columns with null values.
- continue;
- }
-
- row.set(fieldName, convertedValue);
- }
- return row;
- }
-
- // Create a new BigQuery dataset
- private void createDataset(String datasetId) throws IOException, InterruptedException {
- Dataset dataset = new Dataset();
- DatasetReference reference = new DatasetReference();
- reference.setProjectId(projectId);
- reference.setDatasetId(datasetId);
- dataset.setDatasetReference(reference);
-
- String createDatasetError =
- "Error when trying to create the temporary dataset " + datasetId + " in project "
- + projectId;
- executeWithBackOff(
- client.datasets().insert(projectId, dataset), createDatasetError + " :{}");
- }
-
- // Delete the given table that is available in the given dataset.
- private void deleteTable(String datasetId, String tableId)
- throws IOException, InterruptedException {
- executeWithBackOff(
- client.tables().delete(projectId, datasetId, tableId),
- "Error when trying to delete the temporary table " + datasetId + " in dataset " + datasetId
- + " of project " + projectId + ". Manual deletion may be required. Error message : {}");
- }
-
- // Delete the given dataset. This will fail if the given dataset has any tables.
- private void deleteDataset(String datasetId) throws IOException, InterruptedException {
- executeWithBackOff(
- client.datasets().delete(projectId, datasetId),
- "Error when trying to delete the temporary dataset " + datasetId + " in project "
- + projectId + ". Manual deletion may be required. Error message : {}");
- }
-
- /**
- * Executes the specified query and returns a reference to the temporary BigQuery table created
- * to hold the results.
- *
- * @throws IOException if the query fails.
- */
- private TableReference executeQueryAndWaitForCompletion()
- throws IOException, InterruptedException {
- // Create a temporary dataset to store results.
- // Starting dataset name with an "_" so that it is hidden.
- Random rnd = new Random(System.currentTimeMillis());
- temporaryDatasetId = "_dataflow_temporary_dataset_" + rnd.nextInt(1000000);
- temporaryTableId = "dataflow_temporary_table_" + rnd.nextInt(1000000);
-
- createDataset(temporaryDatasetId);
- Job job = new Job();
- JobConfiguration config = new JobConfiguration();
- JobConfigurationQuery queryConfig = new JobConfigurationQuery();
- config.setQuery(queryConfig);
- job.setConfiguration(config);
- queryConfig.setQuery(query);
- queryConfig.setAllowLargeResults(true);
- queryConfig.setFlattenResults(flattenResults);
-
- TableReference destinationTable = new TableReference();
- destinationTable.setProjectId(projectId);
- destinationTable.setDatasetId(temporaryDatasetId);
- destinationTable.setTableId(temporaryTableId);
- queryConfig.setDestinationTable(destinationTable);
-
- Insert insert = client.jobs().insert(projectId, job);
- Job queryJob = executeWithBackOff(
- insert, "Error when trying to execute the job for query " + query + " :{}");
- JobReference jobId = queryJob.getJobReference();
-
- while (true) {
- Job pollJob = executeWithBackOff(
- client.jobs().get(projectId, jobId.getJobId()),
- "Error when trying to get status of the job for query " + query + " :{}");
- JobStatus status = pollJob.getStatus();
- if (status.getState().equals("DONE")) {
- // Job is DONE, but did not necessarily succeed.
- ErrorProto error = status.getErrorResult();
- if (error == null) {
- return pollJob.getConfiguration().getQuery().getDestinationTable();
- } else {
- // There will be no temporary table to delete, so null out the reference.
- temporaryTableId = null;
- throw new IOException("Executing query " + query + " failed: " + error.getMessage());
- }
- }
- Uninterruptibles.sleepUninterruptibly(
- QUERY_COMPLETION_POLL_TIME.getMillis(), TimeUnit.MILLISECONDS);
- }
- }
-
- // Execute a BQ request with exponential backoff and return the result.
- // client - BQ request to be executed
- // error - Formatted message to log if when a request fails. Takes exception message as a
- // formatter parameter.
- public static <T> T executeWithBackOff(AbstractGoogleClientRequest<T> client, String error,
- Object... errorArgs) throws IOException, InterruptedException {
- Sleeper sleeper = Sleeper.DEFAULT;
- BackOff backOff =
- new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_TIME.getMillis());
-
- T result = null;
- while (true) {
- try {
- result = client.execute();
- break;
- } catch (IOException e) {
- LOG.error(String.format(error, errorArgs), e.getMessage());
- if (!BackOffUtils.next(sleeper, backOff)) {
- LOG.error(
- String.format(error, errorArgs), "Failing after retrying " + MAX_RETRIES + " times.");
- throw e;
- }
- }
- }
-
- return result;
- }
-
- @Override
- public void close() {
- // Prevent any further requests.
- lastPage = true;
-
- try {
- // Deleting temporary table and dataset that gets generated when executing a query.
- if (temporaryDatasetId != null) {
- if (temporaryTableId != null) {
- deleteTable(temporaryDatasetId, temporaryTableId);
- }
- deleteDataset(temporaryDatasetId);
- }
- } catch (IOException | InterruptedException e) {
- if (e instanceof InterruptedException) {
- Thread.currentThread().interrupt();
- }
- throw new RuntimeException(e);
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b240525a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
index a616f5a..ca3f0ba 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
@@ -19,13 +19,12 @@ package org.apache.beam.sdk.values;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
+import org.apache.beam.sdk.io.CountingInput;
import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
@@ -40,19 +39,15 @@ import org.apache.beam.sdk.util.WindowingStrategy;
* be passed as the inputs of other PTransforms.
*
* <p>Some root transforms produce bounded {@code PCollections} and others
- * produce unbounded ones. For example, {@link TextIO.Read} reads a static set
- * of files, so it produces a bounded {@link PCollection}.
- * {@link PubsubIO.Read}, on the other hand, receives a potentially infinite stream
- * of Pubsub messages, so it produces an unbounded {@link PCollection}.
+ * produce unbounded ones. For example, {@link CountingInput#upTo} produces a fixed set of integers,
+ * so it produces a bounded {@link PCollection}. {@link CountingInput#unbounded} produces all
+ * integers as an infinite stream, so it produces an unbounded {@link PCollection}.
*
- * <p>Each element in a {@link PCollection} may have an associated implicit
- * timestamp. Readers assign timestamps to elements when they create
- * {@link PCollection PCollections}, and other {@link PTransform PTransforms} propagate these
- * timestamps from their input to their output. For example, {@link PubsubIO.Read}
- * assigns pubsub message timestamps to elements, and {@link TextIO.Read} assigns
- * the default value {@link BoundedWindow#TIMESTAMP_MIN_VALUE} to elements. User code can
- * explicitly assign timestamps to elements with
- * {@link org.apache.beam.sdk.transforms.DoFn.Context#outputWithTimestamp}.
+ * <p>Each element in a {@link PCollection} has an associated timestamp. Readers assign timestamps
+ * to elements when they create {@link PCollection PCollections}, and other
+ * {@link PTransform PTransforms} propagate these timestamps from their input to their output. See
+ * the documentation on {@link BoundedReader} and {@link UnboundedReader} for more information on
+ * how these readers produce timestamps and watermarks.
*
* <p>Additionally, a {@link PCollection} has an associated
* {@link WindowFn} and each element is assigned to a set of windows.
@@ -73,14 +68,11 @@ public class PCollection<T> extends TypedPValue<T> {
*/
public enum IsBounded {
/**
- * Indicates that a {@link PCollection} contains bounded data elements, such as
- * {@link PCollection PCollections} from {@link TextIO}, {@link BigQueryIO},
- * {@link Create} e.t.c.
+ * Indicates that a {@link PCollection} contains a bounded number of elements.
*/
BOUNDED,
/**
- * Indicates that a {@link PCollection} contains unbounded data elements, such as
- * {@link PCollection PCollections} from {@link PubsubIO}.
+ * Indicates that a {@link PCollection} contains an unbounded number of elements.
*/
UNBOUNDED;