You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by lc...@apache.org on 2017/04/18 23:19:27 UTC

[1/7] beam git commit: [BEAM-1871] Create new GCP core module package and move several GCP related classes from beam-sdks-java-core over.

Repository: beam
Updated Branches:
  refs/heads/master ed52d328d -> 6d3fe51ce


http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
new file mode 100644
index 0000000..03668ce
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
@@ -0,0 +1,798 @@
+/*
+ * 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.contains;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.when;
+
+import com.google.api.client.googleapis.batch.BatchRequest;
+import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpResponse;
+import com.google.api.client.http.HttpStatusCodes;
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.http.LowLevelHttpRequest;
+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.JsonFactory;
+import com.google.api.client.json.jackson2.JacksonFactory;
+import com.google.api.client.testing.http.HttpTesting;
+import com.google.api.client.testing.http.MockHttpTransport;
+import com.google.api.client.testing.http.MockLowLevelHttpRequest;
+import com.google.api.client.testing.http.MockLowLevelHttpResponse;
+import com.google.api.client.util.BackOff;
+import com.google.api.services.storage.Storage;
+import com.google.api.services.storage.model.Bucket;
+import com.google.api.services.storage.model.Objects;
+import com.google.api.services.storage.model.StorageObject;
+import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel;
+import com.google.cloud.hadoop.util.ClientRequestHelper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import java.io.ByteArrayInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.math.BigInteger;
+import java.net.SocketTimeoutException;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.AccessDeniedException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
+import org.apache.beam.sdk.util.GcsUtil.StorageObjectOrIOException;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+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.Mockito;
+
+/** Test case for {@link GcsUtil}. */
+@RunWith(JUnit4.class)
+public class GcsUtilTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testGlobTranslation() {
+    assertEquals("foo", GcsUtil.globToRegexp("foo"));
+    assertEquals("fo[^/]*o", GcsUtil.globToRegexp("fo*o"));
+    assertEquals("f[^/]*o\\.[^/]", GcsUtil.globToRegexp("f*o.?"));
+    assertEquals("foo-[0-9][^/]*", GcsUtil.globToRegexp("foo-[0-9]*"));
+  }
+
+  private static GcsOptions gcsOptionsWithTestCredential() {
+    GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class);
+    pipelineOptions.setGcpCredential(new TestCredential());
+    return pipelineOptions;
+  }
+
+  @Test
+  public void testCreationWithDefaultOptions() {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    assertNotNull(pipelineOptions.getGcpCredential());
+  }
+
+  @Test
+  public void testUploadBufferSizeDefault() {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil util = pipelineOptions.getGcsUtil();
+    assertNull(util.getUploadBufferSizeBytes());
+  }
+
+  @Test
+  public void testUploadBufferSizeUserSpecified() {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    pipelineOptions.setGcsUploadBufferSizeBytes(12345);
+    GcsUtil util = pipelineOptions.getGcsUtil();
+    assertEquals((Integer) 12345, util.getUploadBufferSizeBytes());
+  }
+
+  @Test
+  public void testCreationWithExecutorServiceProvided() {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    pipelineOptions.setExecutorService(Executors.newCachedThreadPool());
+    assertSame(pipelineOptions.getExecutorService(), pipelineOptions.getGcsUtil().executorService);
+  }
+
+  @Test
+  public void testCreationWithGcsUtilProvided() {
+    GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class);
+    GcsUtil gcsUtil = Mockito.mock(GcsUtil.class);
+    pipelineOptions.setGcsUtil(gcsUtil);
+    assertSame(gcsUtil, pipelineOptions.getGcsUtil());
+  }
+
+  @Test
+  public void testMultipleThreadsCanCompleteOutOfOrderWithDefaultThreadPool() throws Exception {
+    GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class);
+    ExecutorService executorService = pipelineOptions.getExecutorService();
+
+    int numThreads = 100;
+    final CountDownLatch[] countDownLatches = new CountDownLatch[numThreads];
+    for (int i = 0; i < numThreads; i++) {
+      final int currentLatch = i;
+      countDownLatches[i] = new CountDownLatch(1);
+      executorService.execute(
+          new Runnable() {
+            @Override
+            public void run() {
+              // Wait for latch N and then release latch N - 1
+              try {
+                countDownLatches[currentLatch].await();
+                if (currentLatch > 0) {
+                  countDownLatches[currentLatch - 1].countDown();
+                }
+              } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException(e);
+              }
+            }
+          });
+    }
+
+    // Release the last latch starting the chain reaction.
+    countDownLatches[countDownLatches.length - 1].countDown();
+    executorService.shutdown();
+    assertTrue("Expected tasks to complete",
+        executorService.awaitTermination(10, TimeUnit.SECONDS));
+  }
+
+  @Test
+  public void testGlobExpansion() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
+    Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
+    Storage.Objects.List mockStorageList = Mockito.mock(Storage.Objects.List.class);
+
+    Objects modelObjects = new Objects();
+    List<StorageObject> items = new ArrayList<>();
+    // A directory
+    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/"));
+
+    // Files within the directory
+    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/file1name"));
+    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/file2name"));
+    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/file3name"));
+    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/otherfile"));
+    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/anotherfile"));
+
+    modelObjects.setItems(items);
+
+    when(mockStorage.objects()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.get("testbucket", "testdirectory/otherfile")).thenReturn(
+        mockStorageGet);
+    when(mockStorageObjects.list("testbucket")).thenReturn(mockStorageList);
+    when(mockStorageGet.execute()).thenReturn(
+        new StorageObject().setBucket("testbucket").setName("testdirectory/otherfile"));
+    when(mockStorageList.execute()).thenReturn(modelObjects);
+
+    // Test a single file.
+    {
+      GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/otherfile");
+      List<GcsPath> expectedFiles =
+          ImmutableList.of(GcsPath.fromUri("gs://testbucket/testdirectory/otherfile"));
+
+      assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray()));
+    }
+
+    // Test patterns.
+    {
+      GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/file*");
+      List<GcsPath> expectedFiles = ImmutableList.of(
+          GcsPath.fromUri("gs://testbucket/testdirectory/file1name"),
+          GcsPath.fromUri("gs://testbucket/testdirectory/file2name"),
+          GcsPath.fromUri("gs://testbucket/testdirectory/file3name"));
+
+      assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray()));
+    }
+
+    {
+      GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/file[1-3]*");
+      List<GcsPath> expectedFiles = ImmutableList.of(
+          GcsPath.fromUri("gs://testbucket/testdirectory/file1name"),
+          GcsPath.fromUri("gs://testbucket/testdirectory/file2name"),
+          GcsPath.fromUri("gs://testbucket/testdirectory/file3name"));
+
+      assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray()));
+    }
+
+    {
+      GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/file?name");
+      List<GcsPath> expectedFiles = ImmutableList.of(
+          GcsPath.fromUri("gs://testbucket/testdirectory/file1name"),
+          GcsPath.fromUri("gs://testbucket/testdirectory/file2name"),
+          GcsPath.fromUri("gs://testbucket/testdirectory/file3name"));
+
+      assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray()));
+    }
+
+    {
+      GcsPath pattern = GcsPath.fromUri("gs://testbucket/test*ectory/fi*name");
+      List<GcsPath> expectedFiles = ImmutableList.of(
+          GcsPath.fromUri("gs://testbucket/testdirectory/file1name"),
+          GcsPath.fromUri("gs://testbucket/testdirectory/file2name"),
+          GcsPath.fromUri("gs://testbucket/testdirectory/file3name"));
+
+      assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray()));
+    }
+  }
+
+  // Patterns that contain recursive wildcards ('**') are not supported.
+  @Test
+  public void testRecursiveGlobExpansionFails() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+    GcsPath pattern = GcsPath.fromUri("gs://testbucket/test**");
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Unsupported wildcard usage");
+    gcsUtil.expand(pattern);
+  }
+
+  // GCSUtil.expand() should fail when matching a single object when that object does not exist.
+  // We should return the empty result since GCS get object is strongly consistent.
+  @Test
+  public void testNonExistentObjectReturnsEmptyResult() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
+    Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
+
+    GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/nonexistentfile");
+    GoogleJsonResponseException expectedException =
+        googleJsonResponseException(HttpStatusCodes.STATUS_CODE_NOT_FOUND,
+            "It don't exist", "Nothing here to see");
+
+    when(mockStorage.objects()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.get(pattern.getBucket(), pattern.getObject())).thenReturn(
+        mockStorageGet);
+    when(mockStorageGet.execute()).thenThrow(expectedException);
+
+    assertEquals(Collections.EMPTY_LIST, gcsUtil.expand(pattern));
+  }
+
+  // GCSUtil.expand() should fail for other errors such as access denied.
+  @Test
+  public void testAccessDeniedObjectThrowsIOException() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
+    Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
+
+    GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/accessdeniedfile");
+    GoogleJsonResponseException expectedException =
+        googleJsonResponseException(HttpStatusCodes.STATUS_CODE_FORBIDDEN,
+            "Waves hand mysteriously", "These aren't the buckets you're looking for");
+
+    when(mockStorage.objects()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.get(pattern.getBucket(), pattern.getObject())).thenReturn(
+        mockStorageGet);
+    when(mockStorageGet.execute()).thenThrow(expectedException);
+
+    thrown.expect(IOException.class);
+    thrown.expectMessage("Unable to get the file object for path");
+    gcsUtil.expand(pattern);
+  }
+
+  @Test
+  public void testFileSizeNonBatch() throws Exception {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
+    Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
+
+    when(mockStorage.objects()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet);
+    when(mockStorageGet.execute()).thenReturn(
+            new StorageObject().setSize(BigInteger.valueOf(1000)));
+
+    assertEquals(1000, gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject")));
+  }
+
+  @Test
+  public void testFileSizeWhenFileNotFoundNonBatch() throws Exception {
+    MockLowLevelHttpResponse notFoundResponse = new MockLowLevelHttpResponse();
+    notFoundResponse.setContent("");
+    notFoundResponse.setStatusCode(HttpStatusCodes.STATUS_CODE_NOT_FOUND);
+
+    MockHttpTransport mockTransport =
+            new MockHttpTransport.Builder().setLowLevelHttpResponse(notFoundResponse).build();
+
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    gcsUtil.setStorageClient(new Storage(mockTransport, Transport.getJsonFactory(), null));
+
+    thrown.expect(FileNotFoundException.class);
+    gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject"));
+  }
+
+  @Test
+  public void testRetryFileSizeNonBatch() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
+    Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
+
+    BackOff mockBackOff = FluentBackoff.DEFAULT.withMaxRetries(2).backoff();
+
+    when(mockStorage.objects()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet);
+    when(mockStorageGet.execute())
+            .thenThrow(new SocketTimeoutException("SocketException"))
+            .thenThrow(new SocketTimeoutException("SocketException"))
+            .thenReturn(new StorageObject().setSize(BigInteger.valueOf(1000)));
+
+    assertEquals(1000,
+        gcsUtil.getObject(
+            GcsPath.fromComponents("testbucket", "testobject"),
+            mockBackOff,
+            new FastNanoClockAndSleeper()).getSize().longValue());
+    assertEquals(BackOff.STOP, mockBackOff.nextBackOffMillis());
+  }
+
+  @Test
+  public void testGetSizeBytesWhenFileNotFoundBatch() throws Exception {
+    JsonFactory jsonFactory = new JacksonFactory();
+
+    String contentBoundary = "batch_foobarbaz";
+    String contentBoundaryLine = "--" + contentBoundary;
+    String endOfContentBoundaryLine = "--" + contentBoundary + "--";
+
+    GenericJson error = new GenericJson()
+        .set("error", new GenericJson().set("code", 404));
+    error.setFactory(jsonFactory);
+
+    String content = contentBoundaryLine + "\n"
+        + "Content-Type: application/http\n"
+        + "\n"
+        + "HTTP/1.1 404 Not Found\n"
+        + "Content-Length: -1\n"
+        + "\n"
+        + error.toString()
+        + "\n"
+        + "\n"
+        + endOfContentBoundaryLine
+        + "\n";
+    thrown.expect(FileNotFoundException.class);
+    MockLowLevelHttpResponse notFoundResponse = new MockLowLevelHttpResponse()
+        .setContentType("multipart/mixed; boundary=" + contentBoundary)
+        .setContent(content)
+        .setStatusCode(HttpStatusCodes.STATUS_CODE_OK);
+
+    MockHttpTransport mockTransport =
+        new MockHttpTransport.Builder().setLowLevelHttpResponse(notFoundResponse).build();
+
+    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
+
+    gcsUtil.setStorageClient(new Storage(mockTransport, Transport.getJsonFactory(), null));
+    gcsUtil.fileSizes(ImmutableList.of(GcsPath.fromComponents("testbucket", "testobject")));
+  }
+
+  @Test
+  public void testGetSizeBytesWhenFileNotFoundBatchRetry() throws Exception {
+    JsonFactory jsonFactory = new JacksonFactory();
+
+    String contentBoundary = "batch_foobarbaz";
+    String contentBoundaryLine = "--" + contentBoundary;
+    String endOfContentBoundaryLine = "--" + contentBoundary + "--";
+
+    GenericJson error = new GenericJson()
+        .set("error", new GenericJson().set("code", 404));
+    error.setFactory(jsonFactory);
+
+    String content = contentBoundaryLine + "\n"
+        + "Content-Type: application/http\n"
+        + "\n"
+        + "HTTP/1.1 404 Not Found\n"
+        + "Content-Length: -1\n"
+        + "\n"
+        + error.toString()
+        + "\n"
+        + "\n"
+        + endOfContentBoundaryLine
+        + "\n";
+    thrown.expect(FileNotFoundException.class);
+
+    final LowLevelHttpResponse mockResponse = Mockito.mock(LowLevelHttpResponse.class);
+    when(mockResponse.getContentType()).thenReturn("multipart/mixed; boundary=" + contentBoundary);
+
+    // 429: Too many requests, then 200: OK.
+    when(mockResponse.getStatusCode()).thenReturn(429, 200);
+    when(mockResponse.getContent()).thenReturn(toStream("error"), toStream(content));
+
+    // A mock transport that lets us mock the API responses.
+    MockHttpTransport mockTransport =
+        new MockHttpTransport.Builder()
+            .setLowLevelHttpRequest(
+                new MockLowLevelHttpRequest() {
+                  @Override
+                  public LowLevelHttpResponse execute() throws IOException {
+                    return mockResponse;
+                  }
+                })
+            .build();
+
+    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
+
+        gcsUtil.setStorageClient(
+        new Storage(mockTransport, Transport.getJsonFactory(), new RetryHttpRequestInitializer()));
+    gcsUtil.fileSizes(ImmutableList.of(GcsPath.fromComponents("testbucket", "testobject")));
+  }
+
+  @Test
+  public void testCreateBucket() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Buckets.Insert mockStorageInsert = Mockito.mock(Storage.Buckets.Insert.class);
+
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
+
+    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.insert(
+           any(String.class), any(Bucket.class))).thenReturn(mockStorageInsert);
+    when(mockStorageInsert.execute())
+        .thenThrow(new SocketTimeoutException("SocketException"))
+        .thenReturn(new Bucket());
+
+    gcsUtil.createBucket("a", new Bucket(), mockBackOff, new FastNanoClockAndSleeper());
+  }
+
+  @Test
+  public void testCreateBucketAccessErrors() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
+    Storage.Buckets.Insert mockStorageInsert = Mockito.mock(Storage.Buckets.Insert.class);
+
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
+    GoogleJsonResponseException expectedException =
+        googleJsonResponseException(HttpStatusCodes.STATUS_CODE_FORBIDDEN,
+            "Waves hand mysteriously", "These aren't the buckets you're looking for");
+
+    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.insert(
+           any(String.class), any(Bucket.class))).thenReturn(mockStorageInsert);
+    when(mockStorageInsert.execute())
+        .thenThrow(expectedException);
+
+    thrown.expect(AccessDeniedException.class);
+
+    gcsUtil.createBucket("a", new Bucket(), mockBackOff, new FastNanoClockAndSleeper());
+  }
+
+  @Test
+  public void testBucketAccessible() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
+    Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
+
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
+
+    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
+    when(mockStorageGet.execute())
+        .thenThrow(new SocketTimeoutException("SocketException"))
+        .thenReturn(new Bucket());
+
+    assertTrue(gcsUtil.bucketAccessible(GcsPath.fromComponents("testbucket", "testobject"),
+        mockBackOff, new FastNanoClockAndSleeper()));
+  }
+
+  @Test
+  public void testBucketDoesNotExistBecauseOfAccessError() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
+    Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
+
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
+    GoogleJsonResponseException expectedException =
+        googleJsonResponseException(HttpStatusCodes.STATUS_CODE_FORBIDDEN,
+            "Waves hand mysteriously", "These aren't the buckets you're looking for");
+
+    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
+    when(mockStorageGet.execute())
+        .thenThrow(expectedException);
+
+    assertFalse(gcsUtil.bucketAccessible(GcsPath.fromComponents("testbucket", "testobject"),
+        mockBackOff, new FastNanoClockAndSleeper()));
+  }
+
+  @Test
+  public void testBucketDoesNotExist() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
+    Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
+
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
+
+    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
+    when(mockStorageGet.execute())
+        .thenThrow(googleJsonResponseException(HttpStatusCodes.STATUS_CODE_NOT_FOUND,
+            "It don't exist", "Nothing here to see"));
+
+    assertFalse(gcsUtil.bucketAccessible(GcsPath.fromComponents("testbucket", "testobject"),
+        mockBackOff, new FastNanoClockAndSleeper()));
+  }
+
+  @Test
+  public void testGetBucket() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
+    Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
+
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
+
+    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
+    when(mockStorageGet.execute())
+        .thenThrow(new SocketTimeoutException("SocketException"))
+        .thenReturn(new Bucket());
+
+    assertNotNull(gcsUtil.getBucket(GcsPath.fromComponents("testbucket", "testobject"),
+        mockBackOff, new FastNanoClockAndSleeper()));
+  }
+
+  @Test
+  public void testGetBucketNotExists() throws IOException {
+    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
+    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
+
+    Storage mockStorage = Mockito.mock(Storage.class);
+    gcsUtil.setStorageClient(mockStorage);
+
+    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
+    Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
+
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
+
+    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
+    when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
+    when(mockStorageGet.execute())
+        .thenThrow(googleJsonResponseException(HttpStatusCodes.STATUS_CODE_NOT_FOUND,
+            "It don't exist", "Nothing here to see"));
+
+    thrown.expect(FileNotFoundException.class);
+    thrown.expectMessage("It don't exist");
+    gcsUtil.getBucket(GcsPath.fromComponents("testbucket", "testobject"),
+                      mockBackOff, new FastNanoClockAndSleeper());
+  }
+
+  @Test
+  public void testGCSChannelCloseIdempotent() throws IOException {
+    SeekableByteChannel channel =
+        new GoogleCloudStorageReadChannel(null, "dummybucket", "dummyobject", null,
+        new ClientRequestHelper<StorageObject>());
+    channel.close();
+    channel.close();
+  }
+
+  /**
+   * Builds a fake GoogleJsonResponseException for testing API error handling.
+   */
+  private static GoogleJsonResponseException googleJsonResponseException(
+      final int status, final String reason, final String message) throws IOException {
+    final JsonFactory jsonFactory = new JacksonFactory();
+    HttpTransport transport = new MockHttpTransport() {
+      @Override
+      public LowLevelHttpRequest buildRequest(String method, String url) throws IOException {
+        ErrorInfo errorInfo = new ErrorInfo();
+        errorInfo.setReason(reason);
+        errorInfo.setMessage(message);
+        errorInfo.setFactory(jsonFactory);
+        GenericJson error = new GenericJson();
+        error.set("code", status);
+        error.set("errors", Arrays.asList(errorInfo));
+        error.setFactory(jsonFactory);
+        GenericJson errorResponse = new GenericJson();
+        errorResponse.set("error", error);
+        errorResponse.setFactory(jsonFactory);
+        return new MockLowLevelHttpRequest().setResponse(
+            new MockLowLevelHttpResponse().setContent(errorResponse.toPrettyString())
+            .setContentType(Json.MEDIA_TYPE).setStatusCode(status));
+        }
+    };
+    HttpRequest request =
+        transport.createRequestFactory().buildGetRequest(HttpTesting.SIMPLE_GENERIC_URL);
+    request.setThrowExceptionOnExecuteError(false);
+    HttpResponse response = request.execute();
+    return GoogleJsonResponseException.from(jsonFactory, response);
+  }
+
+  private static List<String> makeStrings(String s, int n) {
+    ImmutableList.Builder<String> ret = ImmutableList.builder();
+    for (int i = 0; i < n; ++i) {
+      ret.add(String.format("gs://bucket/%s%d", s, i));
+    }
+    return ret.build();
+  }
+
+  private static List<GcsPath> makeGcsPaths(String s, int n) {
+    ImmutableList.Builder<GcsPath> ret = ImmutableList.builder();
+    for (int i = 0; i < n; ++i) {
+      ret.add(GcsPath.fromUri(String.format("gs://bucket/%s%d", s, i)));
+    }
+    return ret.build();
+  }
+
+  private static int sumBatchSizes(List<BatchRequest> batches) {
+    int ret = 0;
+    for (BatchRequest b : batches) {
+      ret += b.size();
+      assertThat(b.size(), greaterThan(0));
+    }
+    return ret;
+  }
+
+  @Test
+  public void testMakeCopyBatches() throws IOException {
+    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
+
+    // Small number of files fits in 1 batch
+    List<BatchRequest> batches = gcsUtil.makeCopyBatches(makeStrings("s", 3), makeStrings("d", 3));
+    assertThat(batches.size(), equalTo(1));
+    assertThat(sumBatchSizes(batches), equalTo(3));
+
+    // 1 batch of files fits in 1 batch
+    batches = gcsUtil.makeCopyBatches(makeStrings("s", 100), makeStrings("d", 100));
+    assertThat(batches.size(), equalTo(1));
+    assertThat(sumBatchSizes(batches), equalTo(100));
+
+    // A little more than 5 batches of files fits in 6 batches
+    batches = gcsUtil.makeCopyBatches(makeStrings("s", 501), makeStrings("d", 501));
+    assertThat(batches.size(), equalTo(6));
+    assertThat(sumBatchSizes(batches), equalTo(501));
+  }
+
+  @Test
+  public void testInvalidCopyBatches() throws IOException {
+    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Number of source files 3");
+
+    gcsUtil.makeCopyBatches(makeStrings("s", 3), makeStrings("d", 1));
+  }
+
+  @Test
+  public void testMakeRemoveBatches() throws IOException {
+    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
+
+    // Small number of files fits in 1 batch
+    List<BatchRequest> batches = gcsUtil.makeRemoveBatches(makeStrings("s", 3));
+    assertThat(batches.size(), equalTo(1));
+    assertThat(sumBatchSizes(batches), equalTo(3));
+
+    // 1 batch of files fits in 1 batch
+    batches = gcsUtil.makeRemoveBatches(makeStrings("s", 100));
+    assertThat(batches.size(), equalTo(1));
+    assertThat(sumBatchSizes(batches), equalTo(100));
+
+    // A little more than 5 batches of files fits in 6 batches
+    batches = gcsUtil.makeRemoveBatches(makeStrings("s", 501));
+    assertThat(batches.size(), equalTo(6));
+    assertThat(sumBatchSizes(batches), equalTo(501));
+  }
+
+  @Test
+  public void testMakeGetBatches() throws IOException {
+    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
+
+    // Small number of files fits in 1 batch
+    List<StorageObjectOrIOException[]> results = Lists.newArrayList();
+    List<BatchRequest> batches = gcsUtil.makeGetBatches(makeGcsPaths("s", 3), results);
+    assertThat(batches.size(), equalTo(1));
+    assertThat(sumBatchSizes(batches), equalTo(3));
+    assertEquals(3, results.size());
+
+    // 1 batch of files fits in 1 batch
+    results = Lists.newArrayList();
+    batches = gcsUtil.makeGetBatches(makeGcsPaths("s", 100), results);
+    assertThat(batches.size(), equalTo(1));
+    assertThat(sumBatchSizes(batches), equalTo(100));
+    assertEquals(100, results.size());
+
+    // A little more than 5 batches of files fits in 6 batches
+    results = Lists.newArrayList();
+    batches = gcsUtil.makeGetBatches(makeGcsPaths("s", 501), results);
+    assertThat(batches.size(), equalTo(6));
+    assertThat(sumBatchSizes(batches), equalTo(501));
+    assertEquals(501, results.size());
+  }
+
+  /**
+   * A helper to wrap a {@link GenericJson} object in a content stream.
+   */
+  private static InputStream toStream(String content) throws IOException {
+    return new ByteArrayInputStream(content.getBytes(StandardCharsets.UTF_8));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java
new file mode 100644
index 0000000..8e7878c
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.lessThan;
+import static org.hamcrest.Matchers.lessThanOrEqualTo;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Unit tests for {@link IntervalBoundedExponentialBackOff}. */
+@RunWith(JUnit4.class)
+public class IntervalBoundedExponentialBackOffTest {
+  @Rule public ExpectedException exception = ExpectedException.none();
+
+
+  @Test
+  public void testUsingInvalidInitialInterval() throws Exception {
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage("Initial interval must be greater than zero.");
+    new IntervalBoundedExponentialBackOff(1000L, 0L);
+  }
+
+  @Test
+  public void testUsingInvalidMaximumInterval() throws Exception {
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage("Maximum interval must be greater than zero.");
+    new IntervalBoundedExponentialBackOff(-1L, 10L);
+  }
+
+  @Test
+  public void testThatcertainNumberOfAttemptsReachesMaxInterval() throws Exception {
+    IntervalBoundedExponentialBackOff backOff = new IntervalBoundedExponentialBackOff(1000L, 500);
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+  }
+
+  @Test
+  public void testThatResettingAllowsReuse() throws Exception {
+    IntervalBoundedExponentialBackOff backOff = new IntervalBoundedExponentialBackOff(1000L, 500);
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    backOff.reset();
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+  }
+
+  @Test
+  public void testAtMaxInterval() throws Exception {
+    IntervalBoundedExponentialBackOff backOff = new IntervalBoundedExponentialBackOff(1000L, 500);
+    assertFalse(backOff.atMaxInterval());
+    backOff.nextBackOffMillis();
+    assertFalse(backOff.atMaxInterval());
+    backOff.nextBackOffMillis();
+    assertTrue(backOff.atMaxInterval());
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
new file mode 100644
index 0000000..71554b5
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
@@ -0,0 +1,290 @@
+/*
+ * 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.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyString;
+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 com.google.api.client.auth.oauth2.Credential;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpResponse;
+import com.google.api.client.http.HttpResponseException;
+import com.google.api.client.http.HttpResponseInterceptor;
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.http.LowLevelHttpRequest;
+import com.google.api.client.http.LowLevelHttpResponse;
+import com.google.api.client.json.JsonFactory;
+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.NanoClock;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.storage.Storage;
+import com.google.api.services.storage.Storage.Objects.Get;
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.security.PrivateKey;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicLong;
+import org.hamcrest.Matchers;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+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;
+
+/**
+ * Tests for RetryHttpRequestInitializer.
+ */
+@RunWith(JUnit4.class)
+public class RetryHttpRequestInitializerTest {
+
+  @Mock private Credential mockCredential;
+  @Mock private PrivateKey mockPrivateKey;
+  @Mock private LowLevelHttpRequest mockLowLevelRequest;
+  @Mock private LowLevelHttpResponse mockLowLevelResponse;
+  @Mock private HttpResponseInterceptor mockHttpResponseInterceptor;
+
+  private final JsonFactory jsonFactory = JacksonFactory.getDefaultInstance();
+  private Storage storage;
+
+  // Used to test retrying a request more than the default 10 times.
+  static class MockNanoClock implements NanoClock {
+    private int timesMs[] = {500, 750, 1125, 1688, 2531, 3797, 5695, 8543,
+        12814, 19222, 28833, 43249, 64873, 97310, 145965, 218945, 328420};
+    private int i = 0;
+
+    @Override
+    public long nanoTime() {
+      return timesMs[i++ / 2] * 1000000;
+    }
+  }
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+
+    HttpTransport lowLevelTransport = new HttpTransport() {
+      @Override
+      protected LowLevelHttpRequest buildRequest(String method, String url)
+          throws IOException {
+        return mockLowLevelRequest;
+      }
+    };
+
+    // Retry initializer will pass through to credential, since we can have
+    // only a single HttpRequestInitializer, and we use multiple Credential
+    // types in the SDK, not all of which allow for retry configuration.
+    RetryHttpRequestInitializer initializer = new RetryHttpRequestInitializer(
+        mockCredential, new MockNanoClock(), new Sleeper() {
+          @Override
+          public void sleep(long millis) throws InterruptedException {}
+        }, Arrays.asList(418 /* I'm a teapot */), mockHttpResponseInterceptor);
+    storage = new Storage.Builder(lowLevelTransport, jsonFactory, initializer)
+        .setApplicationName("test").build();
+  }
+
+  @After
+  public void tearDown() {
+    verifyNoMoreInteractions(mockPrivateKey);
+    verifyNoMoreInteractions(mockLowLevelRequest);
+    verifyNoMoreInteractions(mockCredential);
+    verifyNoMoreInteractions(mockHttpResponseInterceptor);
+  }
+
+  @Test
+  public void testBasicOperation() throws IOException {
+    when(mockLowLevelRequest.execute())
+        .thenReturn(mockLowLevelResponse);
+    when(mockLowLevelResponse.getStatusCode())
+        .thenReturn(200);
+
+    Storage.Buckets.Get result = storage.buckets().get("test");
+    HttpResponse response = result.executeUnparsed();
+    assertNotNull(response);
+
+    verify(mockCredential).initialize(any(HttpRequest.class));
+    verify(mockHttpResponseInterceptor).interceptResponse(any(HttpResponse.class));
+    verify(mockLowLevelRequest, atLeastOnce())
+        .addHeader(anyString(), anyString());
+    verify(mockLowLevelRequest).setTimeout(anyInt(), anyInt());
+    verify(mockLowLevelRequest).execute();
+    verify(mockLowLevelResponse).getStatusCode();
+  }
+
+  /**
+   * Tests that a non-retriable error is not retried.
+   */
+  @Test
+  public void testErrorCodeForbidden() throws IOException {
+    when(mockLowLevelRequest.execute())
+        .thenReturn(mockLowLevelResponse);
+    when(mockLowLevelResponse.getStatusCode())
+        .thenReturn(403)  // Non-retryable error.
+        .thenReturn(200); // Shouldn't happen.
+
+    try {
+      Storage.Buckets.Get result = storage.buckets().get("test");
+      HttpResponse response = result.executeUnparsed();
+      assertNotNull(response);
+    } catch (HttpResponseException e) {
+      Assert.assertThat(e.getMessage(), Matchers.containsString("403"));
+    }
+
+    verify(mockCredential).initialize(any(HttpRequest.class));
+    verify(mockHttpResponseInterceptor).interceptResponse(any(HttpResponse.class));
+    verify(mockLowLevelRequest, atLeastOnce())
+        .addHeader(anyString(), anyString());
+    verify(mockLowLevelRequest).setTimeout(anyInt(), anyInt());
+    verify(mockLowLevelRequest).execute();
+    verify(mockLowLevelResponse).getStatusCode();
+  }
+
+  /**
+   * Tests that a retriable error is retried.
+   */
+  @Test
+  public void testRetryableError() throws IOException {
+    when(mockLowLevelRequest.execute())
+        .thenReturn(mockLowLevelResponse)
+        .thenReturn(mockLowLevelResponse)
+        .thenReturn(mockLowLevelResponse);
+    when(mockLowLevelResponse.getStatusCode())
+        .thenReturn(503)  // Retryable
+        .thenReturn(429)  // We also retry on 429 Too Many Requests.
+        .thenReturn(200);
+
+    Storage.Buckets.Get result = storage.buckets().get("test");
+    HttpResponse response = result.executeUnparsed();
+    assertNotNull(response);
+
+    verify(mockCredential).initialize(any(HttpRequest.class));
+    verify(mockHttpResponseInterceptor).interceptResponse(any(HttpResponse.class));
+    verify(mockLowLevelRequest, atLeastOnce())
+        .addHeader(anyString(), anyString());
+    verify(mockLowLevelRequest, times(3)).setTimeout(anyInt(), anyInt());
+    verify(mockLowLevelRequest, times(3)).execute();
+    verify(mockLowLevelResponse, times(3)).getStatusCode();
+  }
+
+  /**
+   * Tests that an IOException is retried.
+   */
+  @Test
+  public void testThrowIOException() throws IOException {
+    when(mockLowLevelRequest.execute())
+        .thenThrow(new IOException("Fake Error"))
+        .thenReturn(mockLowLevelResponse);
+    when(mockLowLevelResponse.getStatusCode())
+        .thenReturn(200);
+
+    Storage.Buckets.Get result = storage.buckets().get("test");
+    HttpResponse response = result.executeUnparsed();
+    assertNotNull(response);
+
+    verify(mockCredential).initialize(any(HttpRequest.class));
+    verify(mockHttpResponseInterceptor).interceptResponse(any(HttpResponse.class));
+    verify(mockLowLevelRequest, atLeastOnce())
+        .addHeader(anyString(), anyString());
+    verify(mockLowLevelRequest, times(2)).setTimeout(anyInt(), anyInt());
+    verify(mockLowLevelRequest, times(2)).execute();
+    verify(mockLowLevelResponse).getStatusCode();
+  }
+
+  /**
+   * Tests that a retryable error is retried enough times.
+   */
+  @Test
+  public void testRetryableErrorRetryEnoughTimes() throws IOException {
+    when(mockLowLevelRequest.execute()).thenReturn(mockLowLevelResponse);
+    final int retries = 10;
+    when(mockLowLevelResponse.getStatusCode()).thenAnswer(new Answer<Integer>(){
+      int n = 0;
+      @Override
+      public Integer answer(InvocationOnMock invocation) {
+        return (n++ < retries - 1) ? 503 : 200;
+      }});
+
+    Storage.Buckets.Get result = storage.buckets().get("test");
+    HttpResponse response = result.executeUnparsed();
+    assertNotNull(response);
+
+    verify(mockCredential).initialize(any(HttpRequest.class));
+    verify(mockHttpResponseInterceptor).interceptResponse(any(HttpResponse.class));
+    verify(mockLowLevelRequest, atLeastOnce()).addHeader(anyString(),
+        anyString());
+    verify(mockLowLevelRequest, times(retries)).setTimeout(anyInt(), anyInt());
+    verify(mockLowLevelRequest, times(retries)).execute();
+    verify(mockLowLevelResponse, times(retries)).getStatusCode();
+  }
+
+  /**
+   * Tests that when RPCs fail with {@link SocketTimeoutException}, the IO exception handler
+   * is invoked.
+   */
+  @Test
+  public void testIOExceptionHandlerIsInvokedOnTimeout() throws Exception {
+    // Counts the number of calls to execute the HTTP request.
+    final AtomicLong executeCount = new AtomicLong();
+
+    // 10 is a private internal constant in the Google API Client library. See
+    // com.google.api.client.http.HttpRequest#setNumberOfRetries
+    // TODO: update this test once the private internal constant is public.
+    final int defaultNumberOfRetries = 10;
+
+    // A mock HTTP request that always throws SocketTimeoutException.
+    MockHttpTransport transport =
+        new MockHttpTransport.Builder().setLowLevelHttpRequest(new MockLowLevelHttpRequest() {
+          @Override
+          public LowLevelHttpResponse execute() throws IOException {
+            executeCount.incrementAndGet();
+            throw new SocketTimeoutException("Fake forced timeout exception");
+          }
+        }).build();
+
+    // A sample HTTP request to Google Cloud Storage that uses both default Transport and default
+    // RetryHttpInitializer.
+    Storage storage = new Storage.Builder(
+        transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer()).build();
+
+    Get getRequest = storage.objects().get("gs://fake", "file");
+
+    try {
+      getRequest.execute();
+      fail();
+    } catch (Throwable e) {
+      assertThat(e, Matchers.<Throwable>instanceOf(SocketTimeoutException.class));
+      assertEquals(1 + defaultNumberOfRetries, executeCount.get());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/pom.xml b/sdks/java/extensions/pom.xml
index 74aba0f..dde8be5 100644
--- a/sdks/java/extensions/pom.xml
+++ b/sdks/java/extensions/pom.xml
@@ -32,6 +32,7 @@
   <name>Apache Beam :: SDKs :: Java :: Extensions</name>
 
   <modules>
+    <module>gcp-core</module>
     <module>jackson</module>
     <module>join-library</module>
     <module>sorter</module>

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/harness/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/harness/pom.xml b/sdks/java/harness/pom.xml
index 3d14401..76ceb3d 100644
--- a/sdks/java/harness/pom.xml
+++ b/sdks/java/harness/pom.xml
@@ -73,6 +73,11 @@
 
     <dependency>
       <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-core-java</artifactId>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/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 d22c6c5..4cd0337 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -66,7 +66,10 @@
       <artifactId>beam-sdks-java-core</artifactId>
     </dependency>
 
-    <!-- Build dependencies -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
+    </dependency>
 
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
@@ -94,12 +97,6 @@
     </dependency>
 
     <dependency>
-      <groupId>com.google.auto.service</groupId>
-      <artifactId>auto-service</artifactId>
-      <optional>true</optional>
-    </dependency>
-
-    <dependency>
       <groupId>com.google.cloud.bigdataoss</groupId>
       <artifactId>util</artifactId>
     </dependency>
@@ -232,13 +229,20 @@
       <artifactId>avro</artifactId>
     </dependency>
 
+    <!-- Build dependencies -->
     <dependency>
       <groupId>com.google.auto.value</groupId>
       <artifactId>auto-value</artifactId>
       <scope>provided</scope>
     </dependency>
 
-    <!--  test -->
+    <dependency>
+      <groupId>com.google.auto.service</groupId>
+      <artifactId>auto-service</artifactId>
+      <optional>true</optional>
+    </dependency>
+
+    <!--  Test dependencies -->
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>


[3/7] beam git commit: [BEAM-1871] Create new GCP core module package and move several GCP related classes from beam-sdks-java-core over.

Posted by lc...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java
new file mode 100644
index 0000000..2187e7d
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java
@@ -0,0 +1,158 @@
+/*
+ * 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.options;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.util.AppEngineEnvironment;
+import org.apache.beam.sdk.util.GcsPathValidator;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.util.PathValidator;
+
+/**
+ * Options used to configure Google Cloud Storage.
+ */
+public interface GcsOptions extends
+    ApplicationNameOptions, GcpOptions, PipelineOptions {
+  /**
+   * The GcsUtil instance that should be used to communicate with Google Cloud Storage.
+   */
+  @JsonIgnore
+  @Description("The GcsUtil instance that should be used to communicate with Google Cloud Storage.")
+  @Default.InstanceFactory(GcsUtil.GcsUtilFactory.class)
+  @Hidden
+  GcsUtil getGcsUtil();
+  void setGcsUtil(GcsUtil value);
+
+  /**
+   * The ExecutorService instance to use to create threads, can be overridden to specify an
+   * ExecutorService that is compatible with the users environment. If unset, the
+   * default is to create an ExecutorService with an unbounded number of threads; this
+   * is compatible with Google AppEngine.
+   */
+  @JsonIgnore
+  @Description("The ExecutorService instance to use to create multiple threads. Can be overridden "
+      + "to specify an ExecutorService that is compatible with the users environment. If unset, "
+      + "the default is to create an ExecutorService with an unbounded number of threads; this "
+      + "is compatible with Google AppEngine.")
+  @Default.InstanceFactory(ExecutorServiceFactory.class)
+  @Hidden
+  ExecutorService getExecutorService();
+  void setExecutorService(ExecutorService value);
+
+  /**
+   * GCS endpoint to use. If unspecified, uses the default endpoint.
+   */
+  @JsonIgnore
+  @Hidden
+  @Description("The URL for the GCS API.")
+  String getGcsEndpoint();
+  void setGcsEndpoint(String value);
+
+  /**
+   * The buffer size (in bytes) to use when uploading files to GCS. Please see the documentation for
+   * {@link AbstractGoogleAsyncWriteChannel#setUploadBufferSize} for more information on the
+   * restrictions and performance implications of this value.
+   */
+  @Description("The buffer size (in bytes) to use when uploading files to GCS. Please see the "
+      + "documentation for AbstractGoogleAsyncWriteChannel.setUploadBufferSize for more "
+      + "information on the restrictions and performance implications of this value.\n\n"
+      + "https://github.com/GoogleCloudPlatform/bigdata-interop/blob/master/util/src/main/java/"
+      + "com/google/cloud/hadoop/util/AbstractGoogleAsyncWriteChannel.java")
+  @Nullable
+  Integer getGcsUploadBufferSizeBytes();
+  void setGcsUploadBufferSizeBytes(@Nullable Integer bytes);
+
+  /**
+   * The class of the validator that should be created and used to validate paths.
+   * If pathValidator has not been set explicitly, an instance of this class will be
+   * constructed and used as the path validator.
+   */
+  @Description("The class of the validator that should be created and used to validate paths. "
+      + "If pathValidator has not been set explicitly, an instance of this class will be "
+      + "constructed and used as the path validator.")
+  @Default.Class(GcsPathValidator.class)
+  Class<? extends PathValidator> getPathValidatorClass();
+  void setPathValidatorClass(Class<? extends PathValidator> validatorClass);
+
+  /**
+   * The path validator instance that should be used to validate paths.
+   * If no path validator has been set explicitly, the default is to use the instance factory that
+   * constructs a path validator based upon the currently set pathValidatorClass.
+   */
+  @JsonIgnore
+  @Description("The path validator instance that should be used to validate paths. "
+      + "If no path validator has been set explicitly, the default is to use the instance factory "
+      + "that constructs a path validator based upon the currently set pathValidatorClass.")
+  @Default.InstanceFactory(PathValidatorFactory.class)
+  PathValidator getPathValidator();
+  void setPathValidator(PathValidator validator);
+
+  /**
+   * Returns the default {@link ExecutorService} to use within the Apache Beam SDK. The
+   * {@link ExecutorService} is compatible with AppEngine.
+   */
+  class ExecutorServiceFactory implements DefaultValueFactory<ExecutorService> {
+    @SuppressWarnings("deprecation")  // IS_APP_ENGINE is deprecated for internal use only.
+    @Override
+    public ExecutorService create(PipelineOptions options) {
+      ThreadFactoryBuilder threadFactoryBuilder = new ThreadFactoryBuilder();
+      threadFactoryBuilder.setThreadFactory(MoreExecutors.platformThreadFactory());
+      if (!AppEngineEnvironment.IS_APP_ENGINE) {
+        // AppEngine doesn't allow modification of threads to be daemon threads.
+        threadFactoryBuilder.setDaemon(true);
+      }
+      /* The SDK requires an unbounded thread pool because a step may create X writers
+       * each requiring their own thread to perform the writes otherwise a writer may
+       * block causing deadlock for the step because the writers buffer is full.
+       * Also, the MapTaskExecutor launches the steps in reverse order and completes
+       * them in forward order thus requiring enough threads so that each step's writers
+       * can be active.
+       */
+      return new ThreadPoolExecutor(
+          0, Integer.MAX_VALUE, // Allow an unlimited number of re-usable threads.
+          Long.MAX_VALUE, TimeUnit.NANOSECONDS, // Keep non-core threads alive forever.
+          new SynchronousQueue<Runnable>(),
+          threadFactoryBuilder.build());
+    }
+  }
+
+  /**
+   * Creates a {@link PathValidator} object using the class specified in
+   * {@link #getPathValidatorClass()}.
+   */
+  class PathValidatorFactory implements DefaultValueFactory<PathValidator> {
+    @Override
+    public PathValidator create(PipelineOptions options) {
+      GcsOptions gcsOptions = options.as(GcsOptions.class);
+      return InstanceBuilder.ofType(PathValidator.class)
+          .fromClass(gcsOptions.getPathValidatorClass())
+          .fromFactoryMethod("fromOptions")
+          .withArg(PipelineOptions.class, options)
+          .build();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java
new file mode 100644
index 0000000..f9cb575
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java
@@ -0,0 +1,87 @@
+/*
+ * 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.options;
+
+import com.google.api.client.googleapis.services.AbstractGoogleClient;
+import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
+import com.google.api.client.googleapis.services.GoogleClientRequestInitializer;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * These options configure debug settings for Google API clients created within the Apache Beam SDK.
+ */
+public interface GoogleApiDebugOptions extends PipelineOptions {
+  /**
+   * This option enables tracing of API calls to Google services used within the Apache
+   * Beam SDK. Values are expected in JSON format <code>{"ApiName":"TraceDestination",...}
+   * </code> where the {@code ApiName} represents the request classes canonical name. The
+   * {@code TraceDestination} is a logical trace consumer to whom the trace will be reported.
+   * Typically, "producer" is the right destination to use: this makes API traces available to the
+   * team offering the API. Note that by enabling this option, the contents of the requests to and
+   * from Google Cloud services will be made available to Google. For example, by specifying
+   * <code>{"Dataflow":"producer"}</code>, all calls to the Dataflow service will be made available
+   * to Google, specifically to the Google Cloud Dataflow team.
+   */
+  @Description("This option enables tracing of API calls to Google services used within the Apache "
+      + "Beam SDK. Values are expected in JSON format {\"ApiName\":\"TraceDestination\",...} "
+      + "where the ApiName represents the request classes canonical name. The TraceDestination is "
+      + "a logical trace consumer to whom the trace will be reported. Typically, \"producer\" is "
+      + "the right destination to use: this makes API traces available to the team offering the "
+      + "API. Note that by enabling this option, the contents of the requests to and from "
+      + "Google Cloud services will be made available to Google. For example, by specifying "
+      + "{\"Dataflow\":\"producer\"}, all calls to the Dataflow service will be made available to "
+      + "Google, specifically to the Google Cloud Dataflow team.")
+  GoogleApiTracer getGoogleApiTrace();
+  void setGoogleApiTrace(GoogleApiTracer commands);
+
+  /**
+   * A {@link GoogleClientRequestInitializer} that adds the trace destination to Google API calls.
+   */
+  class GoogleApiTracer extends HashMap<String, String>
+      implements GoogleClientRequestInitializer {
+    /**
+     * Creates a {@link GoogleApiTracer} that sets the trace destination on all
+     * calls that match the given client type.
+     */
+    public GoogleApiTracer addTraceFor(AbstractGoogleClient client, String traceDestination) {
+      put(client.getClass().getCanonicalName(), traceDestination);
+      return this;
+    }
+
+    /**
+     * Creates a {@link GoogleApiTracer} that sets the trace {@code traceDestination} on all
+     * calls that match for the given request type.
+     */
+    public GoogleApiTracer addTraceFor(
+        AbstractGoogleClientRequest<?> request, String traceDestination) {
+      put(request.getClass().getCanonicalName(), traceDestination);
+      return this;
+    }
+
+    @Override
+    public void initialize(AbstractGoogleClientRequest<?> request) throws IOException {
+      for (Map.Entry<String, String> entry : this.entrySet()) {
+        if (request.getClass().getCanonicalName().contains(entry.getKey())) {
+          request.set("$trace", entry.getValue());
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java
new file mode 100644
index 0000000..b065d19
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java
@@ -0,0 +1,36 @@
+/*
+ * 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.options;
+
+/**
+ * Properties that can be set when using Google Cloud Pub/Sub with the Apache Beam SDK.
+ */
+@Description("Options that are used to configure Google Cloud Pub/Sub. See "
+    + "https://cloud.google.com/pubsub/docs/overview for details on Cloud Pub/Sub.")
+public interface PubsubOptions extends ApplicationNameOptions, GcpOptions,
+    PipelineOptions, StreamingOptions {
+
+  /**
+   * Root URL for use with the Google Cloud Pub/Sub API.
+   */
+  @Description("Root URL for use with the Google Cloud Pub/Sub API")
+  @Default.String("https://pubsub.googleapis.com")
+  @Hidden
+  String getPubsubRootUrl();
+  void setPubsubRootUrl(String value);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/package-info.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/package-info.java
new file mode 100644
index 0000000..465e742
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Defines {@link org.apache.beam.sdk.options.PipelineOptions} for
+ * configuring pipeline execution for Google Cloud Platform components.
+ */
+package org.apache.beam.sdk.options;

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
new file mode 100644
index 0000000..8f752c0
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.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.testing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.api.client.http.HttpTransport;
+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.BigqueryScopes;
+import com.google.api.services.bigquery.model.QueryRequest;
+import com.google.api.services.bigquery.model.QueryResponse;
+import com.google.api.services.bigquery.model.TableCell;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.auth.Credentials;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.util.FluentBackoff;
+import org.apache.beam.sdk.util.Transport;
+import org.hamcrest.Description;
+import org.hamcrest.TypeSafeMatcher;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A matcher to verify data in BigQuery by processing given query
+ * and comparing with content's checksum.
+ *
+ * <p>Example:
+ * <pre>{@code [
+ *   assertThat(job, new BigqueryMatcher(appName, projectId, queryString, expectedChecksum));
+ * ]}</pre>
+ */
+@NotThreadSafe
+public class BigqueryMatcher extends TypeSafeMatcher<PipelineResult>
+    implements SerializableMatcher<PipelineResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(BigqueryMatcher.class);
+
+  // The maximum number of retries to execute a BigQuery RPC
+  static final int MAX_QUERY_RETRIES = 4;
+
+  // The initial backoff for executing a BigQuery RPC
+  private static final Duration INITIAL_BACKOFF = Duration.standardSeconds(1L);
+
+  // The total number of rows in query response to be formatted for debugging purpose
+  private static final int TOTAL_FORMATTED_ROWS = 20;
+
+  // The backoff factory with initial configs
+  static final FluentBackoff BACKOFF_FACTORY =
+      FluentBackoff.DEFAULT
+          .withMaxRetries(MAX_QUERY_RETRIES)
+          .withInitialBackoff(INITIAL_BACKOFF);
+
+  private final String applicationName;
+  private final String projectId;
+  private final String query;
+  private final String expectedChecksum;
+  private String actualChecksum;
+  private transient QueryResponse response;
+
+  public BigqueryMatcher(
+      String applicationName, String projectId, String query, String expectedChecksum) {
+    validateArgument("applicationName", applicationName);
+    validateArgument("projectId", projectId);
+    validateArgument("query", query);
+    validateArgument("expectedChecksum", expectedChecksum);
+
+    this.applicationName = applicationName;
+    this.projectId = projectId;
+    this.query = query;
+    this.expectedChecksum = expectedChecksum;
+  }
+
+  @Override
+  protected boolean matchesSafely(PipelineResult pipelineResult) {
+    LOG.info("Verifying Bigquery data");
+    Bigquery bigqueryClient = newBigqueryClient(applicationName);
+
+    // execute query
+    LOG.debug("Executing query: {}", query);
+    try {
+      QueryRequest queryContent = new QueryRequest();
+      queryContent.setQuery(query);
+
+      response = queryWithRetries(
+          bigqueryClient, queryContent, Sleeper.DEFAULT, BACKOFF_FACTORY.backoff());
+    } catch (IOException | InterruptedException e) {
+      if (e instanceof InterruptedIOException) {
+        Thread.currentThread().interrupt();
+      }
+      throw new RuntimeException("Failed to fetch BigQuery data.", e);
+    }
+
+    if (!response.getJobComplete()) {
+      // query job not complete, verification failed
+      return false;
+    } else {
+      // compute checksum
+      actualChecksum = generateHash(response.getRows());
+      LOG.debug("Generated a SHA1 checksum based on queried data: {}", actualChecksum);
+
+      return expectedChecksum.equals(actualChecksum);
+    }
+  }
+
+  @VisibleForTesting
+  Bigquery newBigqueryClient(String applicationName) {
+    HttpTransport transport = Transport.getTransport();
+    JsonFactory jsonFactory = Transport.getJsonFactory();
+    Credentials credential = getDefaultCredential();
+
+    return new Bigquery.Builder(transport, jsonFactory, new HttpCredentialsAdapter(credential))
+        .setApplicationName(applicationName)
+        .build();
+  }
+
+  @Nonnull
+  @VisibleForTesting
+  QueryResponse queryWithRetries(Bigquery bigqueryClient, QueryRequest queryContent,
+                                 Sleeper sleeper, BackOff backOff)
+      throws IOException, InterruptedException {
+    IOException lastException = null;
+    do {
+      if (lastException != null) {
+        LOG.warn("Retrying query ({}) after exception", queryContent.getQuery(), lastException);
+      }
+      try {
+        QueryResponse response = bigqueryClient.jobs().query(projectId, queryContent).execute();
+        if (response != null) {
+          return response;
+        } else {
+          lastException =
+              new IOException("Expected valid response from query job, but received null.");
+        }
+      } catch (IOException e) {
+        // ignore and retry
+        lastException = e;
+      }
+    } while(BackOffUtils.next(sleeper, backOff));
+
+    throw new RuntimeException(
+        String.format(
+            "Unable to get BigQuery response after retrying %d times using query (%s)",
+            MAX_QUERY_RETRIES,
+            queryContent.getQuery()),
+        lastException);
+  }
+
+  private void validateArgument(String name, String value) {
+    checkArgument(
+        !Strings.isNullOrEmpty(value), "Expected valid %s, but was %s", name, value);
+  }
+
+  private Credentials getDefaultCredential() {
+    GoogleCredentials credential;
+    try {
+      credential = GoogleCredentials.getApplicationDefault();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to get application default credential.", e);
+    }
+
+    if (credential.createScopedRequired()) {
+      Collection<String> bigqueryScope =
+          Lists.newArrayList(BigqueryScopes.CLOUD_PLATFORM_READ_ONLY);
+      credential = credential.createScoped(bigqueryScope);
+    }
+    return credential;
+  }
+
+  private String generateHash(@Nonnull List<TableRow> rows) {
+    List<HashCode> rowHashes = Lists.newArrayList();
+    for (TableRow row : rows) {
+      List<String> cellsInOneRow = Lists.newArrayList();
+      for (TableCell cell : row.getF()) {
+        cellsInOneRow.add(Objects.toString(cell.getV()));
+        Collections.sort(cellsInOneRow);
+      }
+      rowHashes.add(
+          Hashing.sha1().hashString(cellsInOneRow.toString(), StandardCharsets.UTF_8));
+    }
+    return Hashing.combineUnordered(rowHashes).toString();
+  }
+
+  @Override
+  public void describeTo(Description description) {
+    description
+        .appendText("Expected checksum is (")
+        .appendText(expectedChecksum)
+        .appendText(")");
+  }
+
+  @Override
+  public void describeMismatchSafely(PipelineResult pResult, Description description) {
+    String info;
+    if (!response.getJobComplete()) {
+      // query job not complete
+      info = String.format("The query job hasn't completed. Got response: %s", response);
+    } else {
+      // checksum mismatch
+      info = String.format("was (%s).%n"
+          + "\tTotal number of rows are: %d.%n"
+          + "\tQueried data details:%s",
+          actualChecksum, response.getTotalRows(), formatRows(TOTAL_FORMATTED_ROWS));
+    }
+    description.appendText(info);
+  }
+
+  private String formatRows(int totalNumRows) {
+    StringBuilder samples = new StringBuilder();
+    List<TableRow> rows = response.getRows();
+    for (int i = 0; i < totalNumRows && i < rows.size(); i++) {
+      samples.append(String.format("%n\t\t"));
+      for (TableCell field : rows.get(i).getF()) {
+        samples.append(String.format("%-10s", field.getV()));
+      }
+    }
+    if (rows.size() > totalNumRows) {
+      samples.append(String.format("%n\t\t..."));
+    }
+    return samples.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/testing/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/testing/package-info.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/testing/package-info.java
new file mode 100644
index 0000000..1494026
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/testing/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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 utilities for unit testing Google Cloud Platform components of Apache Beam pipelines.
+ */
+package org.apache.beam.sdk.testing;

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java
new file mode 100644
index 0000000..b0fcbd1
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java
@@ -0,0 +1,62 @@
+/*
+ * 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 java.lang.reflect.InvocationTargetException;
+
+/** Stores whether we are running within AppEngine or not. */
+public class AppEngineEnvironment {
+  /**
+   * True if running inside of AppEngine, false otherwise.
+   */
+  @Deprecated
+  public static final boolean IS_APP_ENGINE = isAppEngine();
+
+  /**
+   * Attempts to detect whether we are inside of AppEngine.
+   *
+   * <p>Purposely copied and left private from private <a href="https://code.google.com/p/
+   * guava-libraries/source/browse/guava/src/com/google/common/util/concurrent/
+   * MoreExecutors.java#785">code.google.common.util.concurrent.MoreExecutors#isAppEngine</a>.
+   *
+   * @return true if we are inside of AppEngine, false otherwise.
+   */
+  static boolean isAppEngine() {
+    if (System.getProperty("com.google.appengine.runtime.environment") == null) {
+      return false;
+    }
+    try {
+      // If the current environment is null, we're not inside AppEngine.
+      return Class.forName("com.google.apphosting.api.ApiProxy")
+          .getMethod("getCurrentEnvironment")
+          .invoke(null) != null;
+    } catch (ClassNotFoundException e) {
+      // If ApiProxy doesn't exist, we're not on AppEngine at all.
+      return false;
+    } catch (InvocationTargetException e) {
+      // If ApiProxy throws an exception, we're not in a proper AppEngine environment.
+      return false;
+    } catch (IllegalAccessException e) {
+      // If the method isn't accessible, we're not on a supported version of AppEngine;
+      return false;
+    } catch (NoSuchMethodException e) {
+      // If the method doesn't exist, we're not on a supported version of AppEngine;
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java
new file mode 100644
index 0000000..6229650
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java
@@ -0,0 +1,29 @@
+/*
+ * 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 com.google.auth.Credentials;
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+
+/**
+ * Construct an oauth credential to be used by the SDK and the SDK workers.
+ */
+public interface CredentialFactory {
+  Credentials getCredential() throws IOException, GeneralSecurityException;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/DefaultBucket.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/DefaultBucket.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/DefaultBucket.java
new file mode 100644
index 0000000..75954c0
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/DefaultBucket.java
@@ -0,0 +1,105 @@
+/*
+ * 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.Strings.isNullOrEmpty;
+
+import com.google.api.services.storage.model.Bucket;
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.nio.file.FileAlreadyExistsException;
+import org.apache.beam.sdk.options.CloudResourceManagerOptions;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class for handling default GCS buckets.
+ */
+public class DefaultBucket {
+  static final Logger LOG = LoggerFactory.getLogger(DefaultBucket.class);
+
+  static final String DEFAULT_REGION = "us-central1";
+
+  /**
+   * Creates a default bucket or verifies the existence and proper access control
+   * of an existing default bucket.  Returns the location if successful.
+   */
+  public static String tryCreateDefaultBucket(PipelineOptions options) {
+    GcsOptions gcpOptions = options.as(GcsOptions.class);
+
+    final String projectId = gcpOptions.getProject();
+    checkArgument(!isNullOrEmpty(projectId),
+                  "--project is a required option.");
+
+    // Look up the project number, to create a default bucket with a stable
+    // name with no special characters.
+    long projectNumber = 0L;
+    try {
+      projectNumber = gcpOptions.as(CloudResourceManagerOptions.class)
+          .getGcpProjectUtil().getProjectNumber(projectId);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to verify project with ID " + projectId, e);
+    }
+    String region = DEFAULT_REGION;
+    if (!isNullOrEmpty(gcpOptions.getZone())) {
+      region = getRegionFromZone(gcpOptions.getZone());
+    }
+    final String bucketName =
+      "dataflow-staging-" + region + "-" + projectNumber;
+    LOG.info("No staging location provided, attempting to use default bucket: {}",
+             bucketName);
+    Bucket bucket = new Bucket()
+      .setName(bucketName)
+      .setLocation(region);
+    // Always try to create the bucket before checking access, so that we do not
+    // race with other pipelines that may be attempting to do the same thing.
+    try {
+      gcpOptions.getGcsUtil().createBucket(projectId, bucket);
+    } catch (FileAlreadyExistsException e) {
+      LOG.debug("Bucket '{}'' already exists, verifying access.", bucketName);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable create default bucket.", e);
+    }
+
+    // Once the bucket is expected to exist, verify that it is correctly owned
+    // by the project executing the job.
+    try {
+      long owner = gcpOptions.getGcsUtil().bucketOwner(
+        GcsPath.fromComponents(bucketName, ""));
+      checkArgument(
+        owner == projectNumber,
+        "Bucket owner does not match the project from --project:"
+        + " %s vs. %s", owner, projectNumber);
+    } catch (IOException e) {
+      throw new RuntimeException(
+        "Unable to determine the owner of the default bucket at gs://" + bucketName, e);
+    }
+    return "gs://" + bucketName;
+  }
+
+  @VisibleForTesting
+  static String getRegionFromZone(String zone) {
+    String[] zoneParts = zone.split("-");
+    checkArgument(zoneParts.length >= 2, "Invalid zone provided: %s", zone);
+    return zoneParts[0] + "-" + zoneParts[1];
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java
new file mode 100644
index 0000000..e1fa18f
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java
@@ -0,0 +1,67 @@
+/*
+ * 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 com.google.auth.Credentials;
+import com.google.auth.oauth2.GoogleCredentials;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * Construct an oauth credential to be used by the SDK and the SDK workers.
+ * Returns a GCP credential.
+ */
+public class GcpCredentialFactory implements CredentialFactory {
+  /**
+   * The scope cloud-platform provides access to all Cloud Platform resources.
+   * cloud-platform isn't sufficient yet for talking to datastore so we request
+   * those resources separately.
+   *
+   * <p>Note that trusted scope relationships don't apply to OAuth tokens, so for
+   * services we access directly (GCS) as opposed to through the backend
+   * (BigQuery, GCE), we need to explicitly request that scope.
+   */
+  private static final List<String> SCOPES = Arrays.asList(
+      "https://www.googleapis.com/auth/cloud-platform",
+      "https://www.googleapis.com/auth/devstorage.full_control",
+      "https://www.googleapis.com/auth/userinfo.email",
+      "https://www.googleapis.com/auth/datastore",
+      "https://www.googleapis.com/auth/pubsub");
+
+  private static final GcpCredentialFactory INSTANCE = new GcpCredentialFactory();
+
+  public static GcpCredentialFactory fromOptions(PipelineOptions options) {
+    return INSTANCE;
+  }
+
+  /**
+   * Returns a default GCP {@link Credentials} or null when it fails.
+   */
+  @Override
+  public Credentials getCredential() {
+    try {
+      return GoogleCredentials.getApplicationDefault().createScoped(SCOPES);
+    } catch (IOException e) {
+      // Ignore the exception
+      // Pipelines that only access to public data should be able to run without credentials.
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcpProjectUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcpProjectUtil.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcpProjectUtil.java
new file mode 100644
index 0000000..f73afe0
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcpProjectUtil.java
@@ -0,0 +1,106 @@
+/*
+ * 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 com.google.api.client.util.BackOff;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.cloudresourcemanager.CloudResourceManager;
+import com.google.api.services.cloudresourcemanager.model.Project;
+import com.google.cloud.hadoop.util.ResilientOperation;
+import com.google.cloud.hadoop.util.RetryDeterminer;
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import org.apache.beam.sdk.options.CloudResourceManagerOptions;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides operations on Google Cloud Platform Projects.
+ */
+public class GcpProjectUtil {
+  /**
+   * A {@link DefaultValueFactory} able to create a {@link GcpProjectUtil} using
+   * any transport flags specified on the {@link PipelineOptions}.
+   */
+  public static class GcpProjectUtilFactory implements DefaultValueFactory<GcpProjectUtil> {
+    /**
+     * Returns an instance of {@link GcpProjectUtil} based on the
+     * {@link PipelineOptions}.
+     */
+    @Override
+    public GcpProjectUtil create(PipelineOptions options) {
+      LOG.debug("Creating new GcpProjectUtil");
+      CloudResourceManagerOptions crmOptions = options.as(CloudResourceManagerOptions.class);
+      return new GcpProjectUtil(
+          Transport.newCloudResourceManagerClient(crmOptions).build());
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(GcpProjectUtil.class);
+
+  private static final FluentBackoff BACKOFF_FACTORY =
+      FluentBackoff.DEFAULT.withMaxRetries(3).withInitialBackoff(Duration.millis(200));
+
+  /** Client for the CRM API. */
+  private CloudResourceManager crmClient;
+
+  private GcpProjectUtil(CloudResourceManager crmClient) {
+    this.crmClient = crmClient;
+  }
+
+  // Use this only for testing purposes.
+  @VisibleForTesting
+  void setCrmClient(CloudResourceManager crmClient) {
+    this.crmClient = crmClient;
+  }
+
+  /**
+   * Returns the project number or throws an exception if the project does not
+   * exist or has other access exceptions.
+   */
+  public long getProjectNumber(String projectId) throws IOException {
+    return getProjectNumber(
+      projectId,
+      BACKOFF_FACTORY.backoff(),
+      Sleeper.DEFAULT);
+  }
+
+  /**
+   * Returns the project number or throws an error if the project does not
+   * exist or has other access errors.
+   */
+  @VisibleForTesting
+  long getProjectNumber(String projectId, BackOff backoff, Sleeper sleeper) throws IOException {
+      CloudResourceManager.Projects.Get getProject =
+          crmClient.projects().get(projectId);
+      try {
+        Project project = ResilientOperation.retry(
+            ResilientOperation.getGoogleRequestCallable(getProject),
+            backoff,
+            RetryDeterminer.SOCKET_ERRORS,
+            IOException.class,
+            sleeper);
+        return project.getProjectNumber();
+      } catch (Exception e) {
+        throw new IOException("Unable to get project number", e);
+     }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java
new file mode 100644
index 0000000..745dcb9
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java
@@ -0,0 +1,111 @@
+/*
+ * 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 java.io.IOException;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+
+/**
+ * Implements IOChannelFactory for GCS.
+ */
+public class GcsIOChannelFactory implements IOChannelFactory {
+
+  /**
+   * Create a {@link GcsIOChannelFactory} with the given {@link PipelineOptions}.
+   */
+  public static GcsIOChannelFactory fromOptions(PipelineOptions options) {
+    return new GcsIOChannelFactory(options.as(GcsOptions.class));
+  }
+
+  private final GcsOptions options;
+
+  private GcsIOChannelFactory(GcsOptions options) {
+    this.options = options;
+  }
+
+  @Override
+  public Collection<String> match(String spec) throws IOException {
+    GcsPath path = GcsPath.fromUri(spec);
+    GcsUtil util = options.getGcsUtil();
+    List<GcsPath> matched = util.expand(path);
+
+    List<String> specs = new LinkedList<>();
+    for (GcsPath match : matched) {
+      specs.add(match.toString());
+    }
+
+    return specs;
+  }
+
+  @Override
+  public ReadableByteChannel open(String spec) throws IOException {
+    GcsPath path = GcsPath.fromUri(spec);
+    GcsUtil util = options.getGcsUtil();
+    return util.open(path);
+  }
+
+  @Override
+  public WritableByteChannel create(String spec, String mimeType)
+      throws IOException {
+    GcsPath path = GcsPath.fromUri(spec);
+    GcsUtil util = options.getGcsUtil();
+    return util.create(path, mimeType);
+  }
+
+  @Override
+  public long getSizeBytes(String spec) throws IOException {
+    GcsPath path = GcsPath.fromUri(spec);
+    GcsUtil util = options.getGcsUtil();
+    return util.fileSize(path);
+  }
+
+  @Override
+  public boolean isReadSeekEfficient(String spec) throws IOException {
+    // TODO It is incorrect to return true here for files with content encoding set to gzip.
+    return true;
+  }
+
+  @Override
+  public String resolve(String path, String other) throws IOException {
+    return toPath(path).resolve(other).toString();
+  }
+
+  @Override
+  public Path toPath(String path) {
+    return GcsPath.fromUri(path);
+  }
+
+  @Override
+  public void copy(Iterable<String> srcFilenames, Iterable<String> destFilenames)
+      throws IOException {
+    options.getGcsUtil().copy(srcFilenames, destFilenames);
+  }
+
+  @Override
+  public void remove(Collection<String> filesOrDirs) throws IOException {
+    options.getGcsUtil().remove(filesOrDirs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java
new file mode 100644
index 0000000..b4c457f
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java
@@ -0,0 +1,38 @@
+/*
+ * 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 com.google.auto.service.AutoService;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * {@link AutoService} registrar for the {@link GcsIOChannelFactory}.
+ */
+@AutoService(IOChannelFactoryRegistrar.class)
+public class GcsIOChannelFactoryRegistrar implements IOChannelFactoryRegistrar {
+
+  @Override
+  public GcsIOChannelFactory fromOptions(PipelineOptions options) {
+    return GcsIOChannelFactory.fromOptions(options);
+  }
+
+  @Override
+  public String getScheme() {
+    return "gs";
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
new file mode 100644
index 0000000..a5b951d
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
@@ -0,0 +1,95 @@
+/*
+ * 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 java.io.IOException;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+
+/**
+ * GCP implementation of {@link PathValidator}. Only GCS paths are allowed.
+ */
+public class GcsPathValidator implements PathValidator {
+
+  private GcsOptions gcpOptions;
+
+  private GcsPathValidator(GcsOptions options) {
+    this.gcpOptions = options;
+  }
+
+  public static GcsPathValidator fromOptions(PipelineOptions options) {
+    return new GcsPathValidator(options.as(GcsOptions.class));
+  }
+
+  /**
+   * Validates the the input GCS path is accessible and that the path
+   * is well formed.
+   */
+  @Override
+  public String validateInputFilePatternSupported(String filepattern) {
+    GcsPath gcsPath = getGcsPath(filepattern);
+    checkArgument(gcpOptions.getGcsUtil().isGcsPatternSupported(gcsPath.getObject()));
+    String returnValue = verifyPath(filepattern);
+    verifyPathIsAccessible(filepattern, "Could not find file %s");
+    return returnValue;
+  }
+
+  /**
+   * Validates the the output GCS path is accessible and that the path
+   * is well formed.
+   */
+  @Override
+  public String validateOutputFilePrefixSupported(String filePrefix) {
+    String returnValue = verifyPath(filePrefix);
+    verifyPathIsAccessible(filePrefix, "Output path does not exist or is not writeable: %s");
+    return returnValue;
+  }
+
+  @Override
+  public String verifyPath(String path) {
+    GcsPath gcsPath = getGcsPath(path);
+    checkArgument(gcsPath.isAbsolute(), "Must provide absolute paths for Dataflow");
+    checkArgument(!gcsPath.getObject().contains("//"),
+        "Dataflow Service does not allow objects with consecutive slashes");
+    return gcsPath.toResourceName();
+  }
+
+  private void verifyPathIsAccessible(String path, String errorMessage) {
+    GcsPath gcsPath = getGcsPath(path);
+    try {
+      checkArgument(gcpOptions.getGcsUtil().bucketAccessible(gcsPath),
+        errorMessage, path);
+    } catch (IOException e) {
+      throw new RuntimeException(
+          String.format("Unable to verify that GCS bucket gs://%s exists.", gcsPath.getBucket()),
+          e);
+    }
+  }
+
+  private GcsPath getGcsPath(String path) {
+    try {
+      return GcsPath.fromUri(path);
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException(String.format(
+          "Expected a valid 'gs://' path but was given '%s'", path), e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
new file mode 100644
index 0000000..1c853bb
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
@@ -0,0 +1,798 @@
+/*
+ * 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 com.google.api.client.googleapis.batch.BatchRequest;
+import com.google.api.client.googleapis.batch.json.JsonBatchCallback;
+import com.google.api.client.googleapis.json.GoogleJsonError;
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.http.HttpHeaders;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.storage.Storage;
+import com.google.api.services.storage.model.Bucket;
+import com.google.api.services.storage.model.Objects;
+import com.google.api.services.storage.model.StorageObject;
+import com.google.auto.value.AutoValue;
+import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel;
+import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel;
+import com.google.cloud.hadoop.gcsio.ObjectWriteConditions;
+import com.google.cloud.hadoop.util.ApiErrorExtractor;
+import com.google.cloud.hadoop.util.AsyncWriteChannelOptions;
+import com.google.cloud.hadoop.util.ClientRequestHelper;
+import com.google.cloud.hadoop.util.ResilientOperation;
+import com.google.cloud.hadoop.util.RetryDeterminer;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.AccessDeniedException;
+import java.nio.file.FileAlreadyExistsException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+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.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import javax.annotation.Nullable;
+
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides operations on GCS.
+ */
+public class GcsUtil {
+  /**
+   * This is a {@link DefaultValueFactory} able to create a {@link GcsUtil} using
+   * any transport flags specified on the {@link PipelineOptions}.
+   */
+  public static class GcsUtilFactory implements DefaultValueFactory<GcsUtil> {
+    /**
+     * Returns an instance of {@link GcsUtil} based on the
+     * {@link PipelineOptions}.
+     *
+     * <p>If no instance has previously been created, one is created and the value
+     * stored in {@code options}.
+     */
+    @Override
+    public GcsUtil create(PipelineOptions options) {
+      LOG.debug("Creating new GcsUtil");
+      GcsOptions gcsOptions = options.as(GcsOptions.class);
+      Storage.Builder storageBuilder = Transport.newStorageClient(gcsOptions);
+      return new GcsUtil(
+          storageBuilder.build(),
+          storageBuilder.getHttpRequestInitializer(),
+          gcsOptions.getExecutorService(),
+          gcsOptions.getGcsUploadBufferSizeBytes());
+    }
+
+    /**
+     * Returns an instance of {@link GcsUtil} based on the given parameters.
+     */
+    public static GcsUtil create(
+        Storage storageClient,
+        HttpRequestInitializer httpRequestInitializer,
+        ExecutorService executorService,
+        @Nullable Integer uploadBufferSizeBytes) {
+      return new GcsUtil(
+          storageClient, httpRequestInitializer, executorService, uploadBufferSizeBytes);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(GcsUtil.class);
+
+  /** Maximum number of items to retrieve per Objects.List request. */
+  private static final long MAX_LIST_ITEMS_PER_CALL = 1024;
+
+  /** Matches a glob containing a wildcard, capturing the portion before the first wildcard. */
+  private static final Pattern GLOB_PREFIX = Pattern.compile("(?<PREFIX>[^\\[*?]*)[\\[*?].*");
+
+  private static final String RECURSIVE_WILDCARD = "[*]{2}";
+
+  /**
+   * A {@link Pattern} for globs with a recursive wildcard.
+   */
+  private static final Pattern RECURSIVE_GCS_PATTERN =
+      Pattern.compile(".*" + RECURSIVE_WILDCARD + ".*");
+
+  /**
+   * Maximum number of requests permitted in a GCS batch request.
+   */
+  private static final int MAX_REQUESTS_PER_BATCH = 100;
+  /**
+   * Maximum number of concurrent batches of requests executing on GCS.
+   */
+  private static final int MAX_CONCURRENT_BATCHES = 256;
+
+  private static final FluentBackoff BACKOFF_FACTORY =
+      FluentBackoff.DEFAULT.withMaxRetries(3).withInitialBackoff(Duration.millis(200));
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  /** Client for the GCS API. */
+  private Storage storageClient;
+  private final HttpRequestInitializer httpRequestInitializer;
+  /** Buffer size for GCS uploads (in bytes). */
+  @Nullable private final Integer uploadBufferSizeBytes;
+
+  // Helper delegate for turning IOExceptions from API calls into higher-level semantics.
+  private final ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
+
+  // Exposed for testing.
+  final ExecutorService executorService;
+
+  /**
+   * Returns true if the given GCS pattern is supported otherwise fails with an
+   * exception.
+   */
+  public static boolean isGcsPatternSupported(String gcsPattern) {
+    if (RECURSIVE_GCS_PATTERN.matcher(gcsPattern).matches()) {
+      throw new IllegalArgumentException("Unsupported wildcard usage in \"" + gcsPattern + "\": "
+          + " recursive wildcards are not supported.");
+    }
+    return true;
+  }
+
+  /**
+   * Returns the prefix portion of the glob that doesn't contain wildcards.
+   */
+  public static String getGlobPrefix(String globExp) {
+    checkArgument(isGcsPatternSupported(globExp));
+    Matcher m = GLOB_PREFIX.matcher(globExp);
+    checkArgument(
+        m.matches(),
+        String.format("Glob expression: [%s] is not expandable.", globExp));
+    return m.group("PREFIX");
+  }
+
+  /**
+   * Expands glob expressions to regular expressions.
+   *
+   * @param globExp the glob expression to expand
+   * @return a string with the regular expression this glob expands to
+   */
+  public static String globToRegexp(String globExp) {
+    StringBuilder dst = new StringBuilder();
+    char[] src = globExp.toCharArray();
+    int i = 0;
+    while (i < src.length) {
+      char c = src[i++];
+      switch (c) {
+        case '*':
+          dst.append("[^/]*");
+          break;
+        case '?':
+          dst.append("[^/]");
+          break;
+        case '.':
+        case '+':
+        case '{':
+        case '}':
+        case '(':
+        case ')':
+        case '|':
+        case '^':
+        case '$':
+          // These need to be escaped in regular expressions
+          dst.append('\\').append(c);
+          break;
+        case '\\':
+          i = doubleSlashes(dst, src, i);
+          break;
+        default:
+          dst.append(c);
+          break;
+      }
+    }
+    return dst.toString();
+  }
+
+  /**
+   * Returns true if the given {@code spec} contains glob.
+   */
+  public static boolean isGlob(GcsPath spec) {
+    return GLOB_PREFIX.matcher(spec.getObject()).matches();
+  }
+
+  private GcsUtil(
+      Storage storageClient,
+      HttpRequestInitializer httpRequestInitializer,
+      ExecutorService executorService,
+      @Nullable Integer uploadBufferSizeBytes) {
+    this.storageClient = storageClient;
+    this.httpRequestInitializer = httpRequestInitializer;
+    this.uploadBufferSizeBytes = uploadBufferSizeBytes;
+    this.executorService = executorService;
+  }
+
+  // Use this only for testing purposes.
+  protected void setStorageClient(Storage storageClient) {
+    this.storageClient = storageClient;
+  }
+
+  /**
+   * Expands a pattern into matched paths. The pattern path may contain globs, which are expanded
+   * in the result. For patterns that only match a single object, we ensure that the object
+   * exists.
+   */
+  public List<GcsPath> expand(GcsPath gcsPattern) throws IOException {
+    checkArgument(isGcsPatternSupported(gcsPattern.getObject()));
+    Pattern p = null;
+    String prefix = null;
+    if (!isGlob(gcsPattern)) {
+      // Not a glob.
+      try {
+        // Use a get request to fetch the metadata of the object, and ignore the return value.
+        // The request has strong global consistency.
+        getObject(gcsPattern);
+        return ImmutableList.of(gcsPattern);
+      } catch (FileNotFoundException e) {
+        // If the path was not found, return an empty list.
+        return ImmutableList.of();
+      }
+    } else {
+      // Part before the first wildcard character.
+      prefix = getGlobPrefix(gcsPattern.getObject());
+      p = Pattern.compile(globToRegexp(gcsPattern.getObject()));
+    }
+
+    LOG.debug("matching files in bucket {}, prefix {} against pattern {}", gcsPattern.getBucket(),
+        prefix, p.toString());
+
+    String pageToken = null;
+    List<GcsPath> results = new LinkedList<>();
+    do {
+      Objects objects = listObjects(gcsPattern.getBucket(), prefix, pageToken);
+      if (objects.getItems() == null) {
+        break;
+      }
+
+      // Filter objects based on the regex.
+      for (StorageObject o : objects.getItems()) {
+        String name = o.getName();
+        // Skip directories, which end with a slash.
+        if (p.matcher(name).matches() && !name.endsWith("/")) {
+          LOG.debug("Matched object: {}", name);
+          results.add(GcsPath.fromObject(o));
+        }
+      }
+      pageToken = objects.getNextPageToken();
+    } while (pageToken != null);
+
+    return results;
+  }
+
+  @VisibleForTesting
+  @Nullable
+  Integer getUploadBufferSizeBytes() {
+    return uploadBufferSizeBytes;
+  }
+
+  /**
+   * Returns the file size from GCS or throws {@link FileNotFoundException}
+   * if the resource does not exist.
+   */
+  public long fileSize(GcsPath path) throws IOException {
+    return getObject(path).getSize().longValue();
+  }
+
+  /**
+   * Returns the {@link StorageObject} for the given {@link GcsPath}.
+   */
+  public StorageObject getObject(GcsPath gcsPath) throws IOException {
+    return getObject(gcsPath, BACKOFF_FACTORY.backoff(), Sleeper.DEFAULT);
+  }
+
+  @VisibleForTesting
+  StorageObject getObject(GcsPath gcsPath, BackOff backoff, Sleeper sleeper) throws IOException {
+    Storage.Objects.Get getObject =
+        storageClient.objects().get(gcsPath.getBucket(), gcsPath.getObject());
+    try {
+      return ResilientOperation.retry(
+          ResilientOperation.getGoogleRequestCallable(getObject),
+          backoff,
+          RetryDeterminer.SOCKET_ERRORS,
+          IOException.class,
+          sleeper);
+    } catch (IOException | InterruptedException e) {
+      if (e instanceof InterruptedException) {
+        Thread.currentThread().interrupt();
+      }
+      if (e instanceof IOException && errorExtractor.itemNotFound((IOException) e)) {
+        throw new FileNotFoundException(gcsPath.toString());
+      }
+      throw new IOException(
+          String.format("Unable to get the file object for path %s.", gcsPath),
+          e);
+    }
+  }
+
+  /**
+   * Returns {@link StorageObjectOrIOException StorageObjectOrIOExceptions} for the given
+   * {@link GcsPath GcsPaths}.
+   */
+  public List<StorageObjectOrIOException> getObjects(List<GcsPath> gcsPaths)
+      throws IOException {
+    List<StorageObjectOrIOException[]> results = new ArrayList<>();
+    executeBatches(makeGetBatches(gcsPaths, results));
+    ImmutableList.Builder<StorageObjectOrIOException> ret = ImmutableList.builder();
+    for (StorageObjectOrIOException[] result : results) {
+      ret.add(result[0]);
+    }
+    return ret.build();
+  }
+
+  /**
+   * Lists {@link Objects} given the {@code bucket}, {@code prefix}, {@code pageToken}.
+   */
+  public Objects listObjects(String bucket, String prefix, @Nullable String pageToken)
+      throws IOException {
+    // List all objects that start with the prefix (including objects in sub-directories).
+    Storage.Objects.List listObject = storageClient.objects().list(bucket);
+    listObject.setMaxResults(MAX_LIST_ITEMS_PER_CALL);
+    listObject.setPrefix(prefix);
+
+    if (pageToken != null) {
+      listObject.setPageToken(pageToken);
+    }
+
+    try {
+      return ResilientOperation.retry(
+          ResilientOperation.getGoogleRequestCallable(listObject),
+          BACKOFF_FACTORY.backoff(),
+          RetryDeterminer.SOCKET_ERRORS,
+          IOException.class);
+    } catch (Exception e) {
+      throw new IOException(
+          String.format("Unable to match files in bucket %s, prefix %s.", bucket, prefix),
+          e);
+    }
+  }
+
+  /**
+   * Returns the file size from GCS or throws {@link FileNotFoundException}
+   * if the resource does not exist.
+   */
+  @VisibleForTesting
+  List<Long> fileSizes(List<GcsPath> paths) throws IOException {
+    List<StorageObjectOrIOException> results = getObjects(paths);
+
+    ImmutableList.Builder<Long> ret = ImmutableList.builder();
+    for (StorageObjectOrIOException result : results) {
+      ret.add(toFileSize(result));
+    }
+    return ret.build();
+  }
+
+  private Long toFileSize(StorageObjectOrIOException storageObjectOrIOException)
+      throws IOException {
+    if (storageObjectOrIOException.ioException() != null) {
+      throw storageObjectOrIOException.ioException();
+    } else {
+      return storageObjectOrIOException.storageObject().getSize().longValue();
+    }
+  }
+
+  /**
+   * Opens an object in GCS.
+   *
+   * <p>Returns a SeekableByteChannel that provides access to data in the bucket.
+   *
+   * @param path the GCS filename to read from
+   * @return a SeekableByteChannel that can read the object data
+   */
+  public SeekableByteChannel open(GcsPath path)
+      throws IOException {
+    return new GoogleCloudStorageReadChannel(storageClient, path.getBucket(),
+            path.getObject(), errorExtractor,
+            new ClientRequestHelper<StorageObject>());
+  }
+
+  /**
+   * Creates an object in GCS.
+   *
+   * <p>Returns a WritableByteChannel that can be used to write data to the
+   * object.
+   *
+   * @param path the GCS file to write to
+   * @param type the type of object, eg "text/plain".
+   * @return a Callable object that encloses the operation.
+   */
+  public WritableByteChannel create(GcsPath path,
+      String type) throws IOException {
+    GoogleCloudStorageWriteChannel channel = new GoogleCloudStorageWriteChannel(
+        executorService,
+        storageClient,
+        new ClientRequestHelper<StorageObject>(),
+        path.getBucket(),
+        path.getObject(),
+        AsyncWriteChannelOptions.newBuilder().build(),
+        new ObjectWriteConditions(),
+        Collections.<String, String>emptyMap(),
+        type);
+    if (uploadBufferSizeBytes != null) {
+      channel.setUploadBufferSize(uploadBufferSizeBytes);
+    }
+    channel.initialize();
+    return channel;
+  }
+
+  /**
+   * Returns whether the GCS bucket exists and is accessible.
+   */
+  public boolean bucketAccessible(GcsPath path) throws IOException {
+    return bucketAccessible(
+        path,
+        BACKOFF_FACTORY.backoff(),
+        Sleeper.DEFAULT);
+  }
+
+  /**
+   * Returns the project number of the project which owns this bucket.
+   * If the bucket exists, it must be accessible otherwise the permissions
+   * exception will be propagated.  If the bucket does not exist, an exception
+   * will be thrown.
+   */
+  public long bucketOwner(GcsPath path) throws IOException {
+    return getBucket(
+        path,
+        BACKOFF_FACTORY.backoff(),
+        Sleeper.DEFAULT).getProjectNumber().longValue();
+  }
+
+  /**
+   * Creates a {@link Bucket} under the specified project in Cloud Storage or
+   * propagates an exception.
+   */
+  public void createBucket(String projectId, Bucket bucket) throws IOException {
+    createBucket(
+        projectId, bucket, BACKOFF_FACTORY.backoff(), Sleeper.DEFAULT);
+  }
+
+  /**
+   * Returns whether the GCS bucket exists. This will return false if the bucket
+   * is inaccessible due to permissions.
+   */
+  @VisibleForTesting
+  boolean bucketAccessible(GcsPath path, BackOff backoff, Sleeper sleeper) throws IOException {
+    try {
+      return getBucket(path, backoff, sleeper) != null;
+    } catch (AccessDeniedException | FileNotFoundException e) {
+      return false;
+    }
+  }
+
+  @VisibleForTesting
+  @Nullable
+  Bucket getBucket(GcsPath path, BackOff backoff, Sleeper sleeper) throws IOException {
+    Storage.Buckets.Get getBucket =
+        storageClient.buckets().get(path.getBucket());
+
+      try {
+        Bucket bucket = ResilientOperation.retry(
+            ResilientOperation.getGoogleRequestCallable(getBucket),
+            backoff,
+            new RetryDeterminer<IOException>() {
+              @Override
+              public boolean shouldRetry(IOException e) {
+                if (errorExtractor.itemNotFound(e) || errorExtractor.accessDenied(e)) {
+                  return false;
+                }
+                return RetryDeterminer.SOCKET_ERRORS.shouldRetry(e);
+              }
+            },
+            IOException.class,
+            sleeper);
+
+        return bucket;
+      } catch (GoogleJsonResponseException e) {
+        if (errorExtractor.accessDenied(e)) {
+          throw new AccessDeniedException(path.toString(), null, e.getMessage());
+        }
+        if (errorExtractor.itemNotFound(e)) {
+          throw new FileNotFoundException(e.getMessage());
+        }
+        throw e;
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new IOException(
+            String.format("Error while attempting to verify existence of bucket gs://%s",
+                path.getBucket()), e);
+     }
+  }
+
+  @VisibleForTesting
+  void createBucket(String projectId, Bucket bucket, BackOff backoff, Sleeper sleeper)
+        throws IOException {
+    Storage.Buckets.Insert insertBucket =
+        storageClient.buckets().insert(projectId, bucket);
+    insertBucket.setPredefinedAcl("projectPrivate");
+    insertBucket.setPredefinedDefaultObjectAcl("projectPrivate");
+
+    try {
+      ResilientOperation.retry(
+        ResilientOperation.getGoogleRequestCallable(insertBucket),
+        backoff,
+        new RetryDeterminer<IOException>() {
+          @Override
+          public boolean shouldRetry(IOException e) {
+            if (errorExtractor.itemAlreadyExists(e) || errorExtractor.accessDenied(e)) {
+              return false;
+            }
+            return RetryDeterminer.SOCKET_ERRORS.shouldRetry(e);
+          }
+        },
+        IOException.class,
+        sleeper);
+      return;
+    } catch (GoogleJsonResponseException e) {
+      if (errorExtractor.accessDenied(e)) {
+        throw new AccessDeniedException(bucket.getName(), null, e.getMessage());
+      }
+      if (errorExtractor.itemAlreadyExists(e)) {
+        throw new FileAlreadyExistsException(bucket.getName(), null, e.getMessage());
+      }
+      throw e;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new IOException(
+        String.format("Error while attempting to create bucket gs://%s for rproject %s",
+                      bucket.getName(), projectId), e);
+    }
+  }
+
+  private static void executeBatches(List<BatchRequest> batches) throws IOException {
+    ListeningExecutorService executor = MoreExecutors.listeningDecorator(
+        MoreExecutors.getExitingExecutorService(
+            new ThreadPoolExecutor(MAX_CONCURRENT_BATCHES, MAX_CONCURRENT_BATCHES,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>())));
+
+    List<ListenableFuture<Void>> futures = new LinkedList<>();
+    for (final BatchRequest batch : batches) {
+      futures.add(executor.submit(new Callable<Void>() {
+        public Void call() throws IOException {
+          batch.execute();
+          return null;
+        }
+      }));
+    }
+
+    try {
+      Futures.allAsList(futures).get();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new IOException("Interrupted while executing batch GCS request", e);
+    } catch (ExecutionException e) {
+      if (e.getCause() instanceof FileNotFoundException) {
+        throw (FileNotFoundException) e.getCause();
+      }
+      throw new IOException("Error executing batch GCS request", e);
+    } finally {
+      executor.shutdown();
+    }
+  }
+
+  /**
+   * Makes get {@link BatchRequest BatchRequests}.
+   *
+   * @param paths {@link GcsPath GcsPaths}.
+   * @param results mutable {@link List} for return values.
+   * @return {@link BatchRequest BatchRequests} to execute.
+   * @throws IOException
+   */
+  @VisibleForTesting
+  List<BatchRequest> makeGetBatches(
+      Collection<GcsPath> paths,
+      List<StorageObjectOrIOException[]> results) throws IOException {
+    List<BatchRequest> batches = new LinkedList<>();
+    for (List<GcsPath> filesToGet :
+        Lists.partition(Lists.newArrayList(paths), MAX_REQUESTS_PER_BATCH)) {
+      BatchRequest batch = createBatchRequest();
+      for (GcsPath path : filesToGet) {
+        results.add(enqueueGetFileSize(path, batch));
+      }
+      batches.add(batch);
+    }
+    return batches;
+  }
+
+  public void copy(Iterable<String> srcFilenames,
+                   Iterable<String> destFilenames) throws
+      IOException {
+    executeBatches(makeCopyBatches(srcFilenames, destFilenames));
+  }
+
+  List<BatchRequest> makeCopyBatches(Iterable<String> srcFilenames, Iterable<String> destFilenames)
+      throws IOException {
+    List<String> srcList = Lists.newArrayList(srcFilenames);
+    List<String> destList = Lists.newArrayList(destFilenames);
+    checkArgument(
+        srcList.size() == destList.size(),
+        "Number of source files %s must equal number of destination files %s",
+        srcList.size(),
+        destList.size());
+
+    List<BatchRequest> batches = new LinkedList<>();
+    BatchRequest batch = createBatchRequest();
+    for (int i = 0; i < srcList.size(); i++) {
+      final GcsPath sourcePath = GcsPath.fromUri(srcList.get(i));
+      final GcsPath destPath = GcsPath.fromUri(destList.get(i));
+      enqueueCopy(sourcePath, destPath, batch);
+      if (batch.size() >= MAX_REQUESTS_PER_BATCH) {
+        batches.add(batch);
+        batch = createBatchRequest();
+      }
+    }
+    if (batch.size() > 0) {
+      batches.add(batch);
+    }
+    return batches;
+  }
+
+  List<BatchRequest> makeRemoveBatches(Collection<String> filenames) throws IOException {
+    List<BatchRequest> batches = new LinkedList<>();
+    for (List<String> filesToDelete :
+        Lists.partition(Lists.newArrayList(filenames), MAX_REQUESTS_PER_BATCH)) {
+      BatchRequest batch = createBatchRequest();
+      for (String file : filesToDelete) {
+        enqueueDelete(GcsPath.fromUri(file), batch);
+      }
+      batches.add(batch);
+    }
+    return batches;
+  }
+
+  public void remove(Collection<String> filenames) throws IOException {
+    executeBatches(makeRemoveBatches(filenames));
+  }
+
+  private StorageObjectOrIOException[] enqueueGetFileSize(final GcsPath path, BatchRequest batch)
+      throws IOException {
+    final StorageObjectOrIOException[] ret = new StorageObjectOrIOException[1];
+
+    Storage.Objects.Get getRequest = storageClient.objects()
+        .get(path.getBucket(), path.getObject());
+    getRequest.queue(batch, new JsonBatchCallback<StorageObject>() {
+      @Override
+      public void onSuccess(StorageObject response, HttpHeaders httpHeaders) throws IOException {
+        ret[0] = StorageObjectOrIOException.create(response);
+      }
+
+      @Override
+      public void onFailure(GoogleJsonError e, HttpHeaders httpHeaders) throws IOException {
+        IOException ioException;
+        if (errorExtractor.itemNotFound(e)) {
+          ioException = new FileNotFoundException(path.toString());
+        } else {
+          ioException = new IOException(String.format("Error trying to get %s: %s", path, e));
+        }
+        ret[0] = StorageObjectOrIOException.create(ioException);
+      }
+    });
+    return ret;
+  }
+
+  /**
+   * A class that holds either a {@link StorageObject} or an {@link IOException}.
+   */
+  @AutoValue
+  public abstract static class StorageObjectOrIOException {
+
+    /**
+     * Returns the {@link StorageObject}.
+     */
+    @Nullable
+    public abstract StorageObject storageObject();
+
+    /**
+     * Returns the {@link IOException}.
+     */
+    @Nullable
+    public abstract IOException ioException();
+
+    @VisibleForTesting
+    public static StorageObjectOrIOException create(StorageObject storageObject) {
+      return new AutoValue_GcsUtil_StorageObjectOrIOException(
+          checkNotNull(storageObject, "storageObject"),
+          null /* ioException */);
+    }
+
+    @VisibleForTesting
+    public static StorageObjectOrIOException create(IOException ioException) {
+      return new AutoValue_GcsUtil_StorageObjectOrIOException(
+          null /* storageObject */,
+          checkNotNull(ioException, "ioException"));
+    }
+  }
+
+  private void enqueueCopy(final GcsPath from, final GcsPath to, BatchRequest batch)
+      throws IOException {
+    Storage.Objects.Copy copyRequest = storageClient.objects()
+        .copy(from.getBucket(), from.getObject(), to.getBucket(), to.getObject(), null);
+    copyRequest.queue(batch, new JsonBatchCallback<StorageObject>() {
+      @Override
+      public void onSuccess(StorageObject obj, HttpHeaders responseHeaders) {
+        LOG.debug("Successfully copied {} to {}", from, to);
+      }
+
+      @Override
+      public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException {
+        throw new IOException(
+            String.format("Error trying to copy %s to %s: %s", from, to, e));
+      }
+    });
+  }
+
+  private void enqueueDelete(final GcsPath file, BatchRequest batch) throws IOException {
+    Storage.Objects.Delete deleteRequest = storageClient.objects()
+        .delete(file.getBucket(), file.getObject());
+    deleteRequest.queue(batch, new JsonBatchCallback<Void>() {
+      @Override
+      public void onSuccess(Void obj, HttpHeaders responseHeaders) {
+        LOG.debug("Successfully deleted {}", file);
+      }
+
+      @Override
+      public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException {
+        throw new IOException(String.format("Error trying to delete %s: %s", file, e));
+      }
+    });
+  }
+
+  private BatchRequest createBatchRequest() {
+    return storageClient.batch(httpRequestInitializer);
+  }
+
+  private static int doubleSlashes(StringBuilder dst, char[] src, int i) {
+    // Emit the next character without special interpretation
+    dst.append('\\');
+    if ((i - 1) != src.length) {
+      dst.append(src[i]);
+      i++;
+    } else {
+      // A backslash at the very end is treated like an escaped backslash
+      dst.append('\\');
+    }
+    return i;
+  }
+}


[7/7] beam git commit: [BEAM-1871] Create new GCP core module package and move several GCP related classes from beam-sdks-java-core over.

Posted by lc...@apache.org.
[BEAM-1871] Create new GCP core module package and move several GCP related classes from beam-sdks-java-core over.

This closes #2561


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/6d3fe51c
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/6d3fe51c
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/6d3fe51c

Branch: refs/heads/master
Commit: 6d3fe51ce3240cea54e01cb61723f1547039904d
Parents: ed52d32 be92f59
Author: Luke Cwik <lc...@google.com>
Authored: Tue Apr 18 16:19:16 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Tue Apr 18 16:19:16 2017 -0700

----------------------------------------------------------------------
 examples/java/pom.xml                           |   5 +
 examples/java8/pom.xml                          |   5 +
 pom.xml                                         |   6 +
 runners/flink/examples/pom.xml                  |   4 +
 runners/google-cloud-dataflow-java/pom.xml      |   5 +
 sdks/java/core/pom.xml                          |  46 +-
 .../beam/sdk/options/BigQueryOptions.java       |  32 -
 .../options/CloudResourceManagerOptions.java    |  40 -
 .../DefaultPipelineOptionsRegistrar.java        |   5 -
 .../org/apache/beam/sdk/options/GcpOptions.java | 227 ------
 .../org/apache/beam/sdk/options/GcsOptions.java | 158 ----
 .../beam/sdk/options/GoogleApiDebugOptions.java |  87 --
 .../beam/sdk/options/PipelineOptions.java       |   4 +-
 .../apache/beam/sdk/options/PubsubOptions.java  |  36 -
 .../apache/beam/sdk/runners/PipelineRunner.java |   2 +
 .../beam/sdk/testing/BigqueryMatcher.java       | 256 ------
 .../apache/beam/sdk/testing/TestPipeline.java   |   3 -
 .../beam/sdk/util/AppEngineEnvironment.java     |  62 --
 .../apache/beam/sdk/util/CredentialFactory.java |  29 -
 .../org/apache/beam/sdk/util/DefaultBucket.java | 105 ---
 .../beam/sdk/util/GcpCredentialFactory.java     |  67 --
 .../apache/beam/sdk/util/GcpProjectUtil.java    | 106 ---
 .../beam/sdk/util/GcsIOChannelFactory.java      | 111 ---
 .../sdk/util/GcsIOChannelFactoryRegistrar.java  |  38 -
 .../apache/beam/sdk/util/GcsPathValidator.java  |  95 ---
 .../java/org/apache/beam/sdk/util/GcsUtil.java  | 798 -------------------
 .../util/IntervalBoundedExponentialBackOff.java |  89 ---
 .../beam/sdk/util/NoopCredentialFactory.java    |  68 --
 .../sdk/util/NullCredentialInitializer.java     |  62 --
 .../org/apache/beam/sdk/util/Transport.java     | 178 -----
 .../org/apache/beam/SdkCoreApiSurfaceTest.java  |   2 -
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  97 +--
 .../apache/beam/sdk/options/GcpOptionsTest.java | 171 ----
 .../sdk/options/GoogleApiDebugOptionsTest.java  | 145 ----
 .../sdk/options/PipelineOptionsFactoryTest.java |   4 +-
 .../beam/sdk/runners/PipelineRunnerTest.java    |  46 +-
 .../beam/sdk/testing/BigqueryMatcherTest.java   | 176 ----
 .../beam/sdk/testing/TestPipelineTest.java      |   6 +-
 .../apache/beam/sdk/util/DefaultBucketTest.java | 112 ---
 .../beam/sdk/util/GcpProjectUtilTest.java       |  76 --
 .../util/GcsIOChannelFactoryRegistrarTest.java  |  44 -
 .../beam/sdk/util/GcsIOChannelFactoryTest.java  |  43 -
 .../beam/sdk/util/GcsPathValidatorTest.java     |  87 --
 .../org/apache/beam/sdk/util/GcsUtilTest.java   | 798 -------------------
 .../IntervalBoundedExponentialBackOffTest.java  | 100 ---
 .../util/RetryHttpRequestInitializerTest.java   | 290 -------
 sdks/java/extensions/gcp-core/pom.xml           | 217 +++++
 .../beam/sdk/options/BigQueryOptions.java       |  32 +
 .../options/CloudResourceManagerOptions.java    |  40 +
 .../org/apache/beam/sdk/options/GcpOptions.java | 227 ++++++
 .../options/GcpPipelineOptionsRegistrar.java    |  39 +
 .../org/apache/beam/sdk/options/GcsOptions.java | 158 ++++
 .../beam/sdk/options/GoogleApiDebugOptions.java |  87 ++
 .../apache/beam/sdk/options/PubsubOptions.java  |  36 +
 .../apache/beam/sdk/options/package-info.java   |  22 +
 .../beam/sdk/testing/BigqueryMatcher.java       | 256 ++++++
 .../apache/beam/sdk/testing/package-info.java   |  21 +
 .../beam/sdk/util/AppEngineEnvironment.java     |  62 ++
 .../apache/beam/sdk/util/CredentialFactory.java |  29 +
 .../org/apache/beam/sdk/util/DefaultBucket.java | 105 +++
 .../beam/sdk/util/GcpCredentialFactory.java     |  67 ++
 .../apache/beam/sdk/util/GcpProjectUtil.java    | 106 +++
 .../beam/sdk/util/GcsIOChannelFactory.java      | 111 +++
 .../sdk/util/GcsIOChannelFactoryRegistrar.java  |  38 +
 .../apache/beam/sdk/util/GcsPathValidator.java  |  95 +++
 .../java/org/apache/beam/sdk/util/GcsUtil.java  | 798 +++++++++++++++++++
 .../util/IntervalBoundedExponentialBackOff.java |  89 +++
 .../beam/sdk/util/NoopCredentialFactory.java    |  68 ++
 .../sdk/util/NullCredentialInitializer.java     |  62 ++
 .../org/apache/beam/sdk/util/Transport.java     | 178 +++++
 .../org/apache/beam/sdk/util/package-info.java  |  20 +
 .../org/apache/beam/GcpCoreApiSurfaceTest.java  |  62 ++
 .../apache/beam/sdk/options/GcpOptionsTest.java | 171 ++++
 .../sdk/options/GoogleApiDebugOptionsTest.java  | 145 ++++
 .../beam/sdk/testing/BigqueryMatcherTest.java   | 176 ++++
 .../apache/beam/sdk/util/DefaultBucketTest.java | 112 +++
 .../beam/sdk/util/GcpProjectUtilTest.java       |  76 ++
 .../util/GcsIOChannelFactoryRegistrarTest.java  |  44 +
 .../beam/sdk/util/GcsIOChannelFactoryTest.java  |  43 +
 .../beam/sdk/util/GcsPathValidatorTest.java     |  87 ++
 .../org/apache/beam/sdk/util/GcsUtilTest.java   | 798 +++++++++++++++++++
 .../IntervalBoundedExponentialBackOffTest.java  | 100 +++
 .../util/RetryHttpRequestInitializerTest.java   | 290 +++++++
 sdks/java/extensions/pom.xml                    |   1 +
 sdks/java/harness/pom.xml                       |   5 +
 sdks/java/io/google-cloud-platform/pom.xml      |  20 +-
 86 files changed, 5130 insertions(+), 4889 deletions(-)
----------------------------------------------------------------------



[5/7] beam git commit: [BEAM-1871] Create new GCP core module package and move several GCP related classes from beam-sdks-java-core over.

Posted by lc...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
deleted file mode 100644
index 1c853bb..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
+++ /dev/null
@@ -1,798 +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 com.google.api.client.googleapis.batch.BatchRequest;
-import com.google.api.client.googleapis.batch.json.JsonBatchCallback;
-import com.google.api.client.googleapis.json.GoogleJsonError;
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.http.HttpHeaders;
-import com.google.api.client.http.HttpRequestInitializer;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.storage.Storage;
-import com.google.api.services.storage.model.Bucket;
-import com.google.api.services.storage.model.Objects;
-import com.google.api.services.storage.model.StorageObject;
-import com.google.auto.value.AutoValue;
-import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel;
-import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel;
-import com.google.cloud.hadoop.gcsio.ObjectWriteConditions;
-import com.google.cloud.hadoop.util.ApiErrorExtractor;
-import com.google.cloud.hadoop.util.AsyncWriteChannelOptions;
-import com.google.cloud.hadoop.util.ClientRequestHelper;
-import com.google.cloud.hadoop.util.ResilientOperation;
-import com.google.cloud.hadoop.util.RetryDeterminer;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.channels.SeekableByteChannel;
-import java.nio.channels.WritableByteChannel;
-import java.nio.file.AccessDeniedException;
-import java.nio.file.FileAlreadyExistsException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-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.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-import org.joda.time.Duration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Provides operations on GCS.
- */
-public class GcsUtil {
-  /**
-   * This is a {@link DefaultValueFactory} able to create a {@link GcsUtil} using
-   * any transport flags specified on the {@link PipelineOptions}.
-   */
-  public static class GcsUtilFactory implements DefaultValueFactory<GcsUtil> {
-    /**
-     * Returns an instance of {@link GcsUtil} based on the
-     * {@link PipelineOptions}.
-     *
-     * <p>If no instance has previously been created, one is created and the value
-     * stored in {@code options}.
-     */
-    @Override
-    public GcsUtil create(PipelineOptions options) {
-      LOG.debug("Creating new GcsUtil");
-      GcsOptions gcsOptions = options.as(GcsOptions.class);
-      Storage.Builder storageBuilder = Transport.newStorageClient(gcsOptions);
-      return new GcsUtil(
-          storageBuilder.build(),
-          storageBuilder.getHttpRequestInitializer(),
-          gcsOptions.getExecutorService(),
-          gcsOptions.getGcsUploadBufferSizeBytes());
-    }
-
-    /**
-     * Returns an instance of {@link GcsUtil} based on the given parameters.
-     */
-    public static GcsUtil create(
-        Storage storageClient,
-        HttpRequestInitializer httpRequestInitializer,
-        ExecutorService executorService,
-        @Nullable Integer uploadBufferSizeBytes) {
-      return new GcsUtil(
-          storageClient, httpRequestInitializer, executorService, uploadBufferSizeBytes);
-    }
-  }
-
-  private static final Logger LOG = LoggerFactory.getLogger(GcsUtil.class);
-
-  /** Maximum number of items to retrieve per Objects.List request. */
-  private static final long MAX_LIST_ITEMS_PER_CALL = 1024;
-
-  /** Matches a glob containing a wildcard, capturing the portion before the first wildcard. */
-  private static final Pattern GLOB_PREFIX = Pattern.compile("(?<PREFIX>[^\\[*?]*)[\\[*?].*");
-
-  private static final String RECURSIVE_WILDCARD = "[*]{2}";
-
-  /**
-   * A {@link Pattern} for globs with a recursive wildcard.
-   */
-  private static final Pattern RECURSIVE_GCS_PATTERN =
-      Pattern.compile(".*" + RECURSIVE_WILDCARD + ".*");
-
-  /**
-   * Maximum number of requests permitted in a GCS batch request.
-   */
-  private static final int MAX_REQUESTS_PER_BATCH = 100;
-  /**
-   * Maximum number of concurrent batches of requests executing on GCS.
-   */
-  private static final int MAX_CONCURRENT_BATCHES = 256;
-
-  private static final FluentBackoff BACKOFF_FACTORY =
-      FluentBackoff.DEFAULT.withMaxRetries(3).withInitialBackoff(Duration.millis(200));
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /** Client for the GCS API. */
-  private Storage storageClient;
-  private final HttpRequestInitializer httpRequestInitializer;
-  /** Buffer size for GCS uploads (in bytes). */
-  @Nullable private final Integer uploadBufferSizeBytes;
-
-  // Helper delegate for turning IOExceptions from API calls into higher-level semantics.
-  private final ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
-
-  // Exposed for testing.
-  final ExecutorService executorService;
-
-  /**
-   * Returns true if the given GCS pattern is supported otherwise fails with an
-   * exception.
-   */
-  public static boolean isGcsPatternSupported(String gcsPattern) {
-    if (RECURSIVE_GCS_PATTERN.matcher(gcsPattern).matches()) {
-      throw new IllegalArgumentException("Unsupported wildcard usage in \"" + gcsPattern + "\": "
-          + " recursive wildcards are not supported.");
-    }
-    return true;
-  }
-
-  /**
-   * Returns the prefix portion of the glob that doesn't contain wildcards.
-   */
-  public static String getGlobPrefix(String globExp) {
-    checkArgument(isGcsPatternSupported(globExp));
-    Matcher m = GLOB_PREFIX.matcher(globExp);
-    checkArgument(
-        m.matches(),
-        String.format("Glob expression: [%s] is not expandable.", globExp));
-    return m.group("PREFIX");
-  }
-
-  /**
-   * Expands glob expressions to regular expressions.
-   *
-   * @param globExp the glob expression to expand
-   * @return a string with the regular expression this glob expands to
-   */
-  public static String globToRegexp(String globExp) {
-    StringBuilder dst = new StringBuilder();
-    char[] src = globExp.toCharArray();
-    int i = 0;
-    while (i < src.length) {
-      char c = src[i++];
-      switch (c) {
-        case '*':
-          dst.append("[^/]*");
-          break;
-        case '?':
-          dst.append("[^/]");
-          break;
-        case '.':
-        case '+':
-        case '{':
-        case '}':
-        case '(':
-        case ')':
-        case '|':
-        case '^':
-        case '$':
-          // These need to be escaped in regular expressions
-          dst.append('\\').append(c);
-          break;
-        case '\\':
-          i = doubleSlashes(dst, src, i);
-          break;
-        default:
-          dst.append(c);
-          break;
-      }
-    }
-    return dst.toString();
-  }
-
-  /**
-   * Returns true if the given {@code spec} contains glob.
-   */
-  public static boolean isGlob(GcsPath spec) {
-    return GLOB_PREFIX.matcher(spec.getObject()).matches();
-  }
-
-  private GcsUtil(
-      Storage storageClient,
-      HttpRequestInitializer httpRequestInitializer,
-      ExecutorService executorService,
-      @Nullable Integer uploadBufferSizeBytes) {
-    this.storageClient = storageClient;
-    this.httpRequestInitializer = httpRequestInitializer;
-    this.uploadBufferSizeBytes = uploadBufferSizeBytes;
-    this.executorService = executorService;
-  }
-
-  // Use this only for testing purposes.
-  protected void setStorageClient(Storage storageClient) {
-    this.storageClient = storageClient;
-  }
-
-  /**
-   * Expands a pattern into matched paths. The pattern path may contain globs, which are expanded
-   * in the result. For patterns that only match a single object, we ensure that the object
-   * exists.
-   */
-  public List<GcsPath> expand(GcsPath gcsPattern) throws IOException {
-    checkArgument(isGcsPatternSupported(gcsPattern.getObject()));
-    Pattern p = null;
-    String prefix = null;
-    if (!isGlob(gcsPattern)) {
-      // Not a glob.
-      try {
-        // Use a get request to fetch the metadata of the object, and ignore the return value.
-        // The request has strong global consistency.
-        getObject(gcsPattern);
-        return ImmutableList.of(gcsPattern);
-      } catch (FileNotFoundException e) {
-        // If the path was not found, return an empty list.
-        return ImmutableList.of();
-      }
-    } else {
-      // Part before the first wildcard character.
-      prefix = getGlobPrefix(gcsPattern.getObject());
-      p = Pattern.compile(globToRegexp(gcsPattern.getObject()));
-    }
-
-    LOG.debug("matching files in bucket {}, prefix {} against pattern {}", gcsPattern.getBucket(),
-        prefix, p.toString());
-
-    String pageToken = null;
-    List<GcsPath> results = new LinkedList<>();
-    do {
-      Objects objects = listObjects(gcsPattern.getBucket(), prefix, pageToken);
-      if (objects.getItems() == null) {
-        break;
-      }
-
-      // Filter objects based on the regex.
-      for (StorageObject o : objects.getItems()) {
-        String name = o.getName();
-        // Skip directories, which end with a slash.
-        if (p.matcher(name).matches() && !name.endsWith("/")) {
-          LOG.debug("Matched object: {}", name);
-          results.add(GcsPath.fromObject(o));
-        }
-      }
-      pageToken = objects.getNextPageToken();
-    } while (pageToken != null);
-
-    return results;
-  }
-
-  @VisibleForTesting
-  @Nullable
-  Integer getUploadBufferSizeBytes() {
-    return uploadBufferSizeBytes;
-  }
-
-  /**
-   * Returns the file size from GCS or throws {@link FileNotFoundException}
-   * if the resource does not exist.
-   */
-  public long fileSize(GcsPath path) throws IOException {
-    return getObject(path).getSize().longValue();
-  }
-
-  /**
-   * Returns the {@link StorageObject} for the given {@link GcsPath}.
-   */
-  public StorageObject getObject(GcsPath gcsPath) throws IOException {
-    return getObject(gcsPath, BACKOFF_FACTORY.backoff(), Sleeper.DEFAULT);
-  }
-
-  @VisibleForTesting
-  StorageObject getObject(GcsPath gcsPath, BackOff backoff, Sleeper sleeper) throws IOException {
-    Storage.Objects.Get getObject =
-        storageClient.objects().get(gcsPath.getBucket(), gcsPath.getObject());
-    try {
-      return ResilientOperation.retry(
-          ResilientOperation.getGoogleRequestCallable(getObject),
-          backoff,
-          RetryDeterminer.SOCKET_ERRORS,
-          IOException.class,
-          sleeper);
-    } catch (IOException | InterruptedException e) {
-      if (e instanceof InterruptedException) {
-        Thread.currentThread().interrupt();
-      }
-      if (e instanceof IOException && errorExtractor.itemNotFound((IOException) e)) {
-        throw new FileNotFoundException(gcsPath.toString());
-      }
-      throw new IOException(
-          String.format("Unable to get the file object for path %s.", gcsPath),
-          e);
-    }
-  }
-
-  /**
-   * Returns {@link StorageObjectOrIOException StorageObjectOrIOExceptions} for the given
-   * {@link GcsPath GcsPaths}.
-   */
-  public List<StorageObjectOrIOException> getObjects(List<GcsPath> gcsPaths)
-      throws IOException {
-    List<StorageObjectOrIOException[]> results = new ArrayList<>();
-    executeBatches(makeGetBatches(gcsPaths, results));
-    ImmutableList.Builder<StorageObjectOrIOException> ret = ImmutableList.builder();
-    for (StorageObjectOrIOException[] result : results) {
-      ret.add(result[0]);
-    }
-    return ret.build();
-  }
-
-  /**
-   * Lists {@link Objects} given the {@code bucket}, {@code prefix}, {@code pageToken}.
-   */
-  public Objects listObjects(String bucket, String prefix, @Nullable String pageToken)
-      throws IOException {
-    // List all objects that start with the prefix (including objects in sub-directories).
-    Storage.Objects.List listObject = storageClient.objects().list(bucket);
-    listObject.setMaxResults(MAX_LIST_ITEMS_PER_CALL);
-    listObject.setPrefix(prefix);
-
-    if (pageToken != null) {
-      listObject.setPageToken(pageToken);
-    }
-
-    try {
-      return ResilientOperation.retry(
-          ResilientOperation.getGoogleRequestCallable(listObject),
-          BACKOFF_FACTORY.backoff(),
-          RetryDeterminer.SOCKET_ERRORS,
-          IOException.class);
-    } catch (Exception e) {
-      throw new IOException(
-          String.format("Unable to match files in bucket %s, prefix %s.", bucket, prefix),
-          e);
-    }
-  }
-
-  /**
-   * Returns the file size from GCS or throws {@link FileNotFoundException}
-   * if the resource does not exist.
-   */
-  @VisibleForTesting
-  List<Long> fileSizes(List<GcsPath> paths) throws IOException {
-    List<StorageObjectOrIOException> results = getObjects(paths);
-
-    ImmutableList.Builder<Long> ret = ImmutableList.builder();
-    for (StorageObjectOrIOException result : results) {
-      ret.add(toFileSize(result));
-    }
-    return ret.build();
-  }
-
-  private Long toFileSize(StorageObjectOrIOException storageObjectOrIOException)
-      throws IOException {
-    if (storageObjectOrIOException.ioException() != null) {
-      throw storageObjectOrIOException.ioException();
-    } else {
-      return storageObjectOrIOException.storageObject().getSize().longValue();
-    }
-  }
-
-  /**
-   * Opens an object in GCS.
-   *
-   * <p>Returns a SeekableByteChannel that provides access to data in the bucket.
-   *
-   * @param path the GCS filename to read from
-   * @return a SeekableByteChannel that can read the object data
-   */
-  public SeekableByteChannel open(GcsPath path)
-      throws IOException {
-    return new GoogleCloudStorageReadChannel(storageClient, path.getBucket(),
-            path.getObject(), errorExtractor,
-            new ClientRequestHelper<StorageObject>());
-  }
-
-  /**
-   * Creates an object in GCS.
-   *
-   * <p>Returns a WritableByteChannel that can be used to write data to the
-   * object.
-   *
-   * @param path the GCS file to write to
-   * @param type the type of object, eg "text/plain".
-   * @return a Callable object that encloses the operation.
-   */
-  public WritableByteChannel create(GcsPath path,
-      String type) throws IOException {
-    GoogleCloudStorageWriteChannel channel = new GoogleCloudStorageWriteChannel(
-        executorService,
-        storageClient,
-        new ClientRequestHelper<StorageObject>(),
-        path.getBucket(),
-        path.getObject(),
-        AsyncWriteChannelOptions.newBuilder().build(),
-        new ObjectWriteConditions(),
-        Collections.<String, String>emptyMap(),
-        type);
-    if (uploadBufferSizeBytes != null) {
-      channel.setUploadBufferSize(uploadBufferSizeBytes);
-    }
-    channel.initialize();
-    return channel;
-  }
-
-  /**
-   * Returns whether the GCS bucket exists and is accessible.
-   */
-  public boolean bucketAccessible(GcsPath path) throws IOException {
-    return bucketAccessible(
-        path,
-        BACKOFF_FACTORY.backoff(),
-        Sleeper.DEFAULT);
-  }
-
-  /**
-   * Returns the project number of the project which owns this bucket.
-   * If the bucket exists, it must be accessible otherwise the permissions
-   * exception will be propagated.  If the bucket does not exist, an exception
-   * will be thrown.
-   */
-  public long bucketOwner(GcsPath path) throws IOException {
-    return getBucket(
-        path,
-        BACKOFF_FACTORY.backoff(),
-        Sleeper.DEFAULT).getProjectNumber().longValue();
-  }
-
-  /**
-   * Creates a {@link Bucket} under the specified project in Cloud Storage or
-   * propagates an exception.
-   */
-  public void createBucket(String projectId, Bucket bucket) throws IOException {
-    createBucket(
-        projectId, bucket, BACKOFF_FACTORY.backoff(), Sleeper.DEFAULT);
-  }
-
-  /**
-   * Returns whether the GCS bucket exists. This will return false if the bucket
-   * is inaccessible due to permissions.
-   */
-  @VisibleForTesting
-  boolean bucketAccessible(GcsPath path, BackOff backoff, Sleeper sleeper) throws IOException {
-    try {
-      return getBucket(path, backoff, sleeper) != null;
-    } catch (AccessDeniedException | FileNotFoundException e) {
-      return false;
-    }
-  }
-
-  @VisibleForTesting
-  @Nullable
-  Bucket getBucket(GcsPath path, BackOff backoff, Sleeper sleeper) throws IOException {
-    Storage.Buckets.Get getBucket =
-        storageClient.buckets().get(path.getBucket());
-
-      try {
-        Bucket bucket = ResilientOperation.retry(
-            ResilientOperation.getGoogleRequestCallable(getBucket),
-            backoff,
-            new RetryDeterminer<IOException>() {
-              @Override
-              public boolean shouldRetry(IOException e) {
-                if (errorExtractor.itemNotFound(e) || errorExtractor.accessDenied(e)) {
-                  return false;
-                }
-                return RetryDeterminer.SOCKET_ERRORS.shouldRetry(e);
-              }
-            },
-            IOException.class,
-            sleeper);
-
-        return bucket;
-      } catch (GoogleJsonResponseException e) {
-        if (errorExtractor.accessDenied(e)) {
-          throw new AccessDeniedException(path.toString(), null, e.getMessage());
-        }
-        if (errorExtractor.itemNotFound(e)) {
-          throw new FileNotFoundException(e.getMessage());
-        }
-        throw e;
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new IOException(
-            String.format("Error while attempting to verify existence of bucket gs://%s",
-                path.getBucket()), e);
-     }
-  }
-
-  @VisibleForTesting
-  void createBucket(String projectId, Bucket bucket, BackOff backoff, Sleeper sleeper)
-        throws IOException {
-    Storage.Buckets.Insert insertBucket =
-        storageClient.buckets().insert(projectId, bucket);
-    insertBucket.setPredefinedAcl("projectPrivate");
-    insertBucket.setPredefinedDefaultObjectAcl("projectPrivate");
-
-    try {
-      ResilientOperation.retry(
-        ResilientOperation.getGoogleRequestCallable(insertBucket),
-        backoff,
-        new RetryDeterminer<IOException>() {
-          @Override
-          public boolean shouldRetry(IOException e) {
-            if (errorExtractor.itemAlreadyExists(e) || errorExtractor.accessDenied(e)) {
-              return false;
-            }
-            return RetryDeterminer.SOCKET_ERRORS.shouldRetry(e);
-          }
-        },
-        IOException.class,
-        sleeper);
-      return;
-    } catch (GoogleJsonResponseException e) {
-      if (errorExtractor.accessDenied(e)) {
-        throw new AccessDeniedException(bucket.getName(), null, e.getMessage());
-      }
-      if (errorExtractor.itemAlreadyExists(e)) {
-        throw new FileAlreadyExistsException(bucket.getName(), null, e.getMessage());
-      }
-      throw e;
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw new IOException(
-        String.format("Error while attempting to create bucket gs://%s for rproject %s",
-                      bucket.getName(), projectId), e);
-    }
-  }
-
-  private static void executeBatches(List<BatchRequest> batches) throws IOException {
-    ListeningExecutorService executor = MoreExecutors.listeningDecorator(
-        MoreExecutors.getExitingExecutorService(
-            new ThreadPoolExecutor(MAX_CONCURRENT_BATCHES, MAX_CONCURRENT_BATCHES,
-                0L, TimeUnit.MILLISECONDS,
-                new LinkedBlockingQueue<Runnable>())));
-
-    List<ListenableFuture<Void>> futures = new LinkedList<>();
-    for (final BatchRequest batch : batches) {
-      futures.add(executor.submit(new Callable<Void>() {
-        public Void call() throws IOException {
-          batch.execute();
-          return null;
-        }
-      }));
-    }
-
-    try {
-      Futures.allAsList(futures).get();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw new IOException("Interrupted while executing batch GCS request", e);
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof FileNotFoundException) {
-        throw (FileNotFoundException) e.getCause();
-      }
-      throw new IOException("Error executing batch GCS request", e);
-    } finally {
-      executor.shutdown();
-    }
-  }
-
-  /**
-   * Makes get {@link BatchRequest BatchRequests}.
-   *
-   * @param paths {@link GcsPath GcsPaths}.
-   * @param results mutable {@link List} for return values.
-   * @return {@link BatchRequest BatchRequests} to execute.
-   * @throws IOException
-   */
-  @VisibleForTesting
-  List<BatchRequest> makeGetBatches(
-      Collection<GcsPath> paths,
-      List<StorageObjectOrIOException[]> results) throws IOException {
-    List<BatchRequest> batches = new LinkedList<>();
-    for (List<GcsPath> filesToGet :
-        Lists.partition(Lists.newArrayList(paths), MAX_REQUESTS_PER_BATCH)) {
-      BatchRequest batch = createBatchRequest();
-      for (GcsPath path : filesToGet) {
-        results.add(enqueueGetFileSize(path, batch));
-      }
-      batches.add(batch);
-    }
-    return batches;
-  }
-
-  public void copy(Iterable<String> srcFilenames,
-                   Iterable<String> destFilenames) throws
-      IOException {
-    executeBatches(makeCopyBatches(srcFilenames, destFilenames));
-  }
-
-  List<BatchRequest> makeCopyBatches(Iterable<String> srcFilenames, Iterable<String> destFilenames)
-      throws IOException {
-    List<String> srcList = Lists.newArrayList(srcFilenames);
-    List<String> destList = Lists.newArrayList(destFilenames);
-    checkArgument(
-        srcList.size() == destList.size(),
-        "Number of source files %s must equal number of destination files %s",
-        srcList.size(),
-        destList.size());
-
-    List<BatchRequest> batches = new LinkedList<>();
-    BatchRequest batch = createBatchRequest();
-    for (int i = 0; i < srcList.size(); i++) {
-      final GcsPath sourcePath = GcsPath.fromUri(srcList.get(i));
-      final GcsPath destPath = GcsPath.fromUri(destList.get(i));
-      enqueueCopy(sourcePath, destPath, batch);
-      if (batch.size() >= MAX_REQUESTS_PER_BATCH) {
-        batches.add(batch);
-        batch = createBatchRequest();
-      }
-    }
-    if (batch.size() > 0) {
-      batches.add(batch);
-    }
-    return batches;
-  }
-
-  List<BatchRequest> makeRemoveBatches(Collection<String> filenames) throws IOException {
-    List<BatchRequest> batches = new LinkedList<>();
-    for (List<String> filesToDelete :
-        Lists.partition(Lists.newArrayList(filenames), MAX_REQUESTS_PER_BATCH)) {
-      BatchRequest batch = createBatchRequest();
-      for (String file : filesToDelete) {
-        enqueueDelete(GcsPath.fromUri(file), batch);
-      }
-      batches.add(batch);
-    }
-    return batches;
-  }
-
-  public void remove(Collection<String> filenames) throws IOException {
-    executeBatches(makeRemoveBatches(filenames));
-  }
-
-  private StorageObjectOrIOException[] enqueueGetFileSize(final GcsPath path, BatchRequest batch)
-      throws IOException {
-    final StorageObjectOrIOException[] ret = new StorageObjectOrIOException[1];
-
-    Storage.Objects.Get getRequest = storageClient.objects()
-        .get(path.getBucket(), path.getObject());
-    getRequest.queue(batch, new JsonBatchCallback<StorageObject>() {
-      @Override
-      public void onSuccess(StorageObject response, HttpHeaders httpHeaders) throws IOException {
-        ret[0] = StorageObjectOrIOException.create(response);
-      }
-
-      @Override
-      public void onFailure(GoogleJsonError e, HttpHeaders httpHeaders) throws IOException {
-        IOException ioException;
-        if (errorExtractor.itemNotFound(e)) {
-          ioException = new FileNotFoundException(path.toString());
-        } else {
-          ioException = new IOException(String.format("Error trying to get %s: %s", path, e));
-        }
-        ret[0] = StorageObjectOrIOException.create(ioException);
-      }
-    });
-    return ret;
-  }
-
-  /**
-   * A class that holds either a {@link StorageObject} or an {@link IOException}.
-   */
-  @AutoValue
-  public abstract static class StorageObjectOrIOException {
-
-    /**
-     * Returns the {@link StorageObject}.
-     */
-    @Nullable
-    public abstract StorageObject storageObject();
-
-    /**
-     * Returns the {@link IOException}.
-     */
-    @Nullable
-    public abstract IOException ioException();
-
-    @VisibleForTesting
-    public static StorageObjectOrIOException create(StorageObject storageObject) {
-      return new AutoValue_GcsUtil_StorageObjectOrIOException(
-          checkNotNull(storageObject, "storageObject"),
-          null /* ioException */);
-    }
-
-    @VisibleForTesting
-    public static StorageObjectOrIOException create(IOException ioException) {
-      return new AutoValue_GcsUtil_StorageObjectOrIOException(
-          null /* storageObject */,
-          checkNotNull(ioException, "ioException"));
-    }
-  }
-
-  private void enqueueCopy(final GcsPath from, final GcsPath to, BatchRequest batch)
-      throws IOException {
-    Storage.Objects.Copy copyRequest = storageClient.objects()
-        .copy(from.getBucket(), from.getObject(), to.getBucket(), to.getObject(), null);
-    copyRequest.queue(batch, new JsonBatchCallback<StorageObject>() {
-      @Override
-      public void onSuccess(StorageObject obj, HttpHeaders responseHeaders) {
-        LOG.debug("Successfully copied {} to {}", from, to);
-      }
-
-      @Override
-      public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException {
-        throw new IOException(
-            String.format("Error trying to copy %s to %s: %s", from, to, e));
-      }
-    });
-  }
-
-  private void enqueueDelete(final GcsPath file, BatchRequest batch) throws IOException {
-    Storage.Objects.Delete deleteRequest = storageClient.objects()
-        .delete(file.getBucket(), file.getObject());
-    deleteRequest.queue(batch, new JsonBatchCallback<Void>() {
-      @Override
-      public void onSuccess(Void obj, HttpHeaders responseHeaders) {
-        LOG.debug("Successfully deleted {}", file);
-      }
-
-      @Override
-      public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException {
-        throw new IOException(String.format("Error trying to delete %s: %s", file, e));
-      }
-    });
-  }
-
-  private BatchRequest createBatchRequest() {
-    return storageClient.batch(httpRequestInitializer);
-  }
-
-  private static int doubleSlashes(StringBuilder dst, char[] src, int i) {
-    // Emit the next character without special interpretation
-    dst.append('\\');
-    if ((i - 1) != src.length) {
-      dst.append(src[i]);
-      i++;
-    } else {
-      // A backslash at the very end is treated like an escaped backslash
-      dst.append('\\');
-    }
-    return i;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
deleted file mode 100644
index 6fac6dc..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
+++ /dev/null
@@ -1,89 +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 com.google.api.client.util.BackOff;
-
-
-/**
- * Implementation of {@link BackOff} that increases the back off period for each retry attempt
- * using a randomization function that grows exponentially.
- *
- * <p>Example: The initial interval is .5 seconds and the maximum interval is 60 secs.
- * For 14 tries the sequence will be (values in seconds):
- *
- * <pre>
- * retry#      retry_interval     randomized_interval
- * 1             0.5                [0.25,   0.75]
- * 2             0.75               [0.375,  1.125]
- * 3             1.125              [0.562,  1.687]
- * 4             1.687              [0.8435, 2.53]
- * 5             2.53               [1.265,  3.795]
- * 6             3.795              [1.897,  5.692]
- * 7             5.692              [2.846,  8.538]
- * 8             8.538              [4.269, 12.807]
- * 9            12.807              [6.403, 19.210]
- * 10           28.832              [14.416, 43.248]
- * 11           43.248              [21.624, 64.873]
- * 12           60.0                [30.0, 90.0]
- * 13           60.0                [30.0, 90.0]
- * 14           60.0                [30.0, 90.0]
- * </pre>
- *
- * <p>Implementation is not thread-safe.
- */
-@Deprecated
-public class IntervalBoundedExponentialBackOff implements BackOff {
-  public static final double DEFAULT_MULTIPLIER = 1.5;
-  public static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5;
-  private final long maximumIntervalMillis;
-  private final long initialIntervalMillis;
-  private int currentAttempt;
-
-  public IntervalBoundedExponentialBackOff(long maximumIntervalMillis, long initialIntervalMillis) {
-    checkArgument(maximumIntervalMillis > 0, "Maximum interval must be greater than zero.");
-    checkArgument(initialIntervalMillis > 0, "Initial interval must be greater than zero.");
-    this.maximumIntervalMillis = maximumIntervalMillis;
-    this.initialIntervalMillis = initialIntervalMillis;
-    reset();
-  }
-
-  @Override
-  public void reset() {
-    currentAttempt = 1;
-  }
-
-  @Override
-  public long nextBackOffMillis() {
-    double currentIntervalMillis =
-        Math.min(
-            initialIntervalMillis * Math.pow(DEFAULT_MULTIPLIER, currentAttempt - 1),
-            maximumIntervalMillis);
-    double randomOffset =
-        (Math.random() * 2 - 1) * DEFAULT_RANDOMIZATION_FACTOR * currentIntervalMillis;
-    currentAttempt += 1;
-    return Math.round(currentIntervalMillis + randomOffset);
-  }
-
-  public boolean atMaxInterval() {
-    return initialIntervalMillis * Math.pow(DEFAULT_MULTIPLIER, currentAttempt - 1)
-        >= maximumIntervalMillis;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java
deleted file mode 100644
index f703e4c..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java
+++ /dev/null
@@ -1,68 +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 com.google.auth.Credentials;
-import java.io.IOException;
-import java.net.URI;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-/**
- * Construct an oauth credential to be used by the SDK and the SDK workers.
- * Always returns a null Credential object.
- */
-public class NoopCredentialFactory implements CredentialFactory {
-  private static final NoopCredentialFactory INSTANCE = new NoopCredentialFactory();
-  private static final NoopCredentials NOOP_CREDENTIALS = new NoopCredentials();
-
-  public static NoopCredentialFactory fromOptions(PipelineOptions options) {
-    return INSTANCE;
-  }
-
-  @Override
-  public Credentials getCredential() throws IOException {
-    return NOOP_CREDENTIALS;
-  }
-
-  private static class NoopCredentials extends Credentials {
-    @Override
-    public String getAuthenticationType() {
-      return null;
-    }
-
-    @Override
-    public Map<String, List<String>> getRequestMetadata(URI uri) throws IOException {
-      return null;
-    }
-
-    @Override
-    public boolean hasRequestMetadata() {
-      return false;
-    }
-
-    @Override
-    public boolean hasRequestMetadataOnly() {
-      return false;
-    }
-
-    @Override
-    public void refresh() throws IOException {}
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullCredentialInitializer.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullCredentialInitializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullCredentialInitializer.java
deleted file mode 100644
index 4ed35c6..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullCredentialInitializer.java
+++ /dev/null
@@ -1,62 +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 com.google.api.client.http.HttpRequest;
-import com.google.api.client.http.HttpRequestInitializer;
-import com.google.api.client.http.HttpResponse;
-import com.google.api.client.http.HttpUnsuccessfulResponseHandler;
-import java.io.IOException;
-
-/**
- * A {@link HttpRequestInitializer} for requests that don't have credentials.
- *
- * <p>When the access is denied, it throws {@link IOException} with a detailed error message.
- */
-public class NullCredentialInitializer implements HttpRequestInitializer {
-  private static final int ACCESS_DENIED = 401;
-  private static final String NULL_CREDENTIAL_REASON =
-      "Unable to get application default credentials. Please see "
-      + "https://developers.google.com/accounts/docs/application-default-credentials "
-      + "for details on how to specify credentials. This version of the SDK is "
-      + "dependent on the gcloud core component version 2015.02.05 or newer to "
-      + "be able to get credentials from the currently authorized user via gcloud auth.";
-
-  @Override
-  public void initialize(HttpRequest httpRequest) throws IOException {
-    httpRequest.setUnsuccessfulResponseHandler(new NullCredentialHttpUnsuccessfulResponseHandler());
-  }
-
-  private static class NullCredentialHttpUnsuccessfulResponseHandler
-      implements HttpUnsuccessfulResponseHandler {
-
-    @Override
-    public boolean handleResponse(
-        HttpRequest httpRequest,
-        HttpResponse httpResponse, boolean supportsRetry) throws IOException {
-      if (!httpResponse.isSuccessStatusCode() && httpResponse.getStatusCode() == ACCESS_DENIED) {
-        throwNullCredentialException();
-      }
-      return supportsRetry;
-    }
-  }
-
-  public static void throwNullCredentialException() {
-    throw new RuntimeException(NULL_CREDENTIAL_REASON);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java
deleted file mode 100644
index 80c093b..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java
+++ /dev/null
@@ -1,178 +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 com.google.api.client.googleapis.javanet.GoogleNetHttpTransport;
-import com.google.api.client.http.HttpRequestInitializer;
-import com.google.api.client.http.HttpTransport;
-import com.google.api.client.json.JsonFactory;
-import com.google.api.client.json.jackson2.JacksonFactory;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.cloudresourcemanager.CloudResourceManager;
-import com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.storage.Storage;
-import com.google.auth.Credentials;
-import com.google.auth.http.HttpCredentialsAdapter;
-import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
-import com.google.common.collect.ImmutableList;
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.security.GeneralSecurityException;
-import org.apache.beam.sdk.options.BigQueryOptions;
-import org.apache.beam.sdk.options.CloudResourceManagerOptions;
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PubsubOptions;
-
-/**
- * Helpers for cloud communication.
- */
-public class Transport {
-
-  private static class SingletonHelper {
-    /** Global instance of the JSON factory. */
-    private static final JsonFactory JSON_FACTORY;
-
-    /** Global instance of the HTTP transport. */
-    private static final HttpTransport HTTP_TRANSPORT;
-
-    static {
-      try {
-        JSON_FACTORY = JacksonFactory.getDefaultInstance();
-        HTTP_TRANSPORT = GoogleNetHttpTransport.newTrustedTransport();
-      } catch (GeneralSecurityException | IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-
-  public static HttpTransport getTransport() {
-    return SingletonHelper.HTTP_TRANSPORT;
-  }
-
-  public static JsonFactory getJsonFactory() {
-    return SingletonHelper.JSON_FACTORY;
-  }
-
-  private static class ApiComponents {
-    public String rootUrl;
-    public String servicePath;
-
-    public ApiComponents(String root, String path) {
-      this.rootUrl = root;
-      this.servicePath = path;
-    }
-  }
-
-  private static ApiComponents apiComponentsFromUrl(String urlString) {
-    try {
-      URL url = new URL(urlString);
-      String rootUrl = url.getProtocol() + "://" + url.getHost()
-          + (url.getPort() > 0 ? ":" + url.getPort() : "");
-      return new ApiComponents(rootUrl, url.getPath());
-    } catch (MalformedURLException e) {
-      throw new RuntimeException("Invalid URL: " + urlString);
-    }
-  }
-
-  /**
-   * Returns a BigQuery client builder using the specified {@link BigQueryOptions}.
-   */
-  public static Bigquery.Builder
-      newBigQueryClient(BigQueryOptions options) {
-    return new Bigquery.Builder(getTransport(), getJsonFactory(),
-        chainHttpRequestInitializer(
-            options.getGcpCredential(),
-            // Do not log 404. It clutters the output and is possibly even required by the caller.
-            new RetryHttpRequestInitializer(ImmutableList.of(404))))
-        .setApplicationName(options.getAppName())
-        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
-  }
-
-  /**
-   * Returns a Pubsub client builder using the specified {@link PubsubOptions}.
-   *
-   * @deprecated Use an appropriate org.apache.beam.sdk.util.PubsubClient.PubsubClientFactory
-   */
-  @Deprecated
-  public static Pubsub.Builder
-      newPubsubClient(PubsubOptions options) {
-    return new Pubsub.Builder(getTransport(), getJsonFactory(),
-        chainHttpRequestInitializer(
-            options.getGcpCredential(),
-            // Do not log 404. It clutters the output and is possibly even required by the caller.
-            new RetryHttpRequestInitializer(ImmutableList.of(404))))
-        .setRootUrl(options.getPubsubRootUrl())
-        .setApplicationName(options.getAppName())
-        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
-  }
-
-  /**
-   * Returns a CloudResourceManager client builder using the specified
-   * {@link CloudResourceManagerOptions}.
-   */
-  public static CloudResourceManager.Builder
-      newCloudResourceManagerClient(CloudResourceManagerOptions options) {
-    Credentials credentials = options.getGcpCredential();
-    if (credentials == null) {
-      NullCredentialInitializer.throwNullCredentialException();
-    }
-    return new CloudResourceManager.Builder(getTransport(), getJsonFactory(),
-        chainHttpRequestInitializer(
-            credentials,
-            // Do not log 404. It clutters the output and is possibly even required by the caller.
-            new RetryHttpRequestInitializer(ImmutableList.of(404))))
-        .setApplicationName(options.getAppName())
-        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
-  }
-
-  /**
-   * Returns a Cloud Storage client builder using the specified {@link GcsOptions}.
-   */
-  public static Storage.Builder
-      newStorageClient(GcsOptions options) {
-    String servicePath = options.getGcsEndpoint();
-    Storage.Builder storageBuilder = new Storage.Builder(getTransport(), getJsonFactory(),
-        chainHttpRequestInitializer(
-            options.getGcpCredential(),
-            // Do not log the code 404. Code up the stack will deal with 404's if needed, and
-            // logging it by default clutters the output during file staging.
-            new RetryHttpRequestInitializer(
-                ImmutableList.of(404), new UploadIdResponseInterceptor())))
-        .setApplicationName(options.getAppName())
-        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
-    if (servicePath != null) {
-      ApiComponents components = apiComponentsFromUrl(servicePath);
-      storageBuilder.setRootUrl(components.rootUrl);
-      storageBuilder.setServicePath(components.servicePath);
-    }
-    return storageBuilder;
-  }
-
-  private static HttpRequestInitializer chainHttpRequestInitializer(
-      Credentials credential, HttpRequestInitializer httpRequestInitializer) {
-    if (credential == null) {
-      return new ChainingHttpRequestInitializer(
-          new NullCredentialInitializer(), httpRequestInitializer);
-    } else {
-      return new ChainingHttpRequestInitializer(
-          new HttpCredentialsAdapter(credential),
-          httpRequestInitializer);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java b/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java
index c4b3a9f..66c1393 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java
@@ -40,8 +40,6 @@ public class SdkCoreApiSurfaceTest {
             "org.apache.beam",
             "com.google.api.client",
             "com.google.api.services.bigquery",
-            "com.google.api.services.cloudresourcemanager",
-            "com.google.api.services.pubsub",
             "com.google.api.services.storage",
             "com.google.auth",
             "com.google.protobuf",

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
index 3b6992a..a8655a6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
@@ -48,7 +48,6 @@ import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileOutputStream;
@@ -56,14 +55,11 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.PrintStream;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SeekableByteChannel;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.FileVisitResult;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.SimpleFileVisitor;
-import java.nio.file.StandardOpenOption;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -72,17 +68,13 @@ import java.util.Set;
 import java.util.zip.GZIPOutputStream;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipOutputStream;
-
 import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
 import org.apache.beam.sdk.io.FileBasedSink.WritableByteChannelFactory;
 import org.apache.beam.sdk.io.TextIO.CompressionType;
 import org.apache.beam.sdk.io.TextIO.TextSource;
-import org.apache.beam.sdk.options.GcsOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.ValueProvider;
@@ -96,9 +88,7 @@ import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
 import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.GcsUtil;
 import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream;
 import org.apache.commons.compress.compressors.deflate.DeflateCompressorOutputStream;
@@ -111,9 +101,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
 
 /**
  * Tests for TextIO Read and Write transforms.
@@ -416,7 +403,7 @@ public class TextIOTest {
   }
 
   private static Predicate<List<String>> haveProperHeaderAndFooter(final String header,
-                                                                   final String footer) {
+      final String footer) {
     return new Predicate<List<String>>() {
       @Override
       public boolean apply(List<String> fileLines) {
@@ -577,21 +564,6 @@ public class TextIOTest {
     input.apply(TextIO.Write.to(filename));
   }
 
-  /**
-   * Recursive wildcards are not supported.
-   * This tests "**".
-   */
-  @Test
-  public void testBadWildcardRecursive() throws Exception {
-    p.enableAbandonedNodeEnforcement(false);
-
-    // Check that applying does fail.
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage("wildcard");
-
-    p.apply(TextIO.Read.from("gs://bucket/foo**/baz"));
-  }
-
   /** Options for testing. */
   public interface RuntimeTestOptions extends PipelineOptions {
     ValueProvider<String> getInput();
@@ -1186,70 +1158,5 @@ public class TextIOTest {
     assertThat(splits, hasSize(equalTo(1)));
     SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options);
   }
-
-  ///////////////////////////////////////////////////////////////////////////////////////////////
-  // Test "gs://" paths
-
-  private GcsUtil buildMockGcsUtil() throws IOException {
-    GcsUtil mockGcsUtil = Mockito.mock(GcsUtil.class);
-
-    // Any request to open gets a new bogus channel
-    Mockito
-        .when(mockGcsUtil.open(Mockito.any(GcsPath.class)))
-        .then(new Answer<SeekableByteChannel>() {
-          @Override
-          public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
-            return FileChannel.open(
-                Files.createTempFile("channel-", ".tmp"),
-                StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
-          }
-        });
-
-    // Any request for expansion returns a list containing the original GcsPath
-    // This is required to pass validation that occurs in TextIO during apply()
-    Mockito
-        .when(mockGcsUtil.expand(Mockito.any(GcsPath.class)))
-        .then(new Answer<List<GcsPath>>() {
-          @Override
-          public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
-            return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
-          }
-        });
-
-    return mockGcsUtil;
-  }
-
-  /**
-   * This tests a few corner cases that should not crash.
-   */
-  @Test
-  @Category(NeedsRunner.class)
-  public void testGoodWildcards() throws Exception {
-    GcsOptions options = TestPipeline.testingPipelineOptions().as(GcsOptions.class);
-    options.setGcsUtil(buildMockGcsUtil());
-
-    Pipeline pipeline = Pipeline.create(options);
-
-    applyRead(pipeline, "gs://bucket/foo");
-    applyRead(pipeline, "gs://bucket/foo/");
-    applyRead(pipeline, "gs://bucket/foo/*");
-    applyRead(pipeline, "gs://bucket/foo/?");
-    applyRead(pipeline, "gs://bucket/foo/[0-9]");
-    applyRead(pipeline, "gs://bucket/foo/*baz*");
-    applyRead(pipeline, "gs://bucket/foo/*baz?");
-    applyRead(pipeline, "gs://bucket/foo/[0-9]baz?");
-    applyRead(pipeline, "gs://bucket/foo/baz/*");
-    applyRead(pipeline, "gs://bucket/foo/baz/*wonka*");
-    applyRead(pipeline, "gs://bucket/foo/*baz/wonka*");
-    applyRead(pipeline, "gs://bucket/foo*/baz");
-    applyRead(pipeline, "gs://bucket/foo?/baz");
-    applyRead(pipeline, "gs://bucket/foo[0-9]/baz");
-
-    // Check that running doesn't fail.
-    pipeline.run();
-  }
-
-  private void applyRead(Pipeline pipeline, String path) {
-    pipeline.apply("Read(" + path + ")", TextIO.Read.from(path));
-  }
 }
+

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
deleted file mode 100644
index 288383e..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
+++ /dev/null
@@ -1,171 +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.options;
-
-import static org.hamcrest.Matchers.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.io.Files;
-import java.io.File;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.Map;
-import org.apache.beam.sdk.options.GcpOptions.DefaultProjectFactory;
-import org.apache.beam.sdk.testing.RestoreSystemProperties;
-import org.apache.beam.sdk.util.NoopPathValidator;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
-import org.junit.rules.TestRule;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link GcpOptions}. */
-@RunWith(JUnit4.class)
-public class GcpOptionsTest {
-  @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
-  @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
-  @Rule public ExpectedException thrown = ExpectedException.none();
-
-  @Test
-  public void testGetProjectFromCloudSdkConfigEnv() throws Exception {
-    Map<String, String> environment =
-        ImmutableMap.of("CLOUDSDK_CONFIG", tmpFolder.getRoot().getAbsolutePath());
-    assertEquals("test-project",
-        runGetProjectTest(tmpFolder.newFile("properties"), environment));
-  }
-
-  @Test
-  public void testGetProjectFromAppDataEnv() throws Exception {
-    Map<String, String> environment =
-        ImmutableMap.of("APPDATA", tmpFolder.getRoot().getAbsolutePath());
-    System.setProperty("os.name", "windows");
-    assertEquals("test-project",
-        runGetProjectTest(new File(tmpFolder.newFolder("gcloud"), "properties"),
-            environment));
-  }
-
-  @Test
-  public void testGetProjectFromUserHomeEnvOld() throws Exception {
-    Map<String, String> environment = ImmutableMap.of();
-    System.setProperty("user.home", tmpFolder.getRoot().getAbsolutePath());
-    assertEquals("test-project",
-        runGetProjectTest(
-            new File(tmpFolder.newFolder(".config", "gcloud"), "properties"),
-            environment));
-  }
-
-  @Test
-  public void testGetProjectFromUserHomeEnv() throws Exception {
-    Map<String, String> environment = ImmutableMap.of();
-    System.setProperty("user.home", tmpFolder.getRoot().getAbsolutePath());
-    assertEquals("test-project",
-        runGetProjectTest(
-            new File(tmpFolder.newFolder(".config", "gcloud", "configurations"), "config_default"),
-            environment));
-  }
-
-  @Test
-  public void testGetProjectFromUserHomeOldAndNewPrefersNew() throws Exception {
-    Map<String, String> environment = ImmutableMap.of();
-    System.setProperty("user.home", tmpFolder.getRoot().getAbsolutePath());
-    makePropertiesFileWithProject(new File(tmpFolder.newFolder(".config", "gcloud"), "properties"),
-        "old-project");
-    assertEquals("test-project",
-        runGetProjectTest(
-            new File(tmpFolder.newFolder(".config", "gcloud", "configurations"), "config_default"),
-            environment));
-  }
-
-  @Test
-  public void testUnableToGetDefaultProject() throws Exception {
-    System.setProperty("user.home", tmpFolder.getRoot().getAbsolutePath());
-    DefaultProjectFactory projectFactory = spy(new DefaultProjectFactory());
-    when(projectFactory.getEnvironment()).thenReturn(ImmutableMap.<String, String>of());
-    assertNull(projectFactory.create(PipelineOptionsFactory.create()));
-  }
-
-  @Test
-  public void testEmptyGcpTempLocation() throws Exception {
-    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
-    options.setProject("");
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("--project is a required option");
-    options.getGcpTempLocation();
-  }
-
-  @Test
-  public void testDefaultGcpTempLocation() throws Exception {
-    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
-    String tempLocation = "gs://bucket";
-    options.setTempLocation(tempLocation);
-    options.as(GcsOptions.class).setPathValidatorClass(NoopPathValidator.class);
-    assertEquals(tempLocation, options.getGcpTempLocation());
-  }
-
-  @Test
-  public void testDefaultGcpTempLocationInvalid() throws Exception {
-    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
-    options.setTempLocation("file://");
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(
-        "Error constructing default value for gcpTempLocation: tempLocation is not"
-            + " a valid GCS path");
-    options.getGcpTempLocation();
-  }
-
-  @Test
-  public void testDefaultGcpTempLocationDoesNotExist() {
-    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
-    String tempLocation = "gs://does/not/exist";
-    options.setTempLocation(tempLocation);
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(
-        "Error constructing default value for gcpTempLocation: tempLocation is not"
-            + " a valid GCS path");
-    thrown.expectCause(
-        hasMessage(containsString("Output path does not exist or is not writeable")));
-
-    options.getGcpTempLocation();
-  }
-
-  private static void makePropertiesFileWithProject(File path, String projectId)
-      throws IOException {
-    String properties = String.format("[core]%n"
-        + "account = test-account@google.com%n"
-        + "project = %s%n"
-        + "%n"
-        + "[dataflow]%n"
-        + "magic = true%n", projectId);
-    Files.write(properties, path, StandardCharsets.UTF_8);
-  }
-
-  private static String runGetProjectTest(File path, Map<String, String> environment)
-      throws Exception {
-    makePropertiesFileWithProject(path, "test-project");
-    DefaultProjectFactory projectFactory = spy(new DefaultProjectFactory());
-    when(projectFactory.getEnvironment()).thenReturn(environment);
-    return projectFactory.create(PipelineOptionsFactory.create());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java
deleted file mode 100644
index dae7208..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java
+++ /dev/null
@@ -1,145 +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.options;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.api.services.bigquery.Bigquery.Datasets.Delete;
-import com.google.api.services.storage.Storage;
-import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer;
-import org.apache.beam.sdk.util.TestCredential;
-import org.apache.beam.sdk.util.Transport;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link GoogleApiDebugOptions}. */
-@RunWith(JUnit4.class)
-public class GoogleApiDebugOptionsTest {
-  private static final String STORAGE_GET_TRACE =
-      "--googleApiTrace={\"Objects.Get\":\"GetTraceDestination\"}";
-  private static final String STORAGE_GET_AND_LIST_TRACE =
-      "--googleApiTrace={\"Objects.Get\":\"GetTraceDestination\","
-      + "\"Objects.List\":\"ListTraceDestination\"}";
-  private static final String STORAGE_TRACE = "--googleApiTrace={\"Storage\":\"TraceDestination\"}";
-
-  @Test
-  public void testWhenTracingMatches() throws Exception {
-    String[] args = new String[] {STORAGE_GET_TRACE};
-    GcsOptions options = PipelineOptionsFactory.fromArgs(args).as(GcsOptions.class);
-    options.setGcpCredential(new TestCredential());
-    assertNotNull(options.getGoogleApiTrace());
-
-    Storage.Objects.Get request =
-        Transport.newStorageClient(options).build().objects().get("testBucketId", "testObjectId");
-    assertEquals("GetTraceDestination", request.get("$trace"));
-  }
-
-  @Test
-  public void testWhenTracingDoesNotMatch() throws Exception {
-    String[] args = new String[] {STORAGE_GET_TRACE};
-    GcsOptions options = PipelineOptionsFactory.fromArgs(args).as(GcsOptions.class);
-    options.setGcpCredential(new TestCredential());
-
-    assertNotNull(options.getGoogleApiTrace());
-
-    Storage.Objects.List request =
-        Transport.newStorageClient(options).build().objects().list("testProjectId");
-    assertNull(request.get("$trace"));
-  }
-
-  @Test
-  public void testWithMultipleTraces() throws Exception {
-    String[] args = new String[] {STORAGE_GET_AND_LIST_TRACE};
-    GcsOptions options = PipelineOptionsFactory.fromArgs(args).as(GcsOptions.class);
-    options.setGcpCredential(new TestCredential());
-
-    assertNotNull(options.getGoogleApiTrace());
-
-    Storage.Objects.Get getRequest =
-        Transport.newStorageClient(options).build().objects().get("testBucketId", "testObjectId");
-    assertEquals("GetTraceDestination", getRequest.get("$trace"));
-
-    Storage.Objects.List listRequest =
-        Transport.newStorageClient(options).build().objects().list("testProjectId");
-    assertEquals("ListTraceDestination", listRequest.get("$trace"));
-  }
-
-  @Test
-  public void testMatchingAllCalls() throws Exception {
-    String[] args = new String[] {STORAGE_TRACE};
-    GcsOptions options =
-        PipelineOptionsFactory.fromArgs(args).as(GcsOptions.class);
-    options.setGcpCredential(new TestCredential());
-
-    assertNotNull(options.getGoogleApiTrace());
-
-    Storage.Objects.Get getRequest =
-        Transport.newStorageClient(options).build().objects().get("testBucketId", "testObjectId");
-    assertEquals("TraceDestination", getRequest.get("$trace"));
-
-    Storage.Objects.List listRequest =
-        Transport.newStorageClient(options).build().objects().list("testProjectId");
-    assertEquals("TraceDestination", listRequest.get("$trace"));
-  }
-
-  @Test
-  public void testMatchingAgainstClient() throws Exception {
-    GcsOptions options = PipelineOptionsFactory.as(GcsOptions.class);
-    options.setGcpCredential(new TestCredential());
-    options.setGoogleApiTrace(new GoogleApiTracer().addTraceFor(
-        Transport.newStorageClient(options).build(), "TraceDestination"));
-
-    Storage.Objects.Get getRequest =
-        Transport.newStorageClient(options).build().objects().get("testBucketId", "testObjectId");
-    assertEquals("TraceDestination", getRequest.get("$trace"));
-
-    Delete deleteRequest = Transport.newBigQueryClient(options.as(BigQueryOptions.class))
-        .build().datasets().delete("testProjectId", "testDatasetId");
-    assertNull(deleteRequest.get("$trace"));
-  }
-
-  @Test
-  public void testMatchingAgainstRequestType() throws Exception {
-    GcsOptions options = PipelineOptionsFactory.as(GcsOptions.class);
-    options.setGcpCredential(new TestCredential());
-    options.setGoogleApiTrace(new GoogleApiTracer().addTraceFor(
-        Transport.newStorageClient(options).build().objects()
-            .get("aProjectId", "aObjectId"), "TraceDestination"));
-
-    Storage.Objects.Get getRequest =
-        Transport.newStorageClient(options).build().objects().get("testBucketId", "testObjectId");
-    assertEquals("TraceDestination", getRequest.get("$trace"));
-
-    Storage.Objects.List listRequest =
-        Transport.newStorageClient(options).build().objects().list("testProjectId");
-    assertNull(listRequest.get("$trace"));
-  }
-
-  @Test
-  public void testDeserializationAndSerializationOfGoogleApiTracer() throws Exception {
-    String serializedValue = "{\"Api\":\"Token\"}";
-    ObjectMapper objectMapper = new ObjectMapper();
-    assertEquals(serializedValue,
-        objectMapper.writeValueAsString(
-            objectMapper.readValue(serializedValue, GoogleApiTracer.class)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
index 7d941bf..769a30a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
@@ -691,9 +691,9 @@ public class PipelineOptionsFactoryTest {
   @Test
   public void testPropertyIsSetOnRegisteredPipelineOptionNotPartOfOriginalInterface() {
     PipelineOptions options = PipelineOptionsFactory
-        .fromArgs("--project=testProject")
+        .fromArgs("--streaming")
         .create();
-    assertEquals("testProject", options.as(GcpOptions.class).getProject());
+    assertTrue(options.as(StreamingOptions.class).isStreaming());
   }
 
   /** A test interface containing all the primitives. */

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
index 71ef311..76d8627 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
@@ -19,65 +19,23 @@ package org.apache.beam.sdk.runners;
 
 import static org.junit.Assert.assertTrue;
 
-import org.apache.beam.sdk.options.ApplicationNameOptions;
-import org.apache.beam.sdk.options.GcpOptions;
-import org.apache.beam.sdk.options.GcsOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.CrashingRunner;
-import org.apache.beam.sdk.util.GcsUtil;
-import org.apache.beam.sdk.util.TestCredential;
-import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
 
 /**
- * Tests for PipelineRunner.
+ * Tests for {@link PipelineRunner}.
  */
 @RunWith(JUnit4.class)
 public class PipelineRunnerTest {
-
-  @Mock private GcsUtil mockGcsUtil;
-
-  @Before
-  public void setUp() {
-    MockitoAnnotations.initMocks(this);
-  }
-
-  @Test
-  public void testLongName() {
-    // Check we can create a pipeline runner using the full class name.
-    PipelineOptions options = PipelineOptionsFactory.create();
-    options.as(ApplicationNameOptions.class).setAppName("test");
-    options.as(GcpOptions.class).setProject("test");
-    options.as(GcsOptions.class).setGcsUtil(mockGcsUtil);
-    options.setRunner(CrashingRunner.class);
-    options.as(GcpOptions.class).setGcpCredential(new TestCredential());
-    PipelineRunner<?> runner = PipelineRunner.fromOptions(options);
-    assertTrue(runner instanceof CrashingRunner);
-  }
-
   @Test
-  public void testShortName() {
-    // Check we can create a pipeline runner using the short class name.
+  public void testInstantiation() {
     PipelineOptions options = PipelineOptionsFactory.create();
-    options.as(ApplicationNameOptions.class).setAppName("test");
-    options.as(GcpOptions.class).setProject("test");
-    options.as(GcsOptions.class).setGcsUtil(mockGcsUtil);
     options.setRunner(CrashingRunner.class);
-    options.as(GcpOptions.class).setGcpCredential(new TestCredential());
     PipelineRunner<?> runner = PipelineRunner.fromOptions(options);
     assertTrue(runner instanceof CrashingRunner);
   }
-
-  @Test
-  public void testAppNameDefault() {
-    ApplicationNameOptions options = PipelineOptionsFactory.as(ApplicationNameOptions.class);
-    Assert.assertEquals(PipelineRunnerTest.class.getSimpleName(),
-        options.getAppName());
-  }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
deleted file mode 100644
index 3b35856..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
+++ /dev/null
@@ -1,176 +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.testing;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.atLeast;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.model.QueryRequest;
-import com.google.api.services.bigquery.model.QueryResponse;
-import com.google.api.services.bigquery.model.TableCell;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.math.BigInteger;
-import org.apache.beam.sdk.PipelineResult;
-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;
-
-/**
- * Tests for {@link BigqueryMatcher}.
- */
-@RunWith(JUnit4.class)
-public class BigqueryMatcherTest {
-  private final String appName = "test-app";
-  private final String projectId = "test-project";
-  private final String query = "test-query";
-
-  @Rule public ExpectedException thrown = ExpectedException.none();
-  @Rule public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper();
-  @Mock private Bigquery mockBigqueryClient;
-  @Mock private Bigquery.Jobs mockJobs;
-  @Mock private Bigquery.Jobs.Query mockQuery;
-  @Mock private PipelineResult mockResult;
-
-  @Before
-  public void setUp() throws IOException {
-    MockitoAnnotations.initMocks(this);
-    when(mockBigqueryClient.jobs()).thenReturn(mockJobs);
-    when(mockJobs.query(anyString(), any(QueryRequest.class))).thenReturn(mockQuery);
-  }
-
-  @Test
-  public void testBigqueryMatcherThatSucceeds() throws Exception {
-    BigqueryMatcher matcher = spy(
-        new BigqueryMatcher(
-            appName, projectId, query, "9bb47f5c90d2a99cad526453dff5ed5ec74650dc"));
-    doReturn(mockBigqueryClient).when(matcher).newBigqueryClient(anyString());
-    when(mockQuery.execute()).thenReturn(createResponseContainingTestData());
-
-    assertThat(mockResult, matcher);
-    verify(matcher).newBigqueryClient(eq(appName));
-    verify(mockJobs).query(eq(projectId), eq(new QueryRequest().setQuery(query)));
-  }
-
-  @Test
-  public void testBigqueryMatcherFailsForChecksumMismatch() throws IOException {
-    BigqueryMatcher matcher = spy(
-        new BigqueryMatcher(appName, projectId, query, "incorrect-checksum"));
-    doReturn(mockBigqueryClient).when(matcher).newBigqueryClient(anyString());
-    when(mockQuery.execute()).thenReturn(createResponseContainingTestData());
-
-    thrown.expect(AssertionError.class);
-    thrown.expectMessage("Total number of rows are: 1");
-    thrown.expectMessage("abc");
-    try {
-      assertThat(mockResult, matcher);
-    } finally {
-      verify(matcher).newBigqueryClient(eq(appName));
-      verify(mockJobs).query(eq(projectId), eq(new QueryRequest().setQuery(query)));
-    }
-  }
-
-  @Test
-  public void testBigqueryMatcherFailsWhenQueryJobNotComplete() throws Exception {
-    BigqueryMatcher matcher = spy(
-        new BigqueryMatcher(appName, projectId, query, "some-checksum"));
-    doReturn(mockBigqueryClient).when(matcher).newBigqueryClient(anyString());
-    when(mockQuery.execute()).thenReturn(new QueryResponse().setJobComplete(false));
-
-    thrown.expect(AssertionError.class);
-    thrown.expectMessage("The query job hasn't completed.");
-    thrown.expectMessage("jobComplete=false");
-    try {
-      assertThat(mockResult, matcher);
-    } finally {
-      verify(matcher).newBigqueryClient(eq(appName));
-      verify(mockJobs).query(eq(projectId), eq(new QueryRequest().setQuery(query)));
-    }
-  }
-
-  @Test
-  public void testQueryWithRetriesWhenServiceFails() throws Exception {
-    BigqueryMatcher matcher = spy(
-        new BigqueryMatcher(appName, projectId, query, "some-checksum"));
-    when(mockQuery.execute()).thenThrow(new IOException());
-
-    thrown.expect(RuntimeException.class);
-    thrown.expectMessage("Unable to get BigQuery response after retrying");
-    try {
-      matcher.queryWithRetries(
-          mockBigqueryClient,
-          new QueryRequest(),
-          fastClock,
-          BigqueryMatcher.BACKOFF_FACTORY.backoff());
-    } finally {
-      verify(mockJobs, atLeast(BigqueryMatcher.MAX_QUERY_RETRIES))
-          .query(eq(projectId), eq(new QueryRequest()));
-    }
-  }
-
-  @Test
-  public void testQueryWithRetriesWhenQueryResponseNull() throws Exception {
-    BigqueryMatcher matcher = spy(
-        new BigqueryMatcher(appName, projectId, query, "some-checksum"));
-    when(mockQuery.execute()).thenReturn(null);
-
-    thrown.expect(RuntimeException.class);
-    thrown.expectMessage("Unable to get BigQuery response after retrying");
-    try {
-      matcher.queryWithRetries(
-          mockBigqueryClient,
-          new QueryRequest(),
-          fastClock,
-          BigqueryMatcher.BACKOFF_FACTORY.backoff());
-    } finally {
-      verify(mockJobs, atLeast(BigqueryMatcher.MAX_QUERY_RETRIES))
-          .query(eq(projectId), eq(new QueryRequest()));
-    }
-  }
-
-  private QueryResponse createResponseContainingTestData() {
-    TableCell field1 = new TableCell();
-    field1.setV("abc");
-    TableCell field2 = new TableCell();
-    field2.setV("2");
-    TableCell field3 = new TableCell();
-    field3.setV("testing BigQuery matcher.");
-    TableRow row = new TableRow();
-    row.setF(Lists.newArrayList(field1, field2, field3));
-
-    QueryResponse response = new QueryResponse();
-    response.setJobComplete(true);
-    response.setRows(Lists.newArrayList(row));
-    response.setTotalRows(BigInteger.ONE);
-    return response;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
index 2ddead7..04005c5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
@@ -35,7 +35,6 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.options.ApplicationNameOptions;
-import org.apache.beam.sdk.options.GcpOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.transforms.Create;
@@ -90,12 +89,11 @@ public class TestPipelineTest implements Serializable {
       String stringOptions =
           mapper.writeValueAsString(
               new String[] {
-                "--runner=org.apache.beam.sdk.testing.CrashingRunner", "--project=testProject"
+                "--runner=org.apache.beam.sdk.testing.CrashingRunner"
               });
       System.getProperties().put("beamTestPipelineOptions", stringOptions);
-      GcpOptions options = TestPipeline.testingPipelineOptions().as(GcpOptions.class);
+      PipelineOptions options = TestPipeline.testingPipelineOptions();
       assertEquals(CrashingRunner.class, options.getRunner());
-      assertEquals(options.getProject(), "testProject");
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/util/DefaultBucketTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/DefaultBucketTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/DefaultBucketTest.java
deleted file mode 100644
index 395e1f3..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/DefaultBucketTest.java
+++ /dev/null
@@ -1,112 +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.Mockito.doThrow;
-import static org.mockito.Mockito.when;
-
-import com.google.api.services.storage.model.Bucket;
-import java.io.IOException;
-import org.apache.beam.sdk.options.CloudResourceManagerOptions;
-import org.apache.beam.sdk.options.GcpOptions;
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-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.MockitoAnnotations;
-import org.mockito.MockitoAnnotations.Mock;
-
-/** Tests for DefaultBucket. */
-@RunWith(JUnit4.class)
-public class DefaultBucketTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-
-  private PipelineOptions options;
-  @Mock
-  private GcsUtil gcsUtil;
-  @Mock
-  private GcpProjectUtil gcpUtil;
-
-  @Before
-  public void setUp() {
-    MockitoAnnotations.initMocks(this);
-    options = PipelineOptionsFactory.create();
-    options.as(GcsOptions.class).setGcsUtil(gcsUtil);
-    options.as(CloudResourceManagerOptions.class).setGcpProjectUtil(gcpUtil);
-    options.as(GcpOptions.class).setProject("foo");
-    options.as(GcpOptions.class).setZone("us-north1-a");
-  }
-
-  @Test
-  public void testCreateBucket() {
-    String bucket = DefaultBucket.tryCreateDefaultBucket(options);
-    assertEquals("gs://dataflow-staging-us-north1-0", bucket);
-  }
-
-  @Test
-  public void testCreateBucketProjectLookupFails() throws IOException {
-    when(gcpUtil.getProjectNumber("foo")).thenThrow(new IOException("badness"));
-
-    thrown.expect(RuntimeException.class);
-    thrown.expectMessage("Unable to verify project");
-    DefaultBucket.tryCreateDefaultBucket(options);
-  }
-
-  @Test
-  public void testCreateBucketCreateBucketFails() throws IOException {
-    doThrow(new IOException("badness")).when(
-      gcsUtil).createBucket(any(String.class), any(Bucket.class));
-
-    thrown.expect(RuntimeException.class);
-    thrown.expectMessage("Unable create default bucket");
-    DefaultBucket.tryCreateDefaultBucket(options);
-  }
-
-  @Test
-  public void testCannotGetBucketOwner() throws IOException {
-    when(gcsUtil.bucketOwner(any(GcsPath.class)))
-      .thenThrow(new IOException("badness"));
-
-    thrown.expect(RuntimeException.class);
-    thrown.expectMessage("Unable to determine the owner");
-    DefaultBucket.tryCreateDefaultBucket(options);
-  }
-
-  @Test
-  public void testProjectMismatch() throws IOException {
-    when(gcsUtil.bucketOwner(any(GcsPath.class))).thenReturn(5L);
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Bucket owner does not match the project");
-    DefaultBucket.tryCreateDefaultBucket(options);
-  }
-
-  @Test
-  public void regionFromZone() throws IOException {
-    assertEquals("us-central1", DefaultBucket.getRegionFromZone("us-central1-a"));
-    assertEquals("asia-east", DefaultBucket.getRegionFromZone("asia-east-a"));
-  }
-}


[2/7] beam git commit: [BEAM-1871] Create new GCP core module package and move several GCP related classes from beam-sdks-java-core over.

Posted by lc...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
new file mode 100644
index 0000000..6fac6dc
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
@@ -0,0 +1,89 @@
+/*
+ * 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 com.google.api.client.util.BackOff;
+
+
+/**
+ * Implementation of {@link BackOff} that increases the back off period for each retry attempt
+ * using a randomization function that grows exponentially.
+ *
+ * <p>Example: The initial interval is .5 seconds and the maximum interval is 60 secs.
+ * For 14 tries the sequence will be (values in seconds):
+ *
+ * <pre>
+ * retry#      retry_interval     randomized_interval
+ * 1             0.5                [0.25,   0.75]
+ * 2             0.75               [0.375,  1.125]
+ * 3             1.125              [0.562,  1.687]
+ * 4             1.687              [0.8435, 2.53]
+ * 5             2.53               [1.265,  3.795]
+ * 6             3.795              [1.897,  5.692]
+ * 7             5.692              [2.846,  8.538]
+ * 8             8.538              [4.269, 12.807]
+ * 9            12.807              [6.403, 19.210]
+ * 10           28.832              [14.416, 43.248]
+ * 11           43.248              [21.624, 64.873]
+ * 12           60.0                [30.0, 90.0]
+ * 13           60.0                [30.0, 90.0]
+ * 14           60.0                [30.0, 90.0]
+ * </pre>
+ *
+ * <p>Implementation is not thread-safe.
+ */
+@Deprecated
+public class IntervalBoundedExponentialBackOff implements BackOff {
+  public static final double DEFAULT_MULTIPLIER = 1.5;
+  public static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5;
+  private final long maximumIntervalMillis;
+  private final long initialIntervalMillis;
+  private int currentAttempt;
+
+  public IntervalBoundedExponentialBackOff(long maximumIntervalMillis, long initialIntervalMillis) {
+    checkArgument(maximumIntervalMillis > 0, "Maximum interval must be greater than zero.");
+    checkArgument(initialIntervalMillis > 0, "Initial interval must be greater than zero.");
+    this.maximumIntervalMillis = maximumIntervalMillis;
+    this.initialIntervalMillis = initialIntervalMillis;
+    reset();
+  }
+
+  @Override
+  public void reset() {
+    currentAttempt = 1;
+  }
+
+  @Override
+  public long nextBackOffMillis() {
+    double currentIntervalMillis =
+        Math.min(
+            initialIntervalMillis * Math.pow(DEFAULT_MULTIPLIER, currentAttempt - 1),
+            maximumIntervalMillis);
+    double randomOffset =
+        (Math.random() * 2 - 1) * DEFAULT_RANDOMIZATION_FACTOR * currentIntervalMillis;
+    currentAttempt += 1;
+    return Math.round(currentIntervalMillis + randomOffset);
+  }
+
+  public boolean atMaxInterval() {
+    return initialIntervalMillis * Math.pow(DEFAULT_MULTIPLIER, currentAttempt - 1)
+        >= maximumIntervalMillis;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java
new file mode 100644
index 0000000..f703e4c
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java
@@ -0,0 +1,68 @@
+/*
+ * 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 com.google.auth.Credentials;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * Construct an oauth credential to be used by the SDK and the SDK workers.
+ * Always returns a null Credential object.
+ */
+public class NoopCredentialFactory implements CredentialFactory {
+  private static final NoopCredentialFactory INSTANCE = new NoopCredentialFactory();
+  private static final NoopCredentials NOOP_CREDENTIALS = new NoopCredentials();
+
+  public static NoopCredentialFactory fromOptions(PipelineOptions options) {
+    return INSTANCE;
+  }
+
+  @Override
+  public Credentials getCredential() throws IOException {
+    return NOOP_CREDENTIALS;
+  }
+
+  private static class NoopCredentials extends Credentials {
+    @Override
+    public String getAuthenticationType() {
+      return null;
+    }
+
+    @Override
+    public Map<String, List<String>> getRequestMetadata(URI uri) throws IOException {
+      return null;
+    }
+
+    @Override
+    public boolean hasRequestMetadata() {
+      return false;
+    }
+
+    @Override
+    public boolean hasRequestMetadataOnly() {
+      return false;
+    }
+
+    @Override
+    public void refresh() throws IOException {}
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/NullCredentialInitializer.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/NullCredentialInitializer.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/NullCredentialInitializer.java
new file mode 100644
index 0000000..4ed35c6
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/NullCredentialInitializer.java
@@ -0,0 +1,62 @@
+/*
+* 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 com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.client.http.HttpResponse;
+import com.google.api.client.http.HttpUnsuccessfulResponseHandler;
+import java.io.IOException;
+
+/**
+ * A {@link HttpRequestInitializer} for requests that don't have credentials.
+ *
+ * <p>When the access is denied, it throws {@link IOException} with a detailed error message.
+ */
+public class NullCredentialInitializer implements HttpRequestInitializer {
+  private static final int ACCESS_DENIED = 401;
+  private static final String NULL_CREDENTIAL_REASON =
+      "Unable to get application default credentials. Please see "
+      + "https://developers.google.com/accounts/docs/application-default-credentials "
+      + "for details on how to specify credentials. This version of the SDK is "
+      + "dependent on the gcloud core component version 2015.02.05 or newer to "
+      + "be able to get credentials from the currently authorized user via gcloud auth.";
+
+  @Override
+  public void initialize(HttpRequest httpRequest) throws IOException {
+    httpRequest.setUnsuccessfulResponseHandler(new NullCredentialHttpUnsuccessfulResponseHandler());
+  }
+
+  private static class NullCredentialHttpUnsuccessfulResponseHandler
+      implements HttpUnsuccessfulResponseHandler {
+
+    @Override
+    public boolean handleResponse(
+        HttpRequest httpRequest,
+        HttpResponse httpResponse, boolean supportsRetry) throws IOException {
+      if (!httpResponse.isSuccessStatusCode() && httpResponse.getStatusCode() == ACCESS_DENIED) {
+        throwNullCredentialException();
+      }
+      return supportsRetry;
+    }
+  }
+
+  public static void throwNullCredentialException() {
+    throw new RuntimeException(NULL_CREDENTIAL_REASON);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/Transport.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/Transport.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/Transport.java
new file mode 100644
index 0000000..80c093b
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/Transport.java
@@ -0,0 +1,178 @@
+/*
+ * 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 com.google.api.client.googleapis.javanet.GoogleNetHttpTransport;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.jackson2.JacksonFactory;
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.cloudresourcemanager.CloudResourceManager;
+import com.google.api.services.pubsub.Pubsub;
+import com.google.api.services.storage.Storage;
+import com.google.auth.Credentials;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.security.GeneralSecurityException;
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.options.CloudResourceManagerOptions;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PubsubOptions;
+
+/**
+ * Helpers for cloud communication.
+ */
+public class Transport {
+
+  private static class SingletonHelper {
+    /** Global instance of the JSON factory. */
+    private static final JsonFactory JSON_FACTORY;
+
+    /** Global instance of the HTTP transport. */
+    private static final HttpTransport HTTP_TRANSPORT;
+
+    static {
+      try {
+        JSON_FACTORY = JacksonFactory.getDefaultInstance();
+        HTTP_TRANSPORT = GoogleNetHttpTransport.newTrustedTransport();
+      } catch (GeneralSecurityException | IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  public static HttpTransport getTransport() {
+    return SingletonHelper.HTTP_TRANSPORT;
+  }
+
+  public static JsonFactory getJsonFactory() {
+    return SingletonHelper.JSON_FACTORY;
+  }
+
+  private static class ApiComponents {
+    public String rootUrl;
+    public String servicePath;
+
+    public ApiComponents(String root, String path) {
+      this.rootUrl = root;
+      this.servicePath = path;
+    }
+  }
+
+  private static ApiComponents apiComponentsFromUrl(String urlString) {
+    try {
+      URL url = new URL(urlString);
+      String rootUrl = url.getProtocol() + "://" + url.getHost()
+          + (url.getPort() > 0 ? ":" + url.getPort() : "");
+      return new ApiComponents(rootUrl, url.getPath());
+    } catch (MalformedURLException e) {
+      throw new RuntimeException("Invalid URL: " + urlString);
+    }
+  }
+
+  /**
+   * Returns a BigQuery client builder using the specified {@link BigQueryOptions}.
+   */
+  public static Bigquery.Builder
+      newBigQueryClient(BigQueryOptions options) {
+    return new Bigquery.Builder(getTransport(), getJsonFactory(),
+        chainHttpRequestInitializer(
+            options.getGcpCredential(),
+            // Do not log 404. It clutters the output and is possibly even required by the caller.
+            new RetryHttpRequestInitializer(ImmutableList.of(404))))
+        .setApplicationName(options.getAppName())
+        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
+  }
+
+  /**
+   * Returns a Pubsub client builder using the specified {@link PubsubOptions}.
+   *
+   * @deprecated Use an appropriate org.apache.beam.sdk.util.PubsubClient.PubsubClientFactory
+   */
+  @Deprecated
+  public static Pubsub.Builder
+      newPubsubClient(PubsubOptions options) {
+    return new Pubsub.Builder(getTransport(), getJsonFactory(),
+        chainHttpRequestInitializer(
+            options.getGcpCredential(),
+            // Do not log 404. It clutters the output and is possibly even required by the caller.
+            new RetryHttpRequestInitializer(ImmutableList.of(404))))
+        .setRootUrl(options.getPubsubRootUrl())
+        .setApplicationName(options.getAppName())
+        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
+  }
+
+  /**
+   * Returns a CloudResourceManager client builder using the specified
+   * {@link CloudResourceManagerOptions}.
+   */
+  public static CloudResourceManager.Builder
+      newCloudResourceManagerClient(CloudResourceManagerOptions options) {
+    Credentials credentials = options.getGcpCredential();
+    if (credentials == null) {
+      NullCredentialInitializer.throwNullCredentialException();
+    }
+    return new CloudResourceManager.Builder(getTransport(), getJsonFactory(),
+        chainHttpRequestInitializer(
+            credentials,
+            // Do not log 404. It clutters the output and is possibly even required by the caller.
+            new RetryHttpRequestInitializer(ImmutableList.of(404))))
+        .setApplicationName(options.getAppName())
+        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
+  }
+
+  /**
+   * Returns a Cloud Storage client builder using the specified {@link GcsOptions}.
+   */
+  public static Storage.Builder
+      newStorageClient(GcsOptions options) {
+    String servicePath = options.getGcsEndpoint();
+    Storage.Builder storageBuilder = new Storage.Builder(getTransport(), getJsonFactory(),
+        chainHttpRequestInitializer(
+            options.getGcpCredential(),
+            // Do not log the code 404. Code up the stack will deal with 404's if needed, and
+            // logging it by default clutters the output during file staging.
+            new RetryHttpRequestInitializer(
+                ImmutableList.of(404), new UploadIdResponseInterceptor())))
+        .setApplicationName(options.getAppName())
+        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
+    if (servicePath != null) {
+      ApiComponents components = apiComponentsFromUrl(servicePath);
+      storageBuilder.setRootUrl(components.rootUrl);
+      storageBuilder.setServicePath(components.servicePath);
+    }
+    return storageBuilder;
+  }
+
+  private static HttpRequestInitializer chainHttpRequestInitializer(
+      Credentials credential, HttpRequestInitializer httpRequestInitializer) {
+    if (credential == null) {
+      return new ChainingHttpRequestInitializer(
+          new NullCredentialInitializer(), httpRequestInitializer);
+    } else {
+      return new ChainingHttpRequestInitializer(
+          new HttpCredentialsAdapter(credential),
+          httpRequestInitializer);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/package-info.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/package-info.java
new file mode 100644
index 0000000..f8135e7
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/util/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * 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 Google Cloud Platform component utilities that can be used by Beam runners. */
+package org.apache.beam.sdk.util;

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/GcpCoreApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/GcpCoreApiSurfaceTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/GcpCoreApiSurfaceTest.java
new file mode 100644
index 0000000..37fb42d
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/GcpCoreApiSurfaceTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import static org.apache.beam.sdk.util.ApiSurface.containsOnlyPackages;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import com.google.common.collect.ImmutableSet;
+import java.util.Set;
+import org.apache.beam.sdk.util.ApiSurface;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** API surface verification for Google Cloud Platform core components. */
+@RunWith(JUnit4.class)
+public class GcpCoreApiSurfaceTest {
+
+  @Test
+  public void testApiSurface() throws Exception {
+
+    @SuppressWarnings("unchecked")
+    final Set<String> allowed =
+        ImmutableSet.of(
+            "org.apache.beam",
+            "com.google.api.client",
+            "com.google.api.services.bigquery",
+            "com.google.api.services.cloudresourcemanager",
+            "com.google.api.services.pubsub",
+            "com.google.api.services.storage",
+            "com.google.auth",
+            "com.google.protobuf",
+            "com.fasterxml.jackson.annotation",
+            "com.fasterxml.jackson.core",
+            "com.fasterxml.jackson.databind",
+            "org.apache.avro",
+            "org.hamcrest",
+            // via DataflowMatchers
+            "org.codehaus.jackson",
+            // via Avro
+            "org.joda.time",
+            "org.junit");
+
+    assertThat(
+        ApiSurface.getSdkApiSurface(getClass().getClassLoader()), containsOnlyPackages(allowed));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
new file mode 100644
index 0000000..288383e
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
@@ -0,0 +1,171 @@
+/*
+ * 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.options;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.io.Files;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import org.apache.beam.sdk.options.GcpOptions.DefaultProjectFactory;
+import org.apache.beam.sdk.testing.RestoreSystemProperties;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link GcpOptions}. */
+@RunWith(JUnit4.class)
+public class GcpOptionsTest {
+  @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
+  @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testGetProjectFromCloudSdkConfigEnv() throws Exception {
+    Map<String, String> environment =
+        ImmutableMap.of("CLOUDSDK_CONFIG", tmpFolder.getRoot().getAbsolutePath());
+    assertEquals("test-project",
+        runGetProjectTest(tmpFolder.newFile("properties"), environment));
+  }
+
+  @Test
+  public void testGetProjectFromAppDataEnv() throws Exception {
+    Map<String, String> environment =
+        ImmutableMap.of("APPDATA", tmpFolder.getRoot().getAbsolutePath());
+    System.setProperty("os.name", "windows");
+    assertEquals("test-project",
+        runGetProjectTest(new File(tmpFolder.newFolder("gcloud"), "properties"),
+            environment));
+  }
+
+  @Test
+  public void testGetProjectFromUserHomeEnvOld() throws Exception {
+    Map<String, String> environment = ImmutableMap.of();
+    System.setProperty("user.home", tmpFolder.getRoot().getAbsolutePath());
+    assertEquals("test-project",
+        runGetProjectTest(
+            new File(tmpFolder.newFolder(".config", "gcloud"), "properties"),
+            environment));
+  }
+
+  @Test
+  public void testGetProjectFromUserHomeEnv() throws Exception {
+    Map<String, String> environment = ImmutableMap.of();
+    System.setProperty("user.home", tmpFolder.getRoot().getAbsolutePath());
+    assertEquals("test-project",
+        runGetProjectTest(
+            new File(tmpFolder.newFolder(".config", "gcloud", "configurations"), "config_default"),
+            environment));
+  }
+
+  @Test
+  public void testGetProjectFromUserHomeOldAndNewPrefersNew() throws Exception {
+    Map<String, String> environment = ImmutableMap.of();
+    System.setProperty("user.home", tmpFolder.getRoot().getAbsolutePath());
+    makePropertiesFileWithProject(new File(tmpFolder.newFolder(".config", "gcloud"), "properties"),
+        "old-project");
+    assertEquals("test-project",
+        runGetProjectTest(
+            new File(tmpFolder.newFolder(".config", "gcloud", "configurations"), "config_default"),
+            environment));
+  }
+
+  @Test
+  public void testUnableToGetDefaultProject() throws Exception {
+    System.setProperty("user.home", tmpFolder.getRoot().getAbsolutePath());
+    DefaultProjectFactory projectFactory = spy(new DefaultProjectFactory());
+    when(projectFactory.getEnvironment()).thenReturn(ImmutableMap.<String, String>of());
+    assertNull(projectFactory.create(PipelineOptionsFactory.create()));
+  }
+
+  @Test
+  public void testEmptyGcpTempLocation() throws Exception {
+    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
+    options.setProject("");
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("--project is a required option");
+    options.getGcpTempLocation();
+  }
+
+  @Test
+  public void testDefaultGcpTempLocation() throws Exception {
+    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
+    String tempLocation = "gs://bucket";
+    options.setTempLocation(tempLocation);
+    options.as(GcsOptions.class).setPathValidatorClass(NoopPathValidator.class);
+    assertEquals(tempLocation, options.getGcpTempLocation());
+  }
+
+  @Test
+  public void testDefaultGcpTempLocationInvalid() throws Exception {
+    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
+    options.setTempLocation("file://");
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(
+        "Error constructing default value for gcpTempLocation: tempLocation is not"
+            + " a valid GCS path");
+    options.getGcpTempLocation();
+  }
+
+  @Test
+  public void testDefaultGcpTempLocationDoesNotExist() {
+    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
+    String tempLocation = "gs://does/not/exist";
+    options.setTempLocation(tempLocation);
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(
+        "Error constructing default value for gcpTempLocation: tempLocation is not"
+            + " a valid GCS path");
+    thrown.expectCause(
+        hasMessage(containsString("Output path does not exist or is not writeable")));
+
+    options.getGcpTempLocation();
+  }
+
+  private static void makePropertiesFileWithProject(File path, String projectId)
+      throws IOException {
+    String properties = String.format("[core]%n"
+        + "account = test-account@google.com%n"
+        + "project = %s%n"
+        + "%n"
+        + "[dataflow]%n"
+        + "magic = true%n", projectId);
+    Files.write(properties, path, StandardCharsets.UTF_8);
+  }
+
+  private static String runGetProjectTest(File path, Map<String, String> environment)
+      throws Exception {
+    makePropertiesFileWithProject(path, "test-project");
+    DefaultProjectFactory projectFactory = spy(new DefaultProjectFactory());
+    when(projectFactory.getEnvironment()).thenReturn(environment);
+    return projectFactory.create(PipelineOptionsFactory.create());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java
new file mode 100644
index 0000000..dae7208
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java
@@ -0,0 +1,145 @@
+/*
+ * 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.options;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.api.services.bigquery.Bigquery.Datasets.Delete;
+import com.google.api.services.storage.Storage;
+import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.Transport;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link GoogleApiDebugOptions}. */
+@RunWith(JUnit4.class)
+public class GoogleApiDebugOptionsTest {
+  private static final String STORAGE_GET_TRACE =
+      "--googleApiTrace={\"Objects.Get\":\"GetTraceDestination\"}";
+  private static final String STORAGE_GET_AND_LIST_TRACE =
+      "--googleApiTrace={\"Objects.Get\":\"GetTraceDestination\","
+      + "\"Objects.List\":\"ListTraceDestination\"}";
+  private static final String STORAGE_TRACE = "--googleApiTrace={\"Storage\":\"TraceDestination\"}";
+
+  @Test
+  public void testWhenTracingMatches() throws Exception {
+    String[] args = new String[] {STORAGE_GET_TRACE};
+    GcsOptions options = PipelineOptionsFactory.fromArgs(args).as(GcsOptions.class);
+    options.setGcpCredential(new TestCredential());
+    assertNotNull(options.getGoogleApiTrace());
+
+    Storage.Objects.Get request =
+        Transport.newStorageClient(options).build().objects().get("testBucketId", "testObjectId");
+    assertEquals("GetTraceDestination", request.get("$trace"));
+  }
+
+  @Test
+  public void testWhenTracingDoesNotMatch() throws Exception {
+    String[] args = new String[] {STORAGE_GET_TRACE};
+    GcsOptions options = PipelineOptionsFactory.fromArgs(args).as(GcsOptions.class);
+    options.setGcpCredential(new TestCredential());
+
+    assertNotNull(options.getGoogleApiTrace());
+
+    Storage.Objects.List request =
+        Transport.newStorageClient(options).build().objects().list("testProjectId");
+    assertNull(request.get("$trace"));
+  }
+
+  @Test
+  public void testWithMultipleTraces() throws Exception {
+    String[] args = new String[] {STORAGE_GET_AND_LIST_TRACE};
+    GcsOptions options = PipelineOptionsFactory.fromArgs(args).as(GcsOptions.class);
+    options.setGcpCredential(new TestCredential());
+
+    assertNotNull(options.getGoogleApiTrace());
+
+    Storage.Objects.Get getRequest =
+        Transport.newStorageClient(options).build().objects().get("testBucketId", "testObjectId");
+    assertEquals("GetTraceDestination", getRequest.get("$trace"));
+
+    Storage.Objects.List listRequest =
+        Transport.newStorageClient(options).build().objects().list("testProjectId");
+    assertEquals("ListTraceDestination", listRequest.get("$trace"));
+  }
+
+  @Test
+  public void testMatchingAllCalls() throws Exception {
+    String[] args = new String[] {STORAGE_TRACE};
+    GcsOptions options =
+        PipelineOptionsFactory.fromArgs(args).as(GcsOptions.class);
+    options.setGcpCredential(new TestCredential());
+
+    assertNotNull(options.getGoogleApiTrace());
+
+    Storage.Objects.Get getRequest =
+        Transport.newStorageClient(options).build().objects().get("testBucketId", "testObjectId");
+    assertEquals("TraceDestination", getRequest.get("$trace"));
+
+    Storage.Objects.List listRequest =
+        Transport.newStorageClient(options).build().objects().list("testProjectId");
+    assertEquals("TraceDestination", listRequest.get("$trace"));
+  }
+
+  @Test
+  public void testMatchingAgainstClient() throws Exception {
+    GcsOptions options = PipelineOptionsFactory.as(GcsOptions.class);
+    options.setGcpCredential(new TestCredential());
+    options.setGoogleApiTrace(new GoogleApiTracer().addTraceFor(
+        Transport.newStorageClient(options).build(), "TraceDestination"));
+
+    Storage.Objects.Get getRequest =
+        Transport.newStorageClient(options).build().objects().get("testBucketId", "testObjectId");
+    assertEquals("TraceDestination", getRequest.get("$trace"));
+
+    Delete deleteRequest = Transport.newBigQueryClient(options.as(BigQueryOptions.class))
+        .build().datasets().delete("testProjectId", "testDatasetId");
+    assertNull(deleteRequest.get("$trace"));
+  }
+
+  @Test
+  public void testMatchingAgainstRequestType() throws Exception {
+    GcsOptions options = PipelineOptionsFactory.as(GcsOptions.class);
+    options.setGcpCredential(new TestCredential());
+    options.setGoogleApiTrace(new GoogleApiTracer().addTraceFor(
+        Transport.newStorageClient(options).build().objects()
+            .get("aProjectId", "aObjectId"), "TraceDestination"));
+
+    Storage.Objects.Get getRequest =
+        Transport.newStorageClient(options).build().objects().get("testBucketId", "testObjectId");
+    assertEquals("TraceDestination", getRequest.get("$trace"));
+
+    Storage.Objects.List listRequest =
+        Transport.newStorageClient(options).build().objects().list("testProjectId");
+    assertNull(listRequest.get("$trace"));
+  }
+
+  @Test
+  public void testDeserializationAndSerializationOfGoogleApiTracer() throws Exception {
+    String serializedValue = "{\"Api\":\"Token\"}";
+    ObjectMapper objectMapper = new ObjectMapper();
+    assertEquals(serializedValue,
+        objectMapper.writeValueAsString(
+            objectMapper.readValue(serializedValue, GoogleApiTracer.class)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
new file mode 100644
index 0000000..3b35856
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
@@ -0,0 +1,176 @@
+/*
+ * 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.testing;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.model.QueryRequest;
+import com.google.api.services.bigquery.model.QueryResponse;
+import com.google.api.services.bigquery.model.TableCell;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.math.BigInteger;
+import org.apache.beam.sdk.PipelineResult;
+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;
+
+/**
+ * Tests for {@link BigqueryMatcher}.
+ */
+@RunWith(JUnit4.class)
+public class BigqueryMatcherTest {
+  private final String appName = "test-app";
+  private final String projectId = "test-project";
+  private final String query = "test-query";
+
+  @Rule public ExpectedException thrown = ExpectedException.none();
+  @Rule public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper();
+  @Mock private Bigquery mockBigqueryClient;
+  @Mock private Bigquery.Jobs mockJobs;
+  @Mock private Bigquery.Jobs.Query mockQuery;
+  @Mock private PipelineResult mockResult;
+
+  @Before
+  public void setUp() throws IOException {
+    MockitoAnnotations.initMocks(this);
+    when(mockBigqueryClient.jobs()).thenReturn(mockJobs);
+    when(mockJobs.query(anyString(), any(QueryRequest.class))).thenReturn(mockQuery);
+  }
+
+  @Test
+  public void testBigqueryMatcherThatSucceeds() throws Exception {
+    BigqueryMatcher matcher = spy(
+        new BigqueryMatcher(
+            appName, projectId, query, "9bb47f5c90d2a99cad526453dff5ed5ec74650dc"));
+    doReturn(mockBigqueryClient).when(matcher).newBigqueryClient(anyString());
+    when(mockQuery.execute()).thenReturn(createResponseContainingTestData());
+
+    assertThat(mockResult, matcher);
+    verify(matcher).newBigqueryClient(eq(appName));
+    verify(mockJobs).query(eq(projectId), eq(new QueryRequest().setQuery(query)));
+  }
+
+  @Test
+  public void testBigqueryMatcherFailsForChecksumMismatch() throws IOException {
+    BigqueryMatcher matcher = spy(
+        new BigqueryMatcher(appName, projectId, query, "incorrect-checksum"));
+    doReturn(mockBigqueryClient).when(matcher).newBigqueryClient(anyString());
+    when(mockQuery.execute()).thenReturn(createResponseContainingTestData());
+
+    thrown.expect(AssertionError.class);
+    thrown.expectMessage("Total number of rows are: 1");
+    thrown.expectMessage("abc");
+    try {
+      assertThat(mockResult, matcher);
+    } finally {
+      verify(matcher).newBigqueryClient(eq(appName));
+      verify(mockJobs).query(eq(projectId), eq(new QueryRequest().setQuery(query)));
+    }
+  }
+
+  @Test
+  public void testBigqueryMatcherFailsWhenQueryJobNotComplete() throws Exception {
+    BigqueryMatcher matcher = spy(
+        new BigqueryMatcher(appName, projectId, query, "some-checksum"));
+    doReturn(mockBigqueryClient).when(matcher).newBigqueryClient(anyString());
+    when(mockQuery.execute()).thenReturn(new QueryResponse().setJobComplete(false));
+
+    thrown.expect(AssertionError.class);
+    thrown.expectMessage("The query job hasn't completed.");
+    thrown.expectMessage("jobComplete=false");
+    try {
+      assertThat(mockResult, matcher);
+    } finally {
+      verify(matcher).newBigqueryClient(eq(appName));
+      verify(mockJobs).query(eq(projectId), eq(new QueryRequest().setQuery(query)));
+    }
+  }
+
+  @Test
+  public void testQueryWithRetriesWhenServiceFails() throws Exception {
+    BigqueryMatcher matcher = spy(
+        new BigqueryMatcher(appName, projectId, query, "some-checksum"));
+    when(mockQuery.execute()).thenThrow(new IOException());
+
+    thrown.expect(RuntimeException.class);
+    thrown.expectMessage("Unable to get BigQuery response after retrying");
+    try {
+      matcher.queryWithRetries(
+          mockBigqueryClient,
+          new QueryRequest(),
+          fastClock,
+          BigqueryMatcher.BACKOFF_FACTORY.backoff());
+    } finally {
+      verify(mockJobs, atLeast(BigqueryMatcher.MAX_QUERY_RETRIES))
+          .query(eq(projectId), eq(new QueryRequest()));
+    }
+  }
+
+  @Test
+  public void testQueryWithRetriesWhenQueryResponseNull() throws Exception {
+    BigqueryMatcher matcher = spy(
+        new BigqueryMatcher(appName, projectId, query, "some-checksum"));
+    when(mockQuery.execute()).thenReturn(null);
+
+    thrown.expect(RuntimeException.class);
+    thrown.expectMessage("Unable to get BigQuery response after retrying");
+    try {
+      matcher.queryWithRetries(
+          mockBigqueryClient,
+          new QueryRequest(),
+          fastClock,
+          BigqueryMatcher.BACKOFF_FACTORY.backoff());
+    } finally {
+      verify(mockJobs, atLeast(BigqueryMatcher.MAX_QUERY_RETRIES))
+          .query(eq(projectId), eq(new QueryRequest()));
+    }
+  }
+
+  private QueryResponse createResponseContainingTestData() {
+    TableCell field1 = new TableCell();
+    field1.setV("abc");
+    TableCell field2 = new TableCell();
+    field2.setV("2");
+    TableCell field3 = new TableCell();
+    field3.setV("testing BigQuery matcher.");
+    TableRow row = new TableRow();
+    row.setF(Lists.newArrayList(field1, field2, field3));
+
+    QueryResponse response = new QueryResponse();
+    response.setJobComplete(true);
+    response.setRows(Lists.newArrayList(row));
+    response.setTotalRows(BigInteger.ONE);
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/DefaultBucketTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/DefaultBucketTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/DefaultBucketTest.java
new file mode 100644
index 0000000..395e1f3
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/DefaultBucketTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.Mockito.doThrow;
+import static org.mockito.Mockito.when;
+
+import com.google.api.services.storage.model.Bucket;
+import java.io.IOException;
+import org.apache.beam.sdk.options.CloudResourceManagerOptions;
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+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.MockitoAnnotations;
+import org.mockito.MockitoAnnotations.Mock;
+
+/** Tests for DefaultBucket. */
+@RunWith(JUnit4.class)
+public class DefaultBucketTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  private PipelineOptions options;
+  @Mock
+  private GcsUtil gcsUtil;
+  @Mock
+  private GcpProjectUtil gcpUtil;
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+    options = PipelineOptionsFactory.create();
+    options.as(GcsOptions.class).setGcsUtil(gcsUtil);
+    options.as(CloudResourceManagerOptions.class).setGcpProjectUtil(gcpUtil);
+    options.as(GcpOptions.class).setProject("foo");
+    options.as(GcpOptions.class).setZone("us-north1-a");
+  }
+
+  @Test
+  public void testCreateBucket() {
+    String bucket = DefaultBucket.tryCreateDefaultBucket(options);
+    assertEquals("gs://dataflow-staging-us-north1-0", bucket);
+  }
+
+  @Test
+  public void testCreateBucketProjectLookupFails() throws IOException {
+    when(gcpUtil.getProjectNumber("foo")).thenThrow(new IOException("badness"));
+
+    thrown.expect(RuntimeException.class);
+    thrown.expectMessage("Unable to verify project");
+    DefaultBucket.tryCreateDefaultBucket(options);
+  }
+
+  @Test
+  public void testCreateBucketCreateBucketFails() throws IOException {
+    doThrow(new IOException("badness")).when(
+      gcsUtil).createBucket(any(String.class), any(Bucket.class));
+
+    thrown.expect(RuntimeException.class);
+    thrown.expectMessage("Unable create default bucket");
+    DefaultBucket.tryCreateDefaultBucket(options);
+  }
+
+  @Test
+  public void testCannotGetBucketOwner() throws IOException {
+    when(gcsUtil.bucketOwner(any(GcsPath.class)))
+      .thenThrow(new IOException("badness"));
+
+    thrown.expect(RuntimeException.class);
+    thrown.expectMessage("Unable to determine the owner");
+    DefaultBucket.tryCreateDefaultBucket(options);
+  }
+
+  @Test
+  public void testProjectMismatch() throws IOException {
+    when(gcsUtil.bucketOwner(any(GcsPath.class))).thenReturn(5L);
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Bucket owner does not match the project");
+    DefaultBucket.tryCreateDefaultBucket(options);
+  }
+
+  @Test
+  public void regionFromZone() throws IOException {
+    assertEquals("us-central1", DefaultBucket.getRegionFromZone("us-central1-a"));
+    assertEquals("asia-east", DefaultBucket.getRegionFromZone("asia-east-a"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcpProjectUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcpProjectUtilTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcpProjectUtilTest.java
new file mode 100644
index 0000000..23f0418
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcpProjectUtilTest.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.when;
+
+import com.google.api.client.util.BackOff;
+import com.google.api.services.cloudresourcemanager.CloudResourceManager;
+import com.google.api.services.cloudresourcemanager.model.Project;
+import java.net.SocketTimeoutException;
+import org.apache.beam.sdk.options.CloudResourceManagerOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
+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.Mockito;
+
+/** Test case for {@link GcpProjectUtil}. */
+@RunWith(JUnit4.class)
+public class GcpProjectUtilTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  private static CloudResourceManagerOptions crmOptionsWithTestCredential() {
+    CloudResourceManagerOptions pipelineOptions =
+        PipelineOptionsFactory.as(CloudResourceManagerOptions.class);
+    pipelineOptions.setGcpCredential(new TestCredential());
+    return pipelineOptions;
+  }
+
+  @Test
+  public void testGetProjectNumber() throws Exception {
+    CloudResourceManagerOptions pipelineOptions = crmOptionsWithTestCredential();
+    GcpProjectUtil projectUtil = pipelineOptions.getGcpProjectUtil();
+
+    CloudResourceManager.Projects mockProjects = Mockito.mock(
+        CloudResourceManager.Projects.class);
+    CloudResourceManager mockCrm = Mockito.mock(CloudResourceManager.class);
+    projectUtil.setCrmClient(mockCrm);
+
+    CloudResourceManager.Projects.Get mockProjectsGet =
+        Mockito.mock(CloudResourceManager.Projects.Get.class);
+
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
+    Project project = new Project();
+    project.setProjectNumber(5L);
+
+    when(mockCrm.projects()).thenReturn(mockProjects);
+    when(mockProjects.get(any(String.class))).thenReturn(mockProjectsGet);
+    when(mockProjectsGet.execute())
+      .thenThrow(new SocketTimeoutException("SocketException"))
+      .thenReturn(project);
+
+    assertEquals(5L, projectUtil.getProjectNumber(
+        "foo", mockBackOff, new FastNanoClockAndSleeper()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java
new file mode 100644
index 0000000..a29dd45
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.fail;
+
+import com.google.common.collect.Lists;
+import java.util.ServiceLoader;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link GcsIOChannelFactoryRegistrar}.
+ */
+@RunWith(JUnit4.class)
+public class GcsIOChannelFactoryRegistrarTest {
+
+  @Test
+  public void testServiceLoader() {
+    for (IOChannelFactoryRegistrar registrar
+        : Lists.newArrayList(ServiceLoader.load(IOChannelFactoryRegistrar.class).iterator())) {
+      if (registrar instanceof GcsIOChannelFactoryRegistrar) {
+        return;
+      }
+    }
+    fail("Expected to find " + GcsIOChannelFactoryRegistrar.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java
new file mode 100644
index 0000000..7248b38
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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 org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link GcsIOChannelFactoryTest}. */
+@RunWith(JUnit4.class)
+public class GcsIOChannelFactoryTest {
+  private GcsIOChannelFactory factory;
+
+  @Before
+  public void setUp() {
+    factory = GcsIOChannelFactory.fromOptions(PipelineOptionsFactory.as(GcsOptions.class));
+  }
+
+  @Test
+  public void testResolve() throws Exception {
+    assertEquals("gs://bucket/object", factory.resolve("gs://bucket", "object"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java
new file mode 100644
index 0000000..dc36319
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.mockito.Matchers.any;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+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;
+
+/** Tests for {@link GcsPathValidator}. */
+@RunWith(JUnit4.class)
+public class GcsPathValidatorTest {
+  @Rule public ExpectedException expectedException = ExpectedException.none();
+
+  @Mock private GcsUtil mockGcsUtil;
+  private GcsPathValidator validator;
+
+  @Before
+  public void setUp() throws Exception {
+    MockitoAnnotations.initMocks(this);
+    when(mockGcsUtil.bucketAccessible(any(GcsPath.class))).thenReturn(true);
+    GcsOptions options = PipelineOptionsFactory.as(GcsOptions.class);
+    options.setGcpCredential(new TestCredential());
+    options.setGcsUtil(mockGcsUtil);
+    validator = GcsPathValidator.fromOptions(options);
+  }
+
+  @Test
+  public void testValidFilePattern() {
+    validator.validateInputFilePatternSupported("gs://bucket/path");
+  }
+
+  @Test
+  public void testInvalidFilePattern() {
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "Expected a valid 'gs://' path but was given '/local/path'");
+    validator.validateInputFilePatternSupported("/local/path");
+  }
+
+  @Test
+  public void testWhenBucketDoesNotExist() throws Exception {
+    when(mockGcsUtil.bucketAccessible(any(GcsPath.class))).thenReturn(false);
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "Could not find file gs://non-existent-bucket/location");
+    validator.validateInputFilePatternSupported("gs://non-existent-bucket/location");
+  }
+
+  @Test
+  public void testValidOutputPrefix() {
+    validator.validateOutputFilePrefixSupported("gs://bucket/path");
+  }
+
+  @Test
+  public void testInvalidOutputPrefix() {
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "Expected a valid 'gs://' path but was given '/local/path'");
+    validator.validateOutputFilePrefixSupported("/local/path");
+  }
+}


[6/7] beam git commit: [BEAM-1871] Create new GCP core module package and move several GCP related classes from beam-sdks-java-core over.

Posted by lc...@apache.org.
[BEAM-1871] Create new GCP core module package and move several GCP related classes from beam-sdks-java-core over.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/be92f595
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/be92f595
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/be92f595

Branch: refs/heads/master
Commit: be92f5952706c2cbb980df5073fbc74925ed68e6
Parents: ed52d32
Author: Luke Cwik <lc...@google.com>
Authored: Mon Apr 17 18:02:02 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Tue Apr 18 16:18:53 2017 -0700

----------------------------------------------------------------------
 examples/java/pom.xml                           |   5 +
 examples/java8/pom.xml                          |   5 +
 pom.xml                                         |   6 +
 runners/flink/examples/pom.xml                  |   4 +
 runners/google-cloud-dataflow-java/pom.xml      |   5 +
 sdks/java/core/pom.xml                          |  46 +-
 .../beam/sdk/options/BigQueryOptions.java       |  32 -
 .../options/CloudResourceManagerOptions.java    |  40 -
 .../DefaultPipelineOptionsRegistrar.java        |   5 -
 .../org/apache/beam/sdk/options/GcpOptions.java | 227 ------
 .../org/apache/beam/sdk/options/GcsOptions.java | 158 ----
 .../beam/sdk/options/GoogleApiDebugOptions.java |  87 --
 .../beam/sdk/options/PipelineOptions.java       |   4 +-
 .../apache/beam/sdk/options/PubsubOptions.java  |  36 -
 .../apache/beam/sdk/runners/PipelineRunner.java |   2 +
 .../beam/sdk/testing/BigqueryMatcher.java       | 256 ------
 .../apache/beam/sdk/testing/TestPipeline.java   |   3 -
 .../beam/sdk/util/AppEngineEnvironment.java     |  62 --
 .../apache/beam/sdk/util/CredentialFactory.java |  29 -
 .../org/apache/beam/sdk/util/DefaultBucket.java | 105 ---
 .../beam/sdk/util/GcpCredentialFactory.java     |  67 --
 .../apache/beam/sdk/util/GcpProjectUtil.java    | 106 ---
 .../beam/sdk/util/GcsIOChannelFactory.java      | 111 ---
 .../sdk/util/GcsIOChannelFactoryRegistrar.java  |  38 -
 .../apache/beam/sdk/util/GcsPathValidator.java  |  95 ---
 .../java/org/apache/beam/sdk/util/GcsUtil.java  | 798 -------------------
 .../util/IntervalBoundedExponentialBackOff.java |  89 ---
 .../beam/sdk/util/NoopCredentialFactory.java    |  68 --
 .../sdk/util/NullCredentialInitializer.java     |  62 --
 .../org/apache/beam/sdk/util/Transport.java     | 178 -----
 .../org/apache/beam/SdkCoreApiSurfaceTest.java  |   2 -
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  97 +--
 .../apache/beam/sdk/options/GcpOptionsTest.java | 171 ----
 .../sdk/options/GoogleApiDebugOptionsTest.java  | 145 ----
 .../sdk/options/PipelineOptionsFactoryTest.java |   4 +-
 .../beam/sdk/runners/PipelineRunnerTest.java    |  46 +-
 .../beam/sdk/testing/BigqueryMatcherTest.java   | 176 ----
 .../beam/sdk/testing/TestPipelineTest.java      |   6 +-
 .../apache/beam/sdk/util/DefaultBucketTest.java | 112 ---
 .../beam/sdk/util/GcpProjectUtilTest.java       |  76 --
 .../util/GcsIOChannelFactoryRegistrarTest.java  |  44 -
 .../beam/sdk/util/GcsIOChannelFactoryTest.java  |  43 -
 .../beam/sdk/util/GcsPathValidatorTest.java     |  87 --
 .../org/apache/beam/sdk/util/GcsUtilTest.java   | 798 -------------------
 .../IntervalBoundedExponentialBackOffTest.java  | 100 ---
 .../util/RetryHttpRequestInitializerTest.java   | 290 -------
 sdks/java/extensions/gcp-core/pom.xml           | 217 +++++
 .../beam/sdk/options/BigQueryOptions.java       |  32 +
 .../options/CloudResourceManagerOptions.java    |  40 +
 .../org/apache/beam/sdk/options/GcpOptions.java | 227 ++++++
 .../options/GcpPipelineOptionsRegistrar.java    |  39 +
 .../org/apache/beam/sdk/options/GcsOptions.java | 158 ++++
 .../beam/sdk/options/GoogleApiDebugOptions.java |  87 ++
 .../apache/beam/sdk/options/PubsubOptions.java  |  36 +
 .../apache/beam/sdk/options/package-info.java   |  22 +
 .../beam/sdk/testing/BigqueryMatcher.java       | 256 ++++++
 .../apache/beam/sdk/testing/package-info.java   |  21 +
 .../beam/sdk/util/AppEngineEnvironment.java     |  62 ++
 .../apache/beam/sdk/util/CredentialFactory.java |  29 +
 .../org/apache/beam/sdk/util/DefaultBucket.java | 105 +++
 .../beam/sdk/util/GcpCredentialFactory.java     |  67 ++
 .../apache/beam/sdk/util/GcpProjectUtil.java    | 106 +++
 .../beam/sdk/util/GcsIOChannelFactory.java      | 111 +++
 .../sdk/util/GcsIOChannelFactoryRegistrar.java  |  38 +
 .../apache/beam/sdk/util/GcsPathValidator.java  |  95 +++
 .../java/org/apache/beam/sdk/util/GcsUtil.java  | 798 +++++++++++++++++++
 .../util/IntervalBoundedExponentialBackOff.java |  89 +++
 .../beam/sdk/util/NoopCredentialFactory.java    |  68 ++
 .../sdk/util/NullCredentialInitializer.java     |  62 ++
 .../org/apache/beam/sdk/util/Transport.java     | 178 +++++
 .../org/apache/beam/sdk/util/package-info.java  |  20 +
 .../org/apache/beam/GcpCoreApiSurfaceTest.java  |  62 ++
 .../apache/beam/sdk/options/GcpOptionsTest.java | 171 ++++
 .../sdk/options/GoogleApiDebugOptionsTest.java  | 145 ++++
 .../beam/sdk/testing/BigqueryMatcherTest.java   | 176 ++++
 .../apache/beam/sdk/util/DefaultBucketTest.java | 112 +++
 .../beam/sdk/util/GcpProjectUtilTest.java       |  76 ++
 .../util/GcsIOChannelFactoryRegistrarTest.java  |  44 +
 .../beam/sdk/util/GcsIOChannelFactoryTest.java  |  43 +
 .../beam/sdk/util/GcsPathValidatorTest.java     |  87 ++
 .../org/apache/beam/sdk/util/GcsUtilTest.java   | 798 +++++++++++++++++++
 .../IntervalBoundedExponentialBackOffTest.java  | 100 +++
 .../util/RetryHttpRequestInitializerTest.java   | 290 +++++++
 sdks/java/extensions/pom.xml                    |   1 +
 sdks/java/harness/pom.xml                       |   5 +
 sdks/java/io/google-cloud-platform/pom.xml      |  20 +-
 86 files changed, 5130 insertions(+), 4889 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index 021a819..ae3d63d 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -461,6 +461,11 @@
 
     <dependency>
       <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/examples/java8/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml
index 912c341..cd69acb 100644
--- a/examples/java8/pom.xml
+++ b/examples/java8/pom.xml
@@ -208,6 +208,11 @@
 
     <dependency>
       <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 09f3985..306978d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -357,6 +357,12 @@
 
       <dependency>
         <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.beam</groupId>
         <artifactId>beam-sdks-java-extensions-sorter</artifactId>
         <version>${project.version}</version>
       </dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/runners/flink/examples/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml
index fa642bd..aaf76d9 100644
--- a/runners/flink/examples/pom.xml
+++ b/runners/flink/examples/pom.xml
@@ -65,6 +65,10 @@
   </profiles>
 
   <dependencies>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
+    </dependency>
 
     <dependency>
       <groupId>org.apache.beam</groupId>

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index ff63a31..68d433a 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -171,6 +171,11 @@
 
     <dependency>
       <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-common-runner-api</artifactId>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index 0ac40f4..4ba8e3b 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -145,38 +145,30 @@
     </dependency>
 
     <dependency>
-      <groupId>com.google.auth</groupId>
-      <artifactId>google-auth-library-oauth2-http</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.api-client</groupId>
-      <artifactId>google-api-client</artifactId>
-    </dependency>
-
-    <dependency>
       <groupId>com.google.apis</groupId>
       <artifactId>google-api-services-bigquery</artifactId>
     </dependency>
 
     <dependency>
       <groupId>com.google.apis</groupId>
-      <artifactId>google-api-services-cloudresourcemanager</artifactId>
+      <artifactId>google-api-services-storage</artifactId>
     </dependency>
 
     <dependency>
-      <groupId>com.google.apis</groupId>
-      <artifactId>google-api-services-pubsub</artifactId>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
     </dependency>
 
     <dependency>
-      <groupId>com.google.apis</groupId>
-      <artifactId>google-api-services-storage</artifactId>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>gcsio</artifactId>
+      <scope>runtime</scope>
     </dependency>
 
     <dependency>
-      <groupId>com.google.http-client</groupId>
-      <artifactId>google-http-client</artifactId>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>util</artifactId>
+      <scope>runtime</scope>
     </dependency>
 
     <!-- Required by com.google.apis:google-api-services-datastore-protobuf, but 
@@ -189,31 +181,11 @@
 
     <dependency>
       <groupId>com.google.http-client</groupId>
-      <artifactId>google-http-client-jackson2</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.http-client</groupId>
       <artifactId>google-http-client-protobuf</artifactId>
       <scope>runtime</scope>
     </dependency>
 
     <dependency>
-      <groupId>com.google.oauth-client</groupId>
-      <artifactId>google-oauth-client</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.cloud.bigdataoss</groupId>
-      <artifactId>gcsio</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.cloud.bigdataoss</groupId>
-      <artifactId>util</artifactId>
-    </dependency>
-
-    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java
deleted file mode 100644
index 7672cd7..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java
+++ /dev/null
@@ -1,32 +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.options;
-
-/**
- * Properties needed when using Google BigQuery with the Apache Beam SDK.
- */
-@Description("Options that are used to configure Google BigQuery. See "
-    + "https://cloud.google.com/bigquery/what-is-bigquery for details on BigQuery.")
-public interface BigQueryOptions extends ApplicationNameOptions, GcpOptions,
-    PipelineOptions, StreamingOptions {
-  @Description("Temporary dataset for BigQuery table operations. "
-      + "Supported values are \"bigquery.googleapis.com/{dataset}\"")
-  @Default.String("bigquery.googleapis.com/cloud_dataflow")
-  String getTempDatasetId();
-  void setTempDatasetId(String value);
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/options/CloudResourceManagerOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/CloudResourceManagerOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/CloudResourceManagerOptions.java
deleted file mode 100644
index 13fdaf3..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/CloudResourceManagerOptions.java
+++ /dev/null
@@ -1,40 +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.options;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import org.apache.beam.sdk.util.GcpProjectUtil;
-
-/**
- * Properties needed when using Google CloudResourceManager with the Apache Beam SDK.
- */
-@Description("Options that are used to configure Google CloudResourceManager. See "
-    + "https://cloud.google.com/resource-manager/ for details on CloudResourceManager.")
-public interface CloudResourceManagerOptions extends ApplicationNameOptions, GcpOptions,
-    PipelineOptions, StreamingOptions {
-  /**
-   * The GcpProjectUtil instance that should be used to communicate with Google Cloud Storage.
-   */
-  @JsonIgnore
-  @Description("The GcpProjectUtil instance that should be used to communicate"
-               + " with Google Cloud Resource Manager.")
-  @Default.InstanceFactory(GcpProjectUtil.GcpProjectUtilFactory.class)
-  @Hidden
-  GcpProjectUtil getGcpProjectUtil();
-  void setGcpProjectUtil(GcpProjectUtil value);
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DefaultPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DefaultPipelineOptionsRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DefaultPipelineOptionsRegistrar.java
index 069c109..b0ce812 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DefaultPipelineOptionsRegistrar.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DefaultPipelineOptionsRegistrar.java
@@ -32,11 +32,6 @@ public class DefaultPipelineOptionsRegistrar implements PipelineOptionsRegistrar
         .add(PipelineOptions.class)
         .add(ApplicationNameOptions.class)
         .add(StreamingOptions.class)
-        .add(BigQueryOptions.class)
-        .add(GcpOptions.class)
-        .add(GcsOptions.class)
-        .add(GoogleApiDebugOptions.class)
-        .add(PubsubOptions.class)
         .build();
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
deleted file mode 100644
index d01406f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
+++ /dev/null
@@ -1,227 +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.options;
-
-import static com.google.common.base.Strings.isNullOrEmpty;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.google.auth.Credentials;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.io.Files;
-import java.io.File;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.security.GeneralSecurityException;
-import java.util.Locale;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.util.CredentialFactory;
-import org.apache.beam.sdk.util.DefaultBucket;
-import org.apache.beam.sdk.util.GcpCredentialFactory;
-import org.apache.beam.sdk.util.InstanceBuilder;
-import org.apache.beam.sdk.util.PathValidator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Options used to configure Google Cloud Platform specific options such as the project
- * and credentials.
- *
- * <p>These options defer to the
- * <a href="https://developers.google.com/accounts/docs/application-default-credentials">
- * application default credentials</a> for authentication. See the
- * <a href="https://github.com/google/google-auth-library-java">Google Auth Library</a> for
- * alternative mechanisms for creating credentials.
- */
-@Description("Options used to configure Google Cloud Platform project and credentials.")
-public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions {
-  /**
-   * Project id to use when launching jobs.
-   */
-  @Description("Project id. Required when using Google Cloud Platform services. "
-      + "See https://cloud.google.com/storage/docs/projects for further details.")
-  @Default.InstanceFactory(DefaultProjectFactory.class)
-  String getProject();
-  void setProject(String value);
-
-  /**
-   * GCP <a href="https://developers.google.com/compute/docs/zones"
-   * >availability zone</a> for operations.
-   *
-   * <p>Default is set on a per-service basis.
-   */
-  @Description("GCP availability zone for running GCP operations. "
-      + "Default is up to the individual service.")
-  String getZone();
-  void setZone(String value);
-
-  /**
-   * The class of the credential factory that should be created and used to create
-   * credentials. If gcpCredential has not been set explicitly, an instance of this class will
-   * be constructed and used as a credential factory.
-   */
-  @Description("The class of the credential factory that should be created and used to create "
-      + "credentials. If gcpCredential has not been set explicitly, an instance of this class will "
-      + "be constructed and used as a credential factory.")
-  @Default.Class(GcpCredentialFactory.class)
-  Class<? extends CredentialFactory> getCredentialFactoryClass();
-  void setCredentialFactoryClass(
-      Class<? extends CredentialFactory> credentialFactoryClass);
-
-  /**
-   * The credential instance that should be used to authenticate against GCP services.
-   * If no credential has been set explicitly, the default is to use the instance factory
-   * that constructs a credential based upon the currently set credentialFactoryClass.
-   */
-  @JsonIgnore
-  @Description("The credential instance that should be used to authenticate against GCP services. "
-      + "If no credential has been set explicitly, the default is to use the instance factory "
-      + "that constructs a credential based upon the currently set credentialFactoryClass.")
-  @Default.InstanceFactory(GcpUserCredentialsFactory.class)
-  Credentials getGcpCredential();
-  void setGcpCredential(Credentials value);
-
-  /**
-   * Attempts to infer the default project based upon the environment this application
-   * is executing within. Currently this only supports getting the default project from gcloud.
-   */
-  class DefaultProjectFactory implements DefaultValueFactory<String> {
-    private static final Logger LOG = LoggerFactory.getLogger(DefaultProjectFactory.class);
-
-    @Override
-    public String create(PipelineOptions options) {
-      try {
-        File configFile;
-        if (getEnvironment().containsKey("CLOUDSDK_CONFIG")) {
-          configFile = new File(getEnvironment().get("CLOUDSDK_CONFIG"), "properties");
-        } else if (isWindows() && getEnvironment().containsKey("APPDATA")) {
-          configFile = new File(getEnvironment().get("APPDATA"), "gcloud/properties");
-        } else {
-          // New versions of gcloud use this file
-          configFile = new File(
-              System.getProperty("user.home"),
-              ".config/gcloud/configurations/config_default");
-          if (!configFile.exists()) {
-            // Old versions of gcloud use this file
-            configFile = new File(System.getProperty("user.home"), ".config/gcloud/properties");
-          }
-        }
-        String section = null;
-        Pattern projectPattern = Pattern.compile("^project\\s*=\\s*(.*)$");
-        Pattern sectionPattern = Pattern.compile("^\\[(.*)\\]$");
-        for (String line : Files.readLines(configFile, StandardCharsets.UTF_8)) {
-          line = line.trim();
-          if (line.isEmpty() || line.startsWith(";")) {
-            continue;
-          }
-          Matcher matcher = sectionPattern.matcher(line);
-          if (matcher.matches()) {
-            section = matcher.group(1);
-          } else if (section == null || section.equals("core")) {
-            matcher = projectPattern.matcher(line);
-            if (matcher.matches()) {
-              String project = matcher.group(1).trim();
-              LOG.info("Inferred default GCP project '{}' from gcloud. If this is the incorrect "
-                  + "project, please cancel this Pipeline and specify the command-line "
-                  + "argument --project.", project);
-              return project;
-            }
-          }
-        }
-      } catch (IOException expected) {
-        LOG.debug("Failed to find default project.", expected);
-      }
-      // return null if can't determine
-      return null;
-    }
-
-    /**
-     * Returns true if running on the Windows OS.
-     */
-    private static boolean isWindows() {
-      return System.getProperty("os.name").toLowerCase(Locale.ENGLISH).contains("windows");
-    }
-
-    /**
-     * Used to mock out getting environment variables.
-     */
-    @VisibleForTesting
-    Map<String, String> getEnvironment() {
-        return System.getenv();
-    }
-  }
-
-  /**
-   * Attempts to load the GCP credentials. See
-   * {@link CredentialFactory#getCredential()} for more details.
-   */
-  class GcpUserCredentialsFactory implements DefaultValueFactory<Credentials> {
-    @Override
-    public Credentials create(PipelineOptions options) {
-      GcpOptions gcpOptions = options.as(GcpOptions.class);
-      try {
-        CredentialFactory factory = InstanceBuilder.ofType(CredentialFactory.class)
-            .fromClass(gcpOptions.getCredentialFactoryClass())
-            .fromFactoryMethod("fromOptions")
-            .withArg(PipelineOptions.class, options)
-            .build();
-        return factory.getCredential();
-      } catch (IOException | GeneralSecurityException e) {
-        throw new RuntimeException("Unable to obtain credential", e);
-      }
-    }
-  }
-
-  /**
-   * A GCS path for storing temporary files in GCP.
-   *
-   * <p>Its default to {@link PipelineOptions#getTempLocation}.
-   */
-  @Description("A GCS path for storing temporary files in GCP.")
-  @Default.InstanceFactory(GcpTempLocationFactory.class)
-  @Nullable String getGcpTempLocation();
-  void setGcpTempLocation(String value);
-
-  /**
-   * Returns {@link PipelineOptions#getTempLocation} as the default GCP temp location.
-   */
-  class GcpTempLocationFactory implements DefaultValueFactory<String> {
-
-    @Override
-    @Nullable
-    public String create(PipelineOptions options) {
-      String tempLocation = options.getTempLocation();
-      if (isNullOrEmpty(tempLocation)) {
-        tempLocation = DefaultBucket.tryCreateDefaultBucket(options);
-        options.setTempLocation(tempLocation);
-      } else {
-        try {
-          PathValidator validator = options.as(GcsOptions.class).getPathValidator();
-          validator.validateOutputFilePrefixSupported(tempLocation);
-        } catch (Exception e) {
-          throw new IllegalArgumentException(String.format(
-              "Error constructing default value for gcpTempLocation: tempLocation is not"
-              + " a valid GCS path, %s. ", tempLocation), e);
-        }
-      }
-      return tempLocation;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java
deleted file mode 100644
index 2187e7d..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java
+++ /dev/null
@@ -1,158 +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.options;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.util.AppEngineEnvironment;
-import org.apache.beam.sdk.util.GcsPathValidator;
-import org.apache.beam.sdk.util.GcsUtil;
-import org.apache.beam.sdk.util.InstanceBuilder;
-import org.apache.beam.sdk.util.PathValidator;
-
-/**
- * Options used to configure Google Cloud Storage.
- */
-public interface GcsOptions extends
-    ApplicationNameOptions, GcpOptions, PipelineOptions {
-  /**
-   * The GcsUtil instance that should be used to communicate with Google Cloud Storage.
-   */
-  @JsonIgnore
-  @Description("The GcsUtil instance that should be used to communicate with Google Cloud Storage.")
-  @Default.InstanceFactory(GcsUtil.GcsUtilFactory.class)
-  @Hidden
-  GcsUtil getGcsUtil();
-  void setGcsUtil(GcsUtil value);
-
-  /**
-   * The ExecutorService instance to use to create threads, can be overridden to specify an
-   * ExecutorService that is compatible with the users environment. If unset, the
-   * default is to create an ExecutorService with an unbounded number of threads; this
-   * is compatible with Google AppEngine.
-   */
-  @JsonIgnore
-  @Description("The ExecutorService instance to use to create multiple threads. Can be overridden "
-      + "to specify an ExecutorService that is compatible with the users environment. If unset, "
-      + "the default is to create an ExecutorService with an unbounded number of threads; this "
-      + "is compatible with Google AppEngine.")
-  @Default.InstanceFactory(ExecutorServiceFactory.class)
-  @Hidden
-  ExecutorService getExecutorService();
-  void setExecutorService(ExecutorService value);
-
-  /**
-   * GCS endpoint to use. If unspecified, uses the default endpoint.
-   */
-  @JsonIgnore
-  @Hidden
-  @Description("The URL for the GCS API.")
-  String getGcsEndpoint();
-  void setGcsEndpoint(String value);
-
-  /**
-   * The buffer size (in bytes) to use when uploading files to GCS. Please see the documentation for
-   * {@link AbstractGoogleAsyncWriteChannel#setUploadBufferSize} for more information on the
-   * restrictions and performance implications of this value.
-   */
-  @Description("The buffer size (in bytes) to use when uploading files to GCS. Please see the "
-      + "documentation for AbstractGoogleAsyncWriteChannel.setUploadBufferSize for more "
-      + "information on the restrictions and performance implications of this value.\n\n"
-      + "https://github.com/GoogleCloudPlatform/bigdata-interop/blob/master/util/src/main/java/"
-      + "com/google/cloud/hadoop/util/AbstractGoogleAsyncWriteChannel.java")
-  @Nullable
-  Integer getGcsUploadBufferSizeBytes();
-  void setGcsUploadBufferSizeBytes(@Nullable Integer bytes);
-
-  /**
-   * The class of the validator that should be created and used to validate paths.
-   * If pathValidator has not been set explicitly, an instance of this class will be
-   * constructed and used as the path validator.
-   */
-  @Description("The class of the validator that should be created and used to validate paths. "
-      + "If pathValidator has not been set explicitly, an instance of this class will be "
-      + "constructed and used as the path validator.")
-  @Default.Class(GcsPathValidator.class)
-  Class<? extends PathValidator> getPathValidatorClass();
-  void setPathValidatorClass(Class<? extends PathValidator> validatorClass);
-
-  /**
-   * The path validator instance that should be used to validate paths.
-   * If no path validator has been set explicitly, the default is to use the instance factory that
-   * constructs a path validator based upon the currently set pathValidatorClass.
-   */
-  @JsonIgnore
-  @Description("The path validator instance that should be used to validate paths. "
-      + "If no path validator has been set explicitly, the default is to use the instance factory "
-      + "that constructs a path validator based upon the currently set pathValidatorClass.")
-  @Default.InstanceFactory(PathValidatorFactory.class)
-  PathValidator getPathValidator();
-  void setPathValidator(PathValidator validator);
-
-  /**
-   * Returns the default {@link ExecutorService} to use within the Apache Beam SDK. The
-   * {@link ExecutorService} is compatible with AppEngine.
-   */
-  class ExecutorServiceFactory implements DefaultValueFactory<ExecutorService> {
-    @SuppressWarnings("deprecation")  // IS_APP_ENGINE is deprecated for internal use only.
-    @Override
-    public ExecutorService create(PipelineOptions options) {
-      ThreadFactoryBuilder threadFactoryBuilder = new ThreadFactoryBuilder();
-      threadFactoryBuilder.setThreadFactory(MoreExecutors.platformThreadFactory());
-      if (!AppEngineEnvironment.IS_APP_ENGINE) {
-        // AppEngine doesn't allow modification of threads to be daemon threads.
-        threadFactoryBuilder.setDaemon(true);
-      }
-      /* The SDK requires an unbounded thread pool because a step may create X writers
-       * each requiring their own thread to perform the writes otherwise a writer may
-       * block causing deadlock for the step because the writers buffer is full.
-       * Also, the MapTaskExecutor launches the steps in reverse order and completes
-       * them in forward order thus requiring enough threads so that each step's writers
-       * can be active.
-       */
-      return new ThreadPoolExecutor(
-          0, Integer.MAX_VALUE, // Allow an unlimited number of re-usable threads.
-          Long.MAX_VALUE, TimeUnit.NANOSECONDS, // Keep non-core threads alive forever.
-          new SynchronousQueue<Runnable>(),
-          threadFactoryBuilder.build());
-    }
-  }
-
-  /**
-   * Creates a {@link PathValidator} object using the class specified in
-   * {@link #getPathValidatorClass()}.
-   */
-  class PathValidatorFactory implements DefaultValueFactory<PathValidator> {
-    @Override
-    public PathValidator create(PipelineOptions options) {
-      GcsOptions gcsOptions = options.as(GcsOptions.class);
-      return InstanceBuilder.ofType(PathValidator.class)
-          .fromClass(gcsOptions.getPathValidatorClass())
-          .fromFactoryMethod("fromOptions")
-          .withArg(PipelineOptions.class, options)
-          .build();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java
deleted file mode 100644
index f9cb575..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java
+++ /dev/null
@@ -1,87 +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.options;
-
-import com.google.api.client.googleapis.services.AbstractGoogleClient;
-import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
-import com.google.api.client.googleapis.services.GoogleClientRequestInitializer;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * These options configure debug settings for Google API clients created within the Apache Beam SDK.
- */
-public interface GoogleApiDebugOptions extends PipelineOptions {
-  /**
-   * This option enables tracing of API calls to Google services used within the Apache
-   * Beam SDK. Values are expected in JSON format <code>{"ApiName":"TraceDestination",...}
-   * </code> where the {@code ApiName} represents the request classes canonical name. The
-   * {@code TraceDestination} is a logical trace consumer to whom the trace will be reported.
-   * Typically, "producer" is the right destination to use: this makes API traces available to the
-   * team offering the API. Note that by enabling this option, the contents of the requests to and
-   * from Google Cloud services will be made available to Google. For example, by specifying
-   * <code>{"Dataflow":"producer"}</code>, all calls to the Dataflow service will be made available
-   * to Google, specifically to the Google Cloud Dataflow team.
-   */
-  @Description("This option enables tracing of API calls to Google services used within the Apache "
-      + "Beam SDK. Values are expected in JSON format {\"ApiName\":\"TraceDestination\",...} "
-      + "where the ApiName represents the request classes canonical name. The TraceDestination is "
-      + "a logical trace consumer to whom the trace will be reported. Typically, \"producer\" is "
-      + "the right destination to use: this makes API traces available to the team offering the "
-      + "API. Note that by enabling this option, the contents of the requests to and from "
-      + "Google Cloud services will be made available to Google. For example, by specifying "
-      + "{\"Dataflow\":\"producer\"}, all calls to the Dataflow service will be made available to "
-      + "Google, specifically to the Google Cloud Dataflow team.")
-  GoogleApiTracer getGoogleApiTrace();
-  void setGoogleApiTrace(GoogleApiTracer commands);
-
-  /**
-   * A {@link GoogleClientRequestInitializer} that adds the trace destination to Google API calls.
-   */
-  class GoogleApiTracer extends HashMap<String, String>
-      implements GoogleClientRequestInitializer {
-    /**
-     * Creates a {@link GoogleApiTracer} that sets the trace destination on all
-     * calls that match the given client type.
-     */
-    public GoogleApiTracer addTraceFor(AbstractGoogleClient client, String traceDestination) {
-      put(client.getClass().getCanonicalName(), traceDestination);
-      return this;
-    }
-
-    /**
-     * Creates a {@link GoogleApiTracer} that sets the trace {@code traceDestination} on all
-     * calls that match for the given request type.
-     */
-    public GoogleApiTracer addTraceFor(
-        AbstractGoogleClientRequest<?> request, String traceDestination) {
-      put(request.getClass().getCanonicalName(), traceDestination);
-      return this;
-    }
-
-    @Override
-    public void initialize(AbstractGoogleClientRequest<?> request) throws IOException {
-      for (Map.Entry<String, String> entry : this.entrySet()) {
-        if (request.getClass().getCanonicalName().contains(entry.getKey())) {
-          request.set("$trace", entry.getValue());
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
index 4e7bc89..88d6576 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
@@ -30,7 +30,6 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicLong;
 import javax.annotation.concurrent.ThreadSafe;
 import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer;
 import org.apache.beam.sdk.options.ProxyInvocationHandler.Deserializer;
 import org.apache.beam.sdk.options.ProxyInvocationHandler.Serializer;
 import org.apache.beam.sdk.runners.PipelineRunner;
@@ -187,8 +186,7 @@ import org.joda.time.format.DateTimeFormatter;
  * <a href="https://github.com/FasterXML/jackson-annotations">annotations</a> to aid in
  * serialization of custom types. We point you to the public
  * <a href="https://github.com/FasterXML/jackson">Jackson documentation</a> when attempting
- * to add serialization support for your custom types. See {@link GoogleApiTracer} for an
- * example using the Jackson annotations to serialize and deserialize a custom type.
+ * to add serialization support for your custom types.
  *
  * <p>Note: It is an error to have the same property available in multiple interfaces with only
  * some of them being annotated with {@link JsonIgnore @JsonIgnore}. It is also an error to mark a

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java
deleted file mode 100644
index b065d19..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java
+++ /dev/null
@@ -1,36 +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.options;
-
-/**
- * Properties that can be set when using Google Cloud Pub/Sub with the Apache Beam SDK.
- */
-@Description("Options that are used to configure Google Cloud Pub/Sub. See "
-    + "https://cloud.google.com/pubsub/docs/overview for details on Cloud Pub/Sub.")
-public interface PubsubOptions extends ApplicationNameOptions, GcpOptions,
-    PipelineOptions, StreamingOptions {
-
-  /**
-   * Root URL for use with the Google Cloud Pub/Sub API.
-   */
-  @Description("Root URL for use with the Google Cloud Pub/Sub API")
-  @Default.String("https://pubsub.googleapis.com")
-  @Hidden
-  String getPubsubRootUrl();
-  void setPubsubRootUrl(String value);
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
index 7b2fba3..a318dfc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
@@ -23,6 +23,7 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.io.FileSystems;
 import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
 import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.util.InstanceBuilder;
 
@@ -41,6 +42,7 @@ public abstract class PipelineRunner<ResultT extends PipelineResult> {
    */
   public static PipelineRunner<? extends PipelineResult> fromOptions(PipelineOptions options) {
     checkNotNull(options);
+    PipelineOptionsValidator.validate(PipelineOptions.class, options);
 
     // (Re-)register standard IO factories. Clobbers any prior credentials.
     IOChannelUtils.registerIOFactoriesAllowOverride(options);

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
deleted file mode 100644
index 8f752c0..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.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.testing;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.api.client.http.HttpTransport;
-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.BigqueryScopes;
-import com.google.api.services.bigquery.model.QueryRequest;
-import com.google.api.services.bigquery.model.QueryResponse;
-import com.google.api.services.bigquery.model.TableCell;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.auth.Credentials;
-import com.google.auth.http.HttpCredentialsAdapter;
-import com.google.auth.oauth2.GoogleCredentials;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
-import com.google.common.hash.HashCode;
-import com.google.common.hash.Hashing;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.nio.charset.StandardCharsets;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-import javax.annotation.Nonnull;
-import javax.annotation.concurrent.NotThreadSafe;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.util.FluentBackoff;
-import org.apache.beam.sdk.util.Transport;
-import org.hamcrest.Description;
-import org.hamcrest.TypeSafeMatcher;
-import org.joda.time.Duration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A matcher to verify data in BigQuery by processing given query
- * and comparing with content's checksum.
- *
- * <p>Example:
- * <pre>{@code [
- *   assertThat(job, new BigqueryMatcher(appName, projectId, queryString, expectedChecksum));
- * ]}</pre>
- */
-@NotThreadSafe
-public class BigqueryMatcher extends TypeSafeMatcher<PipelineResult>
-    implements SerializableMatcher<PipelineResult> {
-  private static final Logger LOG = LoggerFactory.getLogger(BigqueryMatcher.class);
-
-  // The maximum number of retries to execute a BigQuery RPC
-  static final int MAX_QUERY_RETRIES = 4;
-
-  // The initial backoff for executing a BigQuery RPC
-  private static final Duration INITIAL_BACKOFF = Duration.standardSeconds(1L);
-
-  // The total number of rows in query response to be formatted for debugging purpose
-  private static final int TOTAL_FORMATTED_ROWS = 20;
-
-  // The backoff factory with initial configs
-  static final FluentBackoff BACKOFF_FACTORY =
-      FluentBackoff.DEFAULT
-          .withMaxRetries(MAX_QUERY_RETRIES)
-          .withInitialBackoff(INITIAL_BACKOFF);
-
-  private final String applicationName;
-  private final String projectId;
-  private final String query;
-  private final String expectedChecksum;
-  private String actualChecksum;
-  private transient QueryResponse response;
-
-  public BigqueryMatcher(
-      String applicationName, String projectId, String query, String expectedChecksum) {
-    validateArgument("applicationName", applicationName);
-    validateArgument("projectId", projectId);
-    validateArgument("query", query);
-    validateArgument("expectedChecksum", expectedChecksum);
-
-    this.applicationName = applicationName;
-    this.projectId = projectId;
-    this.query = query;
-    this.expectedChecksum = expectedChecksum;
-  }
-
-  @Override
-  protected boolean matchesSafely(PipelineResult pipelineResult) {
-    LOG.info("Verifying Bigquery data");
-    Bigquery bigqueryClient = newBigqueryClient(applicationName);
-
-    // execute query
-    LOG.debug("Executing query: {}", query);
-    try {
-      QueryRequest queryContent = new QueryRequest();
-      queryContent.setQuery(query);
-
-      response = queryWithRetries(
-          bigqueryClient, queryContent, Sleeper.DEFAULT, BACKOFF_FACTORY.backoff());
-    } catch (IOException | InterruptedException e) {
-      if (e instanceof InterruptedIOException) {
-        Thread.currentThread().interrupt();
-      }
-      throw new RuntimeException("Failed to fetch BigQuery data.", e);
-    }
-
-    if (!response.getJobComplete()) {
-      // query job not complete, verification failed
-      return false;
-    } else {
-      // compute checksum
-      actualChecksum = generateHash(response.getRows());
-      LOG.debug("Generated a SHA1 checksum based on queried data: {}", actualChecksum);
-
-      return expectedChecksum.equals(actualChecksum);
-    }
-  }
-
-  @VisibleForTesting
-  Bigquery newBigqueryClient(String applicationName) {
-    HttpTransport transport = Transport.getTransport();
-    JsonFactory jsonFactory = Transport.getJsonFactory();
-    Credentials credential = getDefaultCredential();
-
-    return new Bigquery.Builder(transport, jsonFactory, new HttpCredentialsAdapter(credential))
-        .setApplicationName(applicationName)
-        .build();
-  }
-
-  @Nonnull
-  @VisibleForTesting
-  QueryResponse queryWithRetries(Bigquery bigqueryClient, QueryRequest queryContent,
-                                 Sleeper sleeper, BackOff backOff)
-      throws IOException, InterruptedException {
-    IOException lastException = null;
-    do {
-      if (lastException != null) {
-        LOG.warn("Retrying query ({}) after exception", queryContent.getQuery(), lastException);
-      }
-      try {
-        QueryResponse response = bigqueryClient.jobs().query(projectId, queryContent).execute();
-        if (response != null) {
-          return response;
-        } else {
-          lastException =
-              new IOException("Expected valid response from query job, but received null.");
-        }
-      } catch (IOException e) {
-        // ignore and retry
-        lastException = e;
-      }
-    } while(BackOffUtils.next(sleeper, backOff));
-
-    throw new RuntimeException(
-        String.format(
-            "Unable to get BigQuery response after retrying %d times using query (%s)",
-            MAX_QUERY_RETRIES,
-            queryContent.getQuery()),
-        lastException);
-  }
-
-  private void validateArgument(String name, String value) {
-    checkArgument(
-        !Strings.isNullOrEmpty(value), "Expected valid %s, but was %s", name, value);
-  }
-
-  private Credentials getDefaultCredential() {
-    GoogleCredentials credential;
-    try {
-      credential = GoogleCredentials.getApplicationDefault();
-    } catch (IOException e) {
-      throw new RuntimeException("Failed to get application default credential.", e);
-    }
-
-    if (credential.createScopedRequired()) {
-      Collection<String> bigqueryScope =
-          Lists.newArrayList(BigqueryScopes.CLOUD_PLATFORM_READ_ONLY);
-      credential = credential.createScoped(bigqueryScope);
-    }
-    return credential;
-  }
-
-  private String generateHash(@Nonnull List<TableRow> rows) {
-    List<HashCode> rowHashes = Lists.newArrayList();
-    for (TableRow row : rows) {
-      List<String> cellsInOneRow = Lists.newArrayList();
-      for (TableCell cell : row.getF()) {
-        cellsInOneRow.add(Objects.toString(cell.getV()));
-        Collections.sort(cellsInOneRow);
-      }
-      rowHashes.add(
-          Hashing.sha1().hashString(cellsInOneRow.toString(), StandardCharsets.UTF_8));
-    }
-    return Hashing.combineUnordered(rowHashes).toString();
-  }
-
-  @Override
-  public void describeTo(Description description) {
-    description
-        .appendText("Expected checksum is (")
-        .appendText(expectedChecksum)
-        .appendText(")");
-  }
-
-  @Override
-  public void describeMismatchSafely(PipelineResult pResult, Description description) {
-    String info;
-    if (!response.getJobComplete()) {
-      // query job not complete
-      info = String.format("The query job hasn't completed. Got response: %s", response);
-    } else {
-      // checksum mismatch
-      info = String.format("was (%s).%n"
-          + "\tTotal number of rows are: %d.%n"
-          + "\tQueried data details:%s",
-          actualChecksum, response.getTotalRows(), formatRows(TOTAL_FORMATTED_ROWS));
-    }
-    description.appendText(info);
-  }
-
-  private String formatRows(int totalNumRows) {
-    StringBuilder samples = new StringBuilder();
-    List<TableRow> rows = response.getRows();
-    for (int i = 0; i < totalNumRows && i < rows.size(); i++) {
-      samples.append(String.format("%n\t\t"));
-      for (TableCell field : rows.get(i).getF()) {
-        samples.append(String.format("%-10s", field.getV()));
-      }
-    }
-    if (rows.size() > totalNumRows) {
-      samples.append(String.format("%n\t\t..."));
-    }
-    return samples.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
index 3d3de51..1273442 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
@@ -42,14 +42,12 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.io.FileSystems;
 import org.apache.beam.sdk.options.ApplicationNameOptions;
-import org.apache.beam.sdk.options.GcpOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.util.TestCredential;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestRule;
 import org.junit.runner.Description;
@@ -400,7 +398,6 @@ public class TestPipeline extends Pipeline implements TestRule {
         if (!Strings.isNullOrEmpty(useDefaultDummy) && Boolean.valueOf(useDefaultDummy)) {
           options.setRunner(CrashingRunner.class);
         }
-        options.as(GcpOptions.class).setGcpCredential(new TestCredential());
       }
       options.setStableUniqueNames(CheckEnabled.ERROR);
 

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java
deleted file mode 100644
index b0fcbd1..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java
+++ /dev/null
@@ -1,62 +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 java.lang.reflect.InvocationTargetException;
-
-/** Stores whether we are running within AppEngine or not. */
-public class AppEngineEnvironment {
-  /**
-   * True if running inside of AppEngine, false otherwise.
-   */
-  @Deprecated
-  public static final boolean IS_APP_ENGINE = isAppEngine();
-
-  /**
-   * Attempts to detect whether we are inside of AppEngine.
-   *
-   * <p>Purposely copied and left private from private <a href="https://code.google.com/p/
-   * guava-libraries/source/browse/guava/src/com/google/common/util/concurrent/
-   * MoreExecutors.java#785">code.google.common.util.concurrent.MoreExecutors#isAppEngine</a>.
-   *
-   * @return true if we are inside of AppEngine, false otherwise.
-   */
-  static boolean isAppEngine() {
-    if (System.getProperty("com.google.appengine.runtime.environment") == null) {
-      return false;
-    }
-    try {
-      // If the current environment is null, we're not inside AppEngine.
-      return Class.forName("com.google.apphosting.api.ApiProxy")
-          .getMethod("getCurrentEnvironment")
-          .invoke(null) != null;
-    } catch (ClassNotFoundException e) {
-      // If ApiProxy doesn't exist, we're not on AppEngine at all.
-      return false;
-    } catch (InvocationTargetException e) {
-      // If ApiProxy throws an exception, we're not in a proper AppEngine environment.
-      return false;
-    } catch (IllegalAccessException e) {
-      // If the method isn't accessible, we're not on a supported version of AppEngine;
-      return false;
-    } catch (NoSuchMethodException e) {
-      // If the method doesn't exist, we're not on a supported version of AppEngine;
-      return false;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java
deleted file mode 100644
index 6229650..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java
+++ /dev/null
@@ -1,29 +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 com.google.auth.Credentials;
-import java.io.IOException;
-import java.security.GeneralSecurityException;
-
-/**
- * Construct an oauth credential to be used by the SDK and the SDK workers.
- */
-public interface CredentialFactory {
-  Credentials getCredential() throws IOException, GeneralSecurityException;
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DefaultBucket.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DefaultBucket.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DefaultBucket.java
deleted file mode 100644
index 75954c0..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DefaultBucket.java
+++ /dev/null
@@ -1,105 +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.Strings.isNullOrEmpty;
-
-import com.google.api.services.storage.model.Bucket;
-import com.google.common.annotations.VisibleForTesting;
-import java.io.IOException;
-import java.nio.file.FileAlreadyExistsException;
-import org.apache.beam.sdk.options.CloudResourceManagerOptions;
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Utility class for handling default GCS buckets.
- */
-public class DefaultBucket {
-  static final Logger LOG = LoggerFactory.getLogger(DefaultBucket.class);
-
-  static final String DEFAULT_REGION = "us-central1";
-
-  /**
-   * Creates a default bucket or verifies the existence and proper access control
-   * of an existing default bucket.  Returns the location if successful.
-   */
-  public static String tryCreateDefaultBucket(PipelineOptions options) {
-    GcsOptions gcpOptions = options.as(GcsOptions.class);
-
-    final String projectId = gcpOptions.getProject();
-    checkArgument(!isNullOrEmpty(projectId),
-                  "--project is a required option.");
-
-    // Look up the project number, to create a default bucket with a stable
-    // name with no special characters.
-    long projectNumber = 0L;
-    try {
-      projectNumber = gcpOptions.as(CloudResourceManagerOptions.class)
-          .getGcpProjectUtil().getProjectNumber(projectId);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to verify project with ID " + projectId, e);
-    }
-    String region = DEFAULT_REGION;
-    if (!isNullOrEmpty(gcpOptions.getZone())) {
-      region = getRegionFromZone(gcpOptions.getZone());
-    }
-    final String bucketName =
-      "dataflow-staging-" + region + "-" + projectNumber;
-    LOG.info("No staging location provided, attempting to use default bucket: {}",
-             bucketName);
-    Bucket bucket = new Bucket()
-      .setName(bucketName)
-      .setLocation(region);
-    // Always try to create the bucket before checking access, so that we do not
-    // race with other pipelines that may be attempting to do the same thing.
-    try {
-      gcpOptions.getGcsUtil().createBucket(projectId, bucket);
-    } catch (FileAlreadyExistsException e) {
-      LOG.debug("Bucket '{}'' already exists, verifying access.", bucketName);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable create default bucket.", e);
-    }
-
-    // Once the bucket is expected to exist, verify that it is correctly owned
-    // by the project executing the job.
-    try {
-      long owner = gcpOptions.getGcsUtil().bucketOwner(
-        GcsPath.fromComponents(bucketName, ""));
-      checkArgument(
-        owner == projectNumber,
-        "Bucket owner does not match the project from --project:"
-        + " %s vs. %s", owner, projectNumber);
-    } catch (IOException e) {
-      throw new RuntimeException(
-        "Unable to determine the owner of the default bucket at gs://" + bucketName, e);
-    }
-    return "gs://" + bucketName;
-  }
-
-  @VisibleForTesting
-  static String getRegionFromZone(String zone) {
-    String[] zoneParts = zone.split("-");
-    checkArgument(zoneParts.length >= 2, "Invalid zone provided: %s", zone);
-    return zoneParts[0] + "-" + zoneParts[1];
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java
deleted file mode 100644
index e1fa18f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java
+++ /dev/null
@@ -1,67 +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 com.google.auth.Credentials;
-import com.google.auth.oauth2.GoogleCredentials;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-/**
- * Construct an oauth credential to be used by the SDK and the SDK workers.
- * Returns a GCP credential.
- */
-public class GcpCredentialFactory implements CredentialFactory {
-  /**
-   * The scope cloud-platform provides access to all Cloud Platform resources.
-   * cloud-platform isn't sufficient yet for talking to datastore so we request
-   * those resources separately.
-   *
-   * <p>Note that trusted scope relationships don't apply to OAuth tokens, so for
-   * services we access directly (GCS) as opposed to through the backend
-   * (BigQuery, GCE), we need to explicitly request that scope.
-   */
-  private static final List<String> SCOPES = Arrays.asList(
-      "https://www.googleapis.com/auth/cloud-platform",
-      "https://www.googleapis.com/auth/devstorage.full_control",
-      "https://www.googleapis.com/auth/userinfo.email",
-      "https://www.googleapis.com/auth/datastore",
-      "https://www.googleapis.com/auth/pubsub");
-
-  private static final GcpCredentialFactory INSTANCE = new GcpCredentialFactory();
-
-  public static GcpCredentialFactory fromOptions(PipelineOptions options) {
-    return INSTANCE;
-  }
-
-  /**
-   * Returns a default GCP {@link Credentials} or null when it fails.
-   */
-  @Override
-  public Credentials getCredential() {
-    try {
-      return GoogleCredentials.getApplicationDefault().createScoped(SCOPES);
-    } catch (IOException e) {
-      // Ignore the exception
-      // Pipelines that only access to public data should be able to run without credentials.
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpProjectUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpProjectUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpProjectUtil.java
deleted file mode 100644
index f73afe0..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpProjectUtil.java
+++ /dev/null
@@ -1,106 +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 com.google.api.client.util.BackOff;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.cloudresourcemanager.CloudResourceManager;
-import com.google.api.services.cloudresourcemanager.model.Project;
-import com.google.cloud.hadoop.util.ResilientOperation;
-import com.google.cloud.hadoop.util.RetryDeterminer;
-import com.google.common.annotations.VisibleForTesting;
-import java.io.IOException;
-import org.apache.beam.sdk.options.CloudResourceManagerOptions;
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.joda.time.Duration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Provides operations on Google Cloud Platform Projects.
- */
-public class GcpProjectUtil {
-  /**
-   * A {@link DefaultValueFactory} able to create a {@link GcpProjectUtil} using
-   * any transport flags specified on the {@link PipelineOptions}.
-   */
-  public static class GcpProjectUtilFactory implements DefaultValueFactory<GcpProjectUtil> {
-    /**
-     * Returns an instance of {@link GcpProjectUtil} based on the
-     * {@link PipelineOptions}.
-     */
-    @Override
-    public GcpProjectUtil create(PipelineOptions options) {
-      LOG.debug("Creating new GcpProjectUtil");
-      CloudResourceManagerOptions crmOptions = options.as(CloudResourceManagerOptions.class);
-      return new GcpProjectUtil(
-          Transport.newCloudResourceManagerClient(crmOptions).build());
-    }
-  }
-
-  private static final Logger LOG = LoggerFactory.getLogger(GcpProjectUtil.class);
-
-  private static final FluentBackoff BACKOFF_FACTORY =
-      FluentBackoff.DEFAULT.withMaxRetries(3).withInitialBackoff(Duration.millis(200));
-
-  /** Client for the CRM API. */
-  private CloudResourceManager crmClient;
-
-  private GcpProjectUtil(CloudResourceManager crmClient) {
-    this.crmClient = crmClient;
-  }
-
-  // Use this only for testing purposes.
-  @VisibleForTesting
-  void setCrmClient(CloudResourceManager crmClient) {
-    this.crmClient = crmClient;
-  }
-
-  /**
-   * Returns the project number or throws an exception if the project does not
-   * exist or has other access exceptions.
-   */
-  public long getProjectNumber(String projectId) throws IOException {
-    return getProjectNumber(
-      projectId,
-      BACKOFF_FACTORY.backoff(),
-      Sleeper.DEFAULT);
-  }
-
-  /**
-   * Returns the project number or throws an error if the project does not
-   * exist or has other access errors.
-   */
-  @VisibleForTesting
-  long getProjectNumber(String projectId, BackOff backoff, Sleeper sleeper) throws IOException {
-      CloudResourceManager.Projects.Get getProject =
-          crmClient.projects().get(projectId);
-      try {
-        Project project = ResilientOperation.retry(
-            ResilientOperation.getGoogleRequestCallable(getProject),
-            backoff,
-            RetryDeterminer.SOCKET_ERRORS,
-            IOException.class,
-            sleeper);
-        return project.getProjectNumber();
-      } catch (Exception e) {
-        throw new IOException("Unable to get project number", e);
-     }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java
deleted file mode 100644
index 745dcb9..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java
+++ /dev/null
@@ -1,111 +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 java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.WritableByteChannel;
-import java.nio.file.Path;
-import java.util.Collection;
-import java.util.LinkedList;
-import java.util.List;
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-
-/**
- * Implements IOChannelFactory for GCS.
- */
-public class GcsIOChannelFactory implements IOChannelFactory {
-
-  /**
-   * Create a {@link GcsIOChannelFactory} with the given {@link PipelineOptions}.
-   */
-  public static GcsIOChannelFactory fromOptions(PipelineOptions options) {
-    return new GcsIOChannelFactory(options.as(GcsOptions.class));
-  }
-
-  private final GcsOptions options;
-
-  private GcsIOChannelFactory(GcsOptions options) {
-    this.options = options;
-  }
-
-  @Override
-  public Collection<String> match(String spec) throws IOException {
-    GcsPath path = GcsPath.fromUri(spec);
-    GcsUtil util = options.getGcsUtil();
-    List<GcsPath> matched = util.expand(path);
-
-    List<String> specs = new LinkedList<>();
-    for (GcsPath match : matched) {
-      specs.add(match.toString());
-    }
-
-    return specs;
-  }
-
-  @Override
-  public ReadableByteChannel open(String spec) throws IOException {
-    GcsPath path = GcsPath.fromUri(spec);
-    GcsUtil util = options.getGcsUtil();
-    return util.open(path);
-  }
-
-  @Override
-  public WritableByteChannel create(String spec, String mimeType)
-      throws IOException {
-    GcsPath path = GcsPath.fromUri(spec);
-    GcsUtil util = options.getGcsUtil();
-    return util.create(path, mimeType);
-  }
-
-  @Override
-  public long getSizeBytes(String spec) throws IOException {
-    GcsPath path = GcsPath.fromUri(spec);
-    GcsUtil util = options.getGcsUtil();
-    return util.fileSize(path);
-  }
-
-  @Override
-  public boolean isReadSeekEfficient(String spec) throws IOException {
-    // TODO It is incorrect to return true here for files with content encoding set to gzip.
-    return true;
-  }
-
-  @Override
-  public String resolve(String path, String other) throws IOException {
-    return toPath(path).resolve(other).toString();
-  }
-
-  @Override
-  public Path toPath(String path) {
-    return GcsPath.fromUri(path);
-  }
-
-  @Override
-  public void copy(Iterable<String> srcFilenames, Iterable<String> destFilenames)
-      throws IOException {
-    options.getGcsUtil().copy(srcFilenames, destFilenames);
-  }
-
-  @Override
-  public void remove(Collection<String> filesOrDirs) throws IOException {
-    options.getGcsUtil().remove(filesOrDirs);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java
deleted file mode 100644
index b4c457f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java
+++ /dev/null
@@ -1,38 +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 com.google.auto.service.AutoService;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-/**
- * {@link AutoService} registrar for the {@link GcsIOChannelFactory}.
- */
-@AutoService(IOChannelFactoryRegistrar.class)
-public class GcsIOChannelFactoryRegistrar implements IOChannelFactoryRegistrar {
-
-  @Override
-  public GcsIOChannelFactory fromOptions(PipelineOptions options) {
-    return GcsIOChannelFactory.fromOptions(options);
-  }
-
-  @Override
-  public String getScheme() {
-    return "gs";
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
deleted file mode 100644
index a5b951d..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
+++ /dev/null
@@ -1,95 +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 java.io.IOException;
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-
-/**
- * GCP implementation of {@link PathValidator}. Only GCS paths are allowed.
- */
-public class GcsPathValidator implements PathValidator {
-
-  private GcsOptions gcpOptions;
-
-  private GcsPathValidator(GcsOptions options) {
-    this.gcpOptions = options;
-  }
-
-  public static GcsPathValidator fromOptions(PipelineOptions options) {
-    return new GcsPathValidator(options.as(GcsOptions.class));
-  }
-
-  /**
-   * Validates the the input GCS path is accessible and that the path
-   * is well formed.
-   */
-  @Override
-  public String validateInputFilePatternSupported(String filepattern) {
-    GcsPath gcsPath = getGcsPath(filepattern);
-    checkArgument(gcpOptions.getGcsUtil().isGcsPatternSupported(gcsPath.getObject()));
-    String returnValue = verifyPath(filepattern);
-    verifyPathIsAccessible(filepattern, "Could not find file %s");
-    return returnValue;
-  }
-
-  /**
-   * Validates the the output GCS path is accessible and that the path
-   * is well formed.
-   */
-  @Override
-  public String validateOutputFilePrefixSupported(String filePrefix) {
-    String returnValue = verifyPath(filePrefix);
-    verifyPathIsAccessible(filePrefix, "Output path does not exist or is not writeable: %s");
-    return returnValue;
-  }
-
-  @Override
-  public String verifyPath(String path) {
-    GcsPath gcsPath = getGcsPath(path);
-    checkArgument(gcsPath.isAbsolute(), "Must provide absolute paths for Dataflow");
-    checkArgument(!gcsPath.getObject().contains("//"),
-        "Dataflow Service does not allow objects with consecutive slashes");
-    return gcsPath.toResourceName();
-  }
-
-  private void verifyPathIsAccessible(String path, String errorMessage) {
-    GcsPath gcsPath = getGcsPath(path);
-    try {
-      checkArgument(gcpOptions.getGcsUtil().bucketAccessible(gcsPath),
-        errorMessage, path);
-    } catch (IOException e) {
-      throw new RuntimeException(
-          String.format("Unable to verify that GCS bucket gs://%s exists.", gcsPath.getBucket()),
-          e);
-    }
-  }
-
-  private GcsPath getGcsPath(String path) {
-    try {
-      return GcsPath.fromUri(path);
-    } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException(String.format(
-          "Expected a valid 'gs://' path but was given '%s'", path), e);
-    }
-  }
-}


[4/7] beam git commit: [BEAM-1871] Create new GCP core module package and move several GCP related classes from beam-sdks-java-core over.

Posted by lc...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcpProjectUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcpProjectUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcpProjectUtilTest.java
deleted file mode 100644
index 23f0418..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcpProjectUtilTest.java
+++ /dev/null
@@ -1,76 +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.Mockito.when;
-
-import com.google.api.client.util.BackOff;
-import com.google.api.services.cloudresourcemanager.CloudResourceManager;
-import com.google.api.services.cloudresourcemanager.model.Project;
-import java.net.SocketTimeoutException;
-import org.apache.beam.sdk.options.CloudResourceManagerOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
-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.Mockito;
-
-/** Test case for {@link GcpProjectUtil}. */
-@RunWith(JUnit4.class)
-public class GcpProjectUtilTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-
-  private static CloudResourceManagerOptions crmOptionsWithTestCredential() {
-    CloudResourceManagerOptions pipelineOptions =
-        PipelineOptionsFactory.as(CloudResourceManagerOptions.class);
-    pipelineOptions.setGcpCredential(new TestCredential());
-    return pipelineOptions;
-  }
-
-  @Test
-  public void testGetProjectNumber() throws Exception {
-    CloudResourceManagerOptions pipelineOptions = crmOptionsWithTestCredential();
-    GcpProjectUtil projectUtil = pipelineOptions.getGcpProjectUtil();
-
-    CloudResourceManager.Projects mockProjects = Mockito.mock(
-        CloudResourceManager.Projects.class);
-    CloudResourceManager mockCrm = Mockito.mock(CloudResourceManager.class);
-    projectUtil.setCrmClient(mockCrm);
-
-    CloudResourceManager.Projects.Get mockProjectsGet =
-        Mockito.mock(CloudResourceManager.Projects.Get.class);
-
-    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
-    Project project = new Project();
-    project.setProjectNumber(5L);
-
-    when(mockCrm.projects()).thenReturn(mockProjects);
-    when(mockProjects.get(any(String.class))).thenReturn(mockProjectsGet);
-    when(mockProjectsGet.execute())
-      .thenThrow(new SocketTimeoutException("SocketException"))
-      .thenReturn(project);
-
-    assertEquals(5L, projectUtil.getProjectNumber(
-        "foo", mockBackOff, new FastNanoClockAndSleeper()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java
deleted file mode 100644
index a29dd45..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java
+++ /dev/null
@@ -1,44 +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.fail;
-
-import com.google.common.collect.Lists;
-import java.util.ServiceLoader;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link GcsIOChannelFactoryRegistrar}.
- */
-@RunWith(JUnit4.class)
-public class GcsIOChannelFactoryRegistrarTest {
-
-  @Test
-  public void testServiceLoader() {
-    for (IOChannelFactoryRegistrar registrar
-        : Lists.newArrayList(ServiceLoader.load(IOChannelFactoryRegistrar.class).iterator())) {
-      if (registrar instanceof GcsIOChannelFactoryRegistrar) {
-        return;
-      }
-    }
-    fail("Expected to find " + GcsIOChannelFactoryRegistrar.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java
deleted file mode 100644
index 7248b38..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java
+++ /dev/null
@@ -1,43 +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 org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link GcsIOChannelFactoryTest}. */
-@RunWith(JUnit4.class)
-public class GcsIOChannelFactoryTest {
-  private GcsIOChannelFactory factory;
-
-  @Before
-  public void setUp() {
-    factory = GcsIOChannelFactory.fromOptions(PipelineOptionsFactory.as(GcsOptions.class));
-  }
-
-  @Test
-  public void testResolve() throws Exception {
-    assertEquals("gs://bucket/object", factory.resolve("gs://bucket", "object"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java
deleted file mode 100644
index dc36319..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java
+++ /dev/null
@@ -1,87 +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.mockito.Matchers.any;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-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;
-
-/** Tests for {@link GcsPathValidator}. */
-@RunWith(JUnit4.class)
-public class GcsPathValidatorTest {
-  @Rule public ExpectedException expectedException = ExpectedException.none();
-
-  @Mock private GcsUtil mockGcsUtil;
-  private GcsPathValidator validator;
-
-  @Before
-  public void setUp() throws Exception {
-    MockitoAnnotations.initMocks(this);
-    when(mockGcsUtil.bucketAccessible(any(GcsPath.class))).thenReturn(true);
-    GcsOptions options = PipelineOptionsFactory.as(GcsOptions.class);
-    options.setGcpCredential(new TestCredential());
-    options.setGcsUtil(mockGcsUtil);
-    validator = GcsPathValidator.fromOptions(options);
-  }
-
-  @Test
-  public void testValidFilePattern() {
-    validator.validateInputFilePatternSupported("gs://bucket/path");
-  }
-
-  @Test
-  public void testInvalidFilePattern() {
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage(
-        "Expected a valid 'gs://' path but was given '/local/path'");
-    validator.validateInputFilePatternSupported("/local/path");
-  }
-
-  @Test
-  public void testWhenBucketDoesNotExist() throws Exception {
-    when(mockGcsUtil.bucketAccessible(any(GcsPath.class))).thenReturn(false);
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage(
-        "Could not find file gs://non-existent-bucket/location");
-    validator.validateInputFilePatternSupported("gs://non-existent-bucket/location");
-  }
-
-  @Test
-  public void testValidOutputPrefix() {
-    validator.validateOutputFilePrefixSupported("gs://bucket/path");
-  }
-
-  @Test
-  public void testInvalidOutputPrefix() {
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage(
-        "Expected a valid 'gs://' path but was given '/local/path'");
-    validator.validateOutputFilePrefixSupported("/local/path");
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
deleted file mode 100644
index 03668ce..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
+++ /dev/null
@@ -1,798 +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.contains;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.when;
-
-import com.google.api.client.googleapis.batch.BatchRequest;
-import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.http.HttpRequest;
-import com.google.api.client.http.HttpResponse;
-import com.google.api.client.http.HttpStatusCodes;
-import com.google.api.client.http.HttpTransport;
-import com.google.api.client.http.LowLevelHttpRequest;
-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.JsonFactory;
-import com.google.api.client.json.jackson2.JacksonFactory;
-import com.google.api.client.testing.http.HttpTesting;
-import com.google.api.client.testing.http.MockHttpTransport;
-import com.google.api.client.testing.http.MockLowLevelHttpRequest;
-import com.google.api.client.testing.http.MockLowLevelHttpResponse;
-import com.google.api.client.util.BackOff;
-import com.google.api.services.storage.Storage;
-import com.google.api.services.storage.model.Bucket;
-import com.google.api.services.storage.model.Objects;
-import com.google.api.services.storage.model.StorageObject;
-import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel;
-import com.google.cloud.hadoop.util.ClientRequestHelper;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import java.io.ByteArrayInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.math.BigInteger;
-import java.net.SocketTimeoutException;
-import java.nio.channels.SeekableByteChannel;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.AccessDeniedException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
-import org.apache.beam.sdk.util.GcsUtil.StorageObjectOrIOException;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-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.Mockito;
-
-/** Test case for {@link GcsUtil}. */
-@RunWith(JUnit4.class)
-public class GcsUtilTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-
-  @Test
-  public void testGlobTranslation() {
-    assertEquals("foo", GcsUtil.globToRegexp("foo"));
-    assertEquals("fo[^/]*o", GcsUtil.globToRegexp("fo*o"));
-    assertEquals("f[^/]*o\\.[^/]", GcsUtil.globToRegexp("f*o.?"));
-    assertEquals("foo-[0-9][^/]*", GcsUtil.globToRegexp("foo-[0-9]*"));
-  }
-
-  private static GcsOptions gcsOptionsWithTestCredential() {
-    GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class);
-    pipelineOptions.setGcpCredential(new TestCredential());
-    return pipelineOptions;
-  }
-
-  @Test
-  public void testCreationWithDefaultOptions() {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    assertNotNull(pipelineOptions.getGcpCredential());
-  }
-
-  @Test
-  public void testUploadBufferSizeDefault() {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil util = pipelineOptions.getGcsUtil();
-    assertNull(util.getUploadBufferSizeBytes());
-  }
-
-  @Test
-  public void testUploadBufferSizeUserSpecified() {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    pipelineOptions.setGcsUploadBufferSizeBytes(12345);
-    GcsUtil util = pipelineOptions.getGcsUtil();
-    assertEquals((Integer) 12345, util.getUploadBufferSizeBytes());
-  }
-
-  @Test
-  public void testCreationWithExecutorServiceProvided() {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    pipelineOptions.setExecutorService(Executors.newCachedThreadPool());
-    assertSame(pipelineOptions.getExecutorService(), pipelineOptions.getGcsUtil().executorService);
-  }
-
-  @Test
-  public void testCreationWithGcsUtilProvided() {
-    GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class);
-    GcsUtil gcsUtil = Mockito.mock(GcsUtil.class);
-    pipelineOptions.setGcsUtil(gcsUtil);
-    assertSame(gcsUtil, pipelineOptions.getGcsUtil());
-  }
-
-  @Test
-  public void testMultipleThreadsCanCompleteOutOfOrderWithDefaultThreadPool() throws Exception {
-    GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class);
-    ExecutorService executorService = pipelineOptions.getExecutorService();
-
-    int numThreads = 100;
-    final CountDownLatch[] countDownLatches = new CountDownLatch[numThreads];
-    for (int i = 0; i < numThreads; i++) {
-      final int currentLatch = i;
-      countDownLatches[i] = new CountDownLatch(1);
-      executorService.execute(
-          new Runnable() {
-            @Override
-            public void run() {
-              // Wait for latch N and then release latch N - 1
-              try {
-                countDownLatches[currentLatch].await();
-                if (currentLatch > 0) {
-                  countDownLatches[currentLatch - 1].countDown();
-                }
-              } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    }
-
-    // Release the last latch starting the chain reaction.
-    countDownLatches[countDownLatches.length - 1].countDown();
-    executorService.shutdown();
-    assertTrue("Expected tasks to complete",
-        executorService.awaitTermination(10, TimeUnit.SECONDS));
-  }
-
-  @Test
-  public void testGlobExpansion() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
-    Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
-    Storage.Objects.List mockStorageList = Mockito.mock(Storage.Objects.List.class);
-
-    Objects modelObjects = new Objects();
-    List<StorageObject> items = new ArrayList<>();
-    // A directory
-    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/"));
-
-    // Files within the directory
-    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/file1name"));
-    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/file2name"));
-    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/file3name"));
-    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/otherfile"));
-    items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/anotherfile"));
-
-    modelObjects.setItems(items);
-
-    when(mockStorage.objects()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.get("testbucket", "testdirectory/otherfile")).thenReturn(
-        mockStorageGet);
-    when(mockStorageObjects.list("testbucket")).thenReturn(mockStorageList);
-    when(mockStorageGet.execute()).thenReturn(
-        new StorageObject().setBucket("testbucket").setName("testdirectory/otherfile"));
-    when(mockStorageList.execute()).thenReturn(modelObjects);
-
-    // Test a single file.
-    {
-      GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/otherfile");
-      List<GcsPath> expectedFiles =
-          ImmutableList.of(GcsPath.fromUri("gs://testbucket/testdirectory/otherfile"));
-
-      assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray()));
-    }
-
-    // Test patterns.
-    {
-      GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/file*");
-      List<GcsPath> expectedFiles = ImmutableList.of(
-          GcsPath.fromUri("gs://testbucket/testdirectory/file1name"),
-          GcsPath.fromUri("gs://testbucket/testdirectory/file2name"),
-          GcsPath.fromUri("gs://testbucket/testdirectory/file3name"));
-
-      assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray()));
-    }
-
-    {
-      GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/file[1-3]*");
-      List<GcsPath> expectedFiles = ImmutableList.of(
-          GcsPath.fromUri("gs://testbucket/testdirectory/file1name"),
-          GcsPath.fromUri("gs://testbucket/testdirectory/file2name"),
-          GcsPath.fromUri("gs://testbucket/testdirectory/file3name"));
-
-      assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray()));
-    }
-
-    {
-      GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/file?name");
-      List<GcsPath> expectedFiles = ImmutableList.of(
-          GcsPath.fromUri("gs://testbucket/testdirectory/file1name"),
-          GcsPath.fromUri("gs://testbucket/testdirectory/file2name"),
-          GcsPath.fromUri("gs://testbucket/testdirectory/file3name"));
-
-      assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray()));
-    }
-
-    {
-      GcsPath pattern = GcsPath.fromUri("gs://testbucket/test*ectory/fi*name");
-      List<GcsPath> expectedFiles = ImmutableList.of(
-          GcsPath.fromUri("gs://testbucket/testdirectory/file1name"),
-          GcsPath.fromUri("gs://testbucket/testdirectory/file2name"),
-          GcsPath.fromUri("gs://testbucket/testdirectory/file3name"));
-
-      assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray()));
-    }
-  }
-
-  // Patterns that contain recursive wildcards ('**') are not supported.
-  @Test
-  public void testRecursiveGlobExpansionFails() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-    GcsPath pattern = GcsPath.fromUri("gs://testbucket/test**");
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Unsupported wildcard usage");
-    gcsUtil.expand(pattern);
-  }
-
-  // GCSUtil.expand() should fail when matching a single object when that object does not exist.
-  // We should return the empty result since GCS get object is strongly consistent.
-  @Test
-  public void testNonExistentObjectReturnsEmptyResult() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
-    Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
-
-    GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/nonexistentfile");
-    GoogleJsonResponseException expectedException =
-        googleJsonResponseException(HttpStatusCodes.STATUS_CODE_NOT_FOUND,
-            "It don't exist", "Nothing here to see");
-
-    when(mockStorage.objects()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.get(pattern.getBucket(), pattern.getObject())).thenReturn(
-        mockStorageGet);
-    when(mockStorageGet.execute()).thenThrow(expectedException);
-
-    assertEquals(Collections.EMPTY_LIST, gcsUtil.expand(pattern));
-  }
-
-  // GCSUtil.expand() should fail for other errors such as access denied.
-  @Test
-  public void testAccessDeniedObjectThrowsIOException() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
-    Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
-
-    GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/accessdeniedfile");
-    GoogleJsonResponseException expectedException =
-        googleJsonResponseException(HttpStatusCodes.STATUS_CODE_FORBIDDEN,
-            "Waves hand mysteriously", "These aren't the buckets you're looking for");
-
-    when(mockStorage.objects()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.get(pattern.getBucket(), pattern.getObject())).thenReturn(
-        mockStorageGet);
-    when(mockStorageGet.execute()).thenThrow(expectedException);
-
-    thrown.expect(IOException.class);
-    thrown.expectMessage("Unable to get the file object for path");
-    gcsUtil.expand(pattern);
-  }
-
-  @Test
-  public void testFileSizeNonBatch() throws Exception {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
-    Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
-
-    when(mockStorage.objects()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet);
-    when(mockStorageGet.execute()).thenReturn(
-            new StorageObject().setSize(BigInteger.valueOf(1000)));
-
-    assertEquals(1000, gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject")));
-  }
-
-  @Test
-  public void testFileSizeWhenFileNotFoundNonBatch() throws Exception {
-    MockLowLevelHttpResponse notFoundResponse = new MockLowLevelHttpResponse();
-    notFoundResponse.setContent("");
-    notFoundResponse.setStatusCode(HttpStatusCodes.STATUS_CODE_NOT_FOUND);
-
-    MockHttpTransport mockTransport =
-            new MockHttpTransport.Builder().setLowLevelHttpResponse(notFoundResponse).build();
-
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    gcsUtil.setStorageClient(new Storage(mockTransport, Transport.getJsonFactory(), null));
-
-    thrown.expect(FileNotFoundException.class);
-    gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject"));
-  }
-
-  @Test
-  public void testRetryFileSizeNonBatch() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
-    Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
-
-    BackOff mockBackOff = FluentBackoff.DEFAULT.withMaxRetries(2).backoff();
-
-    when(mockStorage.objects()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet);
-    when(mockStorageGet.execute())
-            .thenThrow(new SocketTimeoutException("SocketException"))
-            .thenThrow(new SocketTimeoutException("SocketException"))
-            .thenReturn(new StorageObject().setSize(BigInteger.valueOf(1000)));
-
-    assertEquals(1000,
-        gcsUtil.getObject(
-            GcsPath.fromComponents("testbucket", "testobject"),
-            mockBackOff,
-            new FastNanoClockAndSleeper()).getSize().longValue());
-    assertEquals(BackOff.STOP, mockBackOff.nextBackOffMillis());
-  }
-
-  @Test
-  public void testGetSizeBytesWhenFileNotFoundBatch() throws Exception {
-    JsonFactory jsonFactory = new JacksonFactory();
-
-    String contentBoundary = "batch_foobarbaz";
-    String contentBoundaryLine = "--" + contentBoundary;
-    String endOfContentBoundaryLine = "--" + contentBoundary + "--";
-
-    GenericJson error = new GenericJson()
-        .set("error", new GenericJson().set("code", 404));
-    error.setFactory(jsonFactory);
-
-    String content = contentBoundaryLine + "\n"
-        + "Content-Type: application/http\n"
-        + "\n"
-        + "HTTP/1.1 404 Not Found\n"
-        + "Content-Length: -1\n"
-        + "\n"
-        + error.toString()
-        + "\n"
-        + "\n"
-        + endOfContentBoundaryLine
-        + "\n";
-    thrown.expect(FileNotFoundException.class);
-    MockLowLevelHttpResponse notFoundResponse = new MockLowLevelHttpResponse()
-        .setContentType("multipart/mixed; boundary=" + contentBoundary)
-        .setContent(content)
-        .setStatusCode(HttpStatusCodes.STATUS_CODE_OK);
-
-    MockHttpTransport mockTransport =
-        new MockHttpTransport.Builder().setLowLevelHttpResponse(notFoundResponse).build();
-
-    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
-
-    gcsUtil.setStorageClient(new Storage(mockTransport, Transport.getJsonFactory(), null));
-    gcsUtil.fileSizes(ImmutableList.of(GcsPath.fromComponents("testbucket", "testobject")));
-  }
-
-  @Test
-  public void testGetSizeBytesWhenFileNotFoundBatchRetry() throws Exception {
-    JsonFactory jsonFactory = new JacksonFactory();
-
-    String contentBoundary = "batch_foobarbaz";
-    String contentBoundaryLine = "--" + contentBoundary;
-    String endOfContentBoundaryLine = "--" + contentBoundary + "--";
-
-    GenericJson error = new GenericJson()
-        .set("error", new GenericJson().set("code", 404));
-    error.setFactory(jsonFactory);
-
-    String content = contentBoundaryLine + "\n"
-        + "Content-Type: application/http\n"
-        + "\n"
-        + "HTTP/1.1 404 Not Found\n"
-        + "Content-Length: -1\n"
-        + "\n"
-        + error.toString()
-        + "\n"
-        + "\n"
-        + endOfContentBoundaryLine
-        + "\n";
-    thrown.expect(FileNotFoundException.class);
-
-    final LowLevelHttpResponse mockResponse = Mockito.mock(LowLevelHttpResponse.class);
-    when(mockResponse.getContentType()).thenReturn("multipart/mixed; boundary=" + contentBoundary);
-
-    // 429: Too many requests, then 200: OK.
-    when(mockResponse.getStatusCode()).thenReturn(429, 200);
-    when(mockResponse.getContent()).thenReturn(toStream("error"), toStream(content));
-
-    // A mock transport that lets us mock the API responses.
-    MockHttpTransport mockTransport =
-        new MockHttpTransport.Builder()
-            .setLowLevelHttpRequest(
-                new MockLowLevelHttpRequest() {
-                  @Override
-                  public LowLevelHttpResponse execute() throws IOException {
-                    return mockResponse;
-                  }
-                })
-            .build();
-
-    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
-
-        gcsUtil.setStorageClient(
-        new Storage(mockTransport, Transport.getJsonFactory(), new RetryHttpRequestInitializer()));
-    gcsUtil.fileSizes(ImmutableList.of(GcsPath.fromComponents("testbucket", "testobject")));
-  }
-
-  @Test
-  public void testCreateBucket() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Buckets.Insert mockStorageInsert = Mockito.mock(Storage.Buckets.Insert.class);
-
-    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
-
-    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.insert(
-           any(String.class), any(Bucket.class))).thenReturn(mockStorageInsert);
-    when(mockStorageInsert.execute())
-        .thenThrow(new SocketTimeoutException("SocketException"))
-        .thenReturn(new Bucket());
-
-    gcsUtil.createBucket("a", new Bucket(), mockBackOff, new FastNanoClockAndSleeper());
-  }
-
-  @Test
-  public void testCreateBucketAccessErrors() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
-    Storage.Buckets.Insert mockStorageInsert = Mockito.mock(Storage.Buckets.Insert.class);
-
-    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
-    GoogleJsonResponseException expectedException =
-        googleJsonResponseException(HttpStatusCodes.STATUS_CODE_FORBIDDEN,
-            "Waves hand mysteriously", "These aren't the buckets you're looking for");
-
-    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.insert(
-           any(String.class), any(Bucket.class))).thenReturn(mockStorageInsert);
-    when(mockStorageInsert.execute())
-        .thenThrow(expectedException);
-
-    thrown.expect(AccessDeniedException.class);
-
-    gcsUtil.createBucket("a", new Bucket(), mockBackOff, new FastNanoClockAndSleeper());
-  }
-
-  @Test
-  public void testBucketAccessible() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
-    Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
-
-    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
-
-    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
-    when(mockStorageGet.execute())
-        .thenThrow(new SocketTimeoutException("SocketException"))
-        .thenReturn(new Bucket());
-
-    assertTrue(gcsUtil.bucketAccessible(GcsPath.fromComponents("testbucket", "testobject"),
-        mockBackOff, new FastNanoClockAndSleeper()));
-  }
-
-  @Test
-  public void testBucketDoesNotExistBecauseOfAccessError() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
-    Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
-
-    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
-    GoogleJsonResponseException expectedException =
-        googleJsonResponseException(HttpStatusCodes.STATUS_CODE_FORBIDDEN,
-            "Waves hand mysteriously", "These aren't the buckets you're looking for");
-
-    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
-    when(mockStorageGet.execute())
-        .thenThrow(expectedException);
-
-    assertFalse(gcsUtil.bucketAccessible(GcsPath.fromComponents("testbucket", "testobject"),
-        mockBackOff, new FastNanoClockAndSleeper()));
-  }
-
-  @Test
-  public void testBucketDoesNotExist() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
-    Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
-
-    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
-
-    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
-    when(mockStorageGet.execute())
-        .thenThrow(googleJsonResponseException(HttpStatusCodes.STATUS_CODE_NOT_FOUND,
-            "It don't exist", "Nothing here to see"));
-
-    assertFalse(gcsUtil.bucketAccessible(GcsPath.fromComponents("testbucket", "testobject"),
-        mockBackOff, new FastNanoClockAndSleeper()));
-  }
-
-  @Test
-  public void testGetBucket() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
-    Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
-
-    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
-
-    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
-    when(mockStorageGet.execute())
-        .thenThrow(new SocketTimeoutException("SocketException"))
-        .thenReturn(new Bucket());
-
-    assertNotNull(gcsUtil.getBucket(GcsPath.fromComponents("testbucket", "testobject"),
-        mockBackOff, new FastNanoClockAndSleeper()));
-  }
-
-  @Test
-  public void testGetBucketNotExists() throws IOException {
-    GcsOptions pipelineOptions = gcsOptionsWithTestCredential();
-    GcsUtil gcsUtil = pipelineOptions.getGcsUtil();
-
-    Storage mockStorage = Mockito.mock(Storage.class);
-    gcsUtil.setStorageClient(mockStorage);
-
-    Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
-    Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
-
-    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
-
-    when(mockStorage.buckets()).thenReturn(mockStorageObjects);
-    when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
-    when(mockStorageGet.execute())
-        .thenThrow(googleJsonResponseException(HttpStatusCodes.STATUS_CODE_NOT_FOUND,
-            "It don't exist", "Nothing here to see"));
-
-    thrown.expect(FileNotFoundException.class);
-    thrown.expectMessage("It don't exist");
-    gcsUtil.getBucket(GcsPath.fromComponents("testbucket", "testobject"),
-                      mockBackOff, new FastNanoClockAndSleeper());
-  }
-
-  @Test
-  public void testGCSChannelCloseIdempotent() throws IOException {
-    SeekableByteChannel channel =
-        new GoogleCloudStorageReadChannel(null, "dummybucket", "dummyobject", null,
-        new ClientRequestHelper<StorageObject>());
-    channel.close();
-    channel.close();
-  }
-
-  /**
-   * Builds a fake GoogleJsonResponseException for testing API error handling.
-   */
-  private static GoogleJsonResponseException googleJsonResponseException(
-      final int status, final String reason, final String message) throws IOException {
-    final JsonFactory jsonFactory = new JacksonFactory();
-    HttpTransport transport = new MockHttpTransport() {
-      @Override
-      public LowLevelHttpRequest buildRequest(String method, String url) throws IOException {
-        ErrorInfo errorInfo = new ErrorInfo();
-        errorInfo.setReason(reason);
-        errorInfo.setMessage(message);
-        errorInfo.setFactory(jsonFactory);
-        GenericJson error = new GenericJson();
-        error.set("code", status);
-        error.set("errors", Arrays.asList(errorInfo));
-        error.setFactory(jsonFactory);
-        GenericJson errorResponse = new GenericJson();
-        errorResponse.set("error", error);
-        errorResponse.setFactory(jsonFactory);
-        return new MockLowLevelHttpRequest().setResponse(
-            new MockLowLevelHttpResponse().setContent(errorResponse.toPrettyString())
-            .setContentType(Json.MEDIA_TYPE).setStatusCode(status));
-        }
-    };
-    HttpRequest request =
-        transport.createRequestFactory().buildGetRequest(HttpTesting.SIMPLE_GENERIC_URL);
-    request.setThrowExceptionOnExecuteError(false);
-    HttpResponse response = request.execute();
-    return GoogleJsonResponseException.from(jsonFactory, response);
-  }
-
-  private static List<String> makeStrings(String s, int n) {
-    ImmutableList.Builder<String> ret = ImmutableList.builder();
-    for (int i = 0; i < n; ++i) {
-      ret.add(String.format("gs://bucket/%s%d", s, i));
-    }
-    return ret.build();
-  }
-
-  private static List<GcsPath> makeGcsPaths(String s, int n) {
-    ImmutableList.Builder<GcsPath> ret = ImmutableList.builder();
-    for (int i = 0; i < n; ++i) {
-      ret.add(GcsPath.fromUri(String.format("gs://bucket/%s%d", s, i)));
-    }
-    return ret.build();
-  }
-
-  private static int sumBatchSizes(List<BatchRequest> batches) {
-    int ret = 0;
-    for (BatchRequest b : batches) {
-      ret += b.size();
-      assertThat(b.size(), greaterThan(0));
-    }
-    return ret;
-  }
-
-  @Test
-  public void testMakeCopyBatches() throws IOException {
-    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
-
-    // Small number of files fits in 1 batch
-    List<BatchRequest> batches = gcsUtil.makeCopyBatches(makeStrings("s", 3), makeStrings("d", 3));
-    assertThat(batches.size(), equalTo(1));
-    assertThat(sumBatchSizes(batches), equalTo(3));
-
-    // 1 batch of files fits in 1 batch
-    batches = gcsUtil.makeCopyBatches(makeStrings("s", 100), makeStrings("d", 100));
-    assertThat(batches.size(), equalTo(1));
-    assertThat(sumBatchSizes(batches), equalTo(100));
-
-    // A little more than 5 batches of files fits in 6 batches
-    batches = gcsUtil.makeCopyBatches(makeStrings("s", 501), makeStrings("d", 501));
-    assertThat(batches.size(), equalTo(6));
-    assertThat(sumBatchSizes(batches), equalTo(501));
-  }
-
-  @Test
-  public void testInvalidCopyBatches() throws IOException {
-    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Number of source files 3");
-
-    gcsUtil.makeCopyBatches(makeStrings("s", 3), makeStrings("d", 1));
-  }
-
-  @Test
-  public void testMakeRemoveBatches() throws IOException {
-    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
-
-    // Small number of files fits in 1 batch
-    List<BatchRequest> batches = gcsUtil.makeRemoveBatches(makeStrings("s", 3));
-    assertThat(batches.size(), equalTo(1));
-    assertThat(sumBatchSizes(batches), equalTo(3));
-
-    // 1 batch of files fits in 1 batch
-    batches = gcsUtil.makeRemoveBatches(makeStrings("s", 100));
-    assertThat(batches.size(), equalTo(1));
-    assertThat(sumBatchSizes(batches), equalTo(100));
-
-    // A little more than 5 batches of files fits in 6 batches
-    batches = gcsUtil.makeRemoveBatches(makeStrings("s", 501));
-    assertThat(batches.size(), equalTo(6));
-    assertThat(sumBatchSizes(batches), equalTo(501));
-  }
-
-  @Test
-  public void testMakeGetBatches() throws IOException {
-    GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil();
-
-    // Small number of files fits in 1 batch
-    List<StorageObjectOrIOException[]> results = Lists.newArrayList();
-    List<BatchRequest> batches = gcsUtil.makeGetBatches(makeGcsPaths("s", 3), results);
-    assertThat(batches.size(), equalTo(1));
-    assertThat(sumBatchSizes(batches), equalTo(3));
-    assertEquals(3, results.size());
-
-    // 1 batch of files fits in 1 batch
-    results = Lists.newArrayList();
-    batches = gcsUtil.makeGetBatches(makeGcsPaths("s", 100), results);
-    assertThat(batches.size(), equalTo(1));
-    assertThat(sumBatchSizes(batches), equalTo(100));
-    assertEquals(100, results.size());
-
-    // A little more than 5 batches of files fits in 6 batches
-    results = Lists.newArrayList();
-    batches = gcsUtil.makeGetBatches(makeGcsPaths("s", 501), results);
-    assertThat(batches.size(), equalTo(6));
-    assertThat(sumBatchSizes(batches), equalTo(501));
-    assertEquals(501, results.size());
-  }
-
-  /**
-   * A helper to wrap a {@link GenericJson} object in a content stream.
-   */
-  private static InputStream toStream(String content) throws IOException {
-    return new ByteArrayInputStream(content.getBytes(StandardCharsets.UTF_8));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java
deleted file mode 100644
index 8e7878c..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java
+++ /dev/null
@@ -1,100 +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.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.allOf;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
-import static org.hamcrest.Matchers.lessThan;
-import static org.hamcrest.Matchers.lessThanOrEqualTo;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Unit tests for {@link IntervalBoundedExponentialBackOff}. */
-@RunWith(JUnit4.class)
-public class IntervalBoundedExponentialBackOffTest {
-  @Rule public ExpectedException exception = ExpectedException.none();
-
-
-  @Test
-  public void testUsingInvalidInitialInterval() throws Exception {
-    exception.expect(IllegalArgumentException.class);
-    exception.expectMessage("Initial interval must be greater than zero.");
-    new IntervalBoundedExponentialBackOff(1000L, 0L);
-  }
-
-  @Test
-  public void testUsingInvalidMaximumInterval() throws Exception {
-    exception.expect(IllegalArgumentException.class);
-    exception.expectMessage("Maximum interval must be greater than zero.");
-    new IntervalBoundedExponentialBackOff(-1L, 10L);
-  }
-
-  @Test
-  public void testThatcertainNumberOfAttemptsReachesMaxInterval() throws Exception {
-    IntervalBoundedExponentialBackOff backOff = new IntervalBoundedExponentialBackOff(1000L, 500);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
-        lessThanOrEqualTo(1500L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
-        lessThanOrEqualTo(1500L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
-        lessThanOrEqualTo(1500L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
-        lessThanOrEqualTo(1500L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
-        lessThanOrEqualTo(1500L)));
-  }
-
-  @Test
-  public void testThatResettingAllowsReuse() throws Exception {
-    IntervalBoundedExponentialBackOff backOff = new IntervalBoundedExponentialBackOff(1000L, 500);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
-        lessThanOrEqualTo(1500L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
-        lessThanOrEqualTo(1500L)));
-    backOff.reset();
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
-        lessThanOrEqualTo(1500L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
-        lessThanOrEqualTo(1500L)));
-  }
-
-  @Test
-  public void testAtMaxInterval() throws Exception {
-    IntervalBoundedExponentialBackOff backOff = new IntervalBoundedExponentialBackOff(1000L, 500);
-    assertFalse(backOff.atMaxInterval());
-    backOff.nextBackOffMillis();
-    assertFalse(backOff.atMaxInterval());
-    backOff.nextBackOffMillis();
-    assertTrue(backOff.atMaxInterval());
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
-        lessThanOrEqualTo(1500L)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/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
deleted file mode 100644
index 71554b5..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
+++ /dev/null
@@ -1,290 +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.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyString;
-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 com.google.api.client.auth.oauth2.Credential;
-import com.google.api.client.http.HttpRequest;
-import com.google.api.client.http.HttpResponse;
-import com.google.api.client.http.HttpResponseException;
-import com.google.api.client.http.HttpResponseInterceptor;
-import com.google.api.client.http.HttpTransport;
-import com.google.api.client.http.LowLevelHttpRequest;
-import com.google.api.client.http.LowLevelHttpResponse;
-import com.google.api.client.json.JsonFactory;
-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.NanoClock;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.storage.Storage;
-import com.google.api.services.storage.Storage.Objects.Get;
-import java.io.IOException;
-import java.net.SocketTimeoutException;
-import java.security.PrivateKey;
-import java.util.Arrays;
-import java.util.concurrent.atomic.AtomicLong;
-import org.hamcrest.Matchers;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-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;
-
-/**
- * Tests for RetryHttpRequestInitializer.
- */
-@RunWith(JUnit4.class)
-public class RetryHttpRequestInitializerTest {
-
-  @Mock private Credential mockCredential;
-  @Mock private PrivateKey mockPrivateKey;
-  @Mock private LowLevelHttpRequest mockLowLevelRequest;
-  @Mock private LowLevelHttpResponse mockLowLevelResponse;
-  @Mock private HttpResponseInterceptor mockHttpResponseInterceptor;
-
-  private final JsonFactory jsonFactory = JacksonFactory.getDefaultInstance();
-  private Storage storage;
-
-  // Used to test retrying a request more than the default 10 times.
-  static class MockNanoClock implements NanoClock {
-    private int timesMs[] = {500, 750, 1125, 1688, 2531, 3797, 5695, 8543,
-        12814, 19222, 28833, 43249, 64873, 97310, 145965, 218945, 328420};
-    private int i = 0;
-
-    @Override
-    public long nanoTime() {
-      return timesMs[i++ / 2] * 1000000;
-    }
-  }
-
-  @Before
-  public void setUp() {
-    MockitoAnnotations.initMocks(this);
-
-    HttpTransport lowLevelTransport = new HttpTransport() {
-      @Override
-      protected LowLevelHttpRequest buildRequest(String method, String url)
-          throws IOException {
-        return mockLowLevelRequest;
-      }
-    };
-
-    // Retry initializer will pass through to credential, since we can have
-    // only a single HttpRequestInitializer, and we use multiple Credential
-    // types in the SDK, not all of which allow for retry configuration.
-    RetryHttpRequestInitializer initializer = new RetryHttpRequestInitializer(
-        mockCredential, new MockNanoClock(), new Sleeper() {
-          @Override
-          public void sleep(long millis) throws InterruptedException {}
-        }, Arrays.asList(418 /* I'm a teapot */), mockHttpResponseInterceptor);
-    storage = new Storage.Builder(lowLevelTransport, jsonFactory, initializer)
-        .setApplicationName("test").build();
-  }
-
-  @After
-  public void tearDown() {
-    verifyNoMoreInteractions(mockPrivateKey);
-    verifyNoMoreInteractions(mockLowLevelRequest);
-    verifyNoMoreInteractions(mockCredential);
-    verifyNoMoreInteractions(mockHttpResponseInterceptor);
-  }
-
-  @Test
-  public void testBasicOperation() throws IOException {
-    when(mockLowLevelRequest.execute())
-        .thenReturn(mockLowLevelResponse);
-    when(mockLowLevelResponse.getStatusCode())
-        .thenReturn(200);
-
-    Storage.Buckets.Get result = storage.buckets().get("test");
-    HttpResponse response = result.executeUnparsed();
-    assertNotNull(response);
-
-    verify(mockCredential).initialize(any(HttpRequest.class));
-    verify(mockHttpResponseInterceptor).interceptResponse(any(HttpResponse.class));
-    verify(mockLowLevelRequest, atLeastOnce())
-        .addHeader(anyString(), anyString());
-    verify(mockLowLevelRequest).setTimeout(anyInt(), anyInt());
-    verify(mockLowLevelRequest).execute();
-    verify(mockLowLevelResponse).getStatusCode();
-  }
-
-  /**
-   * Tests that a non-retriable error is not retried.
-   */
-  @Test
-  public void testErrorCodeForbidden() throws IOException {
-    when(mockLowLevelRequest.execute())
-        .thenReturn(mockLowLevelResponse);
-    when(mockLowLevelResponse.getStatusCode())
-        .thenReturn(403)  // Non-retryable error.
-        .thenReturn(200); // Shouldn't happen.
-
-    try {
-      Storage.Buckets.Get result = storage.buckets().get("test");
-      HttpResponse response = result.executeUnparsed();
-      assertNotNull(response);
-    } catch (HttpResponseException e) {
-      Assert.assertThat(e.getMessage(), Matchers.containsString("403"));
-    }
-
-    verify(mockCredential).initialize(any(HttpRequest.class));
-    verify(mockHttpResponseInterceptor).interceptResponse(any(HttpResponse.class));
-    verify(mockLowLevelRequest, atLeastOnce())
-        .addHeader(anyString(), anyString());
-    verify(mockLowLevelRequest).setTimeout(anyInt(), anyInt());
-    verify(mockLowLevelRequest).execute();
-    verify(mockLowLevelResponse).getStatusCode();
-  }
-
-  /**
-   * Tests that a retriable error is retried.
-   */
-  @Test
-  public void testRetryableError() throws IOException {
-    when(mockLowLevelRequest.execute())
-        .thenReturn(mockLowLevelResponse)
-        .thenReturn(mockLowLevelResponse)
-        .thenReturn(mockLowLevelResponse);
-    when(mockLowLevelResponse.getStatusCode())
-        .thenReturn(503)  // Retryable
-        .thenReturn(429)  // We also retry on 429 Too Many Requests.
-        .thenReturn(200);
-
-    Storage.Buckets.Get result = storage.buckets().get("test");
-    HttpResponse response = result.executeUnparsed();
-    assertNotNull(response);
-
-    verify(mockCredential).initialize(any(HttpRequest.class));
-    verify(mockHttpResponseInterceptor).interceptResponse(any(HttpResponse.class));
-    verify(mockLowLevelRequest, atLeastOnce())
-        .addHeader(anyString(), anyString());
-    verify(mockLowLevelRequest, times(3)).setTimeout(anyInt(), anyInt());
-    verify(mockLowLevelRequest, times(3)).execute();
-    verify(mockLowLevelResponse, times(3)).getStatusCode();
-  }
-
-  /**
-   * Tests that an IOException is retried.
-   */
-  @Test
-  public void testThrowIOException() throws IOException {
-    when(mockLowLevelRequest.execute())
-        .thenThrow(new IOException("Fake Error"))
-        .thenReturn(mockLowLevelResponse);
-    when(mockLowLevelResponse.getStatusCode())
-        .thenReturn(200);
-
-    Storage.Buckets.Get result = storage.buckets().get("test");
-    HttpResponse response = result.executeUnparsed();
-    assertNotNull(response);
-
-    verify(mockCredential).initialize(any(HttpRequest.class));
-    verify(mockHttpResponseInterceptor).interceptResponse(any(HttpResponse.class));
-    verify(mockLowLevelRequest, atLeastOnce())
-        .addHeader(anyString(), anyString());
-    verify(mockLowLevelRequest, times(2)).setTimeout(anyInt(), anyInt());
-    verify(mockLowLevelRequest, times(2)).execute();
-    verify(mockLowLevelResponse).getStatusCode();
-  }
-
-  /**
-   * Tests that a retryable error is retried enough times.
-   */
-  @Test
-  public void testRetryableErrorRetryEnoughTimes() throws IOException {
-    when(mockLowLevelRequest.execute()).thenReturn(mockLowLevelResponse);
-    final int retries = 10;
-    when(mockLowLevelResponse.getStatusCode()).thenAnswer(new Answer<Integer>(){
-      int n = 0;
-      @Override
-      public Integer answer(InvocationOnMock invocation) {
-        return (n++ < retries - 1) ? 503 : 200;
-      }});
-
-    Storage.Buckets.Get result = storage.buckets().get("test");
-    HttpResponse response = result.executeUnparsed();
-    assertNotNull(response);
-
-    verify(mockCredential).initialize(any(HttpRequest.class));
-    verify(mockHttpResponseInterceptor).interceptResponse(any(HttpResponse.class));
-    verify(mockLowLevelRequest, atLeastOnce()).addHeader(anyString(),
-        anyString());
-    verify(mockLowLevelRequest, times(retries)).setTimeout(anyInt(), anyInt());
-    verify(mockLowLevelRequest, times(retries)).execute();
-    verify(mockLowLevelResponse, times(retries)).getStatusCode();
-  }
-
-  /**
-   * Tests that when RPCs fail with {@link SocketTimeoutException}, the IO exception handler
-   * is invoked.
-   */
-  @Test
-  public void testIOExceptionHandlerIsInvokedOnTimeout() throws Exception {
-    // Counts the number of calls to execute the HTTP request.
-    final AtomicLong executeCount = new AtomicLong();
-
-    // 10 is a private internal constant in the Google API Client library. See
-    // com.google.api.client.http.HttpRequest#setNumberOfRetries
-    // TODO: update this test once the private internal constant is public.
-    final int defaultNumberOfRetries = 10;
-
-    // A mock HTTP request that always throws SocketTimeoutException.
-    MockHttpTransport transport =
-        new MockHttpTransport.Builder().setLowLevelHttpRequest(new MockLowLevelHttpRequest() {
-          @Override
-          public LowLevelHttpResponse execute() throws IOException {
-            executeCount.incrementAndGet();
-            throw new SocketTimeoutException("Fake forced timeout exception");
-          }
-        }).build();
-
-    // A sample HTTP request to Google Cloud Storage that uses both default Transport and default
-    // RetryHttpInitializer.
-    Storage storage = new Storage.Builder(
-        transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer()).build();
-
-    Get getRequest = storage.objects().get("gs://fake", "file");
-
-    try {
-      getRequest.execute();
-      fail();
-    } catch (Throwable e) {
-      assertThat(e, Matchers.<Throwable>instanceOf(SocketTimeoutException.class));
-      assertEquals(1 + defaultNumberOfRetries, executeCount.get());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/pom.xml b/sdks/java/extensions/gcp-core/pom.xml
new file mode 100644
index 0000000..d566f94
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/pom.xml
@@ -0,0 +1,217 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-java-extensions-parent</artifactId>
+    <version>0.7.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
+  <name>Apache Beam :: SDKs :: Java :: Extensions :: Google Cloud Platform Core</name>
+  <description>Common components used to support multiple
+  Google Cloud Platform specific maven modules.</description>
+
+  <packaging>jar</packaging>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <excludedGroups>
+            org.apache.beam.sdk.testing.NeedsRunner
+          </excludedGroups>
+          <systemPropertyVariables>
+            <beamUseDummyRunner>true</beamUseDummyRunner>
+          </systemPropertyVariables>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+      </plugin>
+
+      <!-- Coverage analysis for unit tests. -->
+      <plugin>
+        <groupId>org.jacoco</groupId>
+        <artifactId>jacoco-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</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>
+
+    <dependency>
+      <groupId>com.google.auth</groupId>
+      <artifactId>google-auth-library-oauth2-http</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.api-client</groupId>
+      <artifactId>google-api-client</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>gcsio</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>util</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-cloudresourcemanager</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-pubsub</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-bigquery</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-storage</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.auth</groupId>
+      <artifactId>google-auth-library-credentials</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+
+    <!-- build dependencies -->
+    <dependency>
+      <groupId>com.google.auto.service</groupId>
+      <artifactId>auto-service</artifactId>
+      <optional>true</optional>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.auto.value</groupId>
+      <artifactId>auto-value</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <!-- test dependencies -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java
new file mode 100644
index 0000000..7672cd7
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java
@@ -0,0 +1,32 @@
+/*
+ * 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.options;
+
+/**
+ * Properties needed when using Google BigQuery with the Apache Beam SDK.
+ */
+@Description("Options that are used to configure Google BigQuery. See "
+    + "https://cloud.google.com/bigquery/what-is-bigquery for details on BigQuery.")
+public interface BigQueryOptions extends ApplicationNameOptions, GcpOptions,
+    PipelineOptions, StreamingOptions {
+  @Description("Temporary dataset for BigQuery table operations. "
+      + "Supported values are \"bigquery.googleapis.com/{dataset}\"")
+  @Default.String("bigquery.googleapis.com/cloud_dataflow")
+  String getTempDatasetId();
+  void setTempDatasetId(String value);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/CloudResourceManagerOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/CloudResourceManagerOptions.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/CloudResourceManagerOptions.java
new file mode 100644
index 0000000..13fdaf3
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/CloudResourceManagerOptions.java
@@ -0,0 +1,40 @@
+/*
+ * 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.options;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.beam.sdk.util.GcpProjectUtil;
+
+/**
+ * Properties needed when using Google CloudResourceManager with the Apache Beam SDK.
+ */
+@Description("Options that are used to configure Google CloudResourceManager. See "
+    + "https://cloud.google.com/resource-manager/ for details on CloudResourceManager.")
+public interface CloudResourceManagerOptions extends ApplicationNameOptions, GcpOptions,
+    PipelineOptions, StreamingOptions {
+  /**
+   * The GcpProjectUtil instance that should be used to communicate with Google Cloud Storage.
+   */
+  @JsonIgnore
+  @Description("The GcpProjectUtil instance that should be used to communicate"
+               + " with Google Cloud Resource Manager.")
+  @Default.InstanceFactory(GcpProjectUtil.GcpProjectUtilFactory.class)
+  @Hidden
+  GcpProjectUtil getGcpProjectUtil();
+  void setGcpProjectUtil(GcpProjectUtil value);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
new file mode 100644
index 0000000..d01406f
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
@@ -0,0 +1,227 @@
+/*
+ * 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.options;
+
+import static com.google.common.base.Strings.isNullOrEmpty;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.auth.Credentials;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.io.Files;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.security.GeneralSecurityException;
+import java.util.Locale;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.util.CredentialFactory;
+import org.apache.beam.sdk.util.DefaultBucket;
+import org.apache.beam.sdk.util.GcpCredentialFactory;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.util.PathValidator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Options used to configure Google Cloud Platform specific options such as the project
+ * and credentials.
+ *
+ * <p>These options defer to the
+ * <a href="https://developers.google.com/accounts/docs/application-default-credentials">
+ * application default credentials</a> for authentication. See the
+ * <a href="https://github.com/google/google-auth-library-java">Google Auth Library</a> for
+ * alternative mechanisms for creating credentials.
+ */
+@Description("Options used to configure Google Cloud Platform project and credentials.")
+public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions {
+  /**
+   * Project id to use when launching jobs.
+   */
+  @Description("Project id. Required when using Google Cloud Platform services. "
+      + "See https://cloud.google.com/storage/docs/projects for further details.")
+  @Default.InstanceFactory(DefaultProjectFactory.class)
+  String getProject();
+  void setProject(String value);
+
+  /**
+   * GCP <a href="https://developers.google.com/compute/docs/zones"
+   * >availability zone</a> for operations.
+   *
+   * <p>Default is set on a per-service basis.
+   */
+  @Description("GCP availability zone for running GCP operations. "
+      + "Default is up to the individual service.")
+  String getZone();
+  void setZone(String value);
+
+  /**
+   * The class of the credential factory that should be created and used to create
+   * credentials. If gcpCredential has not been set explicitly, an instance of this class will
+   * be constructed and used as a credential factory.
+   */
+  @Description("The class of the credential factory that should be created and used to create "
+      + "credentials. If gcpCredential has not been set explicitly, an instance of this class will "
+      + "be constructed and used as a credential factory.")
+  @Default.Class(GcpCredentialFactory.class)
+  Class<? extends CredentialFactory> getCredentialFactoryClass();
+  void setCredentialFactoryClass(
+      Class<? extends CredentialFactory> credentialFactoryClass);
+
+  /**
+   * The credential instance that should be used to authenticate against GCP services.
+   * If no credential has been set explicitly, the default is to use the instance factory
+   * that constructs a credential based upon the currently set credentialFactoryClass.
+   */
+  @JsonIgnore
+  @Description("The credential instance that should be used to authenticate against GCP services. "
+      + "If no credential has been set explicitly, the default is to use the instance factory "
+      + "that constructs a credential based upon the currently set credentialFactoryClass.")
+  @Default.InstanceFactory(GcpUserCredentialsFactory.class)
+  Credentials getGcpCredential();
+  void setGcpCredential(Credentials value);
+
+  /**
+   * Attempts to infer the default project based upon the environment this application
+   * is executing within. Currently this only supports getting the default project from gcloud.
+   */
+  class DefaultProjectFactory implements DefaultValueFactory<String> {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultProjectFactory.class);
+
+    @Override
+    public String create(PipelineOptions options) {
+      try {
+        File configFile;
+        if (getEnvironment().containsKey("CLOUDSDK_CONFIG")) {
+          configFile = new File(getEnvironment().get("CLOUDSDK_CONFIG"), "properties");
+        } else if (isWindows() && getEnvironment().containsKey("APPDATA")) {
+          configFile = new File(getEnvironment().get("APPDATA"), "gcloud/properties");
+        } else {
+          // New versions of gcloud use this file
+          configFile = new File(
+              System.getProperty("user.home"),
+              ".config/gcloud/configurations/config_default");
+          if (!configFile.exists()) {
+            // Old versions of gcloud use this file
+            configFile = new File(System.getProperty("user.home"), ".config/gcloud/properties");
+          }
+        }
+        String section = null;
+        Pattern projectPattern = Pattern.compile("^project\\s*=\\s*(.*)$");
+        Pattern sectionPattern = Pattern.compile("^\\[(.*)\\]$");
+        for (String line : Files.readLines(configFile, StandardCharsets.UTF_8)) {
+          line = line.trim();
+          if (line.isEmpty() || line.startsWith(";")) {
+            continue;
+          }
+          Matcher matcher = sectionPattern.matcher(line);
+          if (matcher.matches()) {
+            section = matcher.group(1);
+          } else if (section == null || section.equals("core")) {
+            matcher = projectPattern.matcher(line);
+            if (matcher.matches()) {
+              String project = matcher.group(1).trim();
+              LOG.info("Inferred default GCP project '{}' from gcloud. If this is the incorrect "
+                  + "project, please cancel this Pipeline and specify the command-line "
+                  + "argument --project.", project);
+              return project;
+            }
+          }
+        }
+      } catch (IOException expected) {
+        LOG.debug("Failed to find default project.", expected);
+      }
+      // return null if can't determine
+      return null;
+    }
+
+    /**
+     * Returns true if running on the Windows OS.
+     */
+    private static boolean isWindows() {
+      return System.getProperty("os.name").toLowerCase(Locale.ENGLISH).contains("windows");
+    }
+
+    /**
+     * Used to mock out getting environment variables.
+     */
+    @VisibleForTesting
+    Map<String, String> getEnvironment() {
+        return System.getenv();
+    }
+  }
+
+  /**
+   * Attempts to load the GCP credentials. See
+   * {@link CredentialFactory#getCredential()} for more details.
+   */
+  class GcpUserCredentialsFactory implements DefaultValueFactory<Credentials> {
+    @Override
+    public Credentials create(PipelineOptions options) {
+      GcpOptions gcpOptions = options.as(GcpOptions.class);
+      try {
+        CredentialFactory factory = InstanceBuilder.ofType(CredentialFactory.class)
+            .fromClass(gcpOptions.getCredentialFactoryClass())
+            .fromFactoryMethod("fromOptions")
+            .withArg(PipelineOptions.class, options)
+            .build();
+        return factory.getCredential();
+      } catch (IOException | GeneralSecurityException e) {
+        throw new RuntimeException("Unable to obtain credential", e);
+      }
+    }
+  }
+
+  /**
+   * A GCS path for storing temporary files in GCP.
+   *
+   * <p>Its default to {@link PipelineOptions#getTempLocation}.
+   */
+  @Description("A GCS path for storing temporary files in GCP.")
+  @Default.InstanceFactory(GcpTempLocationFactory.class)
+  @Nullable String getGcpTempLocation();
+  void setGcpTempLocation(String value);
+
+  /**
+   * Returns {@link PipelineOptions#getTempLocation} as the default GCP temp location.
+   */
+  class GcpTempLocationFactory implements DefaultValueFactory<String> {
+
+    @Override
+    @Nullable
+    public String create(PipelineOptions options) {
+      String tempLocation = options.getTempLocation();
+      if (isNullOrEmpty(tempLocation)) {
+        tempLocation = DefaultBucket.tryCreateDefaultBucket(options);
+        options.setTempLocation(tempLocation);
+      } else {
+        try {
+          PathValidator validator = options.as(GcsOptions.class).getPathValidator();
+          validator.validateOutputFilePrefixSupported(tempLocation);
+        } catch (Exception e) {
+          throw new IllegalArgumentException(String.format(
+              "Error constructing default value for gcpTempLocation: tempLocation is not"
+              + " a valid GCS path, %s. ", tempLocation), e);
+        }
+      }
+      return tempLocation;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/be92f595/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcpPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcpPipelineOptionsRegistrar.java b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcpPipelineOptionsRegistrar.java
new file mode 100644
index 0000000..00be440
--- /dev/null
+++ b/sdks/java/extensions/gcp-core/src/main/java/org/apache/beam/sdk/options/GcpPipelineOptionsRegistrar.java
@@ -0,0 +1,39 @@
+/*
+ * 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.options;
+
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableList;
+
+/**
+ * A registrar containing the default GCP options.
+ */
+@AutoService(PipelineOptionsRegistrar.class)
+public class GcpPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
+  @Override
+  public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+    return ImmutableList.<Class<? extends PipelineOptions>>builder()
+        .add(BigQueryOptions.class)
+        .add(GcpOptions.class)
+        .add(GcsOptions.class)
+        .add(GoogleApiDebugOptions.class)
+        .add(PubsubOptions.class)
+        .build();
+  }
+}