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;