You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by dh...@apache.org on 2017/04/12 15:16:58 UTC

[1/8] beam git commit: [BEAM-1722] Move PubsubIO into the google-cloud-platform module

Repository: beam
Updated Branches:
  refs/heads/master 82f2f2cff -> 7e603d5c7


http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java
new file mode 100644
index 0000000..d290994
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.api.services.pubsub.Pubsub;
+import com.google.api.services.pubsub.model.PublishRequest;
+import com.google.api.services.pubsub.model.PublishResponse;
+import com.google.api.services.pubsub.model.PubsubMessage;
+import com.google.api.services.pubsub.model.PullRequest;
+import com.google.api.services.pubsub.model.PullResponse;
+import com.google.api.services.pubsub.model.ReceivedMessage;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.IncomingMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mockito;
+
+/**
+ * Tests for PubsubJsonClient.
+ */
+@RunWith(JUnit4.class)
+public class PubsubJsonClientTest {
+  private Pubsub mockPubsub;
+  private PubsubClient client;
+
+  private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic");
+  private static final SubscriptionPath SUBSCRIPTION =
+      PubsubClient.subscriptionPathFromName("testProject", "testSubscription");
+  private static final long REQ_TIME = 1234L;
+  private static final long PUB_TIME = 3456L;
+  private static final long MESSAGE_TIME = 6789L;
+  private static final String TIMESTAMP_LABEL = "timestamp";
+  private static final String ID_LABEL = "id";
+  private static final String MESSAGE_ID = "testMessageId";
+  private static final String DATA = "testData";
+  private static final String RECORD_ID = "testRecordId";
+  private static final String ACK_ID = "testAckId";
+
+  @Before
+  public void setup() throws IOException {
+    mockPubsub = Mockito.mock(Pubsub.class, Mockito.RETURNS_DEEP_STUBS);
+    client = new PubsubJsonClient(TIMESTAMP_LABEL, ID_LABEL, mockPubsub);
+  }
+
+  @After
+  public void teardown() throws IOException {
+    client.close();
+    client = null;
+    mockPubsub = null;
+  }
+
+  @Test
+  public void pullOneMessage() throws IOException {
+    String expectedSubscription = SUBSCRIPTION.getPath();
+    PullRequest expectedRequest =
+        new PullRequest().setReturnImmediately(true).setMaxMessages(10);
+    PubsubMessage expectedPubsubMessage = new PubsubMessage()
+        .setMessageId(MESSAGE_ID)
+        .encodeData(DATA.getBytes())
+        .setPublishTime(String.valueOf(PUB_TIME))
+        .setAttributes(
+            ImmutableMap.of(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME),
+                            ID_LABEL, RECORD_ID));
+    ReceivedMessage expectedReceivedMessage =
+        new ReceivedMessage().setMessage(expectedPubsubMessage)
+                             .setAckId(ACK_ID);
+    PullResponse expectedResponse =
+        new PullResponse().setReceivedMessages(ImmutableList.of(expectedReceivedMessage));
+    Mockito.when((Object) (mockPubsub.projects()
+                               .subscriptions()
+                               .pull(expectedSubscription, expectedRequest)
+                               .execute()))
+           .thenReturn(expectedResponse);
+    List<IncomingMessage> acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true);
+    assertEquals(1, acutalMessages.size());
+    IncomingMessage actualMessage = acutalMessages.get(0);
+    assertEquals(ACK_ID, actualMessage.ackId);
+    assertEquals(DATA, new String(actualMessage.elementBytes));
+    assertEquals(RECORD_ID, actualMessage.recordId);
+    assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch);
+    assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch);
+  }
+
+  @Test
+  public void publishOneMessage() throws IOException {
+    String expectedTopic = TOPIC.getPath();
+    PubsubMessage expectedPubsubMessage = new PubsubMessage()
+        .encodeData(DATA.getBytes())
+        .setAttributes(
+            ImmutableMap.<String, String> builder()
+                    .put(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME))
+                    .put(ID_LABEL, RECORD_ID)
+                    .put("k", "v").build());
+    PublishRequest expectedRequest = new PublishRequest()
+        .setMessages(ImmutableList.of(expectedPubsubMessage));
+    PublishResponse expectedResponse = new PublishResponse()
+        .setMessageIds(ImmutableList.of(MESSAGE_ID));
+    Mockito.when((Object) (mockPubsub.projects()
+                                .topics()
+                                .publish(expectedTopic, expectedRequest)
+                                .execute()))
+           .thenReturn(expectedResponse);
+    Map<String, String> attrs = new HashMap<>();
+    attrs.put("k", "v");
+    OutgoingMessage actualMessage = new OutgoingMessage(
+            DATA.getBytes(), attrs, MESSAGE_TIME, RECORD_ID);
+    int n = client.publish(TOPIC, ImmutableList.of(actualMessage));
+    assertEquals(1, n);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java
new file mode 100644
index 0000000..18180af
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.api.client.util.Clock;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.IncomingMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubTestClient.PubsubTestClientFactory;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for PubsubTestClient.
+ */
+@RunWith(JUnit4.class)
+public class PubsubTestClientTest {
+  private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic");
+  private static final SubscriptionPath SUBSCRIPTION =
+      PubsubClient.subscriptionPathFromName("testProject", "testSubscription");
+  private static final long REQ_TIME = 1234L;
+  private static final long MESSAGE_TIME = 6789L;
+  private static final String MESSAGE_ID = "testMessageId";
+  private static final String DATA = "testData";
+  private static final String ACK_ID = "testAckId";
+  private static final int ACK_TIMEOUT_S = 60;
+
+  @Test
+  public void pullOneMessage() throws IOException {
+    final AtomicLong now = new AtomicLong();
+    Clock clock = new Clock() {
+      @Override
+      public long currentTimeMillis() {
+        return now.get();
+      }
+    };
+    IncomingMessage expectedIncomingMessage =
+        new IncomingMessage(DATA.getBytes(), null, MESSAGE_TIME, REQ_TIME, ACK_ID, MESSAGE_ID);
+    try (PubsubTestClientFactory factory =
+             PubsubTestClient.createFactoryForPull(clock, SUBSCRIPTION, ACK_TIMEOUT_S,
+                                                   Lists.newArrayList(expectedIncomingMessage))) {
+      try (PubsubTestClient client = (PubsubTestClient) factory.newClient(null, null, null)) {
+        now.set(REQ_TIME);
+        client.advance();
+        List<IncomingMessage> incomingMessages = client.pull(now.get(), SUBSCRIPTION, 1, true);
+        assertEquals(1, incomingMessages.size());
+        assertEquals(expectedIncomingMessage, incomingMessages.get(0));
+        // Timeout on ACK.
+        now.addAndGet((ACK_TIMEOUT_S + 10) * 1000);
+        client.advance();
+        incomingMessages = client.pull(now.get(), SUBSCRIPTION, 1, true);
+        assertEquals(1, incomingMessages.size());
+        assertEquals(expectedIncomingMessage.withRequestTime(now.get()), incomingMessages.get(0));
+        now.addAndGet(10 * 1000);
+        client.advance();
+        // Extend ack
+        client.modifyAckDeadline(SUBSCRIPTION, ImmutableList.of(ACK_ID), 20);
+        // Timeout on extended ACK
+        now.addAndGet(30 * 1000);
+        client.advance();
+        incomingMessages = client.pull(now.get(), SUBSCRIPTION, 1, true);
+        assertEquals(1, incomingMessages.size());
+        assertEquals(expectedIncomingMessage.withRequestTime(now.get()), incomingMessages.get(0));
+        // Extend ack
+        client.modifyAckDeadline(SUBSCRIPTION, ImmutableList.of(ACK_ID), 20);
+        // Ack
+        now.addAndGet(15 * 1000);
+        client.advance();
+        client.acknowledge(SUBSCRIPTION, ImmutableList.of(ACK_ID));
+      }
+    }
+  }
+
+  @Test
+  public void publishOneMessage() throws IOException {
+    OutgoingMessage expectedOutgoingMessage =
+        new OutgoingMessage(DATA.getBytes(), null, MESSAGE_TIME, MESSAGE_ID);
+    try (PubsubTestClientFactory factory =
+             PubsubTestClient.createFactoryForPublish(
+                 TOPIC,
+                 Sets.newHashSet(expectedOutgoingMessage),
+                 ImmutableList.<OutgoingMessage>of())) {
+      try (PubsubTestClient client = (PubsubTestClient) factory.newClient(null, null, null)) {
+        client.publish(TOPIC, ImmutableList.of(expectedOutgoingMessage));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java
new file mode 100644
index 0000000..be425d4
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.hash.Hashing;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubTestClient.PubsubTestClientFactory;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSink.RecordIdMethod;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test PubsubUnboundedSink.
+ */
+@RunWith(JUnit4.class)
+public class PubsubUnboundedSinkTest implements Serializable {
+  private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic");
+  private static final String DATA = "testData";
+  private static final Map<String, String> ATTRIBUTES =
+          ImmutableMap.<String, String>builder().put("a", "b").put("c", "d").build();
+  private static final long TIMESTAMP = 1234L;
+  private static final String TIMESTAMP_LABEL = "timestamp";
+  private static final String ID_LABEL = "id";
+  private static final int NUM_SHARDS = 10;
+
+  private static class Stamp extends DoFn<String, String> {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.outputWithTimestamp(c.element(), new Instant(TIMESTAMP));
+    }
+  }
+
+  private String getRecordId(String data) {
+    return Hashing.murmur3_128().hashBytes(data.getBytes()).toString();
+  }
+
+  @Rule
+  public transient TestPipeline p = TestPipeline.create();
+
+  @Test
+  public void saneCoder() throws Exception {
+    OutgoingMessage message = new OutgoingMessage(
+            DATA.getBytes(), ImmutableMap.<String, String>of(), TIMESTAMP, getRecordId(DATA));
+    CoderProperties.coderDecodeEncodeEqual(PubsubUnboundedSink.CODER, message);
+    CoderProperties.coderSerializable(PubsubUnboundedSink.CODER);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void sendOneMessage() throws IOException {
+    List<OutgoingMessage> outgoing =
+        ImmutableList.of(new OutgoingMessage(
+                DATA.getBytes(),
+                ATTRIBUTES,
+                TIMESTAMP, getRecordId(DATA)));
+    int batchSize = 1;
+    int batchBytes = 1;
+    try (PubsubTestClientFactory factory =
+             PubsubTestClient.createFactoryForPublish(TOPIC, outgoing,
+                                                      ImmutableList.<OutgoingMessage>of())) {
+      PubsubUnboundedSink<String> sink =
+          new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), StringUtf8Coder.of(),
+              TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes,
+              Duration.standardSeconds(2),
+              new SimpleFunction<String, PubsubIO.PubsubMessage>() {
+                @Override
+                public PubsubIO.PubsubMessage apply(String input) {
+                  return new PubsubIO.PubsubMessage(input.getBytes(), ATTRIBUTES);
+                }
+              },
+              RecordIdMethod.DETERMINISTIC);
+      p.apply(Create.of(ImmutableList.of(DATA)))
+       .apply(ParDo.of(new Stamp()))
+       .apply(sink);
+      p.run();
+    }
+    // The PubsubTestClientFactory will assert fail on close if the actual published
+    // message does not match the expected publish message.
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void sendMoreThanOneBatchByNumMessages() throws IOException {
+    List<OutgoingMessage> outgoing = new ArrayList<>();
+    List<String> data = new ArrayList<>();
+    int batchSize = 2;
+    int batchBytes = 1000;
+    for (int i = 0; i < batchSize * 10; i++) {
+      String str = String.valueOf(i);
+      outgoing.add(new OutgoingMessage(
+              str.getBytes(), ImmutableMap.<String, String>of(), TIMESTAMP, getRecordId(str)));
+      data.add(str);
+    }
+    try (PubsubTestClientFactory factory =
+             PubsubTestClient.createFactoryForPublish(TOPIC, outgoing,
+                                                      ImmutableList.<OutgoingMessage>of())) {
+      PubsubUnboundedSink<String> sink =
+          new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), StringUtf8Coder.of(),
+              TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes,
+              Duration.standardSeconds(2), null, RecordIdMethod.DETERMINISTIC);
+      p.apply(Create.of(data))
+       .apply(ParDo.of(new Stamp()))
+       .apply(sink);
+      p.run();
+    }
+    // The PubsubTestClientFactory will assert fail on close if the actual published
+    // message does not match the expected publish message.
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void sendMoreThanOneBatchByByteSize() throws IOException {
+    List<OutgoingMessage> outgoing = new ArrayList<>();
+    List<String> data = new ArrayList<>();
+    int batchSize = 100;
+    int batchBytes = 10;
+    int n = 0;
+    while (n < batchBytes * 10) {
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < batchBytes; i++) {
+        sb.append(String.valueOf(n));
+      }
+      String str = sb.toString();
+      outgoing.add(new OutgoingMessage(
+              str.getBytes(), ImmutableMap.<String, String>of(), TIMESTAMP, getRecordId(str)));
+      data.add(str);
+      n += str.length();
+    }
+    try (PubsubTestClientFactory factory =
+             PubsubTestClient.createFactoryForPublish(TOPIC, outgoing,
+                                                      ImmutableList.<OutgoingMessage>of())) {
+      PubsubUnboundedSink<String> sink =
+          new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC),
+              StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL,
+              NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2),
+              null, RecordIdMethod.DETERMINISTIC);
+      p.apply(Create.of(data))
+       .apply(ParDo.of(new Stamp()))
+       .apply(sink);
+      p.run();
+    }
+    // The PubsubTestClientFactory will assert fail on close if the actual published
+    // message does not match the expected publish message.
+  }
+
+  // TODO: We would like to test that failed Pubsub publish calls cause the already assigned
+  // (and random) record ids to be reused. However that can't be done without the test runnner
+  // supporting retrying bundles.
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java
new file mode 100644
index 0000000..d2e88c3
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java
@@ -0,0 +1,409 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static junit.framework.TestCase.assertFalse;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.lessThanOrEqualTo;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import com.google.api.client.util.Clock;
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.IncomingMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubTestClient.PubsubTestClientFactory;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSource.PubsubCheckpoint;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSource.PubsubReader;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSource.PubsubSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.joda.time.Instant;
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test PubsubUnboundedSource.
+ */
+@RunWith(JUnit4.class)
+public class PubsubUnboundedSourceTest {
+  private static final SubscriptionPath SUBSCRIPTION =
+      PubsubClient.subscriptionPathFromName("testProject", "testSubscription");
+  private static final String DATA = "testData";
+  private static final long TIMESTAMP = 1234L;
+  private static final long REQ_TIME = 6373L;
+  private static final String TIMESTAMP_LABEL = "timestamp";
+  private static final String ID_LABEL = "id";
+  private static final String ACK_ID = "testAckId";
+  private static final String RECORD_ID = "testRecordId";
+  private static final int ACK_TIMEOUT_S = 60;
+
+  private AtomicLong now;
+  private Clock clock;
+  private PubsubTestClientFactory factory;
+  private PubsubSource<String> primSource;
+
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
+  private void setupOneMessage(Iterable<IncomingMessage> incoming) {
+    now = new AtomicLong(REQ_TIME);
+    clock = new Clock() {
+      @Override
+      public long currentTimeMillis() {
+        return now.get();
+      }
+    };
+    factory = PubsubTestClient.createFactoryForPull(clock, SUBSCRIPTION, ACK_TIMEOUT_S, incoming);
+    PubsubUnboundedSource<String> source =
+        new PubsubUnboundedSource<>(
+            clock, factory, null, null, StaticValueProvider.of(SUBSCRIPTION),
+            StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL, null);
+    primSource = new PubsubSource<>(source);
+  }
+
+  private void setupOneMessage() {
+    setupOneMessage(ImmutableList.of(
+        new IncomingMessage(DATA.getBytes(), null, TIMESTAMP, 0, ACK_ID, RECORD_ID)));
+  }
+
+  @After
+  public void after() throws IOException {
+    factory.close();
+    now = null;
+    clock = null;
+    primSource = null;
+    factory = null;
+  }
+
+  @Test
+  public void checkpointCoderIsSane() throws Exception {
+    setupOneMessage(ImmutableList.<IncomingMessage>of());
+    CoderProperties.coderSerializable(primSource.getCheckpointMarkCoder());
+    // Since we only serialize/deserialize the 'notYetReadIds', and we don't want to make
+    // equals on checkpoints ignore those fields, we'll test serialization and deserialization
+    // of checkpoints in multipleReaders below.
+  }
+
+  @Test
+  public void readOneMessage() throws IOException {
+    setupOneMessage();
+    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
+    // Read one message.
+    assertTrue(reader.start());
+    assertEquals(DATA, reader.getCurrent());
+    assertFalse(reader.advance());
+    // ACK the message.
+    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
+    checkpoint.finalizeCheckpoint();
+    reader.close();
+  }
+
+  @Test
+  public void timeoutAckAndRereadOneMessage() throws IOException {
+    setupOneMessage();
+    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
+    PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
+    assertTrue(reader.start());
+    assertEquals(DATA, reader.getCurrent());
+    // Let the ACK deadline for the above expire.
+    now.addAndGet(65 * 1000);
+    pubsubClient.advance();
+    // We'll now receive the same message again.
+    assertTrue(reader.advance());
+    assertEquals(DATA, reader.getCurrent());
+    assertFalse(reader.advance());
+    // Now ACK the message.
+    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
+    checkpoint.finalizeCheckpoint();
+    reader.close();
+  }
+
+  @Test
+  public void extendAck() throws IOException {
+    setupOneMessage();
+    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
+    PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
+    // Pull the first message but don't take a checkpoint for it.
+    assertTrue(reader.start());
+    assertEquals(DATA, reader.getCurrent());
+    // Extend the ack
+    now.addAndGet(55 * 1000);
+    pubsubClient.advance();
+    assertFalse(reader.advance());
+    // Extend the ack again
+    now.addAndGet(25 * 1000);
+    pubsubClient.advance();
+    assertFalse(reader.advance());
+    // Now ACK the message.
+    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
+    checkpoint.finalizeCheckpoint();
+    reader.close();
+  }
+
+  @Test
+  public void timeoutAckExtensions() throws IOException {
+    setupOneMessage();
+    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
+    PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
+    // Pull the first message but don't take a checkpoint for it.
+    assertTrue(reader.start());
+    assertEquals(DATA, reader.getCurrent());
+    // Extend the ack.
+    now.addAndGet(55 * 1000);
+    pubsubClient.advance();
+    assertFalse(reader.advance());
+    // Let the ack expire.
+    for (int i = 0; i < 3; i++) {
+      now.addAndGet(25 * 1000);
+      pubsubClient.advance();
+      assertFalse(reader.advance());
+    }
+    // Wait for resend.
+    now.addAndGet(25 * 1000);
+    pubsubClient.advance();
+    // Reread the same message.
+    assertTrue(reader.advance());
+    assertEquals(DATA, reader.getCurrent());
+    // Now ACK the message.
+    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
+    checkpoint.finalizeCheckpoint();
+    reader.close();
+  }
+
+  @Test
+  public void multipleReaders() throws IOException {
+    List<IncomingMessage> incoming = new ArrayList<>();
+    for (int i = 0; i < 2; i++) {
+      String data = String.format("data_%d", i);
+      String ackid = String.format("ackid_%d", i);
+      incoming.add(new IncomingMessage(data.getBytes(), null, TIMESTAMP, 0, ackid, RECORD_ID));
+    }
+    setupOneMessage(incoming);
+    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
+    // Consume two messages, only read one.
+    assertTrue(reader.start());
+    assertEquals("data_0", reader.getCurrent());
+
+    // Grab checkpoint.
+    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
+    checkpoint.finalizeCheckpoint();
+    assertEquals(1, checkpoint.notYetReadIds.size());
+    assertEquals("ackid_1", checkpoint.notYetReadIds.get(0));
+
+    // Read second message.
+    assertTrue(reader.advance());
+    assertEquals("data_1", reader.getCurrent());
+
+    // Restore from checkpoint.
+    byte[] checkpointBytes =
+        CoderUtils.encodeToByteArray(primSource.getCheckpointMarkCoder(), checkpoint);
+    checkpoint = CoderUtils.decodeFromByteArray(primSource.getCheckpointMarkCoder(),
+                                                checkpointBytes);
+    assertEquals(1, checkpoint.notYetReadIds.size());
+    assertEquals("ackid_1", checkpoint.notYetReadIds.get(0));
+
+    // Re-read second message.
+    reader = primSource.createReader(p.getOptions(), checkpoint);
+    assertTrue(reader.start());
+    assertEquals("data_1", reader.getCurrent());
+
+    // We are done.
+    assertFalse(reader.advance());
+
+    // ACK final message.
+    checkpoint = reader.getCheckpointMark();
+    checkpoint.finalizeCheckpoint();
+    reader.close();
+  }
+
+  private long messageNumToTimestamp(int messageNum) {
+    return TIMESTAMP + messageNum * 100;
+  }
+
+  @Test
+  public void readManyMessages() throws IOException {
+    Map<String, Integer> dataToMessageNum = new HashMap<>();
+
+    final int m = 97;
+    final int n = 10000;
+    List<IncomingMessage> incoming = new ArrayList<>();
+    for (int i = 0; i < n; i++) {
+      // Make the messages timestamps slightly out of order.
+      int messageNum = ((i / m) * m) + (m - 1) - (i % m);
+      String data = String.format("data_%d", messageNum);
+      dataToMessageNum.put(data, messageNum);
+      String recid = String.format("recordid_%d", messageNum);
+      String ackId = String.format("ackid_%d", messageNum);
+      incoming.add(new IncomingMessage(data.getBytes(), null, messageNumToTimestamp(messageNum), 0,
+                                       ackId, recid));
+    }
+    setupOneMessage(incoming);
+
+    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
+    PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
+
+    for (int i = 0; i < n; i++) {
+      if (i == 0) {
+        assertTrue(reader.start());
+      } else {
+        assertTrue(reader.advance());
+      }
+      // We'll checkpoint and ack within the 2min limit.
+      now.addAndGet(30);
+      pubsubClient.advance();
+      String data = reader.getCurrent();
+      Integer messageNum = dataToMessageNum.remove(data);
+      // No duplicate messages.
+      assertNotNull(messageNum);
+      // Preserve timestamp.
+      assertEquals(new Instant(messageNumToTimestamp(messageNum)), reader.getCurrentTimestamp());
+      // Preserve record id.
+      String recid = String.format("recordid_%d", messageNum);
+      assertArrayEquals(recid.getBytes(), reader.getCurrentRecordId());
+
+      if (i % 1000 == 999) {
+        // Estimated watermark can never get ahead of actual outstanding messages.
+        long watermark = reader.getWatermark().getMillis();
+        long minOutstandingTimestamp = Long.MAX_VALUE;
+        for (Integer outstandingMessageNum : dataToMessageNum.values()) {
+          minOutstandingTimestamp =
+              Math.min(minOutstandingTimestamp, messageNumToTimestamp(outstandingMessageNum));
+        }
+        assertThat(watermark, lessThanOrEqualTo(minOutstandingTimestamp));
+        // Ack messages, but only every other finalization.
+        PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
+        if (i % 2000 == 1999) {
+          checkpoint.finalizeCheckpoint();
+        }
+      }
+    }
+    // We are done.
+    assertFalse(reader.advance());
+    // We saw each message exactly once.
+    assertTrue(dataToMessageNum.isEmpty());
+    reader.close();
+  }
+
+  @Test
+  public void noSubscriptionSplitIntoBundlesGeneratesSubscription() throws Exception {
+    TopicPath topicPath = PubsubClient.topicPathFromName("my_project", "my_topic");
+    factory = PubsubTestClient.createFactoryForCreateSubscription();
+    PubsubUnboundedSource<String> source =
+        new PubsubUnboundedSource<>(
+            factory,
+            StaticValueProvider.of(PubsubClient.projectPathFromId("my_project")),
+            StaticValueProvider.of(topicPath),
+            null,
+            StringUtf8Coder.of(),
+            null,
+            null,
+            null);
+    assertThat(source.getSubscription(), nullValue());
+
+    assertThat(source.getSubscription(), nullValue());
+
+    PipelineOptions options = PipelineOptionsFactory.create();
+    List<PubsubSource<String>> splits =
+        (new PubsubSource<>(source)).generateInitialSplits(3, options);
+    // We have at least one returned split
+    assertThat(splits, hasSize(greaterThan(0)));
+    for (PubsubSource<String> split : splits) {
+      // Each split is equal
+      assertThat(split, equalTo(splits.get(0)));
+    }
+
+    assertThat(splits.get(0).subscriptionPath, not(nullValue()));
+  }
+
+  @Test
+  public void noSubscriptionNoSplitGeneratesSubscription() throws Exception {
+    TopicPath topicPath = PubsubClient.topicPathFromName("my_project", "my_topic");
+    factory = PubsubTestClient.createFactoryForCreateSubscription();
+    PubsubUnboundedSource<String> source =
+        new PubsubUnboundedSource<>(
+            factory,
+            StaticValueProvider.of(PubsubClient.projectPathFromId("my_project")),
+            StaticValueProvider.of(topicPath),
+            null,
+            StringUtf8Coder.of(),
+            null,
+            null,
+            null);
+    assertThat(source.getSubscription(), nullValue());
+
+    assertThat(source.getSubscription(), nullValue());
+
+    PipelineOptions options = PipelineOptionsFactory.create();
+    PubsubSource<String> actualSource = new PubsubSource<>(source);
+    PubsubReader<String> reader = actualSource.createReader(options, null);
+    SubscriptionPath createdSubscription = reader.subscription;
+    assertThat(createdSubscription, not(nullValue()));
+
+    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
+    assertThat(checkpoint.subscriptionPath, equalTo(createdSubscription.getPath()));
+
+    checkpoint.finalizeCheckpoint();
+    PubsubCheckpoint<String> deserCheckpoint =
+        CoderUtils.clone(actualSource.getCheckpointMarkCoder(), checkpoint);
+    assertThat(checkpoint.subscriptionPath, not(nullValue()));
+    assertThat(checkpoint.subscriptionPath, equalTo(deserCheckpoint.subscriptionPath));
+
+    PubsubReader<String> readerFromOriginal = actualSource.createReader(options, checkpoint);
+    PubsubReader<String> readerFromDeser = actualSource.createReader(options, deserCheckpoint);
+
+    assertThat(readerFromOriginal.subscription, equalTo(createdSubscription));
+    assertThat(readerFromDeser.subscription, equalTo(createdSubscription));
+  }
+
+  /**
+   * Tests that checkpoints finalized after the reader is closed succeed.
+   */
+  @Test
+  public void closeWithActiveCheckpoints() throws Exception {
+    setupOneMessage();
+    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
+    reader.start();
+    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
+    reader.close();
+    checkpoint.finalizeCheckpoint();
+  }
+}


[8/8] beam git commit: This closes #2503

Posted by dh...@apache.org.
This closes #2503


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

Branch: refs/heads/master
Commit: 7e603d5c763ee4b95dffb94e129603de55e5c33d
Parents: 82f2f2c 5bcb8c5
Author: Dan Halperin <dh...@google.com>
Authored: Wed Apr 12 08:16:47 2017 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Apr 12 08:16:47 2017 -0700

----------------------------------------------------------------------
 .../beam/examples/complete/game/GameStats.java  |    2 +-
 .../examples/complete/game/LeaderBoard.java     |    2 +-
 .../triggers/AfterWatermarkStateMachine.java    |   14 +-
 runners/google-cloud-dataflow-java/pom.xml      |    5 +-
 .../beam/runners/dataflow/DataflowRunner.java   |    4 +-
 sdks/java/core/pom.xml                          |   44 -
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 1016 ------------
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |  494 ------
 .../beam/sdk/io/PubsubUnboundedSource.java      | 1463 ------------------
 .../apache/beam/sdk/transforms/GroupByKey.java  |    4 +-
 .../transforms/windowing/AfterWatermark.java    |   14 +-
 .../org/apache/beam/sdk/util/PubsubClient.java  |  544 -------
 .../apache/beam/sdk/util/PubsubGrpcClient.java  |  424 -----
 .../apache/beam/sdk/util/PubsubJsonClient.java  |  317 ----
 .../apache/beam/sdk/util/PubsubTestClient.java  |  436 ------
 .../org/apache/beam/sdk/util/Transport.java     |    3 +-
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |  189 ---
 .../beam/sdk/io/PubsubUnboundedSinkTest.java    |  190 ---
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |  411 -----
 .../apache/beam/sdk/util/PubsubClientTest.java  |  189 ---
 .../beam/sdk/util/PubsubGrpcClientTest.java     |  207 ---
 .../beam/sdk/util/PubsubJsonClientTest.java     |  140 --
 .../beam/sdk/util/PubsubTestClientTest.java     |  114 --
 sdks/java/io/google-cloud-platform/pom.xml      |   44 +
 .../beam/sdk/io/gcp/pubsub/PubsubClient.java    |  544 +++++++
 .../sdk/io/gcp/pubsub/PubsubGrpcClient.java     |  424 +++++
 .../apache/beam/sdk/io/gcp/pubsub/PubsubIO.java | 1014 ++++++++++++
 .../sdk/io/gcp/pubsub/PubsubJsonClient.java     |  319 ++++
 .../sdk/io/gcp/pubsub/PubsubTestClient.java     |  436 ++++++
 .../sdk/io/gcp/pubsub/PubsubUnboundedSink.java  |  490 ++++++
 .../io/gcp/pubsub/PubsubUnboundedSource.java    | 1463 ++++++++++++++++++
 .../beam/sdk/io/gcp/pubsub/package-info.java    |   24 +
 .../beam/sdk/io/gcp/GcpApiSurfaceTest.java      |    5 +-
 .../sdk/io/gcp/pubsub/PubsubClientTest.java     |  189 +++
 .../sdk/io/gcp/pubsub/PubsubGrpcClientTest.java |  208 +++
 .../beam/sdk/io/gcp/pubsub/PubsubIOTest.java    |  189 +++
 .../sdk/io/gcp/pubsub/PubsubJsonClientTest.java |  139 ++
 .../sdk/io/gcp/pubsub/PubsubTestClientTest.java |  114 ++
 .../io/gcp/pubsub/PubsubUnboundedSinkTest.java  |  188 +++
 .../gcp/pubsub/PubsubUnboundedSourceTest.java   |  409 +++++
 40 files changed, 6218 insertions(+), 6207 deletions(-)
----------------------------------------------------------------------



[5/8] beam git commit: [BEAM-1722] Move PubsubIO into the google-cloud-platform module

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java
deleted file mode 100644
index 4a6ddac..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java
+++ /dev/null
@@ -1,424 +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.checkState;
-
-import com.google.auth.Credentials;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Timestamp;
-import com.google.pubsub.v1.AcknowledgeRequest;
-import com.google.pubsub.v1.DeleteSubscriptionRequest;
-import com.google.pubsub.v1.DeleteTopicRequest;
-import com.google.pubsub.v1.GetSubscriptionRequest;
-import com.google.pubsub.v1.ListSubscriptionsRequest;
-import com.google.pubsub.v1.ListSubscriptionsResponse;
-import com.google.pubsub.v1.ListTopicsRequest;
-import com.google.pubsub.v1.ListTopicsResponse;
-import com.google.pubsub.v1.ModifyAckDeadlineRequest;
-import com.google.pubsub.v1.PublishRequest;
-import com.google.pubsub.v1.PublishResponse;
-import com.google.pubsub.v1.PublisherGrpc;
-import com.google.pubsub.v1.PublisherGrpc.PublisherBlockingStub;
-import com.google.pubsub.v1.PubsubMessage;
-import com.google.pubsub.v1.PullRequest;
-import com.google.pubsub.v1.PullResponse;
-import com.google.pubsub.v1.ReceivedMessage;
-import com.google.pubsub.v1.SubscriberGrpc;
-import com.google.pubsub.v1.SubscriberGrpc.SubscriberBlockingStub;
-import com.google.pubsub.v1.Subscription;
-import com.google.pubsub.v1.Topic;
-import io.grpc.Channel;
-import io.grpc.ClientInterceptors;
-import io.grpc.ManagedChannel;
-import io.grpc.auth.ClientAuthInterceptor;
-import io.grpc.netty.GrpcSslContexts;
-import io.grpc.netty.NegotiationType;
-import io.grpc.netty.NettyChannelBuilder;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.options.GcpOptions;
-import org.apache.beam.sdk.options.PubsubOptions;
-
-/**
- * A helper class for talking to Pubsub via grpc.
- *
- * <p>CAUTION: Currently uses the application default credentials and does not respect any
- * credentials-related arguments in {@link GcpOptions}.
- */
-public class PubsubGrpcClient extends PubsubClient {
-  private static final String PUBSUB_ADDRESS = "pubsub.googleapis.com";
-  private static final int PUBSUB_PORT = 443;
-  private static final int LIST_BATCH_SIZE = 1000;
-
-  private static final int DEFAULT_TIMEOUT_S = 15;
-
-  private static class PubsubGrpcClientFactory implements PubsubClientFactory {
-    @Override
-    public PubsubClient newClient(
-        @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
-        throws IOException {
-      ManagedChannel channel = NettyChannelBuilder
-          .forAddress(PUBSUB_ADDRESS, PUBSUB_PORT)
-          .negotiationType(NegotiationType.TLS)
-          .sslContext(GrpcSslContexts.forClient().ciphers(null).build())
-          .build();
-
-      return new PubsubGrpcClient(timestampLabel,
-                                  idLabel,
-                                  DEFAULT_TIMEOUT_S,
-                                  channel,
-                                  options.getGcpCredential());
-    }
-
-    @Override
-    public String getKind() {
-      return "Grpc";
-    }
-  }
-
-  /**
-   * Factory for creating Pubsub clients using gRCP transport.
-   */
-  public static final PubsubClientFactory FACTORY = new PubsubGrpcClientFactory();
-
-  /**
-   * Timeout for grpc calls (in s).
-   */
-  private final int timeoutSec;
-
-  /**
-   * Underlying netty channel, or {@literal null} if closed.
-   */
-  @Nullable
-  private ManagedChannel publisherChannel;
-
-  /**
-   * Credentials determined from options and environment.
-   */
-  private final Credentials credentials;
-
-  /**
-   * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time
-   * instead.
-   */
-  @Nullable
-  private final String timestampLabel;
-
-  /**
-   * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids.
-   */
-  @Nullable
-  private final String idLabel;
-
-
-  /**
-   * Cached stubs, or null if not cached.
-   */
-  @Nullable
-  private PublisherGrpc.PublisherBlockingStub cachedPublisherStub;
-  private SubscriberGrpc.SubscriberBlockingStub cachedSubscriberStub;
-
-  @VisibleForTesting
-  PubsubGrpcClient(
-      @Nullable String timestampLabel,
-      @Nullable String idLabel,
-      int timeoutSec,
-      ManagedChannel publisherChannel,
-      Credentials credentials) {
-    this.timestampLabel = timestampLabel;
-    this.idLabel = idLabel;
-    this.timeoutSec = timeoutSec;
-    this.publisherChannel = publisherChannel;
-    this.credentials = credentials;
-  }
-
-  /**
-   * Gracefully close the underlying netty channel.
-   */
-  @Override
-  public void close() {
-    if (publisherChannel == null) {
-      // Already closed.
-      return;
-    }
-    // Can gc the underlying stubs.
-    cachedPublisherStub = null;
-    cachedSubscriberStub = null;
-    // Mark the client as having been closed before going further
-    // in case we have an exception from the channel.
-    ManagedChannel publisherChannel = this.publisherChannel;
-    this.publisherChannel = null;
-    // Gracefully shutdown the channel.
-    publisherChannel.shutdown();
-    try {
-      publisherChannel.awaitTermination(timeoutSec, TimeUnit.SECONDS);
-    } catch (InterruptedException e) {
-      // Ignore.
-      Thread.currentThread().interrupt();
-    }
-  }
-
-  /**
-   * Return channel with interceptor for returning credentials.
-   */
-  private Channel newChannel() throws IOException {
-    checkState(publisherChannel != null, "PubsubGrpcClient has been closed");
-    ClientAuthInterceptor interceptor =
-        new ClientAuthInterceptor(credentials, Executors.newSingleThreadExecutor());
-    return ClientInterceptors.intercept(publisherChannel, interceptor);
-  }
-
-  /**
-   * Return a stub for making a publish request with a timeout.
-   */
-  private PublisherBlockingStub publisherStub() throws IOException {
-    if (cachedPublisherStub == null) {
-      cachedPublisherStub = PublisherGrpc.newBlockingStub(newChannel());
-    }
-    return cachedPublisherStub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS);
-  }
-
-  /**
-   * Return a stub for making a subscribe request with a timeout.
-   */
-  private SubscriberBlockingStub subscriberStub() throws IOException {
-    if (cachedSubscriberStub == null) {
-      cachedSubscriberStub = SubscriberGrpc.newBlockingStub(newChannel());
-    }
-    return cachedSubscriberStub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS);
-  }
-
-  @Override
-  public int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages)
-      throws IOException {
-    PublishRequest.Builder request = PublishRequest.newBuilder()
-                                                   .setTopic(topic.getPath());
-    for (OutgoingMessage outgoingMessage : outgoingMessages) {
-      PubsubMessage.Builder message =
-          PubsubMessage.newBuilder()
-                       .setData(ByteString.copyFrom(outgoingMessage.elementBytes));
-
-      if (outgoingMessage.attributes != null) {
-        message.putAllAttributes(outgoingMessage.attributes);
-      }
-
-      if (timestampLabel != null) {
-        message.getMutableAttributes()
-               .put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch));
-      }
-
-      if (idLabel != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) {
-        message.getMutableAttributes().put(idLabel, outgoingMessage.recordId);
-      }
-
-      request.addMessages(message);
-    }
-
-    PublishResponse response = publisherStub().publish(request.build());
-    return response.getMessageIdsCount();
-  }
-
-  @Override
-  public List<IncomingMessage> pull(
-      long requestTimeMsSinceEpoch,
-      SubscriptionPath subscription,
-      int batchSize,
-      boolean returnImmediately) throws IOException {
-    PullRequest request = PullRequest.newBuilder()
-                                     .setSubscription(subscription.getPath())
-                                     .setReturnImmediately(returnImmediately)
-                                     .setMaxMessages(batchSize)
-                                     .build();
-    PullResponse response = subscriberStub().pull(request);
-    if (response.getReceivedMessagesCount() == 0) {
-      return ImmutableList.of();
-    }
-    List<IncomingMessage> incomingMessages = new ArrayList<>(response.getReceivedMessagesCount());
-    for (ReceivedMessage message : response.getReceivedMessagesList()) {
-      PubsubMessage pubsubMessage = message.getMessage();
-      @Nullable Map<String, String> attributes = pubsubMessage.getAttributes();
-
-      // Payload.
-      byte[] elementBytes = pubsubMessage.getData().toByteArray();
-
-      // Timestamp.
-      String pubsubTimestampString = null;
-      Timestamp timestampProto = pubsubMessage.getPublishTime();
-      if (timestampProto != null) {
-        pubsubTimestampString = String.valueOf(timestampProto.getSeconds()
-                                               + timestampProto.getNanos() / 1000L);
-      }
-      long timestampMsSinceEpoch =
-          extractTimestamp(timestampLabel, pubsubTimestampString, attributes);
-
-      // Ack id.
-      String ackId = message.getAckId();
-      checkState(!Strings.isNullOrEmpty(ackId));
-
-      // Record id, if any.
-      @Nullable String recordId = null;
-      if (idLabel != null && attributes != null) {
-        recordId = attributes.get(idLabel);
-      }
-      if (Strings.isNullOrEmpty(recordId)) {
-        // Fall back to the Pubsub provided message id.
-        recordId = pubsubMessage.getMessageId();
-      }
-
-      incomingMessages.add(new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch,
-                                               requestTimeMsSinceEpoch, ackId, recordId));
-    }
-    return incomingMessages;
-  }
-
-  @Override
-  public void acknowledge(SubscriptionPath subscription, List<String> ackIds)
-      throws IOException {
-    AcknowledgeRequest request = AcknowledgeRequest.newBuilder()
-                                                   .setSubscription(subscription.getPath())
-                                                   .addAllAckIds(ackIds)
-                                                   .build();
-    subscriberStub().acknowledge(request); // ignore Empty result.
-  }
-
-  @Override
-  public void modifyAckDeadline(
-      SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds)
-      throws IOException {
-    ModifyAckDeadlineRequest request =
-        ModifyAckDeadlineRequest.newBuilder()
-                                .setSubscription(subscription.getPath())
-                                .addAllAckIds(ackIds)
-                                .setAckDeadlineSeconds(deadlineSeconds)
-                                .build();
-    subscriberStub().modifyAckDeadline(request); // ignore Empty result.
-  }
-
-  @Override
-  public void createTopic(TopicPath topic) throws IOException {
-    Topic request = Topic.newBuilder()
-                         .setName(topic.getPath())
-                         .build();
-    publisherStub().createTopic(request); // ignore Topic result.
-  }
-
-  @Override
-  public void deleteTopic(TopicPath topic) throws IOException {
-    DeleteTopicRequest request = DeleteTopicRequest.newBuilder()
-                                                   .setTopic(topic.getPath())
-                                                   .build();
-    publisherStub().deleteTopic(request); // ignore Empty result.
-  }
-
-  @Override
-  public List<TopicPath> listTopics(ProjectPath project) throws IOException {
-    ListTopicsRequest.Builder request =
-        ListTopicsRequest.newBuilder()
-                         .setProject(project.getPath())
-                         .setPageSize(LIST_BATCH_SIZE);
-    ListTopicsResponse response = publisherStub().listTopics(request.build());
-    if (response.getTopicsCount() == 0) {
-      return ImmutableList.of();
-    }
-    List<TopicPath> topics = new ArrayList<>(response.getTopicsCount());
-    while (true) {
-      for (Topic topic : response.getTopicsList()) {
-        topics.add(topicPathFromPath(topic.getName()));
-      }
-      if (response.getNextPageToken().isEmpty()) {
-        break;
-      }
-      request.setPageToken(response.getNextPageToken());
-      response = publisherStub().listTopics(request.build());
-    }
-    return topics;
-  }
-
-  @Override
-  public void createSubscription(
-      TopicPath topic, SubscriptionPath subscription,
-      int ackDeadlineSeconds) throws IOException {
-    Subscription request = Subscription.newBuilder()
-                                       .setTopic(topic.getPath())
-                                       .setName(subscription.getPath())
-                                       .setAckDeadlineSeconds(ackDeadlineSeconds)
-                                       .build();
-    subscriberStub().createSubscription(request); // ignore Subscription result.
-  }
-
-  @Override
-  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
-    DeleteSubscriptionRequest request =
-        DeleteSubscriptionRequest.newBuilder()
-                                 .setSubscription(subscription.getPath())
-                                 .build();
-    subscriberStub().deleteSubscription(request); // ignore Empty result.
-  }
-
-  @Override
-  public List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
-      throws IOException {
-    ListSubscriptionsRequest.Builder request =
-        ListSubscriptionsRequest.newBuilder()
-                                .setProject(project.getPath())
-                                .setPageSize(LIST_BATCH_SIZE);
-    ListSubscriptionsResponse response = subscriberStub().listSubscriptions(request.build());
-    if (response.getSubscriptionsCount() == 0) {
-      return ImmutableList.of();
-    }
-    List<SubscriptionPath> subscriptions = new ArrayList<>(response.getSubscriptionsCount());
-    while (true) {
-      for (Subscription subscription : response.getSubscriptionsList()) {
-        if (subscription.getTopic().equals(topic.getPath())) {
-          subscriptions.add(subscriptionPathFromPath(subscription.getName()));
-        }
-      }
-      if (response.getNextPageToken().isEmpty()) {
-        break;
-      }
-      request.setPageToken(response.getNextPageToken());
-      response = subscriberStub().listSubscriptions(request.build());
-    }
-    return subscriptions;
-  }
-
-  @Override
-  public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException {
-    GetSubscriptionRequest request =
-        GetSubscriptionRequest.newBuilder()
-                              .setSubscription(subscription.getPath())
-                              .build();
-    Subscription response = subscriberStub().getSubscription(request);
-    return response.getAckDeadlineSeconds();
-  }
-
-  @Override
-  public boolean isEOF() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java
deleted file mode 100644
index ef8abfd..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java
+++ /dev/null
@@ -1,317 +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.checkState;
-
-import com.google.api.client.http.HttpRequestInitializer;
-import com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.pubsub.Pubsub.Builder;
-import com.google.api.services.pubsub.model.AcknowledgeRequest;
-import com.google.api.services.pubsub.model.ListSubscriptionsResponse;
-import com.google.api.services.pubsub.model.ListTopicsResponse;
-import com.google.api.services.pubsub.model.ModifyAckDeadlineRequest;
-import com.google.api.services.pubsub.model.PublishRequest;
-import com.google.api.services.pubsub.model.PublishResponse;
-import com.google.api.services.pubsub.model.PubsubMessage;
-import com.google.api.services.pubsub.model.PullRequest;
-import com.google.api.services.pubsub.model.PullResponse;
-import com.google.api.services.pubsub.model.ReceivedMessage;
-import com.google.api.services.pubsub.model.Subscription;
-import com.google.api.services.pubsub.model.Topic;
-import com.google.auth.Credentials;
-import com.google.auth.http.HttpCredentialsAdapter;
-import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.options.PubsubOptions;
-
-/**
- * A Pubsub client using JSON transport.
- */
-public class PubsubJsonClient extends PubsubClient {
-
-  private static class PubsubJsonClientFactory implements PubsubClientFactory {
-    private static HttpRequestInitializer chainHttpRequestInitializer(
-        Credentials credential, HttpRequestInitializer httpRequestInitializer) {
-      if (credential == null) {
-        return httpRequestInitializer;
-      } else {
-        return new ChainingHttpRequestInitializer(
-            new HttpCredentialsAdapter(credential),
-            httpRequestInitializer);
-      }
-    }
-
-    @Override
-    public PubsubClient newClient(
-        @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
-        throws IOException {
-      Pubsub pubsub = new Builder(
-          Transport.getTransport(),
-          Transport.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())
-          .build();
-      return new PubsubJsonClient(timestampLabel, idLabel, pubsub);
-    }
-
-    @Override
-    public String getKind() {
-      return "Json";
-    }
-  }
-
-  /**
-   * Factory for creating Pubsub clients using Json transport.
-   */
-  public static final PubsubClientFactory FACTORY = new PubsubJsonClientFactory();
-
-  /**
-   * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time
-   * instead.
-   */
-  @Nullable
-  private final String timestampLabel;
-
-  /**
-   * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids.
-   */
-  @Nullable
-  private final String idLabel;
-
-  /**
-   * Underlying JSON transport.
-   */
-  private Pubsub pubsub;
-
-  @VisibleForTesting
-  PubsubJsonClient(
-      @Nullable String timestampLabel,
-      @Nullable String idLabel,
-      Pubsub pubsub) {
-    this.timestampLabel = timestampLabel;
-    this.idLabel = idLabel;
-    this.pubsub = pubsub;
-  }
-
-  @Override
-  public void close() {
-    // Nothing to close.
-  }
-
-  @Override
-  public int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages)
-      throws IOException {
-    List<PubsubMessage> pubsubMessages = new ArrayList<>(outgoingMessages.size());
-    for (OutgoingMessage outgoingMessage : outgoingMessages) {
-      PubsubMessage pubsubMessage = new PubsubMessage().encodeData(outgoingMessage.elementBytes);
-
-      Map<String, String> attributes = outgoingMessage.attributes;
-      if ((timestampLabel != null || idLabel != null) && attributes == null) {
-        attributes = new TreeMap<>();
-      }
-      if (attributes != null) {
-        pubsubMessage.setAttributes(attributes);
-      }
-
-      if (timestampLabel != null) {
-        attributes.put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch));
-      }
-
-      if (idLabel != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) {
-        attributes.put(idLabel, outgoingMessage.recordId);
-      }
-
-      pubsubMessages.add(pubsubMessage);
-    }
-    PublishRequest request = new PublishRequest().setMessages(pubsubMessages);
-    PublishResponse response = pubsub.projects()
-                                     .topics()
-                                     .publish(topic.getPath(), request)
-                                     .execute();
-    return response.getMessageIds().size();
-  }
-
-  @Override
-  public List<IncomingMessage> pull(
-      long requestTimeMsSinceEpoch,
-      SubscriptionPath subscription,
-      int batchSize,
-      boolean returnImmediately) throws IOException {
-    PullRequest request = new PullRequest()
-        .setReturnImmediately(returnImmediately)
-        .setMaxMessages(batchSize);
-    PullResponse response = pubsub.projects()
-                                  .subscriptions()
-                                  .pull(subscription.getPath(), request)
-                                  .execute();
-    if (response.getReceivedMessages() == null || response.getReceivedMessages().size() == 0) {
-      return ImmutableList.of();
-    }
-    List<IncomingMessage> incomingMessages = new ArrayList<>(response.getReceivedMessages().size());
-    for (ReceivedMessage message : response.getReceivedMessages()) {
-      PubsubMessage pubsubMessage = message.getMessage();
-      @Nullable Map<String, String> attributes = pubsubMessage.getAttributes();
-
-      // Payload.
-      byte[] elementBytes = pubsubMessage.decodeData();
-
-      // Timestamp.
-      long timestampMsSinceEpoch =
-          extractTimestamp(timestampLabel, message.getMessage().getPublishTime(), attributes);
-
-      // Ack id.
-      String ackId = message.getAckId();
-      checkState(!Strings.isNullOrEmpty(ackId));
-
-      // Record id, if any.
-      @Nullable String recordId = null;
-      if (idLabel != null && attributes != null) {
-        recordId = attributes.get(idLabel);
-      }
-      if (Strings.isNullOrEmpty(recordId)) {
-        // Fall back to the Pubsub provided message id.
-        recordId = pubsubMessage.getMessageId();
-      }
-
-      incomingMessages.add(new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch,
-                                               requestTimeMsSinceEpoch, ackId, recordId));
-    }
-
-    return incomingMessages;
-  }
-
-  @Override
-  public void acknowledge(SubscriptionPath subscription, List<String> ackIds) throws IOException {
-    AcknowledgeRequest request = new AcknowledgeRequest().setAckIds(ackIds);
-    pubsub.projects()
-          .subscriptions()
-          .acknowledge(subscription.getPath(), request)
-          .execute(); // ignore Empty result.
-  }
-
-  @Override
-  public void modifyAckDeadline(
-      SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds)
-      throws IOException {
-    ModifyAckDeadlineRequest request =
-        new ModifyAckDeadlineRequest().setAckIds(ackIds)
-                                      .setAckDeadlineSeconds(deadlineSeconds);
-    pubsub.projects()
-          .subscriptions()
-          .modifyAckDeadline(subscription.getPath(), request)
-          .execute(); // ignore Empty result.
-  }
-
-  @Override
-  public void createTopic(TopicPath topic) throws IOException {
-    pubsub.projects()
-          .topics()
-          .create(topic.getPath(), new Topic())
-          .execute(); // ignore Topic result.
-  }
-
-  @Override
-  public void deleteTopic(TopicPath topic) throws IOException {
-    pubsub.projects()
-          .topics()
-          .delete(topic.getPath())
-          .execute(); // ignore Empty result.
-  }
-
-  @Override
-  public List<TopicPath> listTopics(ProjectPath project) throws IOException {
-    ListTopicsResponse response = pubsub.projects()
-                                        .topics()
-                                        .list(project.getPath())
-                                        .execute();
-    if (response.getTopics() == null || response.getTopics().isEmpty()) {
-      return ImmutableList.of();
-    }
-    List<TopicPath> topics = new ArrayList<>(response.getTopics().size());
-    for (Topic topic : response.getTopics()) {
-      topics.add(topicPathFromPath(topic.getName()));
-    }
-    return topics;
-  }
-
-  @Override
-  public void createSubscription(
-      TopicPath topic, SubscriptionPath subscription,
-      int ackDeadlineSeconds) throws IOException {
-    Subscription request = new Subscription()
-        .setTopic(topic.getPath())
-        .setAckDeadlineSeconds(ackDeadlineSeconds);
-    pubsub.projects()
-          .subscriptions()
-          .create(subscription.getPath(), request)
-          .execute(); // ignore Subscription result.
-  }
-
-  @Override
-  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
-    pubsub.projects()
-          .subscriptions()
-          .delete(subscription.getPath())
-          .execute(); // ignore Empty result.
-  }
-
-  @Override
-  public List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
-      throws IOException {
-    ListSubscriptionsResponse response = pubsub.projects()
-                                               .subscriptions()
-                                               .list(project.getPath())
-                                               .execute();
-    if (response.getSubscriptions() == null || response.getSubscriptions().isEmpty()) {
-      return ImmutableList.of();
-    }
-    List<SubscriptionPath> subscriptions = new ArrayList<>(response.getSubscriptions().size());
-    for (Subscription subscription : response.getSubscriptions()) {
-      if (subscription.getTopic().equals(topic.getPath())) {
-        subscriptions.add(subscriptionPathFromPath(subscription.getName()));
-      }
-    }
-    return subscriptions;
-  }
-
-  @Override
-  public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException {
-    Subscription response = pubsub.projects().subscriptions().get(subscription.getPath()).execute();
-    return response.getAckDeadlineSeconds();
-  }
-
-  @Override
-  public boolean isEOF() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java
deleted file mode 100644
index 61479f9..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java
+++ /dev/null
@@ -1,436 +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.checkState;
-
-import com.google.api.client.util.Clock;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.options.PubsubOptions;
-
-/**
- * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for
- * testing {@link #publish}, {@link #pull}, {@link #acknowledge} and {@link #modifyAckDeadline}
- * methods. Relies on statics to mimic the Pubsub service, though we try to hide that.
- */
-public class PubsubTestClient extends PubsubClient implements Serializable {
-  /**
-   * Mimic the state of the simulated Pubsub 'service'.
-   *
-   * <p>Note that the {@link PubsubTestClientFactory} is serialized/deserialized even when running
-   * test pipelines. Meanwhile it is valid for multiple {@link PubsubTestClient}s to be created
-   * from the same client factory and run in parallel. Thus we can't enforce aliasing of the
-   * following data structures over all clients and must resort to a static.
-   */
-  private static class State {
-    /**
-     * True if has been primed for a test but not yet validated.
-     */
-    boolean isActive;
-
-    /**
-     * Publish mode only: Only publish calls for this topic are allowed.
-     */
-    @Nullable
-    TopicPath expectedTopic;
-
-    /**
-     * Publish mode only: Messages yet to seen in a {@link #publish} call.
-     */
-    @Nullable
-    Set<OutgoingMessage> remainingExpectedOutgoingMessages;
-
-    /**
-     * Publish mode only: Messages which should throw when first sent to simulate transient publish
-     * failure.
-     */
-    @Nullable
-    Set<OutgoingMessage> remainingFailingOutgoingMessages;
-
-    /**
-     * Pull mode only: Clock from which to get current time.
-     */
-    @Nullable
-    Clock clock;
-
-    /**
-     * Pull mode only: Only pull calls for this subscription are allowed.
-     */
-    @Nullable
-    SubscriptionPath expectedSubscription;
-
-    /**
-     * Pull mode only: Timeout to simulate.
-     */
-    int ackTimeoutSec;
-
-    /**
-     * Pull mode only: Messages waiting to be received by a {@link #pull} call.
-     */
-    @Nullable
-    List<IncomingMessage> remainingPendingIncomingMessages;
-
-    /**
-     * Pull mode only: Messages which have been returned from a {@link #pull} call and
-     * not yet ACKed by an {@link #acknowledge} call.
-     */
-    @Nullable
-    Map<String, IncomingMessage> pendingAckIncomingMessages;
-
-    /**
-     * Pull mode only: When above messages are due to have their ACK deadlines expire.
-     */
-    @Nullable
-    Map<String, Long> ackDeadline;
-  }
-
-  private static final State STATE = new State();
-
-  /** Closing the factory will validate all expected messages were processed. */
-  public interface PubsubTestClientFactory
-          extends PubsubClientFactory, Closeable, Serializable {
-  }
-
-  /**
-   * Return a factory for testing publishers. Only one factory may be in-flight at a time.
-   * The factory must be closed when the test is complete, at which point final validation will
-   * occur.
-   */
-  public static PubsubTestClientFactory createFactoryForPublish(
-      final TopicPath expectedTopic,
-      final Iterable<OutgoingMessage> expectedOutgoingMessages,
-      final Iterable<OutgoingMessage> failingOutgoingMessages) {
-    synchronized (STATE) {
-      checkState(!STATE.isActive, "Test still in flight");
-      STATE.expectedTopic = expectedTopic;
-      STATE.remainingExpectedOutgoingMessages = Sets.newHashSet(expectedOutgoingMessages);
-      STATE.remainingFailingOutgoingMessages = Sets.newHashSet(failingOutgoingMessages);
-      STATE.isActive = true;
-    }
-    return new PubsubTestClientFactory() {
-      @Override
-      public PubsubClient newClient(
-          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
-          throws IOException {
-        return new PubsubTestClient();
-      }
-
-      @Override
-      public String getKind() {
-        return "PublishTest";
-      }
-
-      @Override
-      public void close() {
-        synchronized (STATE) {
-          checkState(STATE.isActive, "No test still in flight");
-          checkState(STATE.remainingExpectedOutgoingMessages.isEmpty(),
-                     "Still waiting for %s messages to be published",
-                     STATE.remainingExpectedOutgoingMessages.size());
-          STATE.isActive = false;
-          STATE.remainingExpectedOutgoingMessages = null;
-        }
-      }
-    };
-  }
-
-  /**
-   * Return a factory for testing subscribers. Only one factory may be in-flight at a time.
-   * The factory must be closed when the test in complete
-   */
-  public static PubsubTestClientFactory createFactoryForPull(
-      final Clock clock,
-      final SubscriptionPath expectedSubscription,
-      final int ackTimeoutSec,
-      final Iterable<IncomingMessage> expectedIncomingMessages) {
-    synchronized (STATE) {
-      checkState(!STATE.isActive, "Test still in flight");
-      STATE.clock = clock;
-      STATE.expectedSubscription = expectedSubscription;
-      STATE.ackTimeoutSec = ackTimeoutSec;
-      STATE.remainingPendingIncomingMessages = Lists.newArrayList(expectedIncomingMessages);
-      STATE.pendingAckIncomingMessages = new HashMap<>();
-      STATE.ackDeadline = new HashMap<>();
-      STATE.isActive = true;
-    }
-    return new PubsubTestClientFactory() {
-      @Override
-      public PubsubClient newClient(
-          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
-          throws IOException {
-        return new PubsubTestClient();
-      }
-
-      @Override
-      public String getKind() {
-        return "PullTest";
-      }
-
-      @Override
-      public void close() {
-        synchronized (STATE) {
-          checkState(STATE.isActive, "No test still in flight");
-          checkState(STATE.remainingPendingIncomingMessages.isEmpty(),
-                     "Still waiting for %s messages to be pulled",
-                     STATE.remainingPendingIncomingMessages.size());
-          checkState(STATE.pendingAckIncomingMessages.isEmpty(),
-                     "Still waiting for %s messages to be ACKed",
-                     STATE.pendingAckIncomingMessages.size());
-          checkState(STATE.ackDeadline.isEmpty(),
-                     "Still waiting for %s messages to be ACKed",
-                     STATE.ackDeadline.size());
-          STATE.isActive = false;
-          STATE.remainingPendingIncomingMessages = null;
-          STATE.pendingAckIncomingMessages = null;
-          STATE.ackDeadline = null;
-        }
-      }
-    };
-  }
-
-  public static PubsubTestClientFactory createFactoryForCreateSubscription() {
-    return new PubsubTestClientFactory() {
-      int numCalls = 0;
-
-      @Override
-      public void close() throws IOException {
-        checkState(
-            numCalls == 1, "Expected exactly one subscription to be created, got %s", numCalls);
-      }
-
-      @Override
-      public PubsubClient newClient(
-          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
-          throws IOException {
-        return new PubsubTestClient() {
-          @Override
-          public void createSubscription(
-              TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds)
-              throws IOException {
-            checkState(numCalls == 0, "Expected at most one subscription to be created");
-            numCalls++;
-          }
-        };
-      }
-
-      @Override
-      public String getKind() {
-        return "CreateSubscriptionTest";
-      }
-    };
-  }
-
-  /**
-   * Return true if in pull mode.
-   */
-  private boolean inPullMode() {
-    checkState(STATE.isActive, "No test is active");
-    return STATE.expectedSubscription != null;
-  }
-
-  /**
-   * Return true if in publish mode.
-   */
-  private boolean inPublishMode() {
-    checkState(STATE.isActive, "No test is active");
-    return STATE.expectedTopic != null;
-  }
-
-  /**
-   * For subscription mode only:
-   * Track progression of time according to the {@link Clock} passed . This will simulate Pubsub
-   * expiring
-   * outstanding ACKs.
-   */
-  public void advance() {
-    synchronized (STATE) {
-      checkState(inPullMode(), "Can only advance in pull mode");
-      // Any messages who's ACKs timed out are available for re-pulling.
-      Iterator<Map.Entry<String, Long>> deadlineItr = STATE.ackDeadline.entrySet().iterator();
-      while (deadlineItr.hasNext()) {
-        Map.Entry<String, Long> entry = deadlineItr.next();
-        if (entry.getValue() <= STATE.clock.currentTimeMillis()) {
-          STATE.remainingPendingIncomingMessages.add(
-              STATE.pendingAckIncomingMessages.remove(entry.getKey()));
-          deadlineItr.remove();
-        }
-      }
-    }
-  }
-
-  @Override
-  public void close() {
-  }
-
-  @Override
-  public int publish(
-      TopicPath topic, List<OutgoingMessage> outgoingMessages) throws IOException {
-    synchronized (STATE) {
-      checkState(inPublishMode(), "Can only publish in publish mode");
-      checkState(topic.equals(STATE.expectedTopic), "Topic %s does not match expected %s", topic,
-                 STATE.expectedTopic);
-      for (OutgoingMessage outgoingMessage : outgoingMessages) {
-        if (STATE.remainingFailingOutgoingMessages.remove(outgoingMessage)) {
-          throw new RuntimeException("Simulating failure for " + outgoingMessage);
-        }
-        checkState(STATE.remainingExpectedOutgoingMessages.remove(outgoingMessage),
-                   "Unexpected outgoing message %s", outgoingMessage);
-      }
-      return outgoingMessages.size();
-    }
-  }
-
-  @Override
-  public List<IncomingMessage> pull(
-      long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize,
-      boolean returnImmediately) throws IOException {
-    synchronized (STATE) {
-      checkState(inPullMode(), "Can only pull in pull mode");
-      long now = STATE.clock.currentTimeMillis();
-      checkState(requestTimeMsSinceEpoch == now,
-                 "Simulated time %s does not match request time %s", now, requestTimeMsSinceEpoch);
-      checkState(subscription.equals(STATE.expectedSubscription),
-                 "Subscription %s does not match expected %s", subscription,
-                 STATE.expectedSubscription);
-      checkState(returnImmediately, "Pull only supported if returning immediately");
-
-      List<IncomingMessage> incomingMessages = new ArrayList<>();
-      Iterator<IncomingMessage> pendItr = STATE.remainingPendingIncomingMessages.iterator();
-      while (pendItr.hasNext()) {
-        IncomingMessage incomingMessage = pendItr.next();
-        pendItr.remove();
-        IncomingMessage incomingMessageWithRequestTime =
-            incomingMessage.withRequestTime(requestTimeMsSinceEpoch);
-        incomingMessages.add(incomingMessageWithRequestTime);
-        STATE.pendingAckIncomingMessages.put(incomingMessageWithRequestTime.ackId,
-                                             incomingMessageWithRequestTime);
-        STATE.ackDeadline.put(incomingMessageWithRequestTime.ackId,
-                              requestTimeMsSinceEpoch + STATE.ackTimeoutSec * 1000);
-        if (incomingMessages.size() >= batchSize) {
-          break;
-        }
-      }
-      return incomingMessages;
-    }
-  }
-
-  @Override
-  public void acknowledge(
-      SubscriptionPath subscription,
-      List<String> ackIds) throws IOException {
-    synchronized (STATE) {
-      checkState(inPullMode(), "Can only acknowledge in pull mode");
-      checkState(subscription.equals(STATE.expectedSubscription),
-                 "Subscription %s does not match expected %s", subscription,
-                 STATE.expectedSubscription);
-
-      for (String ackId : ackIds) {
-        checkState(STATE.ackDeadline.remove(ackId) != null,
-                   "No message with ACK id %s is waiting for an ACK", ackId);
-        checkState(STATE.pendingAckIncomingMessages.remove(ackId) != null,
-                   "No message with ACK id %s is waiting for an ACK", ackId);
-      }
-    }
-  }
-
-  @Override
-  public void modifyAckDeadline(
-      SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds) throws IOException {
-    synchronized (STATE) {
-      checkState(inPullMode(), "Can only modify ack deadline in pull mode");
-      checkState(subscription.equals(STATE.expectedSubscription),
-                 "Subscription %s does not match expected %s", subscription,
-                 STATE.expectedSubscription);
-
-      for (String ackId : ackIds) {
-        if (deadlineSeconds > 0) {
-          checkState(STATE.ackDeadline.remove(ackId) != null,
-                     "No message with ACK id %s is waiting for an ACK", ackId);
-          checkState(STATE.pendingAckIncomingMessages.containsKey(ackId),
-                     "No message with ACK id %s is waiting for an ACK", ackId);
-          STATE.ackDeadline.put(ackId, STATE.clock.currentTimeMillis() + deadlineSeconds * 1000);
-        } else {
-          checkState(STATE.ackDeadline.remove(ackId) != null,
-                     "No message with ACK id %s is waiting for an ACK", ackId);
-          IncomingMessage message = STATE.pendingAckIncomingMessages.remove(ackId);
-          checkState(message != null, "No message with ACK id %s is waiting for an ACK", ackId);
-          STATE.remainingPendingIncomingMessages.add(message);
-        }
-      }
-    }
-  }
-
-  @Override
-  public void createTopic(TopicPath topic) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void deleteTopic(TopicPath topic) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public List<TopicPath> listTopics(ProjectPath project) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void createSubscription(
-      TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public List<SubscriptionPath> listSubscriptions(
-      ProjectPath project, TopicPath topic) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException {
-    synchronized (STATE) {
-      return STATE.ackTimeoutSec;
-    }
-  }
-
-  @Override
-  public boolean isEOF() {
-    synchronized (STATE) {
-      checkState(inPullMode(), "Can only check EOF in pull mode");
-      return STATE.remainingPendingIncomingMessages.isEmpty();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/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
index 1edfa1d..80c093b 100644
--- 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
@@ -107,8 +107,7 @@ public class Transport {
   /**
    * Returns a Pubsub client builder using the specified {@link PubsubOptions}.
    *
-   * @deprecated Use an appropriate
-   * {@link org.apache.beam.sdk.util.PubsubClient.PubsubClientFactory}
+   * @deprecated Use an appropriate org.apache.beam.sdk.util.PubsubClient.PubsubClientFactory
    */
   @Deprecated
   public static Pubsub.Builder

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
deleted file mode 100644
index c996409..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.io;
-
-import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
-import static org.hamcrest.Matchers.hasItem;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-
-import java.util.Set;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
-import org.apache.beam.sdk.testing.UsesUnboundedPCollections;
-import org.apache.beam.sdk.testing.ValidatesRunner;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
-import org.joda.time.Duration;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for PubsubIO Read and Write transforms.
- */
-@RunWith(JUnit4.class)
-public class PubsubIOTest {
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  @Test
-  public void testPubsubIOGetName() {
-    assertEquals("PubsubIO.Read",
-        PubsubIO.<String>read().topic("projects/myproject/topics/mytopic").getName());
-    assertEquals("PubsubIO.Write",
-        PubsubIO.<String>write().topic("projects/myproject/topics/mytopic").getName());
-  }
-
-  @Test
-  public void testTopicValidationSuccess() throws Exception {
-    PubsubIO.<String>read().topic("projects/my-project/topics/abc");
-    PubsubIO.<String>read().topic("projects/my-project/topics/ABC");
-    PubsubIO.<String>read().topic("projects/my-project/topics/AbC-DeF");
-    PubsubIO.<String>read().topic("projects/my-project/topics/AbC-1234");
-    PubsubIO.<String>read().topic("projects/my-project/topics/AbC-1234-_.~%+-_.~%+-_.~%+-abc");
-    PubsubIO.<String>read().topic(new StringBuilder()
-        .append("projects/my-project/topics/A-really-long-one-")
-        .append("111111111111111111111111111111111111111111111111111111111111111111111111111111111")
-        .append("111111111111111111111111111111111111111111111111111111111111111111111111111111111")
-        .append("11111111111111111111111111111111111111111111111111111111111111111111111111")
-        .toString());
-  }
-
-  @Test
-  public void testTopicValidationBadCharacter() throws Exception {
-    thrown.expect(IllegalArgumentException.class);
-    PubsubIO.<String>read().topic("projects/my-project/topics/abc-*-abc");
-  }
-
-  @Test
-  public void testTopicValidationTooLong() throws Exception {
-    thrown.expect(IllegalArgumentException.class);
-    PubsubIO.<String>read().topic(new StringBuilder().append
-        ("projects/my-project/topics/A-really-long-one-")
-        .append("111111111111111111111111111111111111111111111111111111111111111111111111111111111")
-        .append("111111111111111111111111111111111111111111111111111111111111111111111111111111111")
-        .append("1111111111111111111111111111111111111111111111111111111111111111111111111111")
-        .toString());
-  }
-
-  @Test
-  public void testReadTopicDisplayData() {
-    String topic = "projects/project/topics/topic";
-    String subscription = "projects/project/subscriptions/subscription";
-    Duration maxReadTime = Duration.standardMinutes(5);
-    PubsubIO.Read<String> read = PubsubIO.<String>read()
-        .topic(StaticValueProvider.of(topic))
-        .timestampLabel("myTimestamp")
-        .idLabel("myId");
-
-    DisplayData displayData = DisplayData.from(read);
-
-    assertThat(displayData, hasDisplayItem("topic", topic));
-    assertThat(displayData, hasDisplayItem("timestampLabel", "myTimestamp"));
-    assertThat(displayData, hasDisplayItem("idLabel", "myId"));
-  }
-
-  @Test
-  public void testReadSubscriptionDisplayData() {
-    String topic = "projects/project/topics/topic";
-    String subscription = "projects/project/subscriptions/subscription";
-    Duration maxReadTime = Duration.standardMinutes(5);
-    PubsubIO.Read<String> read = PubsubIO.<String>read()
-        .subscription(StaticValueProvider.of(subscription))
-        .timestampLabel("myTimestamp")
-        .idLabel("myId");
-
-    DisplayData displayData = DisplayData.from(read);
-
-    assertThat(displayData, hasDisplayItem("subscription", subscription));
-    assertThat(displayData, hasDisplayItem("timestampLabel", "myTimestamp"));
-    assertThat(displayData, hasDisplayItem("idLabel", "myId"));
-  }
-
-  @Test
-  public void testNullTopic() {
-    String subscription = "projects/project/subscriptions/subscription";
-    PubsubIO.Read<String> read = PubsubIO.<String>read()
-        .subscription(StaticValueProvider.of(subscription));
-    assertNull(read.getTopic());
-    assertNotNull(read.getSubscription());
-    assertNotNull(DisplayData.from(read));
-  }
-
-  @Test
-  public void testNullSubscription() {
-    String topic = "projects/project/topics/topic";
-    PubsubIO.Read<String> read = PubsubIO.<String>read()
-        .topic(StaticValueProvider.of(topic));
-    assertNotNull(read.getTopic());
-    assertNull(read.getSubscription());
-    assertNotNull(DisplayData.from(read));
-  }
-
-  @Test
-  @Category({ValidatesRunner.class, UsesUnboundedPCollections.class})
-  public void testPrimitiveReadDisplayData() {
-    DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
-    Set<DisplayData> displayData;
-    PubsubIO.Read<String> read = PubsubIO.<String>read().withCoder(StringUtf8Coder.of());
-
-    // Reading from a subscription.
-    read = read.subscription("projects/project/subscriptions/subscription");
-    displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
-    assertThat("PubsubIO.Read should include the subscription in its primitive display data",
-        displayData, hasItem(hasDisplayItem("subscription")));
-
-    // Reading from a topic.
-    read = read.topic("projects/project/topics/topic");
-    displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
-    assertThat("PubsubIO.Read should include the topic in its primitive display data",
-        displayData, hasItem(hasDisplayItem("topic")));
-  }
-
-  @Test
-  public void testWriteDisplayData() {
-    String topic = "projects/project/topics/topic";
-    PubsubIO.Write<?> write = PubsubIO.<String>write()
-        .topic(topic)
-        .timestampLabel("myTimestamp")
-        .idLabel("myId");
-
-    DisplayData displayData = DisplayData.from(write);
-
-    assertThat(displayData, hasDisplayItem("topic", topic));
-    assertThat(displayData, hasDisplayItem("timestampLabel", "myTimestamp"));
-    assertThat(displayData, hasDisplayItem("idLabel", "myId"));
-  }
-
-  @Test
-  @Category(ValidatesRunner.class)
-  public void testPrimitiveWriteDisplayData() {
-    DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
-    PubsubIO.Write<?> write = PubsubIO.<String>write().topic("projects/project/topics/topic");
-
-    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(write);
-    assertThat("PubsubIO.Write should include the topic in its primitive display data",
-        displayData, hasItem(hasDisplayItem("topic")));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java
deleted file mode 100644
index 7a4be62..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.sdk.io;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.hash.Hashing;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.io.PubsubUnboundedSink.RecordIdMethod;
-import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
-import org.apache.beam.sdk.testing.CoderProperties;
-import org.apache.beam.sdk.testing.NeedsRunner;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.util.PubsubClient;
-import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage;
-import org.apache.beam.sdk.util.PubsubClient.TopicPath;
-import org.apache.beam.sdk.util.PubsubTestClient;
-import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Test PubsubUnboundedSink.
- */
-@RunWith(JUnit4.class)
-public class PubsubUnboundedSinkTest implements Serializable {
-  private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic");
-  private static final String DATA = "testData";
-  private static final Map<String, String> ATTRIBUTES =
-          ImmutableMap.<String, String>builder().put("a", "b").put("c", "d").build();
-  private static final long TIMESTAMP = 1234L;
-  private static final String TIMESTAMP_LABEL = "timestamp";
-  private static final String ID_LABEL = "id";
-  private static final int NUM_SHARDS = 10;
-
-  private static class Stamp extends DoFn<String, String> {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      c.outputWithTimestamp(c.element(), new Instant(TIMESTAMP));
-    }
-  }
-
-  private String getRecordId(String data) {
-    return Hashing.murmur3_128().hashBytes(data.getBytes()).toString();
-  }
-
-  @Rule
-  public transient TestPipeline p = TestPipeline.create();
-
-  @Test
-  public void saneCoder() throws Exception {
-    OutgoingMessage message = new OutgoingMessage(
-            DATA.getBytes(), ImmutableMap.<String, String>of(), TIMESTAMP, getRecordId(DATA));
-    CoderProperties.coderDecodeEncodeEqual(PubsubUnboundedSink.CODER, message);
-    CoderProperties.coderSerializable(PubsubUnboundedSink.CODER);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void sendOneMessage() throws IOException {
-    List<OutgoingMessage> outgoing =
-        ImmutableList.of(new OutgoingMessage(
-                DATA.getBytes(),
-                ATTRIBUTES,
-                TIMESTAMP, getRecordId(DATA)));
-    int batchSize = 1;
-    int batchBytes = 1;
-    try (PubsubTestClientFactory factory =
-             PubsubTestClient.createFactoryForPublish(TOPIC, outgoing,
-                                                      ImmutableList.<OutgoingMessage>of())) {
-      PubsubUnboundedSink<String> sink =
-          new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), StringUtf8Coder.of(),
-              TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes,
-              Duration.standardSeconds(2),
-              new SimpleFunction<String, PubsubIO.PubsubMessage>() {
-                @Override
-                public PubsubIO.PubsubMessage apply(String input) {
-                  return new PubsubIO.PubsubMessage(input.getBytes(), ATTRIBUTES);
-                }
-              },
-              RecordIdMethod.DETERMINISTIC);
-      p.apply(Create.of(ImmutableList.of(DATA)))
-       .apply(ParDo.of(new Stamp()))
-       .apply(sink);
-      p.run();
-    }
-    // The PubsubTestClientFactory will assert fail on close if the actual published
-    // message does not match the expected publish message.
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void sendMoreThanOneBatchByNumMessages() throws IOException {
-    List<OutgoingMessage> outgoing = new ArrayList<>();
-    List<String> data = new ArrayList<>();
-    int batchSize = 2;
-    int batchBytes = 1000;
-    for (int i = 0; i < batchSize * 10; i++) {
-      String str = String.valueOf(i);
-      outgoing.add(new OutgoingMessage(
-              str.getBytes(), ImmutableMap.<String, String>of(), TIMESTAMP, getRecordId(str)));
-      data.add(str);
-    }
-    try (PubsubTestClientFactory factory =
-             PubsubTestClient.createFactoryForPublish(TOPIC, outgoing,
-                                                      ImmutableList.<OutgoingMessage>of())) {
-      PubsubUnboundedSink<String> sink =
-          new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), StringUtf8Coder.of(),
-              TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes,
-              Duration.standardSeconds(2), null, RecordIdMethod.DETERMINISTIC);
-      p.apply(Create.of(data))
-       .apply(ParDo.of(new Stamp()))
-       .apply(sink);
-      p.run();
-    }
-    // The PubsubTestClientFactory will assert fail on close if the actual published
-    // message does not match the expected publish message.
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void sendMoreThanOneBatchByByteSize() throws IOException {
-    List<OutgoingMessage> outgoing = new ArrayList<>();
-    List<String> data = new ArrayList<>();
-    int batchSize = 100;
-    int batchBytes = 10;
-    int n = 0;
-    while (n < batchBytes * 10) {
-      StringBuilder sb = new StringBuilder();
-      for (int i = 0; i < batchBytes; i++) {
-        sb.append(String.valueOf(n));
-      }
-      String str = sb.toString();
-      outgoing.add(new OutgoingMessage(
-              str.getBytes(), ImmutableMap.<String, String>of(), TIMESTAMP, getRecordId(str)));
-      data.add(str);
-      n += str.length();
-    }
-    try (PubsubTestClientFactory factory =
-             PubsubTestClient.createFactoryForPublish(TOPIC, outgoing,
-                                                      ImmutableList.<OutgoingMessage>of())) {
-      PubsubUnboundedSink<String> sink =
-          new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC),
-              StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL,
-              NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2),
-              null, RecordIdMethod.DETERMINISTIC);
-      p.apply(Create.of(data))
-       .apply(ParDo.of(new Stamp()))
-       .apply(sink);
-      p.run();
-    }
-    // The PubsubTestClientFactory will assert fail on close if the actual published
-    // message does not match the expected publish message.
-  }
-
-  // TODO: We would like to test that failed Pubsub publish calls cause the already assigned
-  // (and random) record ids to be reused. However that can't be done without the test runnner
-  // supporting retrying bundles.
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java
deleted file mode 100644
index d9df2ca..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java
+++ /dev/null
@@ -1,411 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.sdk.io;
-
-import static junit.framework.TestCase.assertFalse;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.hasSize;
-import static org.hamcrest.Matchers.lessThanOrEqualTo;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-import com.google.api.client.util.Clock;
-import com.google.common.collect.ImmutableList;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.io.PubsubUnboundedSource.PubsubCheckpoint;
-import org.apache.beam.sdk.io.PubsubUnboundedSource.PubsubReader;
-import org.apache.beam.sdk.io.PubsubUnboundedSource.PubsubSource;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
-import org.apache.beam.sdk.testing.CoderProperties;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.PubsubClient;
-import org.apache.beam.sdk.util.PubsubClient.IncomingMessage;
-import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath;
-import org.apache.beam.sdk.util.PubsubClient.TopicPath;
-import org.apache.beam.sdk.util.PubsubTestClient;
-import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory;
-import org.joda.time.Instant;
-import org.junit.After;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Test PubsubUnboundedSource.
- */
-@RunWith(JUnit4.class)
-public class PubsubUnboundedSourceTest {
-  private static final SubscriptionPath SUBSCRIPTION =
-      PubsubClient.subscriptionPathFromName("testProject", "testSubscription");
-  private static final String DATA = "testData";
-  private static final long TIMESTAMP = 1234L;
-  private static final long REQ_TIME = 6373L;
-  private static final String TIMESTAMP_LABEL = "timestamp";
-  private static final String ID_LABEL = "id";
-  private static final String ACK_ID = "testAckId";
-  private static final String RECORD_ID = "testRecordId";
-  private static final int ACK_TIMEOUT_S = 60;
-
-  private AtomicLong now;
-  private Clock clock;
-  private PubsubTestClientFactory factory;
-  private PubsubSource<String> primSource;
-
-  @Rule
-  public TestPipeline p = TestPipeline.create();
-
-  private void setupOneMessage(Iterable<IncomingMessage> incoming) {
-    now = new AtomicLong(REQ_TIME);
-    clock = new Clock() {
-      @Override
-      public long currentTimeMillis() {
-        return now.get();
-      }
-    };
-    factory = PubsubTestClient.createFactoryForPull(clock, SUBSCRIPTION, ACK_TIMEOUT_S, incoming);
-    PubsubUnboundedSource<String> source =
-        new PubsubUnboundedSource<>(
-            clock, factory, null, null, StaticValueProvider.of(SUBSCRIPTION),
-            StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL, null);
-    primSource = new PubsubSource<>(source);
-  }
-
-  private void setupOneMessage() {
-    setupOneMessage(ImmutableList.of(
-        new IncomingMessage(DATA.getBytes(), null, TIMESTAMP, 0, ACK_ID, RECORD_ID)));
-  }
-
-  @After
-  public void after() throws IOException {
-    factory.close();
-    now = null;
-    clock = null;
-    primSource = null;
-    factory = null;
-  }
-
-  @Test
-  public void checkpointCoderIsSane() throws Exception {
-    setupOneMessage(ImmutableList.<IncomingMessage>of());
-    CoderProperties.coderSerializable(primSource.getCheckpointMarkCoder());
-    // Since we only serialize/deserialize the 'notYetReadIds', and we don't want to make
-    // equals on checkpoints ignore those fields, we'll test serialization and deserialization
-    // of checkpoints in multipleReaders below.
-  }
-
-  @Test
-  public void readOneMessage() throws IOException {
-    setupOneMessage();
-    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
-    // Read one message.
-    assertTrue(reader.start());
-    assertEquals(DATA, reader.getCurrent());
-    assertFalse(reader.advance());
-    // ACK the message.
-    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
-    checkpoint.finalizeCheckpoint();
-    reader.close();
-  }
-
-  @Test
-  public void timeoutAckAndRereadOneMessage() throws IOException {
-    setupOneMessage();
-    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
-    PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
-    assertTrue(reader.start());
-    assertEquals(DATA, reader.getCurrent());
-    // Let the ACK deadline for the above expire.
-    now.addAndGet(65 * 1000);
-    pubsubClient.advance();
-    // We'll now receive the same message again.
-    assertTrue(reader.advance());
-    assertEquals(DATA, reader.getCurrent());
-    assertFalse(reader.advance());
-    // Now ACK the message.
-    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
-    checkpoint.finalizeCheckpoint();
-    reader.close();
-  }
-
-  @Test
-  public void extendAck() throws IOException {
-    setupOneMessage();
-    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
-    PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
-    // Pull the first message but don't take a checkpoint for it.
-    assertTrue(reader.start());
-    assertEquals(DATA, reader.getCurrent());
-    // Extend the ack
-    now.addAndGet(55 * 1000);
-    pubsubClient.advance();
-    assertFalse(reader.advance());
-    // Extend the ack again
-    now.addAndGet(25 * 1000);
-    pubsubClient.advance();
-    assertFalse(reader.advance());
-    // Now ACK the message.
-    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
-    checkpoint.finalizeCheckpoint();
-    reader.close();
-  }
-
-  @Test
-  public void timeoutAckExtensions() throws IOException {
-    setupOneMessage();
-    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
-    PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
-    // Pull the first message but don't take a checkpoint for it.
-    assertTrue(reader.start());
-    assertEquals(DATA, reader.getCurrent());
-    // Extend the ack.
-    now.addAndGet(55 * 1000);
-    pubsubClient.advance();
-    assertFalse(reader.advance());
-    // Let the ack expire.
-    for (int i = 0; i < 3; i++) {
-      now.addAndGet(25 * 1000);
-      pubsubClient.advance();
-      assertFalse(reader.advance());
-    }
-    // Wait for resend.
-    now.addAndGet(25 * 1000);
-    pubsubClient.advance();
-    // Reread the same message.
-    assertTrue(reader.advance());
-    assertEquals(DATA, reader.getCurrent());
-    // Now ACK the message.
-    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
-    checkpoint.finalizeCheckpoint();
-    reader.close();
-  }
-
-  @Test
-  public void multipleReaders() throws IOException {
-    List<IncomingMessage> incoming = new ArrayList<>();
-    for (int i = 0; i < 2; i++) {
-      String data = String.format("data_%d", i);
-      String ackid = String.format("ackid_%d", i);
-      incoming.add(new IncomingMessage(data.getBytes(), null, TIMESTAMP, 0, ackid, RECORD_ID));
-    }
-    setupOneMessage(incoming);
-    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
-    // Consume two messages, only read one.
-    assertTrue(reader.start());
-    assertEquals("data_0", reader.getCurrent());
-
-    // Grab checkpoint.
-    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
-    checkpoint.finalizeCheckpoint();
-    assertEquals(1, checkpoint.notYetReadIds.size());
-    assertEquals("ackid_1", checkpoint.notYetReadIds.get(0));
-
-    // Read second message.
-    assertTrue(reader.advance());
-    assertEquals("data_1", reader.getCurrent());
-
-    // Restore from checkpoint.
-    byte[] checkpointBytes =
-        CoderUtils.encodeToByteArray(primSource.getCheckpointMarkCoder(), checkpoint);
-    checkpoint = CoderUtils.decodeFromByteArray(primSource.getCheckpointMarkCoder(),
-                                                checkpointBytes);
-    assertEquals(1, checkpoint.notYetReadIds.size());
-    assertEquals("ackid_1", checkpoint.notYetReadIds.get(0));
-
-    // Re-read second message.
-    reader = primSource.createReader(p.getOptions(), checkpoint);
-    assertTrue(reader.start());
-    assertEquals("data_1", reader.getCurrent());
-
-    // We are done.
-    assertFalse(reader.advance());
-
-    // ACK final message.
-    checkpoint = reader.getCheckpointMark();
-    checkpoint.finalizeCheckpoint();
-    reader.close();
-  }
-
-  private long messageNumToTimestamp(int messageNum) {
-    return TIMESTAMP + messageNum * 100;
-  }
-
-  @Test
-  public void readManyMessages() throws IOException {
-    Map<String, Integer> dataToMessageNum = new HashMap<>();
-
-    final int m = 97;
-    final int n = 10000;
-    List<IncomingMessage> incoming = new ArrayList<>();
-    for (int i = 0; i < n; i++) {
-      // Make the messages timestamps slightly out of order.
-      int messageNum = ((i / m) * m) + (m - 1) - (i % m);
-      String data = String.format("data_%d", messageNum);
-      dataToMessageNum.put(data, messageNum);
-      String recid = String.format("recordid_%d", messageNum);
-      String ackId = String.format("ackid_%d", messageNum);
-      incoming.add(new IncomingMessage(data.getBytes(), null, messageNumToTimestamp(messageNum), 0,
-                                       ackId, recid));
-    }
-    setupOneMessage(incoming);
-
-    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
-    PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
-
-    for (int i = 0; i < n; i++) {
-      if (i == 0) {
-        assertTrue(reader.start());
-      } else {
-        assertTrue(reader.advance());
-      }
-      // We'll checkpoint and ack within the 2min limit.
-      now.addAndGet(30);
-      pubsubClient.advance();
-      String data = reader.getCurrent();
-      Integer messageNum = dataToMessageNum.remove(data);
-      // No duplicate messages.
-      assertNotNull(messageNum);
-      // Preserve timestamp.
-      assertEquals(new Instant(messageNumToTimestamp(messageNum)), reader.getCurrentTimestamp());
-      // Preserve record id.
-      String recid = String.format("recordid_%d", messageNum);
-      assertArrayEquals(recid.getBytes(), reader.getCurrentRecordId());
-
-      if (i % 1000 == 999) {
-        // Estimated watermark can never get ahead of actual outstanding messages.
-        long watermark = reader.getWatermark().getMillis();
-        long minOutstandingTimestamp = Long.MAX_VALUE;
-        for (Integer outstandingMessageNum : dataToMessageNum.values()) {
-          minOutstandingTimestamp =
-              Math.min(minOutstandingTimestamp, messageNumToTimestamp(outstandingMessageNum));
-        }
-        assertThat(watermark, lessThanOrEqualTo(minOutstandingTimestamp));
-        // Ack messages, but only every other finalization.
-        PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
-        if (i % 2000 == 1999) {
-          checkpoint.finalizeCheckpoint();
-        }
-      }
-    }
-    // We are done.
-    assertFalse(reader.advance());
-    // We saw each message exactly once.
-    assertTrue(dataToMessageNum.isEmpty());
-    reader.close();
-  }
-
-  @Test
-  public void noSubscriptionSplitIntoBundlesGeneratesSubscription() throws Exception {
-    TopicPath topicPath = PubsubClient.topicPathFromName("my_project", "my_topic");
-    factory = PubsubTestClient.createFactoryForCreateSubscription();
-    PubsubUnboundedSource<String> source =
-        new PubsubUnboundedSource<>(
-            factory,
-            StaticValueProvider.of(PubsubClient.projectPathFromId("my_project")),
-            StaticValueProvider.of(topicPath),
-            null,
-            StringUtf8Coder.of(),
-            null,
-            null,
-            null);
-    assertThat(source.getSubscription(), nullValue());
-
-    assertThat(source.getSubscription(), nullValue());
-
-    PipelineOptions options = PipelineOptionsFactory.create();
-    List<PubsubSource<String>> splits =
-        (new PubsubSource<>(source)).generateInitialSplits(3, options);
-    // We have at least one returned split
-    assertThat(splits, hasSize(greaterThan(0)));
-    for (PubsubSource<String> split : splits) {
-      // Each split is equal
-      assertThat(split, equalTo(splits.get(0)));
-    }
-
-    assertThat(splits.get(0).subscriptionPath, not(nullValue()));
-  }
-
-  @Test
-  public void noSubscriptionNoSplitGeneratesSubscription() throws Exception {
-    TopicPath topicPath = PubsubClient.topicPathFromName("my_project", "my_topic");
-    factory = PubsubTestClient.createFactoryForCreateSubscription();
-    PubsubUnboundedSource<String> source =
-        new PubsubUnboundedSource<>(
-            factory,
-            StaticValueProvider.of(PubsubClient.projectPathFromId("my_project")),
-            StaticValueProvider.of(topicPath),
-            null,
-            StringUtf8Coder.of(),
-            null,
-            null,
-            null);
-    assertThat(source.getSubscription(), nullValue());
-
-    assertThat(source.getSubscription(), nullValue());
-
-    PipelineOptions options = PipelineOptionsFactory.create();
-    PubsubSource<String> actualSource = new PubsubSource<>(source);
-    PubsubReader<String> reader = actualSource.createReader(options, null);
-    SubscriptionPath createdSubscription = reader.subscription;
-    assertThat(createdSubscription, not(nullValue()));
-
-    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
-    assertThat(checkpoint.subscriptionPath, equalTo(createdSubscription.getPath()));
-
-    checkpoint.finalizeCheckpoint();
-    PubsubCheckpoint<String> deserCheckpoint =
-        CoderUtils.clone(actualSource.getCheckpointMarkCoder(), checkpoint);
-    assertThat(checkpoint.subscriptionPath, not(nullValue()));
-    assertThat(checkpoint.subscriptionPath, equalTo(deserCheckpoint.subscriptionPath));
-
-    PubsubReader<String> readerFromOriginal = actualSource.createReader(options, checkpoint);
-    PubsubReader<String> readerFromDeser = actualSource.createReader(options, deserCheckpoint);
-
-    assertThat(readerFromOriginal.subscription, equalTo(createdSubscription));
-    assertThat(readerFromDeser.subscription, equalTo(createdSubscription));
-  }
-
-  /**
-   * Tests that checkpoints finalized after the reader is closed succeed.
-   */
-  @Test
-  public void closeWithActiveCheckpoints() throws Exception {
-    setupOneMessage();
-    PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
-    reader.start();
-    PubsubCheckpoint<String> checkpoint = reader.getCheckpointMark();
-    reader.close();
-    checkpoint.finalizeCheckpoint();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java
deleted file mode 100644
index 1a99d38..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java
+++ /dev/null
@@ -1,189 +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 com.google.common.collect.ImmutableMap;
-import java.util.Map;
-import org.apache.beam.sdk.util.PubsubClient.ProjectPath;
-import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath;
-import org.apache.beam.sdk.util.PubsubClient.TopicPath;
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for helper classes and methods in PubsubClient.
- */
-@RunWith(JUnit4.class)
-public class PubsubClientTest {
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  //
-  // Timestamp handling
-  //
-
-  private long parse(String timestamp) {
-    Map<String, String> map = ImmutableMap.of("myLabel", timestamp);
-    return PubsubClient.extractTimestamp("myLabel", null, map);
-  }
-
-  private void roundTripRfc339(String timestamp) {
-    assertEquals(Instant.parse(timestamp).getMillis(), parse(timestamp));
-  }
-
-  private void truncatedRfc339(String timestamp, String truncatedTimestmap) {
-    assertEquals(Instant.parse(truncatedTimestmap).getMillis(), parse(timestamp));
-  }
-
-  @Test
-  public void noTimestampLabelReturnsPubsubPublish() {
-    final long time = 987654321L;
-    long timestamp = PubsubClient.extractTimestamp(null, String.valueOf(time), null);
-    assertEquals(time, timestamp);
-  }
-
-  @Test
-  public void noTimestampLabelAndInvalidPubsubPublishThrowsError() {
-    thrown.expect(NumberFormatException.class);
-    PubsubClient.extractTimestamp(null, "not-a-date", null);
-  }
-
-  @Test
-  public void timestampLabelWithNullAttributesThrowsError() {
-    thrown.expect(RuntimeException.class);
-    thrown.expectMessage("PubSub message is missing a value for timestamp label myLabel");
-    PubsubClient.extractTimestamp("myLabel", null, null);
-  }
-
-  @Test
-  public void timestampLabelSetWithMissingAttributeThrowsError() {
-    thrown.expect(RuntimeException.class);
-    thrown.expectMessage("PubSub message is missing a value for timestamp label myLabel");
-    Map<String, String> map = ImmutableMap.of("otherLabel", "whatever");
-    PubsubClient.extractTimestamp("myLabel", null, map);
-  }
-
-  @Test
-  public void timestampLabelParsesMillisecondsSinceEpoch() {
-    long time = 1446162101123L;
-    Map<String, String> map = ImmutableMap.of("myLabel", String.valueOf(time));
-    long timestamp = PubsubClient.extractTimestamp("myLabel", null, map);
-    assertEquals(time, timestamp);
-  }
-
-  @Test
-  public void timestampLabelParsesRfc3339Seconds() {
-    roundTripRfc339("2015-10-29T23:41:41Z");
-  }
-
-  @Test
-  public void timestampLabelParsesRfc3339Tenths() {
-    roundTripRfc339("2015-10-29T23:41:41.1Z");
-  }
-
-  @Test
-  public void timestampLabelParsesRfc3339Hundredths() {
-    roundTripRfc339("2015-10-29T23:41:41.12Z");
-  }
-
-  @Test
-  public void timestampLabelParsesRfc3339Millis() {
-    roundTripRfc339("2015-10-29T23:41:41.123Z");
-  }
-
-  @Test
-  public void timestampLabelParsesRfc3339Micros() {
-    // Note: micros part 456/1000 is dropped.
-    truncatedRfc339("2015-10-29T23:41:41.123456Z", "2015-10-29T23:41:41.123Z");
-  }
-
-  @Test
-  public void timestampLabelParsesRfc3339MicrosRounding() {
-    // Note: micros part 999/1000 is dropped, not rounded up.
-    truncatedRfc339("2015-10-29T23:41:41.123999Z", "2015-10-29T23:41:41.123Z");
-  }
-
-  @Test
-  public void timestampLabelWithInvalidFormatThrowsError() {
-    thrown.expect(NumberFormatException.class);
-    parse("not-a-timestamp");
-  }
-
-  @Test
-  public void timestampLabelWithInvalidFormat2ThrowsError() {
-    thrown.expect(NumberFormatException.class);
-    parse("null");
-  }
-
-  @Test
-  public void timestampLabelWithInvalidFormat3ThrowsError() {
-    thrown.expect(NumberFormatException.class);
-    parse("2015-10");
-  }
-
-  @Test
-  public void timestampLabelParsesRfc3339WithSmallYear() {
-    // Google and JodaTime agree on dates after 1582-10-15, when the Gregorian Calendar was adopted
-    // This is therefore a "small year" until this difference is reconciled.
-    roundTripRfc339("1582-10-15T01:23:45.123Z");
-  }
-
-  @Test
-  public void timestampLabelParsesRfc3339WithLargeYear() {
-    // Year 9999 in range.
-    roundTripRfc339("9999-10-29T23:41:41.123999Z");
-  }
-
-  @Test
-  public void timestampLabelRfc3339WithTooLargeYearThrowsError() {
-    thrown.expect(NumberFormatException.class);
-    // Year 10000 out of range.
-    parse("10000-10-29T23:41:41.123999Z");
-  }
-
-  //
-  // Paths
-  //
-
-  @Test
-  public void projectPathFromIdWellFormed() {
-    ProjectPath path = PubsubClient.projectPathFromId("test");
-    assertEquals("projects/test", path.getPath());
-  }
-
-  @Test
-  public void subscriptionPathFromNameWellFormed() {
-    SubscriptionPath path = PubsubClient.subscriptionPathFromName("test", "something");
-    assertEquals("projects/test/subscriptions/something", path.getPath());
-    assertEquals("/subscriptions/test/something", path.getV1Beta1Path());
-  }
-
-  @Test
-  public void topicPathFromNameWellFormed() {
-    TopicPath path = PubsubClient.topicPathFromName("test", "something");
-    assertEquals("projects/test/topics/something", path.getPath());
-    assertEquals("/topics/test/something", path.getV1Beta1Path());
-  }
-}


[7/8] beam git commit: [BEAM-1722] Move PubsubIO into the google-cloud-platform module

Posted by dh...@apache.org.
[BEAM-1722] Move PubsubIO into the google-cloud-platform module


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

Branch: refs/heads/master
Commit: 5bcb8c579656bc2d6e4d2a8dd5dcb2a46875812f
Parents: 82f2f2c
Author: Isma�l Mej�a <ie...@apache.org>
Authored: Wed Apr 12 14:49:58 2017 +0200
Committer: Isma�l Mej�a <ie...@apache.org>
Committed: Wed Apr 12 14:49:58 2017 +0200

----------------------------------------------------------------------
 .../beam/examples/complete/game/GameStats.java  |    2 +-
 .../examples/complete/game/LeaderBoard.java     |    2 +-
 .../triggers/AfterWatermarkStateMachine.java    |   14 +-
 runners/google-cloud-dataflow-java/pom.xml      |    5 +-
 .../beam/runners/dataflow/DataflowRunner.java   |    4 +-
 sdks/java/core/pom.xml                          |   44 -
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 1016 ------------
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |  494 ------
 .../beam/sdk/io/PubsubUnboundedSource.java      | 1463 ------------------
 .../apache/beam/sdk/transforms/GroupByKey.java  |    4 +-
 .../transforms/windowing/AfterWatermark.java    |   14 +-
 .../org/apache/beam/sdk/util/PubsubClient.java  |  544 -------
 .../apache/beam/sdk/util/PubsubGrpcClient.java  |  424 -----
 .../apache/beam/sdk/util/PubsubJsonClient.java  |  317 ----
 .../apache/beam/sdk/util/PubsubTestClient.java  |  436 ------
 .../org/apache/beam/sdk/util/Transport.java     |    3 +-
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |  189 ---
 .../beam/sdk/io/PubsubUnboundedSinkTest.java    |  190 ---
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |  411 -----
 .../apache/beam/sdk/util/PubsubClientTest.java  |  189 ---
 .../beam/sdk/util/PubsubGrpcClientTest.java     |  207 ---
 .../beam/sdk/util/PubsubJsonClientTest.java     |  140 --
 .../beam/sdk/util/PubsubTestClientTest.java     |  114 --
 sdks/java/io/google-cloud-platform/pom.xml      |   44 +
 .../beam/sdk/io/gcp/pubsub/PubsubClient.java    |  544 +++++++
 .../sdk/io/gcp/pubsub/PubsubGrpcClient.java     |  424 +++++
 .../apache/beam/sdk/io/gcp/pubsub/PubsubIO.java | 1014 ++++++++++++
 .../sdk/io/gcp/pubsub/PubsubJsonClient.java     |  319 ++++
 .../sdk/io/gcp/pubsub/PubsubTestClient.java     |  436 ++++++
 .../sdk/io/gcp/pubsub/PubsubUnboundedSink.java  |  490 ++++++
 .../io/gcp/pubsub/PubsubUnboundedSource.java    | 1463 ++++++++++++++++++
 .../beam/sdk/io/gcp/pubsub/package-info.java    |   24 +
 .../beam/sdk/io/gcp/GcpApiSurfaceTest.java      |    5 +-
 .../sdk/io/gcp/pubsub/PubsubClientTest.java     |  189 +++
 .../sdk/io/gcp/pubsub/PubsubGrpcClientTest.java |  208 +++
 .../beam/sdk/io/gcp/pubsub/PubsubIOTest.java    |  189 +++
 .../sdk/io/gcp/pubsub/PubsubJsonClientTest.java |  139 ++
 .../sdk/io/gcp/pubsub/PubsubTestClientTest.java |  114 ++
 .../io/gcp/pubsub/PubsubUnboundedSinkTest.java  |  188 +++
 .../gcp/pubsub/PubsubUnboundedSourceTest.java   |  409 +++++
 40 files changed, 6218 insertions(+), 6207 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
index 93e8254..6874953 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
@@ -25,7 +25,7 @@ import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
index 35b586b..96f4291 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
@@ -28,7 +28,7 @@ import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java
index 0b12005..1b117d2 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java
@@ -31,18 +31,16 @@ import org.apache.beam.sdk.util.TimeDomain;
  * lower-bound, sometimes heuristically established, on event times that have been fully processed
  * by the pipeline.
  *
- * <p>For sources that provide non-heuristic watermarks (e.g.
- * {@link org.apache.beam.sdk.io.PubsubIO} when using arrival times as event times), the
- * watermark is a strict guarantee that no data with an event time earlier than
+ * <p>For sources that provide non-heuristic watermarks (e.g. PubsubIO when using arrival times as
+ * event times), the watermark is a strict guarantee that no data with an event time earlier than
  * that watermark will ever be observed in the pipeline. In this case, it's safe to assume that any
  * pane triggered by an {@code AfterWatermark} trigger with a reference point at or beyond the end
  * of the window will be the last pane ever for that window.
  *
- * <p>For sources that provide heuristic watermarks (e.g.
- * {@link org.apache.beam.sdk.io.PubsubIO} when using user-supplied event times), the
- * watermark itself becomes an <i>estimate</i> that no data with an event time earlier than that
- * watermark (i.e. "late data") will ever be observed in the pipeline. These heuristics can
- * often be quite accurate, but the chance of seeing late data for any given window is non-zero.
+ * <p>For sources that provide heuristic watermarks (e.g. PubsubIO when using user-supplied event
+ * times), the watermark itself becomes an <i>estimate</i> that no data with an event time earlier
+ * than that watermark (i.e. "late data") will ever be observed in the pipeline. These heuristics
+ * can often be quite accurate, but the chance of seeing late data for any given window is non-zero.
  * Thus, if absolute correctness over time is important to your use case, you may want to consider
  * using a trigger that accounts for late data. The default trigger,
  * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}, which fires

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/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 a57744c..96eced8 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -188,13 +188,12 @@
 
     <dependency>
       <groupId>org.apache.beam</groupId>
-      <artifactId>beam-runners-core-construction-java</artifactId>
+      <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
     </dependency>
 
     <dependency>
       <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
-      <scope>test</scope>
+      <artifactId>beam-runners-core-construction-java</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 8726635..684dc14 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -82,11 +82,11 @@ import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.io.BoundedSource;
 import org.apache.beam.sdk.io.FileBasedSink;
-import org.apache.beam.sdk.io.PubsubUnboundedSink;
-import org.apache.beam.sdk.io.PubsubUnboundedSource;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.io.Write;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSink;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSource;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
 import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index d117d5a..0ac40f4 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -140,40 +140,6 @@
     </dependency>
 
     <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-auth</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-core</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-netty</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-stub</artifactId>
-    </dependency>
-
-    <!-- grpc-all does not obey IWYU, so we need to exclude from compile
-         scope and depend on it at runtime. -->
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-all</artifactId>
-      <scope>runtime</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-protobuf</artifactId>
-      <scope>runtime</scope>
-    </dependency>
-
-    <dependency>
       <groupId>com.google.auth</groupId>
       <artifactId>google-auth-library-credentials</artifactId>
     </dependency>
@@ -184,16 +150,6 @@
     </dependency>
 
     <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-handler</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.api.grpc</groupId>
-      <artifactId>grpc-google-pubsub-v1</artifactId>
-    </dependency>
-
-    <dependency>
       <groupId>com.google.api-client</groupId>
       <artifactId>google-api-client</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
deleted file mode 100644
index 67ab2ec..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
+++ /dev/null
@@ -1,1016 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.io;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.options.PubsubOptions;
-import org.apache.beam.sdk.options.ValueProvider;
-import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
-import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.PubsubClient;
-import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage;
-import org.apache.beam.sdk.util.PubsubClient.ProjectPath;
-import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath;
-import org.apache.beam.sdk.util.PubsubClient.TopicPath;
-import org.apache.beam.sdk.util.PubsubJsonClient;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Read and Write {@link PTransform}s for Cloud Pub/Sub streams. These transforms create
- * and consume unbounded {@link PCollection PCollections}.
- *
- * <h3>Permissions</h3>
- *
- * <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
- * Beam pipeline. Please refer to the documentation of corresponding
- * {@link PipelineRunner PipelineRunners} for more details.
- */
-public class PubsubIO {
-
-  private static final Logger LOG = LoggerFactory.getLogger(PubsubIO.class);
-
-  /** Factory for creating pubsub client to manage transport. */
-  private static final PubsubClient.PubsubClientFactory FACTORY = PubsubJsonClient.FACTORY;
-
-  /**
-   * Project IDs must contain 6-63 lowercase letters, digits, or dashes.
-   * IDs must start with a letter and may not end with a dash.
-   * This regex isn't exact - this allows for patterns that would be rejected by
-   * the service, but this is sufficient for basic parsing of table references.
-   */
-  private static final Pattern PROJECT_ID_REGEXP =
-      Pattern.compile("[a-z][-a-z0-9:.]{4,61}[a-z0-9]");
-
-  private static final Pattern SUBSCRIPTION_REGEXP =
-      Pattern.compile("projects/([^/]+)/subscriptions/(.+)");
-
-  private static final Pattern TOPIC_REGEXP = Pattern.compile("projects/([^/]+)/topics/(.+)");
-
-  private static final Pattern V1BETA1_SUBSCRIPTION_REGEXP =
-      Pattern.compile("/subscriptions/([^/]+)/(.+)");
-
-  private static final Pattern V1BETA1_TOPIC_REGEXP = Pattern.compile("/topics/([^/]+)/(.+)");
-
-  private static final Pattern PUBSUB_NAME_REGEXP = Pattern.compile("[a-zA-Z][-._~%+a-zA-Z0-9]+");
-
-  private static final int PUBSUB_NAME_MIN_LENGTH = 3;
-  private static final int PUBSUB_NAME_MAX_LENGTH = 255;
-
-  private static final String SUBSCRIPTION_RANDOM_TEST_PREFIX = "_random/";
-  private static final String SUBSCRIPTION_STARTING_SIGNAL = "_starting_signal/";
-  private static final String TOPIC_DEV_NULL_TEST_NAME = "/topics/dev/null";
-
-  private static void validateProjectName(String project) {
-    Matcher match = PROJECT_ID_REGEXP.matcher(project);
-    if (!match.matches()) {
-      throw new IllegalArgumentException(
-          "Illegal project name specified in Pubsub subscription: " + project);
-    }
-  }
-
-  private static void validatePubsubName(String name) {
-    if (name.length() < PUBSUB_NAME_MIN_LENGTH) {
-      throw new IllegalArgumentException(
-          "Pubsub object name is shorter than 3 characters: " + name);
-    }
-    if (name.length() > PUBSUB_NAME_MAX_LENGTH) {
-      throw new IllegalArgumentException(
-          "Pubsub object name is longer than 255 characters: " + name);
-    }
-
-    if (name.startsWith("goog")) {
-      throw new IllegalArgumentException("Pubsub object name cannot start with goog: " + name);
-    }
-
-    Matcher match = PUBSUB_NAME_REGEXP.matcher(name);
-    if (!match.matches()) {
-      throw new IllegalArgumentException("Illegal Pubsub object name specified: " + name
-          + " Please see Javadoc for naming rules.");
-    }
-  }
-
-  /**
-   * Populate common {@link DisplayData} between Pubsub source and sink.
-   */
-  private static void populateCommonDisplayData(DisplayData.Builder builder,
-      String timestampLabel, String idLabel, ValueProvider<PubsubTopic> topic) {
-    builder
-        .addIfNotNull(DisplayData.item("timestampLabel", timestampLabel)
-            .withLabel("Timestamp Label Attribute"))
-        .addIfNotNull(DisplayData.item("idLabel", idLabel)
-            .withLabel("ID Label Attribute"));
-
-    if (topic != null) {
-      String topicString = topic.isAccessible() ? topic.get().asPath()
-          : topic.toString();
-      builder.add(DisplayData.item("topic", topicString)
-          .withLabel("Pubsub Topic"));
-    }
-  }
-
-  /**
-   * Class representing a Pub/Sub message. Each message contains a single message payload and
-   * a map of attached attributes.
-   */
-  public static class PubsubMessage {
-
-    private byte[] message;
-    private Map<String, String> attributes;
-
-    public PubsubMessage(byte[] message, Map<String, String> attributes) {
-      this.message = message;
-      this.attributes = attributes;
-    }
-
-    /**
-     * Returns the main PubSub message.
-     */
-    public byte[] getMessage() {
-      return message;
-    }
-
-    /**
-     * Returns the given attribute value. If not such attribute exists, returns null.
-     */
-    @Nullable
-    public String getAttribute(String attribute) {
-      checkNotNull(attribute, "attribute");
-      return attributes.get(attribute);
-    }
-
-    /**
-     * Returns the full map of attributes. This is an unmodifiable map.
-     */
-    public Map<String, String> getAttributeMap() {
-      return attributes;
-    }
-  }
-
-  /**
-   * Class representing a Cloud Pub/Sub Subscription.
-   */
-  public static class PubsubSubscription implements Serializable {
-
-    private enum Type {NORMAL, FAKE}
-
-    private final Type type;
-    private final String project;
-    private final String subscription;
-
-    private PubsubSubscription(Type type, String project, String subscription) {
-      this.type = type;
-      this.project = project;
-      this.subscription = subscription;
-    }
-
-    /**
-     * Creates a class representing a Pub/Sub subscription from the specified subscription path.
-     *
-     * <p>Cloud Pub/Sub subscription names should be of the form
-     * {@code projects/<project>/subscriptions/<subscription>}, where {@code <project>} is the name
-     * of the project the subscription belongs to. The {@code <subscription>} component must comply
-     * with the following requirements:
-     *
-     * <ul>
-     * <li>Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods
-     * ('.').</li>
-     * <li>Must be between 3 and 255 characters.</li>
-     * <li>Must begin with a letter.</li>
-     * <li>Must end with a letter or a number.</li>
-     * <li>Cannot begin with {@code 'goog'} prefix.</li>
-     * </ul>
-     */
-    public static PubsubSubscription fromPath(String path) {
-      if (path.startsWith(SUBSCRIPTION_RANDOM_TEST_PREFIX)
-          || path.startsWith(SUBSCRIPTION_STARTING_SIGNAL)) {
-        return new PubsubSubscription(Type.FAKE, "", path);
-      }
-
-      String projectName, subscriptionName;
-
-      Matcher v1beta1Match = V1BETA1_SUBSCRIPTION_REGEXP.matcher(path);
-      if (v1beta1Match.matches()) {
-        LOG.warn("Saw subscription in v1beta1 format. Subscriptions should be in the format "
-            + "projects/<project_id>/subscriptions/<subscription_name>");
-        projectName = v1beta1Match.group(1);
-        subscriptionName = v1beta1Match.group(2);
-      } else {
-        Matcher match = SUBSCRIPTION_REGEXP.matcher(path);
-        if (!match.matches()) {
-          throw new IllegalArgumentException("Pubsub subscription is not in "
-              + "projects/<project_id>/subscriptions/<subscription_name> format: " + path);
-        }
-        projectName = match.group(1);
-        subscriptionName = match.group(2);
-      }
-
-      validateProjectName(projectName);
-      validatePubsubName(subscriptionName);
-      return new PubsubSubscription(Type.NORMAL, projectName, subscriptionName);
-    }
-
-    /**
-     * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub
-     * v1beta1 API.
-     *
-     * @deprecated the v1beta1 API for Cloud Pub/Sub is deprecated.
-     */
-    @Deprecated
-    public String asV1Beta1Path() {
-      if (type == Type.NORMAL) {
-        return "/subscriptions/" + project + "/" + subscription;
-      } else {
-        return subscription;
-      }
-    }
-
-    /**
-     * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub
-     * v1beta2 API.
-     *
-     * @deprecated the v1beta2 API for Cloud Pub/Sub is deprecated.
-     */
-    @Deprecated
-    public String asV1Beta2Path() {
-      if (type == Type.NORMAL) {
-        return "projects/" + project + "/subscriptions/" + subscription;
-      } else {
-        return subscription;
-      }
-    }
-
-    /**
-     * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub
-     * API.
-     */
-    public String asPath() {
-      if (type == Type.NORMAL) {
-        return "projects/" + project + "/subscriptions/" + subscription;
-      } else {
-        return subscription;
-      }
-    }
-  }
-
-  /**
-   * Used to build a {@link ValueProvider} for {@link PubsubSubscription}.
-   */
-  private static class SubscriptionTranslator
-      implements SerializableFunction<String, PubsubSubscription> {
-
-    @Override
-    public PubsubSubscription apply(String from) {
-      return PubsubSubscription.fromPath(from);
-    }
-  }
-
-  /**
-   * Used to build a {@link ValueProvider} for {@link SubscriptionPath}.
-   */
-  private static class SubscriptionPathTranslator
-      implements SerializableFunction<PubsubSubscription, SubscriptionPath> {
-
-    @Override
-    public SubscriptionPath apply(PubsubSubscription from) {
-      return PubsubClient.subscriptionPathFromName(from.project, from.subscription);
-    }
-  }
-
-  /**
-   * Used to build a {@link ValueProvider} for {@link PubsubTopic}.
-   */
-  private static class TopicTranslator
-      implements SerializableFunction<String, PubsubTopic> {
-
-    @Override
-    public PubsubTopic apply(String from) {
-      return PubsubTopic.fromPath(from);
-    }
-  }
-
-  /**
-   * Used to build a {@link ValueProvider} for {@link TopicPath}.
-   */
-  private static class TopicPathTranslator
-      implements SerializableFunction<PubsubTopic, TopicPath> {
-
-    @Override
-    public TopicPath apply(PubsubTopic from) {
-      return PubsubClient.topicPathFromName(from.project, from.topic);
-    }
-  }
-
-  /**
-   * Used to build a {@link ValueProvider} for {@link ProjectPath}.
-   */
-  private static class ProjectPathTranslator
-      implements SerializableFunction<PubsubTopic, ProjectPath> {
-
-    @Override
-    public ProjectPath apply(PubsubTopic from) {
-      return PubsubClient.projectPathFromId(from.project);
-    }
-  }
-
-  /**
-   * Class representing a Cloud Pub/Sub Topic.
-   */
-  public static class PubsubTopic implements Serializable {
-
-    private enum Type {NORMAL, FAKE}
-
-    private final Type type;
-    private final String project;
-    private final String topic;
-
-    private PubsubTopic(Type type, String project, String topic) {
-      this.type = type;
-      this.project = project;
-      this.topic = topic;
-    }
-
-    /**
-     * Creates a class representing a Cloud Pub/Sub topic from the specified topic path.
-     *
-     * <p>Cloud Pub/Sub topic names should be of the form
-     * {@code /topics/<project>/<topic>}, where {@code <project>} is the name of
-     * the publishing project. The {@code <topic>} component must comply with
-     * the following requirements:
-     *
-     * <ul>
-     * <li>Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods
-     * ('.').</li>
-     * <li>Must be between 3 and 255 characters.</li>
-     * <li>Must begin with a letter.</li>
-     * <li>Must end with a letter or a number.</li>
-     * <li>Cannot begin with 'goog' prefix.</li>
-     * </ul>
-     */
-    public static PubsubTopic fromPath(String path) {
-      if (path.equals(TOPIC_DEV_NULL_TEST_NAME)) {
-        return new PubsubTopic(Type.FAKE, "", path);
-      }
-
-      String projectName, topicName;
-
-      Matcher v1beta1Match = V1BETA1_TOPIC_REGEXP.matcher(path);
-      if (v1beta1Match.matches()) {
-        LOG.warn("Saw topic in v1beta1 format.  Topics should be in the format "
-            + "projects/<project_id>/topics/<topic_name>");
-        projectName = v1beta1Match.group(1);
-        topicName = v1beta1Match.group(2);
-      } else {
-        Matcher match = TOPIC_REGEXP.matcher(path);
-        if (!match.matches()) {
-          throw new IllegalArgumentException(
-              "Pubsub topic is not in projects/<project_id>/topics/<topic_name> format: " + path);
-        }
-        projectName = match.group(1);
-        topicName = match.group(2);
-      }
-
-      validateProjectName(projectName);
-      validatePubsubName(topicName);
-      return new PubsubTopic(Type.NORMAL, projectName, topicName);
-    }
-
-    /**
-     * Returns the string representation of this topic as a path used in the Cloud Pub/Sub
-     * v1beta1 API.
-     *
-     * @deprecated the v1beta1 API for Cloud Pub/Sub is deprecated.
-     */
-    @Deprecated
-    public String asV1Beta1Path() {
-      if (type == Type.NORMAL) {
-        return "/topics/" + project + "/" + topic;
-      } else {
-        return topic;
-      }
-    }
-
-    /**
-     * Returns the string representation of this topic as a path used in the Cloud Pub/Sub
-     * v1beta2 API.
-     *
-     * @deprecated the v1beta2 API for Cloud Pub/Sub is deprecated.
-     */
-    @Deprecated
-    public String asV1Beta2Path() {
-      if (type == Type.NORMAL) {
-        return "projects/" + project + "/topics/" + topic;
-      } else {
-        return topic;
-      }
-    }
-
-    /**
-     * Returns the string representation of this topic as a path used in the Cloud Pub/Sub
-     * API.
-     */
-    public String asPath() {
-      if (type == Type.NORMAL) {
-        return "projects/" + project + "/topics/" + topic;
-      } else {
-        return topic;
-      }
-    }
-  }
-
-  public static <T> Read<T> read() {
-    return new Read<>();
-  }
-
-  public static <T> Write<T> write() {
-    return new Write<>();
-  }
-
-  /**
-   * A {@link PTransform} that continuously reads from a Google Cloud Pub/Sub stream and
-   * returns a {@link PCollection} of {@link String Strings} containing the items from
-   * the stream.
-   */
-  public static class Read<T> extends PTransform<PBegin, PCollection<T>> {
-
-    /** The Cloud Pub/Sub topic to read from. */
-    @Nullable
-    private final ValueProvider<PubsubTopic> topic;
-
-    /** The Cloud Pub/Sub subscription to read from. */
-    @Nullable
-    private final ValueProvider<PubsubSubscription> subscription;
-
-    /** The name of the message attribute to read timestamps from. */
-    @Nullable
-    private final String timestampLabel;
-
-    /** The name of the message attribute to read unique message IDs from. */
-    @Nullable
-    private final String idLabel;
-
-    /** The coder used to decode each record. */
-    @Nullable
-    private final Coder<T> coder;
-
-    /** User function for parsing PubsubMessage object. */
-    SimpleFunction<PubsubMessage, T> parseFn;
-
-    private Read() {
-      this(null, null, null, null, null, null, null);
-    }
-
-    private Read(String name, ValueProvider<PubsubSubscription> subscription,
-        ValueProvider<PubsubTopic> topic, String timestampLabel, Coder<T> coder,
-        String idLabel,
-        SimpleFunction<PubsubMessage, T> parseFn) {
-      super(name);
-      this.subscription = subscription;
-      this.topic = topic;
-      this.timestampLabel = timestampLabel;
-      this.coder = coder;
-      this.idLabel = idLabel;
-      this.parseFn = parseFn;
-    }
-
-    /**
-     * Returns a transform that's like this one but reading from the
-     * given subscription.
-     *
-     * <p>See {@link PubsubIO.PubsubSubscription#fromPath(String)} for more details on the format
-     * of the {@code subscription} string.
-     *
-     * <p>Multiple readers reading from the same subscription will each receive
-     * some arbitrary portion of the data.  Most likely, separate readers should
-     * use their own subscriptions.
-     *
-     * <p>Does not modify this object.
-     */
-    public Read<T> subscription(String subscription) {
-      return subscription(StaticValueProvider.of(subscription));
-    }
-
-    /**
-     * Like {@code subscription()} but with a {@link ValueProvider}.
-     */
-    public Read<T> subscription(ValueProvider<String> subscription) {
-      if (subscription.isAccessible()) {
-        // Validate.
-        PubsubSubscription.fromPath(subscription.get());
-      }
-      return new Read<>(
-          name, NestedValueProvider.of(subscription, new SubscriptionTranslator()),
-          null /* reset topic to null */, timestampLabel, coder, idLabel, parseFn);
-    }
-
-    /**
-     * Creates and returns a transform for reading from a Cloud Pub/Sub topic. Mutually exclusive
-     * with {@link #subscription(String)}.
-     *
-     * <p>See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format
-     * of the {@code topic} string.
-     *
-     * <p>The Beam runner will start reading data published on this topic from the time the
-     * pipeline is started. Any data published on the topic before the pipeline is started will
-     * not be read by the runner.
-     */
-    public Read<T> topic(String topic) {
-      return topic(StaticValueProvider.of(topic));
-    }
-
-    /**
-     * Like {@code topic()} but with a {@link ValueProvider}.
-     */
-    public Read<T> topic(ValueProvider<String> topic) {
-      if (topic.isAccessible()) {
-        // Validate.
-        PubsubTopic.fromPath(topic.get());
-      }
-      return new Read<>(name, null /* reset subscription to null */,
-          NestedValueProvider.of(topic, new TopicTranslator()),
-          timestampLabel, coder, idLabel, parseFn);
-    }
-
-    /**
-     * Creates and returns a transform reading from Cloud Pub/Sub where record timestamps are
-     * expected to be provided as Pub/Sub message attributes. The {@code timestampLabel}
-     * parameter specifies the name of the attribute that contains the timestamp.
-     *
-     * <p>The timestamp value is expected to be represented in the attribute as either:
-     *
-     * <ul>
-     * <li>a numerical value representing the number of milliseconds since the Unix epoch. For
-     * example, if using the Joda time classes, {@link Instant#getMillis()} returns the correct
-     * value for this attribute.
-     * <li>a String in RFC 3339 format. For example, {@code 2015-10-29T23:41:41.123Z}. The
-     * sub-second component of the timestamp is optional, and digits beyond the first three
-     * (i.e., time units smaller than milliseconds) will be ignored.
-     * </ul>
-     *
-     * <p>If {@code timestampLabel} is not provided, the system will generate record timestamps
-     * the first time it sees each record. All windowing will be done relative to these
-     * timestamps.
-     *
-     * <p>By default, windows are emitted based on an estimate of when this source is likely
-     * done producing data for a given timestamp (referred to as the Watermark; see
-     * {@link AfterWatermark} for more details). Any late data will be handled by the trigger
-     * specified with the windowing strategy &ndash; by default it will be output immediately.
-     *
-     * <p>Note that the system can guarantee that no late data will ever be seen when it assigns
-     * timestamps by arrival time (i.e. {@code timestampLabel} is not provided).
-     *
-     * @see <a href="https://www.ietf.org/rfc/rfc3339.txt">RFC 3339</a>
-     */
-    public Read<T> timestampLabel(String timestampLabel) {
-      return new Read<>(
-          name, subscription, topic, timestampLabel, coder, idLabel,
-          parseFn);
-    }
-
-    /**
-     * Creates and returns a transform for reading from Cloud Pub/Sub where unique record
-     * identifiers are expected to be provided as Pub/Sub message attributes. The {@code idLabel}
-     * parameter specifies the attribute name. The value of the attribute can be any string
-     * that uniquely identifies this record.
-     *
-     * <p>Pub/Sub cannot guarantee that no duplicate data will be delivered on the Pub/Sub stream.
-     * If {@code idLabel} is not provided, Beam cannot guarantee that no duplicate data will
-     * be delivered, and deduplication of the stream will be strictly best effort.
-     */
-    public Read<T> idLabel(String idLabel) {
-      return new Read<>(
-          name, subscription, topic, timestampLabel, coder, idLabel,
-          parseFn);
-    }
-
-    /**
-     * Returns a transform that's like this one but that uses the given
-     * {@link Coder} to decode each record into a value of type {@code T}.
-     *
-     * <p>Does not modify this object.
-     */
-    public Read<T> withCoder(Coder<T> coder) {
-      return new Read<>(
-          name, subscription, topic, timestampLabel, coder, idLabel,
-          parseFn);
-    }
-
-    /**
-     * Causes the source to return a PubsubMessage that includes Pubsub attributes.
-     * The user must supply a parsing function to transform the PubsubMessage into an output type.
-     * A Coder for the output type T must be registered or set on the output via
-     * {@link PCollection#setCoder(Coder)}.
-     */
-    public Read<T> withAttributes(SimpleFunction<PubsubMessage, T> parseFn) {
-      return new Read<T>(
-          name, subscription, topic, timestampLabel, coder, idLabel,
-          parseFn);
-    }
-
-    @Override
-    public PCollection<T> expand(PBegin input) {
-      if (topic == null && subscription == null) {
-        throw new IllegalStateException("Need to set either the topic or the subscription for "
-            + "a PubsubIO.Read transform");
-      }
-      if (topic != null && subscription != null) {
-        throw new IllegalStateException("Can't set both the topic and the subscription for "
-            + "a PubsubIO.Read transform");
-      }
-      if (coder == null) {
-        throw new IllegalStateException("PubsubIO.Read requires that a coder be set using "
-            + "the withCoder method.");
-      }
-
-      @Nullable ValueProvider<ProjectPath> projectPath =
-          topic == null ? null : NestedValueProvider.of(topic, new ProjectPathTranslator());
-      @Nullable ValueProvider<TopicPath> topicPath =
-          topic == null ? null : NestedValueProvider.of(topic, new TopicPathTranslator());
-      @Nullable ValueProvider<SubscriptionPath> subscriptionPath =
-          subscription == null
-              ? null
-              : NestedValueProvider.of(subscription, new SubscriptionPathTranslator());
-      PubsubUnboundedSource<T> source = new PubsubUnboundedSource<T>(
-              FACTORY, projectPath, topicPath, subscriptionPath,
-              coder, timestampLabel, idLabel, parseFn);
-      return input.getPipeline().apply(source);
-    }
-
-    @Override
-    public void populateDisplayData(DisplayData.Builder builder) {
-      super.populateDisplayData(builder);
-      populateCommonDisplayData(builder, timestampLabel, idLabel, topic);
-
-      if (subscription != null) {
-        String subscriptionString = subscription.isAccessible()
-            ? subscription.get().asPath() : subscription.toString();
-        builder.add(DisplayData.item("subscription", subscriptionString)
-            .withLabel("Pubsub Subscription"));
-      }
-    }
-
-    @Override
-    protected Coder<T> getDefaultOutputCoder() {
-      return coder;
-    }
-
-    /**
-     * Get the topic being read from.
-     */
-    @Nullable
-    public PubsubTopic getTopic() {
-      return topic == null ? null : topic.get();
-    }
-
-    /**
-     * Get the {@link ValueProvider} for the topic being read from.
-     */
-    public ValueProvider<PubsubTopic> getTopicProvider() {
-      return topic;
-    }
-
-    /**
-     * Get the subscription being read from.
-     */
-    @Nullable
-    public PubsubSubscription getSubscription() {
-      return subscription == null ? null : subscription.get();
-    }
-
-    /**
-     * Get the {@link ValueProvider} for the subscription being read from.
-     */
-    public ValueProvider<PubsubSubscription> getSubscriptionProvider() {
-      return subscription;
-    }
-
-    /**
-     * Get the timestamp label.
-     */
-    @Nullable
-    public String getTimestampLabel() {
-      return timestampLabel;
-    }
-
-    /**
-     * Get the id label.
-     */
-    @Nullable
-    public String getIdLabel() {
-      return idLabel;
-    }
-
-
-    /**
-     * Get the {@link Coder} used for the transform's output.
-     */
-    @Nullable
-    public Coder<T> getCoder() {
-      return coder;
-    }
-
-    /**
-     * Get the parse function used for PubSub attributes.
-     */
-    @Nullable
-    public SimpleFunction<PubsubMessage, T> getPubSubMessageParseFn() {
-      return parseFn;
-    }
-
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /** Disallow construction of utility class. */
-  private PubsubIO() {}
-
-
-  /**
-   * A {@link PTransform} that writes an unbounded {@link PCollection} of {@link String Strings}
-   * to a Cloud Pub/Sub stream.
-   */
-  public static class Write<T> extends PTransform<PCollection<T>, PDone> {
-
-    /** The Cloud Pub/Sub topic to publish to. */
-    @Nullable
-    private final ValueProvider<PubsubTopic> topic;
-    /** The name of the message attribute to publish message timestamps in. */
-    @Nullable
-    private final String timestampLabel;
-    /** The name of the message attribute to publish unique message IDs in. */
-    @Nullable
-    private final String idLabel;
-    /** The input type Coder. */
-    private final Coder<T> coder;
-    /** The format function for input PubsubMessage objects. */
-    SimpleFunction<T, PubsubMessage> formatFn;
-
-    private Write() {
-      this(null, null, null, null, null, null);
-    }
-
-    private Write(
-        String name, ValueProvider<PubsubTopic> topic, String timestampLabel,
-        String idLabel, Coder<T> coder, SimpleFunction<T, PubsubMessage> formatFn) {
-      super(name);
-      this.topic = topic;
-      this.timestampLabel = timestampLabel;
-      this.idLabel = idLabel;
-      this.coder = coder;
-      this.formatFn = formatFn;
-    }
-
-    /**
-     * Creates a transform that publishes to the specified topic.
-     *
-     * <p>See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format of the
-     * {@code topic} string.
-     */
-    public Write<T> topic(String topic) {
-      return topic(StaticValueProvider.of(topic));
-    }
-
-    /**
-     * Like {@code topic()} but with a {@link ValueProvider}.
-     */
-    public Write<T> topic(ValueProvider<String> topic) {
-      return new Write<>(name, NestedValueProvider.of(topic, new TopicTranslator()),
-          timestampLabel, idLabel, coder, formatFn);
-    }
-
-    /**
-     * Creates a transform that writes to Pub/Sub, adds each record's timestamp to the published
-     * messages in an attribute with the specified name. The value of the attribute will be a number
-     * representing the number of milliseconds since the Unix epoch. For example, if using the Joda
-     * time classes, {@link Instant#Instant(long)} can be used to parse this value.
-     *
-     * <p>If the output from this sink is being read by another Beam pipeline, then
-     * {@link PubsubIO.Read#timestampLabel(String)} can be used to ensure the other source reads
-     * these timestamps from the appropriate attribute.
-     */
-    public Write<T> timestampLabel(String timestampLabel) {
-      return new Write<>(name, topic, timestampLabel, idLabel, coder, formatFn);
-    }
-
-    /**
-     * Creates a transform that writes to Pub/Sub, adding each record's unique identifier to the
-     * published messages in an attribute with the specified name. The value of the attribute is an
-     * opaque string.
-     *
-     * <p>If the the output from this sink is being read by another Beam pipeline, then
-     * {@link PubsubIO.Read#idLabel(String)} can be used to ensure that* the other source reads
-     * these unique identifiers from the appropriate attribute.
-     */
-    public Write<T> idLabel(String idLabel) {
-      return new Write<>(name, topic, timestampLabel, idLabel, coder, formatFn);
-    }
-
-    /**
-     * Returns a new transform that's like this one
-     * but that uses the given {@link Coder} to encode each of
-     * the elements of the input {@link PCollection} into an
-     * output record.
-     *
-     * <p>Does not modify this object.
-     */
-    public Write<T> withCoder(Coder<T> coder) {
-      return new Write<>(name, topic, timestampLabel, idLabel, coder, formatFn);
-    }
-
-    /**
-     * Used to write a PubSub message together with PubSub attributes. The user-supplied format
-     * function translates the input type T to a PubsubMessage object, which is used by the sink
-     * to separately set the PubSub message's payload and attributes.
-     */
-    public Write<T> withAttributes(SimpleFunction<T, PubsubMessage> formatFn) {
-      return new Write<T>(name, topic, timestampLabel, idLabel, coder, formatFn);
-    }
-
-    @Override
-    public PDone expand(PCollection<T> input) {
-      if (topic == null) {
-        throw new IllegalStateException("need to set the topic of a PubsubIO.Write transform");
-      }
-      switch (input.isBounded()) {
-        case BOUNDED:
-          input.apply(ParDo.of(new PubsubBoundedWriter()));
-          return PDone.in(input.getPipeline());
-        case UNBOUNDED:
-          return input.apply(new PubsubUnboundedSink<T>(
-              FACTORY,
-              NestedValueProvider.of(topic, new TopicPathTranslator()),
-              coder,
-              timestampLabel,
-              idLabel,
-              formatFn,
-              100 /* numShards */));
-      }
-      throw new RuntimeException(); // cases are exhaustive.
-    }
-
-    @Override
-    public void populateDisplayData(DisplayData.Builder builder) {
-      super.populateDisplayData(builder);
-      populateCommonDisplayData(builder, timestampLabel, idLabel, topic);
-    }
-
-    @Override
-    protected Coder<Void> getDefaultOutputCoder() {
-      return VoidCoder.of();
-    }
-
-    /**
-     * Returns the PubSub topic being written to.
-     */
-    @Nullable
-    public PubsubTopic getTopic() {
-      return (topic == null) ? null : topic.get();
-    }
-
-    /**
-     * Returns the {@link ValueProvider} for the topic being written to.
-     */
-    @Nullable
-    public ValueProvider<PubsubTopic> getTopicProvider() {
-      return topic;
-    }
-
-    /**
-     * Returns the timestamp label.
-     */
-    @Nullable
-    public String getTimestampLabel() {
-      return timestampLabel;
-    }
-
-    /**
-     * Returns the id label.
-     */
-    @Nullable
-    public String getIdLabel() {
-      return idLabel;
-    }
-
-    /**
-     * Returns the output coder.
-     */
-    @Nullable
-    public Coder<T> getCoder() {
-      return coder;
-    }
-
-    /**
-     * Returns the formatting function used if publishing attributes.
-     */
-    @Nullable
-    public SimpleFunction<T, PubsubMessage> getFormatFn() {
-      return formatFn;
-    }
-
-    /**
-     * Writer to Pubsub which batches messages from bounded collections.
-     *
-     * <p>Public so can be suppressed by runners.
-     */
-    public class PubsubBoundedWriter extends DoFn<T, Void> {
-
-      private static final int MAX_PUBLISH_BATCH_SIZE = 100;
-      private transient List<OutgoingMessage> output;
-      private transient PubsubClient pubsubClient;
-
-      @StartBundle
-      public void startBundle(Context c) throws IOException {
-        this.output = new ArrayList<>();
-        // NOTE: idLabel is ignored.
-        this.pubsubClient =
-            FACTORY.newClient(timestampLabel, null,
-                c.getPipelineOptions().as(PubsubOptions.class));
-      }
-
-      @ProcessElement
-      public void processElement(ProcessContext c) throws IOException {
-        byte[] payload = null;
-        Map<String, String> attributes = null;
-        if (formatFn != null) {
-          PubsubMessage message = formatFn.apply(c.element());
-          payload = message.getMessage();
-          attributes = message.getAttributeMap();
-        } else {
-          payload = CoderUtils.encodeToByteArray(getCoder(), c.element());
-        }
-        // NOTE: The record id is always null.
-        OutgoingMessage message =
-            new OutgoingMessage(payload, attributes, c.timestamp().getMillis(), null);
-        output.add(message);
-
-        if (output.size() >= MAX_PUBLISH_BATCH_SIZE) {
-          publish();
-        }
-      }
-
-      @FinishBundle
-      public void finishBundle(Context c) throws IOException {
-        if (!output.isEmpty()) {
-          publish();
-        }
-        output = null;
-        pubsubClient.close();
-        pubsubClient = null;
-      }
-
-      private void publish() throws IOException {
-        int n = pubsubClient.publish(
-            PubsubClient.topicPathFromName(getTopic().project, getTopic().topic),
-            output);
-        checkState(n == output.size());
-        output.clear();
-      }
-
-      @Override
-      public void populateDisplayData(DisplayData.Builder builder) {
-        super.populateDisplayData(builder);
-        builder.delegate(Write.this);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java
deleted file mode 100644
index 55605b3..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java
+++ /dev/null
@@ -1,494 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.sdk.io;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.hash.Hashing;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.MapCoder;
-import org.apache.beam.sdk.coders.NullableCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.io.PubsubIO.PubsubMessage;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.options.PubsubOptions;
-import org.apache.beam.sdk.options.ValueProvider;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
-import org.apache.beam.sdk.transforms.windowing.AfterFirst;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.PubsubClient;
-import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage;
-import org.apache.beam.sdk.util.PubsubClient.PubsubClientFactory;
-import org.apache.beam.sdk.util.PubsubClient.TopicPath;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.joda.time.Duration;
-
-/**
- * A PTransform which streams messages to Pubsub.
- * <ul>
- * <li>The underlying implementation is just a {@link GroupByKey} followed by a {@link ParDo} which
- * publishes as a side effect. (In the future we want to design and switch to a custom
- * {@code UnboundedSink} implementation so as to gain access to system watermark and
- * end-of-pipeline cleanup.)
- * <li>We try to send messages in batches while also limiting send latency.
- * <li>No stats are logged. Rather some counters are used to keep track of elements and batches.
- * <li>Though some background threads are used by the underlying netty system all actual Pubsub
- * calls are blocking. We rely on the underlying runner to allow multiple {@link DoFn} instances
- * to execute concurrently and hide latency.
- * <li>A failed bundle will cause messages to be resent. Thus we rely on the Pubsub consumer
- * to dedup messages.
- * </ul>
- */
-public class PubsubUnboundedSink<T> extends PTransform<PCollection<T>, PDone> {
-  /**
-   * Default maximum number of messages per publish.
-   */
-  private static final int DEFAULT_PUBLISH_BATCH_SIZE = 1000;
-
-  /**
-   * Default maximum size of a publish batch, in bytes.
-   */
-  private static final int DEFAULT_PUBLISH_BATCH_BYTES = 400000;
-
-  /**
-   * Default longest delay between receiving a message and pushing it to Pubsub.
-   */
-  private static final Duration DEFAULT_MAX_LATENCY = Duration.standardSeconds(2);
-
-  /**
-   * Coder for conveying outgoing messages between internal stages.
-   */
-  private static class OutgoingMessageCoder extends AtomicCoder<OutgoingMessage> {
-    private static final NullableCoder<String> RECORD_ID_CODER =
-        NullableCoder.of(StringUtf8Coder.of());
-    private static final NullableCoder<Map<String, String>> ATTRIBUTES_CODER =
-            NullableCoder.of(MapCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
-
-    @Override
-    public void encode(
-        OutgoingMessage value, OutputStream outStream, Context context)
-        throws CoderException, IOException {
-      ByteArrayCoder.of().encode(value.elementBytes, outStream, context.nested());
-      ATTRIBUTES_CODER.encode(value.attributes, outStream, context.nested());
-      BigEndianLongCoder.of().encode(value.timestampMsSinceEpoch, outStream, context.nested());
-      RECORD_ID_CODER.encode(value.recordId, outStream, context.nested());
-    }
-
-    @Override
-    public OutgoingMessage decode(
-        InputStream inStream, Context context) throws CoderException, IOException {
-      byte[] elementBytes = ByteArrayCoder.of().decode(inStream, context.nested());
-      Map<String, String> attributes = ATTRIBUTES_CODER.decode(inStream, context.nested());
-      long timestampMsSinceEpoch = BigEndianLongCoder.of().decode(inStream, context.nested());
-      @Nullable String recordId = RECORD_ID_CODER.decode(inStream, context.nested());
-      return new OutgoingMessage(elementBytes, attributes, timestampMsSinceEpoch, recordId);
-    }
-  }
-
-  @VisibleForTesting
-  static final Coder<OutgoingMessage> CODER = new OutgoingMessageCoder();
-
-  // ================================================================================
-  // RecordIdMethod
-  // ================================================================================
-
-  /**
-   * Specify how record ids are to be generated.
-   */
-  @VisibleForTesting
-  enum RecordIdMethod {
-    /** Leave null. */
-    NONE,
-    /** Generate randomly. */
-    RANDOM,
-    /** Generate deterministically. For testing only. */
-    DETERMINISTIC
-  }
-
-  // ================================================================================
-  // ShardFn
-  // ================================================================================
-
-  /**
-   * Convert elements to messages and shard them.
-   */
-  private static class ShardFn<T> extends DoFn<T, KV<Integer, OutgoingMessage>> {
-    private final Counter elementCounter = Metrics.counter(ShardFn.class, "elements");
-    private final Coder<T> elementCoder;
-    private final int numShards;
-    private final RecordIdMethod recordIdMethod;
-    private final SimpleFunction<T, PubsubMessage> formatFn;
-
-    ShardFn(Coder<T> elementCoder, int numShards,
-            SimpleFunction<T, PubsubIO.PubsubMessage> formatFn, RecordIdMethod recordIdMethod) {
-      this.elementCoder = elementCoder;
-      this.numShards = numShards;
-      this.formatFn = formatFn;
-      this.recordIdMethod = recordIdMethod;
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext c) throws Exception {
-      elementCounter.inc();
-      byte[] elementBytes = null;
-      Map<String, String> attributes = ImmutableMap.<String, String>of();
-      if (formatFn != null) {
-        PubsubIO.PubsubMessage message = formatFn.apply(c.element());
-        elementBytes = message.getMessage();
-        attributes = message.getAttributeMap();
-      } else {
-        elementBytes = CoderUtils.encodeToByteArray(elementCoder, c.element());
-      }
-
-      long timestampMsSinceEpoch = c.timestamp().getMillis();
-      @Nullable String recordId = null;
-      switch (recordIdMethod) {
-        case NONE:
-          break;
-        case DETERMINISTIC:
-          recordId = Hashing.murmur3_128().hashBytes(elementBytes).toString();
-          break;
-        case RANDOM:
-          // Since these elements go through a GroupByKey, any  failures while sending to
-          // Pubsub will be retried without falling back and generating a new record id.
-          // Thus even though we may send the same message to Pubsub twice, it is guaranteed
-          // to have the same record id.
-          recordId = UUID.randomUUID().toString();
-          break;
-      }
-      c.output(KV.of(ThreadLocalRandom.current().nextInt(numShards),
-                     new OutgoingMessage(elementBytes, attributes, timestampMsSinceEpoch,
-                             recordId)));
-    }
-
-    @Override
-    public void populateDisplayData(Builder builder) {
-      super.populateDisplayData(builder);
-      builder.add(DisplayData.item("numShards", numShards));
-    }
-  }
-
-  // ================================================================================
-  // WriterFn
-  // ================================================================================
-
-  /**
-   * Publish messages to Pubsub in batches.
-   */
-  private static class WriterFn
-      extends DoFn<KV<Integer, Iterable<OutgoingMessage>>, Void> {
-    private final PubsubClientFactory pubsubFactory;
-    private final ValueProvider<TopicPath> topic;
-    private final String timestampLabel;
-    private final String idLabel;
-    private final int publishBatchSize;
-    private final int publishBatchBytes;
-
-    /**
-     * Client on which to talk to Pubsub. Null until created by {@link #startBundle}.
-     */
-    @Nullable
-    private transient PubsubClient pubsubClient;
-
-    private final Counter batchCounter = Metrics.counter(WriterFn.class, "batches");
-    private final Counter elementCounter = Metrics.counter(WriterFn.class, "elements");
-    private final Counter byteCounter = Metrics.counter(WriterFn.class, "bytes");
-
-    WriterFn(
-        PubsubClientFactory pubsubFactory, ValueProvider<TopicPath> topic,
-        String timestampLabel, String idLabel, int publishBatchSize, int publishBatchBytes) {
-      this.pubsubFactory = pubsubFactory;
-      this.topic = topic;
-      this.timestampLabel = timestampLabel;
-      this.idLabel = idLabel;
-      this.publishBatchSize = publishBatchSize;
-      this.publishBatchBytes = publishBatchBytes;
-    }
-
-    /**
-     * BLOCKING
-     * Send {@code messages} as a batch to Pubsub.
-     */
-    private void publishBatch(List<OutgoingMessage> messages, int bytes)
-        throws IOException {
-      int n = pubsubClient.publish(topic.get(), messages);
-      checkState(n == messages.size(), "Attempted to publish %s messages but %s were successful",
-                 messages.size(), n);
-      batchCounter.inc();
-      elementCounter.inc(messages.size());
-      byteCounter.inc(bytes);
-    }
-
-    @StartBundle
-    public void startBundle(Context c) throws Exception {
-      checkState(pubsubClient == null, "startBundle invoked without prior finishBundle");
-      pubsubClient = pubsubFactory.newClient(timestampLabel, idLabel,
-                                             c.getPipelineOptions().as(PubsubOptions.class));
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext c) throws Exception {
-      List<OutgoingMessage> pubsubMessages = new ArrayList<>(publishBatchSize);
-      int bytes = 0;
-      for (OutgoingMessage message : c.element().getValue()) {
-        if (!pubsubMessages.isEmpty()
-            && bytes + message.elementBytes.length > publishBatchBytes) {
-          // Break large (in bytes) batches into smaller.
-          // (We've already broken by batch size using the trigger below, though that may
-          // run slightly over the actual PUBLISH_BATCH_SIZE. We'll consider that ok since
-          // the hard limit from Pubsub is by bytes rather than number of messages.)
-          // BLOCKS until published.
-          publishBatch(pubsubMessages, bytes);
-          pubsubMessages.clear();
-          bytes = 0;
-        }
-        pubsubMessages.add(message);
-        bytes += message.elementBytes.length;
-      }
-      if (!pubsubMessages.isEmpty()) {
-        // BLOCKS until published.
-        publishBatch(pubsubMessages, bytes);
-      }
-    }
-
-    @FinishBundle
-    public void finishBundle(Context c) throws Exception {
-      pubsubClient.close();
-      pubsubClient = null;
-    }
-
-    @Override
-    public void populateDisplayData(Builder builder) {
-      super.populateDisplayData(builder);
-        String topicString =
-            topic == null ? null
-            : topic.isAccessible() ? topic.get().getPath()
-            : topic.toString();
-      builder.add(DisplayData.item("topic", topicString));
-      builder.add(DisplayData.item("transport", pubsubFactory.getKind()));
-      builder.addIfNotNull(DisplayData.item("timestampLabel", timestampLabel));
-      builder.addIfNotNull(DisplayData.item("idLabel", idLabel));
-    }
-  }
-
-  // ================================================================================
-  // PubsubUnboundedSink
-  // ================================================================================
-
-  /**
-   * Which factory to use for creating Pubsub transport.
-   */
-  private final PubsubClientFactory pubsubFactory;
-
-  /**
-   * Pubsub topic to publish to.
-   */
-  private final ValueProvider<TopicPath> topic;
-
-  /**
-   * Coder for elements. It is the responsibility of the underlying Pubsub transport to
-   * re-encode element bytes if necessary, eg as Base64 strings.
-   */
-  private final Coder<T> elementCoder;
-
-  /**
-   * Pubsub metadata field holding timestamp of each element, or {@literal null} if should use
-   * Pubsub message publish timestamp instead.
-   */
-  @Nullable
-  private final String timestampLabel;
-
-  /**
-   * Pubsub metadata field holding id for each element, or {@literal null} if need to generate
-   * a unique id ourselves.
-   */
-  @Nullable
-  private final String idLabel;
-
-  /**
-   * Number of 'shards' to use so that latency in Pubsub publish can be hidden. Generally this
-   * should be a small multiple of the number of available cores. Too smoll a number results
-   * in too much time lost to blocking Pubsub calls. To large a number results in too many
-   * single-element batches being sent to Pubsub with high per-batch overhead.
-   */
-  private final int numShards;
-
-  /**
-   * Maximum number of messages per publish.
-   */
-  private final int publishBatchSize;
-
-  /**
-   * Maximum size of a publish batch, in bytes.
-   */
-  private final int publishBatchBytes;
-
-  /**
-   * Longest delay between receiving a message and pushing it to Pubsub.
-   */
-  private final Duration maxLatency;
-
-  /**
-   * How record ids should be generated for each record (if {@link #idLabel} is non-{@literal
-   * null}).
-   */
-  private final RecordIdMethod recordIdMethod;
-
-  /**
-   * In order to publish attributes, a formatting function is used to format the output into
-   * a {@link PubsubIO.PubsubMessage}.
-   */
-  private final SimpleFunction<T, PubsubIO.PubsubMessage> formatFn;
-
-  @VisibleForTesting
-  PubsubUnboundedSink(
-      PubsubClientFactory pubsubFactory,
-      ValueProvider<TopicPath> topic,
-      Coder<T> elementCoder,
-      String timestampLabel,
-      String idLabel,
-      int numShards,
-      int publishBatchSize,
-      int publishBatchBytes,
-      Duration maxLatency,
-      SimpleFunction<T, PubsubIO.PubsubMessage> formatFn,
-      RecordIdMethod recordIdMethod) {
-    this.pubsubFactory = pubsubFactory;
-    this.topic = topic;
-    this.elementCoder = elementCoder;
-    this.timestampLabel = timestampLabel;
-    this.idLabel = idLabel;
-    this.numShards = numShards;
-    this.publishBatchSize = publishBatchSize;
-    this.publishBatchBytes = publishBatchBytes;
-    this.maxLatency = maxLatency;
-    this.formatFn = formatFn;
-    this.recordIdMethod = idLabel == null ? RecordIdMethod.NONE : recordIdMethod;
-  }
-
-  public PubsubUnboundedSink(
-      PubsubClientFactory pubsubFactory,
-      ValueProvider<TopicPath> topic,
-      Coder<T> elementCoder,
-      String timestampLabel,
-      String idLabel,
-      SimpleFunction<T, PubsubIO.PubsubMessage> formatFn,
-      int numShards) {
-    this(pubsubFactory, topic, elementCoder, timestampLabel, idLabel, numShards,
-         DEFAULT_PUBLISH_BATCH_SIZE, DEFAULT_PUBLISH_BATCH_BYTES, DEFAULT_MAX_LATENCY,
-         formatFn, RecordIdMethod.RANDOM);
-  }
-
-  /**
-   * Get the topic being written to.
-   */
-  public TopicPath getTopic() {
-    return topic.get();
-  }
-
-  /**
-   * Get the {@link ValueProvider} for the topic being written to.
-   */
-  public ValueProvider<TopicPath> getTopicProvider() {
-    return topic;
-  }
-
-  /**
-   * Get the timestamp label.
-   */
-  @Nullable
-  public String getTimestampLabel() {
-    return timestampLabel;
-  }
-
-  /**
-   * Get the id label.
-   */
-  @Nullable
-  public String getIdLabel() {
-    return idLabel;
-  }
-
-  /**
-   * Get the format function used for PubSub attributes.
-   */
-  @Nullable
-  public SimpleFunction<T, PubsubIO.PubsubMessage> getFormatFn() {
-    return formatFn;
-  }
-
-  /**
-   * Get the Coder used to encode output elements.
-   */
-  public Coder<T> getElementCoder() {
-    return elementCoder;
-  }
-
-  @Override
-  public PDone expand(PCollection<T> input) {
-    input.apply("PubsubUnboundedSink.Window", Window.<T>into(new GlobalWindows())
-        .triggering(
-            Repeatedly.forever(
-                AfterFirst.of(AfterPane.elementCountAtLeast(publishBatchSize),
-                    AfterProcessingTime.pastFirstElementInPane()
-                    .plusDelayOf(maxLatency))))
-            .discardingFiredPanes())
-         .apply("PubsubUnboundedSink.Shard",
-             ParDo.of(new ShardFn<T>(elementCoder, numShards, formatFn, recordIdMethod)))
-         .setCoder(KvCoder.of(VarIntCoder.of(), CODER))
-         .apply(GroupByKey.<Integer, OutgoingMessage>create())
-         .apply("PubsubUnboundedSink.Writer",
-             ParDo.of(new WriterFn(pubsubFactory, topic, timestampLabel, idLabel,
-                 publishBatchSize, publishBatchBytes)));
-    return PDone.in(input.getPipeline());
-  }
-}


[4/8] beam git commit: [BEAM-1722] Move PubsubIO into the google-cloud-platform module

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java
deleted file mode 100644
index 6d4cf4e..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java
+++ /dev/null
@@ -1,207 +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 com.google.auth.Credentials;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Timestamp;
-import com.google.pubsub.v1.PublishRequest;
-import com.google.pubsub.v1.PublishResponse;
-import com.google.pubsub.v1.PublisherGrpc.PublisherImplBase;
-import com.google.pubsub.v1.PubsubMessage;
-import com.google.pubsub.v1.PullRequest;
-import com.google.pubsub.v1.PullResponse;
-import com.google.pubsub.v1.ReceivedMessage;
-import com.google.pubsub.v1.SubscriberGrpc.SubscriberImplBase;
-import io.grpc.ManagedChannel;
-import io.grpc.Server;
-import io.grpc.inprocess.InProcessChannelBuilder;
-import io.grpc.inprocess.InProcessServerBuilder;
-import io.grpc.stub.StreamObserver;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-import org.apache.beam.sdk.util.PubsubClient.IncomingMessage;
-import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage;
-import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath;
-import org.apache.beam.sdk.util.PubsubClient.TopicPath;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for PubsubGrpcClient.
- */
-@RunWith(JUnit4.class)
-public class PubsubGrpcClientTest {
-  private ManagedChannel inProcessChannel;
-  private Credentials testCredentials;
-
-  private PubsubClient client;
-  private String channelName;
-
-  private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic");
-  private static final SubscriptionPath SUBSCRIPTION =
-      PubsubClient.subscriptionPathFromName("testProject", "testSubscription");
-  private static final long REQ_TIME = 1234L;
-  private static final long PUB_TIME = 3456L;
-  private static final long MESSAGE_TIME = 6789L;
-  private static final String TIMESTAMP_LABEL = "timestamp";
-  private static final String ID_LABEL = "id";
-  private static final String MESSAGE_ID = "testMessageId";
-  private static final String DATA = "testData";
-  private static final String RECORD_ID = "testRecordId";
-  private static final String ACK_ID = "testAckId";
-  private static final Map<String, String> ATTRIBUTES =
-          ImmutableMap.<String, String>builder().put("a", "b").put("c", "d").build();
-
-  @Before
-  public void setup() {
-    channelName = String.format("%s-%s",
-        PubsubGrpcClientTest.class.getName(), ThreadLocalRandom.current().nextInt());
-    inProcessChannel = InProcessChannelBuilder.forName(channelName).directExecutor().build();
-    testCredentials = new TestCredential();
-    client = new PubsubGrpcClient(TIMESTAMP_LABEL, ID_LABEL, 10, inProcessChannel, testCredentials);
-  }
-
-  @After
-  public void teardown() throws IOException {
-    client.close();
-    inProcessChannel.shutdownNow();
-  }
-
-  @Test
-  public void pullOneMessage() throws IOException {
-    String expectedSubscription = SUBSCRIPTION.getPath();
-    final PullRequest expectedRequest =
-        PullRequest.newBuilder()
-                   .setSubscription(expectedSubscription)
-                   .setReturnImmediately(true)
-                   .setMaxMessages(10)
-                   .build();
-    Timestamp timestamp = Timestamp.newBuilder()
-                                   .setSeconds(PUB_TIME / 1000)
-                                   .setNanos((int) (PUB_TIME % 1000) * 1000)
-                                   .build();
-    PubsubMessage expectedPubsubMessage =
-        PubsubMessage.newBuilder()
-                     .setMessageId(MESSAGE_ID)
-                     .setData(
-                         ByteString.copyFrom(DATA.getBytes()))
-                     .setPublishTime(timestamp)
-                     .putAllAttributes(ATTRIBUTES)
-                     .putAllAttributes(
-                         ImmutableMap.of(TIMESTAMP_LABEL,
-                                         String.valueOf(MESSAGE_TIME),
-                                         ID_LABEL, RECORD_ID))
-                     .build();
-    ReceivedMessage expectedReceivedMessage =
-        ReceivedMessage.newBuilder()
-                       .setMessage(expectedPubsubMessage)
-                       .setAckId(ACK_ID)
-                       .build();
-    final PullResponse response =
-        PullResponse.newBuilder()
-                    .addAllReceivedMessages(ImmutableList.of(expectedReceivedMessage))
-                    .build();
-
-    final List<PullRequest> requestsReceived = new ArrayList<>();
-    SubscriberImplBase subscriberImplBase = new SubscriberImplBase() {
-      @Override
-      public void pull(PullRequest request, StreamObserver<PullResponse> responseObserver) {
-        requestsReceived.add(request);
-        responseObserver.onNext(response);
-        responseObserver.onCompleted();
-      }
-    };
-    Server server = InProcessServerBuilder.forName(channelName)
-        .addService(subscriberImplBase)
-        .build()
-        .start();
-    try {
-      List<IncomingMessage> acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true);
-      assertEquals(1, acutalMessages.size());
-      IncomingMessage actualMessage = acutalMessages.get(0);
-      assertEquals(ACK_ID, actualMessage.ackId);
-      assertEquals(DATA, new String(actualMessage.elementBytes));
-      assertEquals(RECORD_ID, actualMessage.recordId);
-      assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch);
-      assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch);
-      assertEquals(expectedRequest, Iterables.getOnlyElement(requestsReceived));
-    } finally {
-      server.shutdownNow();
-    }
-  }
-
-  @Test
-  public void publishOneMessage() throws IOException {
-    String expectedTopic = TOPIC.getPath();
-    PubsubMessage expectedPubsubMessage =
-        PubsubMessage.newBuilder()
-                     .setData(ByteString.copyFrom(DATA.getBytes()))
-                     .putAllAttributes(ATTRIBUTES)
-                     .putAllAttributes(
-                         ImmutableMap.of(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME),
-                                         ID_LABEL, RECORD_ID))
-                     .build();
-    final PublishRequest expectedRequest =
-        PublishRequest.newBuilder()
-                      .setTopic(expectedTopic)
-                      .addAllMessages(
-                          ImmutableList.of(expectedPubsubMessage))
-                      .build();
-    final PublishResponse response =
-        PublishResponse.newBuilder()
-                       .addAllMessageIds(ImmutableList.of(MESSAGE_ID))
-                       .build();
-
-    final List<PublishRequest> requestsReceived = new ArrayList<>();
-    PublisherImplBase publisherImplBase = new PublisherImplBase() {
-      @Override
-      public void publish(
-          PublishRequest request, StreamObserver<PublishResponse> responseObserver) {
-        requestsReceived.add(request);
-        responseObserver.onNext(response);
-        responseObserver.onCompleted();
-      }
-    };
-    Server server = InProcessServerBuilder.forName(channelName)
-        .addService(publisherImplBase)
-        .build()
-        .start();
-    try {
-      OutgoingMessage actualMessage = new OutgoingMessage(
-              DATA.getBytes(), ATTRIBUTES, MESSAGE_TIME, RECORD_ID);
-      int n = client.publish(TOPIC, ImmutableList.of(actualMessage));
-      assertEquals(1, n);
-      assertEquals(expectedRequest, Iterables.getOnlyElement(requestsReceived));
-    } finally {
-      server.shutdownNow();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java
deleted file mode 100644
index 019190b..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java
+++ /dev/null
@@ -1,140 +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 com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.pubsub.model.PublishRequest;
-import com.google.api.services.pubsub.model.PublishResponse;
-import com.google.api.services.pubsub.model.PubsubMessage;
-import com.google.api.services.pubsub.model.PullRequest;
-import com.google.api.services.pubsub.model.PullResponse;
-import com.google.api.services.pubsub.model.ReceivedMessage;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.beam.sdk.util.PubsubClient.IncomingMessage;
-import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage;
-import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath;
-import org.apache.beam.sdk.util.PubsubClient.TopicPath;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mockito;
-
-/**
- * Tests for PubsubJsonClient.
- */
-@RunWith(JUnit4.class)
-public class PubsubJsonClientTest {
-  private Pubsub mockPubsub;
-  private PubsubClient client;
-
-  private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic");
-  private static final SubscriptionPath SUBSCRIPTION =
-      PubsubClient.subscriptionPathFromName("testProject", "testSubscription");
-  private static final long REQ_TIME = 1234L;
-  private static final long PUB_TIME = 3456L;
-  private static final long MESSAGE_TIME = 6789L;
-  private static final String TIMESTAMP_LABEL = "timestamp";
-  private static final String ID_LABEL = "id";
-  private static final String MESSAGE_ID = "testMessageId";
-  private static final String DATA = "testData";
-  private static final String RECORD_ID = "testRecordId";
-  private static final String ACK_ID = "testAckId";
-
-  @Before
-  public void setup() throws IOException {
-    mockPubsub = Mockito.mock(Pubsub.class, Mockito.RETURNS_DEEP_STUBS);
-    client = new PubsubJsonClient(TIMESTAMP_LABEL, ID_LABEL, mockPubsub);
-  }
-
-  @After
-  public void teardown() throws IOException {
-    client.close();
-    client = null;
-    mockPubsub = null;
-  }
-
-  @Test
-  public void pullOneMessage() throws IOException {
-    String expectedSubscription = SUBSCRIPTION.getPath();
-    PullRequest expectedRequest =
-        new PullRequest().setReturnImmediately(true).setMaxMessages(10);
-    PubsubMessage expectedPubsubMessage = new PubsubMessage()
-        .setMessageId(MESSAGE_ID)
-        .encodeData(DATA.getBytes())
-        .setPublishTime(String.valueOf(PUB_TIME))
-        .setAttributes(
-            ImmutableMap.of(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME),
-                            ID_LABEL, RECORD_ID));
-    ReceivedMessage expectedReceivedMessage =
-        new ReceivedMessage().setMessage(expectedPubsubMessage)
-                             .setAckId(ACK_ID);
-    PullResponse expectedResponse =
-        new PullResponse().setReceivedMessages(ImmutableList.of(expectedReceivedMessage));
-    Mockito.when((Object) (mockPubsub.projects()
-                               .subscriptions()
-                               .pull(expectedSubscription, expectedRequest)
-                               .execute()))
-           .thenReturn(expectedResponse);
-    List<IncomingMessage> acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true);
-    assertEquals(1, acutalMessages.size());
-    IncomingMessage actualMessage = acutalMessages.get(0);
-    assertEquals(ACK_ID, actualMessage.ackId);
-    assertEquals(DATA, new String(actualMessage.elementBytes));
-    assertEquals(RECORD_ID, actualMessage.recordId);
-    assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch);
-    assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch);
-  }
-
-  @Test
-  public void publishOneMessage() throws IOException {
-    String expectedTopic = TOPIC.getPath();
-    PubsubMessage expectedPubsubMessage = new PubsubMessage()
-        .encodeData(DATA.getBytes())
-        .setAttributes(
-            ImmutableMap.<String, String> builder()
-                    .put(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME))
-                    .put(ID_LABEL, RECORD_ID)
-                    .put("k", "v").build());
-    PublishRequest expectedRequest = new PublishRequest()
-        .setMessages(ImmutableList.of(expectedPubsubMessage));
-    PublishResponse expectedResponse = new PublishResponse()
-        .setMessageIds(ImmutableList.of(MESSAGE_ID));
-    Mockito.when((Object) (mockPubsub.projects()
-                                .topics()
-                                .publish(expectedTopic, expectedRequest)
-                                .execute()))
-           .thenReturn(expectedResponse);
-    Map<String, String> attrs = new HashMap<>();
-    attrs.put("k", "v");
-    OutgoingMessage actualMessage = new OutgoingMessage(
-            DATA.getBytes(), attrs, MESSAGE_TIME, RECORD_ID);
-    int n = client.publish(TOPIC, ImmutableList.of(actualMessage));
-    assertEquals(1, n);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java
deleted file mode 100644
index a1b7daf..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java
+++ /dev/null
@@ -1,114 +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 com.google.api.client.util.Clock;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.beam.sdk.util.PubsubClient.IncomingMessage;
-import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage;
-import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath;
-import org.apache.beam.sdk.util.PubsubClient.TopicPath;
-import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for PubsubTestClient.
- */
-@RunWith(JUnit4.class)
-public class PubsubTestClientTest {
-  private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic");
-  private static final SubscriptionPath SUBSCRIPTION =
-      PubsubClient.subscriptionPathFromName("testProject", "testSubscription");
-  private static final long REQ_TIME = 1234L;
-  private static final long MESSAGE_TIME = 6789L;
-  private static final String MESSAGE_ID = "testMessageId";
-  private static final String DATA = "testData";
-  private static final String ACK_ID = "testAckId";
-  private static final int ACK_TIMEOUT_S = 60;
-
-  @Test
-  public void pullOneMessage() throws IOException {
-    final AtomicLong now = new AtomicLong();
-    Clock clock = new Clock() {
-      @Override
-      public long currentTimeMillis() {
-        return now.get();
-      }
-    };
-    IncomingMessage expectedIncomingMessage =
-        new IncomingMessage(DATA.getBytes(), null, MESSAGE_TIME, REQ_TIME, ACK_ID, MESSAGE_ID);
-    try (PubsubTestClientFactory factory =
-             PubsubTestClient.createFactoryForPull(clock, SUBSCRIPTION, ACK_TIMEOUT_S,
-                                                   Lists.newArrayList(expectedIncomingMessage))) {
-      try (PubsubTestClient client = (PubsubTestClient) factory.newClient(null, null, null)) {
-        now.set(REQ_TIME);
-        client.advance();
-        List<IncomingMessage> incomingMessages = client.pull(now.get(), SUBSCRIPTION, 1, true);
-        assertEquals(1, incomingMessages.size());
-        assertEquals(expectedIncomingMessage, incomingMessages.get(0));
-        // Timeout on ACK.
-        now.addAndGet((ACK_TIMEOUT_S + 10) * 1000);
-        client.advance();
-        incomingMessages = client.pull(now.get(), SUBSCRIPTION, 1, true);
-        assertEquals(1, incomingMessages.size());
-        assertEquals(expectedIncomingMessage.withRequestTime(now.get()), incomingMessages.get(0));
-        now.addAndGet(10 * 1000);
-        client.advance();
-        // Extend ack
-        client.modifyAckDeadline(SUBSCRIPTION, ImmutableList.of(ACK_ID), 20);
-        // Timeout on extended ACK
-        now.addAndGet(30 * 1000);
-        client.advance();
-        incomingMessages = client.pull(now.get(), SUBSCRIPTION, 1, true);
-        assertEquals(1, incomingMessages.size());
-        assertEquals(expectedIncomingMessage.withRequestTime(now.get()), incomingMessages.get(0));
-        // Extend ack
-        client.modifyAckDeadline(SUBSCRIPTION, ImmutableList.of(ACK_ID), 20);
-        // Ack
-        now.addAndGet(15 * 1000);
-        client.advance();
-        client.acknowledge(SUBSCRIPTION, ImmutableList.of(ACK_ID));
-      }
-    }
-  }
-
-  @Test
-  public void publishOneMessage() throws IOException {
-    OutgoingMessage expectedOutgoingMessage =
-        new OutgoingMessage(DATA.getBytes(), null, MESSAGE_TIME, MESSAGE_ID);
-    try (PubsubTestClientFactory factory =
-             PubsubTestClient.createFactoryForPublish(
-                 TOPIC,
-                 Sets.newHashSet(expectedOutgoingMessage),
-                 ImmutableList.<OutgoingMessage>of())) {
-      try (PubsubTestClient client = (PubsubTestClient) factory.newClient(null, null, null)) {
-        client.publish(TOPIC, ImmutableList.of(expectedOutgoingMessage));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/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 1a2e100..d22c6c5 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -84,6 +84,16 @@
     </dependency>
 
     <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-pubsub</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.api.grpc</groupId>
+      <artifactId>grpc-google-pubsub-v1</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>com.google.auto.service</groupId>
       <artifactId>auto-service</artifactId>
       <optional>true</optional>
@@ -106,10 +116,44 @@
 
     <dependency>
       <groupId>io.grpc</groupId>
+      <artifactId>grpc-auth</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
       <artifactId>grpc-core</artifactId>
     </dependency>
 
     <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-handler</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+    </dependency>
+
+    <!-- grpc-all does not obey IWYU, so we need to exclude from compile
+     scope and depend on it at runtime. -->
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-all</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
       <groupId>joda-time</groupId>
       <artifactId>joda-time</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java
new file mode 100644
index 0000000..750178c
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java
@@ -0,0 +1,544 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.api.client.util.DateTime;
+import com.google.common.base.Objects;
+import com.google.common.base.Strings;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.PubsubOptions;
+
+/**
+ * An (abstract) helper class for talking to Pubsub via an underlying transport.
+ */
+abstract class PubsubClient implements Closeable {
+  /**
+   * Factory for creating clients.
+   */
+  public interface PubsubClientFactory extends Serializable {
+    /**
+     * Construct a new Pubsub client. It should be closed via {@link #close} in order
+     * to ensure tidy cleanup of underlying netty resources (or use the try-with-resources
+     * construct). Uses {@code options} to derive pubsub endpoints and application credentials.
+     * If non-{@literal null}, use {@code timestampLabel} and {@code idLabel} to store custom
+     * timestamps/ids within message metadata.
+     */
+    PubsubClient newClient(
+        @Nullable String timestampLabel,
+        @Nullable String idLabel,
+        PubsubOptions options) throws IOException;
+
+    /**
+     * Return the display name for this factory. Eg "Json", "gRPC".
+     */
+    String getKind();
+  }
+
+  /**
+   * Return timestamp as ms-since-unix-epoch corresponding to {@code timestamp}.
+   * Return {@literal null} if no timestamp could be found. Throw {@link IllegalArgumentException}
+   * if timestamp cannot be recognized.
+   */
+  @Nullable
+  private static Long asMsSinceEpoch(@Nullable String timestamp) {
+    if (Strings.isNullOrEmpty(timestamp)) {
+      return null;
+    }
+    try {
+      // Try parsing as milliseconds since epoch. Note there is no way to parse a
+      // string in RFC 3339 format here.
+      // Expected IllegalArgumentException if parsing fails; we use that to fall back
+      // to RFC 3339.
+      return Long.parseLong(timestamp);
+    } catch (IllegalArgumentException e1) {
+      // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an
+      // IllegalArgumentException if parsing fails, and the caller should handle.
+      return DateTime.parseRfc3339(timestamp).getValue();
+    }
+  }
+
+  /**
+   * Return the timestamp (in ms since unix epoch) to use for a Pubsub message with {@code
+   * attributes} and {@code pubsubTimestamp}.
+   *
+   * <p>If {@code timestampLabel} is non-{@literal null} then the message attributes must contain
+   * that label, and the value of that label will be taken as the timestamp.
+   * Otherwise the timestamp will be taken from the Pubsub publish timestamp {@code
+   * pubsubTimestamp}.
+   *
+   * @throws IllegalArgumentException if the timestamp cannot be recognized as a ms-since-unix-epoch
+   * or RFC3339 time.
+   */
+  protected static long extractTimestamp(
+      @Nullable String timestampLabel,
+      @Nullable String pubsubTimestamp,
+      @Nullable Map<String, String> attributes) {
+    Long timestampMsSinceEpoch;
+    if (Strings.isNullOrEmpty(timestampLabel)) {
+      timestampMsSinceEpoch = asMsSinceEpoch(pubsubTimestamp);
+      checkArgument(timestampMsSinceEpoch != null,
+                    "Cannot interpret PubSub publish timestamp: %s",
+                    pubsubTimestamp);
+    } else {
+      String value = attributes == null ? null : attributes.get(timestampLabel);
+      checkArgument(value != null,
+                    "PubSub message is missing a value for timestamp label %s",
+                    timestampLabel);
+      timestampMsSinceEpoch = asMsSinceEpoch(value);
+      checkArgument(timestampMsSinceEpoch != null,
+                    "Cannot interpret value of label %s as timestamp: %s",
+                    timestampLabel, value);
+    }
+    return timestampMsSinceEpoch;
+  }
+
+  /**
+   * Path representing a cloud project id.
+   */
+  static class ProjectPath implements Serializable {
+    private final String projectId;
+
+    /**
+     * Creates a {@link ProjectPath} from a {@link String} representation, which
+     * must be of the form {@code "projects/" + projectId}.
+     */
+    ProjectPath(String path) {
+      String[] splits = path.split("/");
+      checkArgument(
+          splits.length == 2 && splits[0].equals("projects"),
+          "Malformed project path \"%s\": must be of the form \"projects/\" + <project id>",
+          path);
+      this.projectId = splits[1];
+    }
+
+    public String getPath() {
+      return String.format("projects/%s", projectId);
+    }
+
+    public String getId() {
+      return projectId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      ProjectPath that = (ProjectPath) o;
+
+      return projectId.equals(that.projectId);
+    }
+
+    @Override
+    public int hashCode() {
+      return projectId.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return getPath();
+    }
+  }
+
+  public static ProjectPath projectPathFromPath(String path) {
+    return new ProjectPath(path);
+  }
+
+  public static ProjectPath projectPathFromId(String projectId) {
+    return new ProjectPath(String.format("projects/%s", projectId));
+  }
+
+  /**
+   * Path representing a Pubsub subscription.
+   */
+  public static class SubscriptionPath implements Serializable {
+    private final String projectId;
+    private final String subscriptionName;
+
+    SubscriptionPath(String path) {
+      String[] splits = path.split("/");
+      checkState(
+          splits.length == 4 && splits[0].equals("projects") && splits[2].equals("subscriptions"),
+          "Malformed subscription path %s: "
+          + "must be of the form \"projects/\" + <project id> + \"subscriptions\"", path);
+      this.projectId = splits[1];
+      this.subscriptionName = splits[3];
+    }
+
+    public String getPath() {
+      return String.format("projects/%s/subscriptions/%s", projectId, subscriptionName);
+    }
+
+    public String getName() {
+      return subscriptionName;
+    }
+
+    public String getV1Beta1Path() {
+      return String.format("/subscriptions/%s/%s", projectId, subscriptionName);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      SubscriptionPath that = (SubscriptionPath) o;
+      return this.subscriptionName.equals(that.subscriptionName)
+          && this.projectId.equals(that.projectId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(projectId, subscriptionName);
+    }
+
+    @Override
+    public String toString() {
+      return getPath();
+    }
+  }
+
+  public static SubscriptionPath subscriptionPathFromPath(String path) {
+    return new SubscriptionPath(path);
+  }
+
+  public static SubscriptionPath subscriptionPathFromName(
+      String projectId, String subscriptionName) {
+    return new SubscriptionPath(String.format("projects/%s/subscriptions/%s",
+                                              projectId, subscriptionName));
+  }
+
+  /**
+   * Path representing a Pubsub topic.
+   */
+  public static class TopicPath implements Serializable {
+    private final String path;
+
+    TopicPath(String path) {
+      this.path = path;
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+    public String getName() {
+      String[] splits = path.split("/");
+      checkState(splits.length == 4, "Malformed topic path %s", path);
+      return splits[3];
+    }
+
+    public String getV1Beta1Path() {
+      String[] splits = path.split("/");
+      checkState(splits.length == 4, "Malformed topic path %s", path);
+      return String.format("/topics/%s/%s", splits[1], splits[3]);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      TopicPath topicPath = (TopicPath) o;
+      return path.equals(topicPath.path);
+    }
+
+    @Override
+    public int hashCode() {
+      return path.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return path;
+    }
+  }
+
+  public static TopicPath topicPathFromPath(String path) {
+    return new TopicPath(path);
+  }
+
+  public static TopicPath topicPathFromName(String projectId, String topicName) {
+    return new TopicPath(String.format("projects/%s/topics/%s", projectId, topicName));
+  }
+
+  /**
+   * A message to be sent to Pubsub.
+   *
+   * <p>NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}.
+   * Java serialization is never used for non-test clients.
+   */
+  static class OutgoingMessage implements Serializable {
+    /**
+     * Underlying (encoded) element.
+     */
+    public final byte[] elementBytes;
+
+    public final Map<String, String> attributes;
+
+    /**
+     * Timestamp for element (ms since epoch).
+     */
+    public final long timestampMsSinceEpoch;
+
+    /**
+     * If using an id label, the record id to associate with this record's metadata so the receiver
+     * can reject duplicates. Otherwise {@literal null}.
+     */
+    @Nullable
+    public final String recordId;
+
+    public OutgoingMessage(byte[] elementBytes, Map<String, String> attributes,
+                           long timestampMsSinceEpoch, @Nullable String recordId) {
+      this.elementBytes = elementBytes;
+      this.attributes = attributes;
+      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
+      this.recordId = recordId;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("OutgoingMessage(%db, %dms)",
+                           elementBytes.length, timestampMsSinceEpoch);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      OutgoingMessage that = (OutgoingMessage) o;
+
+      return timestampMsSinceEpoch == that.timestampMsSinceEpoch
+              && Arrays.equals(elementBytes, that.elementBytes)
+              && Objects.equal(attributes, that.attributes)
+              && Objects.equal(recordId, that.recordId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch,
+              recordId);
+    }
+  }
+
+  /**
+   * A message received from Pubsub.
+   *
+   * <p>NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}.
+   * Java serialization is never used for non-test clients.
+   */
+  static class IncomingMessage implements Serializable {
+    /**
+     * Underlying (encoded) element.
+     */
+    public final byte[] elementBytes;
+
+    public Map<String, String> attributes;
+
+    /**
+     * Timestamp for element (ms since epoch). Either Pubsub's processing time,
+     * or the custom timestamp associated with the message.
+     */
+    public final long timestampMsSinceEpoch;
+
+    /**
+     * Timestamp (in system time) at which we requested the message (ms since epoch).
+     */
+    public final long requestTimeMsSinceEpoch;
+
+    /**
+     * Id to pass back to Pubsub to acknowledge receipt of this message.
+     */
+    public final String ackId;
+
+    /**
+     * Id to pass to the runner to distinguish this message from all others.
+     */
+    public final String recordId;
+
+    public IncomingMessage(
+        byte[] elementBytes,
+        Map<String, String> attributes,
+        long timestampMsSinceEpoch,
+        long requestTimeMsSinceEpoch,
+        String ackId,
+        String recordId) {
+      this.elementBytes = elementBytes;
+      this.attributes = attributes;
+      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
+      this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch;
+      this.ackId = ackId;
+      this.recordId = recordId;
+    }
+
+    public IncomingMessage withRequestTime(long requestTimeMsSinceEpoch) {
+      return new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch,
+              requestTimeMsSinceEpoch, ackId, recordId);
+    }
+
+    @Override
+    public String toString() {
+      return String.format("IncomingMessage(%db, %dms)",
+                           elementBytes.length, timestampMsSinceEpoch);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      IncomingMessage that = (IncomingMessage) o;
+
+      return timestampMsSinceEpoch == that.timestampMsSinceEpoch
+             && requestTimeMsSinceEpoch == that.requestTimeMsSinceEpoch
+             && ackId.equals(that.ackId)
+             && recordId.equals(that.recordId)
+             && Arrays.equals(elementBytes, that.elementBytes)
+              && Objects.equal(attributes, that.attributes);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch,
+                              requestTimeMsSinceEpoch,
+                              ackId, recordId);
+    }
+  }
+
+  /**
+   * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages
+   * published.
+   */
+  public abstract int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages)
+      throws IOException;
+
+  /**
+   * Request the next batch of up to {@code batchSize} messages from {@code subscription}.
+   * Return the received messages, or empty collection if none were available. Does not
+   * wait for messages to arrive if {@code returnImmediately} is {@literal true}.
+   * Returned messages will record their request time as {@code requestTimeMsSinceEpoch}.
+   */
+  public abstract List<IncomingMessage> pull(
+      long requestTimeMsSinceEpoch,
+      SubscriptionPath subscription,
+      int batchSize,
+      boolean returnImmediately)
+      throws IOException;
+
+  /**
+   * Acknowldege messages from {@code subscription} with {@code ackIds}.
+   */
+  public abstract void acknowledge(SubscriptionPath subscription, List<String> ackIds)
+      throws IOException;
+
+  /**
+   * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to
+   * be {@code deadlineSeconds} from now.
+   */
+  public abstract void modifyAckDeadline(
+      SubscriptionPath subscription, List<String> ackIds,
+      int deadlineSeconds) throws IOException;
+
+  /**
+   * Create {@code topic}.
+   */
+  public abstract void createTopic(TopicPath topic) throws IOException;
+
+  /*
+   * Delete {@code topic}.
+   */
+  public abstract void deleteTopic(TopicPath topic) throws IOException;
+
+  /**
+   * Return a list of topics for {@code project}.
+   */
+  public abstract List<TopicPath> listTopics(ProjectPath project) throws IOException;
+
+  /**
+   * Create {@code subscription} to {@code topic}.
+   */
+  public abstract void createSubscription(
+      TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException;
+
+  /**
+   * Create a random subscription for {@code topic}. Return the {@link SubscriptionPath}. It
+   * is the responsibility of the caller to later delete the subscription.
+   */
+  public SubscriptionPath createRandomSubscription(
+      ProjectPath project, TopicPath topic, int ackDeadlineSeconds) throws IOException {
+    // Create a randomized subscription derived from the topic name.
+    String subscriptionName = topic.getName() + "_beam_" + ThreadLocalRandom.current().nextLong();
+    SubscriptionPath subscription =
+        PubsubClient.subscriptionPathFromName(project.getId(), subscriptionName);
+    createSubscription(topic, subscription, ackDeadlineSeconds);
+    return subscription;
+  }
+
+  /**
+   * Delete {@code subscription}.
+   */
+  public abstract void deleteSubscription(SubscriptionPath subscription) throws IOException;
+
+  /**
+   * Return a list of subscriptions for {@code topic} in {@code project}.
+   */
+  public abstract List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
+      throws IOException;
+
+  /**
+   * Return the ack deadline, in seconds, for {@code subscription}.
+   */
+  public abstract int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException;
+
+  /**
+   * Return {@literal true} if {@link #pull} will always return empty list. Actual clients
+   * will return {@literal false}. Test clients may return {@literal true} to signal that all
+   * expected messages have been pulled and the test may complete.
+   */
+  public abstract boolean isEOF();
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java
new file mode 100644
index 0000000..912d59c
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java
@@ -0,0 +1,424 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.auth.Credentials;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Timestamp;
+import com.google.pubsub.v1.AcknowledgeRequest;
+import com.google.pubsub.v1.DeleteSubscriptionRequest;
+import com.google.pubsub.v1.DeleteTopicRequest;
+import com.google.pubsub.v1.GetSubscriptionRequest;
+import com.google.pubsub.v1.ListSubscriptionsRequest;
+import com.google.pubsub.v1.ListSubscriptionsResponse;
+import com.google.pubsub.v1.ListTopicsRequest;
+import com.google.pubsub.v1.ListTopicsResponse;
+import com.google.pubsub.v1.ModifyAckDeadlineRequest;
+import com.google.pubsub.v1.PublishRequest;
+import com.google.pubsub.v1.PublishResponse;
+import com.google.pubsub.v1.PublisherGrpc;
+import com.google.pubsub.v1.PublisherGrpc.PublisherBlockingStub;
+import com.google.pubsub.v1.PubsubMessage;
+import com.google.pubsub.v1.PullRequest;
+import com.google.pubsub.v1.PullResponse;
+import com.google.pubsub.v1.ReceivedMessage;
+import com.google.pubsub.v1.SubscriberGrpc;
+import com.google.pubsub.v1.SubscriberGrpc.SubscriberBlockingStub;
+import com.google.pubsub.v1.Subscription;
+import com.google.pubsub.v1.Topic;
+import io.grpc.Channel;
+import io.grpc.ClientInterceptors;
+import io.grpc.ManagedChannel;
+import io.grpc.auth.ClientAuthInterceptor;
+import io.grpc.netty.GrpcSslContexts;
+import io.grpc.netty.NegotiationType;
+import io.grpc.netty.NettyChannelBuilder;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.PubsubOptions;
+
+/**
+ * A helper class for talking to Pubsub via grpc.
+ *
+ * <p>CAUTION: Currently uses the application default credentials and does not respect any
+ * credentials-related arguments in {@link GcpOptions}.
+ */
+class PubsubGrpcClient extends PubsubClient {
+  private static final String PUBSUB_ADDRESS = "pubsub.googleapis.com";
+  private static final int PUBSUB_PORT = 443;
+  private static final int LIST_BATCH_SIZE = 1000;
+
+  private static final int DEFAULT_TIMEOUT_S = 15;
+
+  private static class PubsubGrpcClientFactory implements PubsubClientFactory {
+    @Override
+    public PubsubClient newClient(
+        @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
+        throws IOException {
+      ManagedChannel channel = NettyChannelBuilder
+          .forAddress(PUBSUB_ADDRESS, PUBSUB_PORT)
+          .negotiationType(NegotiationType.TLS)
+          .sslContext(GrpcSslContexts.forClient().ciphers(null).build())
+          .build();
+
+      return new PubsubGrpcClient(timestampLabel,
+                                  idLabel,
+                                  DEFAULT_TIMEOUT_S,
+                                  channel,
+                                  options.getGcpCredential());
+    }
+
+    @Override
+    public String getKind() {
+      return "Grpc";
+    }
+  }
+
+  /**
+   * Factory for creating Pubsub clients using gRCP transport.
+   */
+  public static final PubsubClientFactory FACTORY = new PubsubGrpcClientFactory();
+
+  /**
+   * Timeout for grpc calls (in s).
+   */
+  private final int timeoutSec;
+
+  /**
+   * Underlying netty channel, or {@literal null} if closed.
+   */
+  @Nullable
+  private ManagedChannel publisherChannel;
+
+  /**
+   * Credentials determined from options and environment.
+   */
+  private final Credentials credentials;
+
+  /**
+   * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time
+   * instead.
+   */
+  @Nullable
+  private final String timestampLabel;
+
+  /**
+   * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids.
+   */
+  @Nullable
+  private final String idLabel;
+
+
+  /**
+   * Cached stubs, or null if not cached.
+   */
+  @Nullable
+  private PublisherGrpc.PublisherBlockingStub cachedPublisherStub;
+  private SubscriberGrpc.SubscriberBlockingStub cachedSubscriberStub;
+
+  @VisibleForTesting
+  PubsubGrpcClient(
+      @Nullable String timestampLabel,
+      @Nullable String idLabel,
+      int timeoutSec,
+      ManagedChannel publisherChannel,
+      Credentials credentials) {
+    this.timestampLabel = timestampLabel;
+    this.idLabel = idLabel;
+    this.timeoutSec = timeoutSec;
+    this.publisherChannel = publisherChannel;
+    this.credentials = credentials;
+  }
+
+  /**
+   * Gracefully close the underlying netty channel.
+   */
+  @Override
+  public void close() {
+    if (publisherChannel == null) {
+      // Already closed.
+      return;
+    }
+    // Can gc the underlying stubs.
+    cachedPublisherStub = null;
+    cachedSubscriberStub = null;
+    // Mark the client as having been closed before going further
+    // in case we have an exception from the channel.
+    ManagedChannel publisherChannel = this.publisherChannel;
+    this.publisherChannel = null;
+    // Gracefully shutdown the channel.
+    publisherChannel.shutdown();
+    try {
+      publisherChannel.awaitTermination(timeoutSec, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      // Ignore.
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  /**
+   * Return channel with interceptor for returning credentials.
+   */
+  private Channel newChannel() throws IOException {
+    checkState(publisherChannel != null, "PubsubGrpcClient has been closed");
+    ClientAuthInterceptor interceptor =
+        new ClientAuthInterceptor(credentials, Executors.newSingleThreadExecutor());
+    return ClientInterceptors.intercept(publisherChannel, interceptor);
+  }
+
+  /**
+   * Return a stub for making a publish request with a timeout.
+   */
+  private PublisherBlockingStub publisherStub() throws IOException {
+    if (cachedPublisherStub == null) {
+      cachedPublisherStub = PublisherGrpc.newBlockingStub(newChannel());
+    }
+    return cachedPublisherStub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS);
+  }
+
+  /**
+   * Return a stub for making a subscribe request with a timeout.
+   */
+  private SubscriberBlockingStub subscriberStub() throws IOException {
+    if (cachedSubscriberStub == null) {
+      cachedSubscriberStub = SubscriberGrpc.newBlockingStub(newChannel());
+    }
+    return cachedSubscriberStub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages)
+      throws IOException {
+    PublishRequest.Builder request = PublishRequest.newBuilder()
+                                                   .setTopic(topic.getPath());
+    for (OutgoingMessage outgoingMessage : outgoingMessages) {
+      PubsubMessage.Builder message =
+          PubsubMessage.newBuilder()
+                       .setData(ByteString.copyFrom(outgoingMessage.elementBytes));
+
+      if (outgoingMessage.attributes != null) {
+        message.putAllAttributes(outgoingMessage.attributes);
+      }
+
+      if (timestampLabel != null) {
+        message.getMutableAttributes()
+               .put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch));
+      }
+
+      if (idLabel != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) {
+        message.getMutableAttributes().put(idLabel, outgoingMessage.recordId);
+      }
+
+      request.addMessages(message);
+    }
+
+    PublishResponse response = publisherStub().publish(request.build());
+    return response.getMessageIdsCount();
+  }
+
+  @Override
+  public List<IncomingMessage> pull(
+      long requestTimeMsSinceEpoch,
+      SubscriptionPath subscription,
+      int batchSize,
+      boolean returnImmediately) throws IOException {
+    PullRequest request = PullRequest.newBuilder()
+                                     .setSubscription(subscription.getPath())
+                                     .setReturnImmediately(returnImmediately)
+                                     .setMaxMessages(batchSize)
+                                     .build();
+    PullResponse response = subscriberStub().pull(request);
+    if (response.getReceivedMessagesCount() == 0) {
+      return ImmutableList.of();
+    }
+    List<IncomingMessage> incomingMessages = new ArrayList<>(response.getReceivedMessagesCount());
+    for (ReceivedMessage message : response.getReceivedMessagesList()) {
+      PubsubMessage pubsubMessage = message.getMessage();
+      @Nullable Map<String, String> attributes = pubsubMessage.getAttributes();
+
+      // Payload.
+      byte[] elementBytes = pubsubMessage.getData().toByteArray();
+
+      // Timestamp.
+      String pubsubTimestampString = null;
+      Timestamp timestampProto = pubsubMessage.getPublishTime();
+      if (timestampProto != null) {
+        pubsubTimestampString = String.valueOf(timestampProto.getSeconds()
+                                               + timestampProto.getNanos() / 1000L);
+      }
+      long timestampMsSinceEpoch =
+          extractTimestamp(timestampLabel, pubsubTimestampString, attributes);
+
+      // Ack id.
+      String ackId = message.getAckId();
+      checkState(!Strings.isNullOrEmpty(ackId));
+
+      // Record id, if any.
+      @Nullable String recordId = null;
+      if (idLabel != null && attributes != null) {
+        recordId = attributes.get(idLabel);
+      }
+      if (Strings.isNullOrEmpty(recordId)) {
+        // Fall back to the Pubsub provided message id.
+        recordId = pubsubMessage.getMessageId();
+      }
+
+      incomingMessages.add(new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch,
+                                               requestTimeMsSinceEpoch, ackId, recordId));
+    }
+    return incomingMessages;
+  }
+
+  @Override
+  public void acknowledge(SubscriptionPath subscription, List<String> ackIds)
+      throws IOException {
+    AcknowledgeRequest request = AcknowledgeRequest.newBuilder()
+                                                   .setSubscription(subscription.getPath())
+                                                   .addAllAckIds(ackIds)
+                                                   .build();
+    subscriberStub().acknowledge(request); // ignore Empty result.
+  }
+
+  @Override
+  public void modifyAckDeadline(
+      SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds)
+      throws IOException {
+    ModifyAckDeadlineRequest request =
+        ModifyAckDeadlineRequest.newBuilder()
+                                .setSubscription(subscription.getPath())
+                                .addAllAckIds(ackIds)
+                                .setAckDeadlineSeconds(deadlineSeconds)
+                                .build();
+    subscriberStub().modifyAckDeadline(request); // ignore Empty result.
+  }
+
+  @Override
+  public void createTopic(TopicPath topic) throws IOException {
+    Topic request = Topic.newBuilder()
+                         .setName(topic.getPath())
+                         .build();
+    publisherStub().createTopic(request); // ignore Topic result.
+  }
+
+  @Override
+  public void deleteTopic(TopicPath topic) throws IOException {
+    DeleteTopicRequest request = DeleteTopicRequest.newBuilder()
+                                                   .setTopic(topic.getPath())
+                                                   .build();
+    publisherStub().deleteTopic(request); // ignore Empty result.
+  }
+
+  @Override
+  public List<TopicPath> listTopics(ProjectPath project) throws IOException {
+    ListTopicsRequest.Builder request =
+        ListTopicsRequest.newBuilder()
+                         .setProject(project.getPath())
+                         .setPageSize(LIST_BATCH_SIZE);
+    ListTopicsResponse response = publisherStub().listTopics(request.build());
+    if (response.getTopicsCount() == 0) {
+      return ImmutableList.of();
+    }
+    List<TopicPath> topics = new ArrayList<>(response.getTopicsCount());
+    while (true) {
+      for (Topic topic : response.getTopicsList()) {
+        topics.add(topicPathFromPath(topic.getName()));
+      }
+      if (response.getNextPageToken().isEmpty()) {
+        break;
+      }
+      request.setPageToken(response.getNextPageToken());
+      response = publisherStub().listTopics(request.build());
+    }
+    return topics;
+  }
+
+  @Override
+  public void createSubscription(
+      TopicPath topic, SubscriptionPath subscription,
+      int ackDeadlineSeconds) throws IOException {
+    Subscription request = Subscription.newBuilder()
+                                       .setTopic(topic.getPath())
+                                       .setName(subscription.getPath())
+                                       .setAckDeadlineSeconds(ackDeadlineSeconds)
+                                       .build();
+    subscriberStub().createSubscription(request); // ignore Subscription result.
+  }
+
+  @Override
+  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
+    DeleteSubscriptionRequest request =
+        DeleteSubscriptionRequest.newBuilder()
+                                 .setSubscription(subscription.getPath())
+                                 .build();
+    subscriberStub().deleteSubscription(request); // ignore Empty result.
+  }
+
+  @Override
+  public List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
+      throws IOException {
+    ListSubscriptionsRequest.Builder request =
+        ListSubscriptionsRequest.newBuilder()
+                                .setProject(project.getPath())
+                                .setPageSize(LIST_BATCH_SIZE);
+    ListSubscriptionsResponse response = subscriberStub().listSubscriptions(request.build());
+    if (response.getSubscriptionsCount() == 0) {
+      return ImmutableList.of();
+    }
+    List<SubscriptionPath> subscriptions = new ArrayList<>(response.getSubscriptionsCount());
+    while (true) {
+      for (Subscription subscription : response.getSubscriptionsList()) {
+        if (subscription.getTopic().equals(topic.getPath())) {
+          subscriptions.add(subscriptionPathFromPath(subscription.getName()));
+        }
+      }
+      if (response.getNextPageToken().isEmpty()) {
+        break;
+      }
+      request.setPageToken(response.getNextPageToken());
+      response = subscriberStub().listSubscriptions(request.build());
+    }
+    return subscriptions;
+  }
+
+  @Override
+  public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException {
+    GetSubscriptionRequest request =
+        GetSubscriptionRequest.newBuilder()
+                              .setSubscription(subscription.getPath())
+                              .build();
+    Subscription response = subscriberStub().getSubscription(request);
+    return response.getAckDeadlineSeconds();
+  }
+
+  @Override
+  public boolean isEOF() {
+    return false;
+  }
+}


[6/8] beam git commit: [BEAM-1722] Move PubsubIO into the google-cloud-platform module

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java
deleted file mode 100644
index 0e6bec8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java
+++ /dev/null
@@ -1,1463 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.sdk.io;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.api.client.util.Clock;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.security.GeneralSecurityException;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.coders.NullableCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.io.PubsubIO.PubsubMessage;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PubsubOptions;
-import org.apache.beam.sdk.options.ValueProvider;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.BucketingFunction;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.MovingFunction;
-import org.apache.beam.sdk.util.PubsubClient;
-import org.apache.beam.sdk.util.PubsubClient.ProjectPath;
-import org.apache.beam.sdk.util.PubsubClient.PubsubClientFactory;
-import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath;
-import org.apache.beam.sdk.util.PubsubClient.TopicPath;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A PTransform which streams messages from Pubsub.
- * <ul>
- * <li>The underlying implementation in an {@link UnboundedSource} which receives messages
- * in batches and hands them out one at a time.
- * <li>The watermark (either in Pubsub processing time or custom timestamp time) is estimated
- * by keeping track of the minimum of the last minutes worth of messages. This assumes Pubsub
- * delivers the oldest (in Pubsub processing time) available message at least once a minute,
- * and that custom timestamps are 'mostly' monotonic with Pubsub processing time. Unfortunately
- * both of those assumptions are fragile. Thus the estimated watermark may get ahead of
- * the 'true' watermark and cause some messages to be late.
- * <li>Checkpoints are used both to ACK received messages back to Pubsub (so that they may
- * be retired on the Pubsub end), and to NACK already consumed messages should a checkpoint
- * need to be restored (so that Pubsub will resend those messages promptly).
- * <li>The backlog is determined by each reader using the messages which have been pulled from
- * Pubsub but not yet consumed downstream. The backlog does not take account of any messages queued
- * by Pubsub for the subscription. Unfortunately there is currently no API to determine
- * the size of the Pubsub queue's backlog.
- * <li>The subscription must already exist.
- * <li>The subscription timeout is read whenever a reader is started. However it is not
- * checked thereafter despite the timeout being user-changeable on-the-fly.
- * <li>We log vital stats every 30 seconds.
- * <li>Though some background threads may be used by the underlying transport all Pubsub calls
- * are blocking. We rely on the underlying runner to allow multiple
- * {@link UnboundedSource.UnboundedReader} instances to execute concurrently and thus hide latency.
- * </ul>
- */
-public class PubsubUnboundedSource<T> extends PTransform<PBegin, PCollection<T>> {
-  private static final Logger LOG = LoggerFactory.getLogger(PubsubUnboundedSource.class);
-
-  /**
-   * Default ACK timeout for created subscriptions.
-   */
-  private static final int DEAULT_ACK_TIMEOUT_SEC = 60;
-
-  /**
-   * Coder for checkpoints.
-   */
-  private static final PubsubCheckpointCoder<?> CHECKPOINT_CODER = new PubsubCheckpointCoder<>();
-
-  /**
-   * Maximum number of messages per pull.
-   */
-  private static final int PULL_BATCH_SIZE = 1000;
-
-  /**
-   * Maximum number of ACK ids per ACK or ACK extension call.
-   */
-  private static final int ACK_BATCH_SIZE = 2000;
-
-  /**
-   * Maximum number of messages in flight.
-   */
-  private static final int MAX_IN_FLIGHT = 20000;
-
-  /**
-   * Timeout for round trip from receiving a message to finally ACKing it back to Pubsub.
-   */
-  private static final Duration PROCESSING_TIMEOUT = Duration.standardSeconds(120);
-
-  /**
-   * Percentage of ack timeout by which to extend acks when they are near timeout.
-   */
-  private static final int ACK_EXTENSION_PCT = 50;
-
-  /**
-   * Percentage of ack timeout we should use as a safety margin. We'll try to extend acks
-   * by this margin before the ack actually expires.
-   */
-  private static final int ACK_SAFETY_PCT = 20;
-
-  /**
-   * For stats only: How close we can get to an ack deadline before we risk it being already
-   * considered passed by Pubsub.
-   */
-  private static final Duration ACK_TOO_LATE = Duration.standardSeconds(2);
-
-  /**
-   * Period of samples to determine watermark and other stats.
-   */
-  private static final Duration SAMPLE_PERIOD = Duration.standardMinutes(1);
-
-  /**
-   * Period of updates to determine watermark and other stats.
-   */
-  private static final Duration SAMPLE_UPDATE = Duration.standardSeconds(5);
-
-  /**
-   * Period for logging stats.
-   */
-  private static final Duration LOG_PERIOD = Duration.standardSeconds(30);
-
-  /**
-   * Minimum number of unread messages required before considering updating watermark.
-   */
-  private static final int MIN_WATERMARK_MESSAGES = 10;
-
-  /**
-   * Minimum number of SAMPLE_UPDATE periods over which unread messages shoud be spread
-   * before considering updating watermark.
-   */
-  private static final int MIN_WATERMARK_SPREAD = 2;
-
-  /**
-   * Additional sharding so that we can hide read message latency.
-   */
-  private static final int SCALE_OUT = 4;
-
-  // TODO: Would prefer to use MinLongFn but it is a BinaryCombineFn<Long> rather
-  // than a BinaryCombineLongFn. [BEAM-285]
-  private static final Combine.BinaryCombineLongFn MIN =
-      new Combine.BinaryCombineLongFn() {
-        @Override
-        public long apply(long left, long right) {
-          return Math.min(left, right);
-        }
-
-        @Override
-        public long identity() {
-          return Long.MAX_VALUE;
-        }
-      };
-
-  private static final Combine.BinaryCombineLongFn MAX =
-      new Combine.BinaryCombineLongFn() {
-        @Override
-        public long apply(long left, long right) {
-          return Math.max(left, right);
-        }
-
-        @Override
-        public long identity() {
-          return Long.MIN_VALUE;
-        }
-      };
-
-  private static final Combine.BinaryCombineLongFn SUM = Sum.ofLongs();
-
-  // ================================================================================
-  // Checkpoint
-  // ================================================================================
-
-  /**
-   * Which messages have been durably committed and thus can now be ACKed.
-   * Which messages have been read but not yet committed, in which case they should be NACKed if
-   * we need to restore.
-   */
-  @VisibleForTesting
-  static class PubsubCheckpoint<T> implements UnboundedSource.CheckpointMark {
-    /**
-     * The {@link SubscriptionPath} to the subscription the reader is reading from. May be
-     * {@code null} if the {@link PubsubUnboundedSource} contains the subscription.
-     */
-    @VisibleForTesting
-    @Nullable String subscriptionPath;
-
-    /**
-     * If the checkpoint is for persisting: the reader who's snapshotted state we are persisting.
-     * If the checkpoint is for restoring: {@literal null}.
-     * Not persisted in durable checkpoint.
-     * CAUTION: Between a checkpoint being taken and {@link #finalizeCheckpoint()} being called
-     * the 'true' active reader may have changed.
-     */
-    @Nullable
-    private PubsubReader<T> reader;
-
-    /**
-     * If the checkpoint is for persisting: The ACK ids of messages which have been passed
-     * downstream since the last checkpoint.
-     * If the checkpoint is for restoring: {@literal null}.
-     * Not persisted in durable checkpoint.
-     */
-    @Nullable
-    private List<String> safeToAckIds;
-
-    /**
-     * If the checkpoint is for persisting: The ACK ids of messages which have been received
-     * from Pubsub but not yet passed downstream at the time of the snapshot.
-     * If the checkpoint is for restoring: Same, but recovered from durable storage.
-     */
-    @VisibleForTesting
-    final List<String> notYetReadIds;
-
-    public PubsubCheckpoint(
-        @Nullable String subscriptionPath,
-        @Nullable PubsubReader<T> reader,
-        @Nullable List<String> safeToAckIds,
-        List<String> notYetReadIds) {
-      this.subscriptionPath = subscriptionPath;
-      this.reader = reader;
-      this.safeToAckIds = safeToAckIds;
-      this.notYetReadIds = notYetReadIds;
-    }
-
-    @Nullable
-    private SubscriptionPath getSubscription() {
-      return subscriptionPath == null
-          ? null
-          : PubsubClient.subscriptionPathFromPath(subscriptionPath);
-    }
-
-    /**
-     * BLOCKING
-     * All messages which have been passed downstream have now been durably committed.
-     * We can ACK them upstream.
-     * CAUTION: This may never be called.
-     */
-    @Override
-    public void finalizeCheckpoint() throws IOException {
-      checkState(reader != null && safeToAckIds != null, "Cannot finalize a restored checkpoint");
-      // Even if the 'true' active reader has changed since the checkpoint was taken we are
-      // fine:
-      // - The underlying Pubsub topic will not have changed, so the following ACKs will still
-      // go to the right place.
-      // - We'll delete the ACK ids from the readers in-flight state, but that only effects
-      // flow control and stats, neither of which are relevant anymore.
-      try {
-        int n = safeToAckIds.size();
-        List<String> batchSafeToAckIds = new ArrayList<>(Math.min(n, ACK_BATCH_SIZE));
-        for (String ackId : safeToAckIds) {
-          batchSafeToAckIds.add(ackId);
-          if (batchSafeToAckIds.size() >= ACK_BATCH_SIZE) {
-            reader.ackBatch(batchSafeToAckIds);
-            n -= batchSafeToAckIds.size();
-            // CAUTION: Don't reuse the same list since ackBatch holds on to it.
-            batchSafeToAckIds = new ArrayList<>(Math.min(n, ACK_BATCH_SIZE));
-          }
-        }
-        if (!batchSafeToAckIds.isEmpty()) {
-          reader.ackBatch(batchSafeToAckIds);
-        }
-      } finally {
-        int remainingInFlight = reader.numInFlightCheckpoints.decrementAndGet();
-        checkState(remainingInFlight >= 0,
-                   "Miscounted in-flight checkpoints");
-        reader.maybeCloseClient();
-        reader = null;
-        safeToAckIds = null;
-      }
-    }
-
-    /**
-     * Return current time according to {@code reader}.
-     */
-    private static long now(PubsubReader<?> reader) {
-      if (reader.outer.outer.clock == null) {
-        return System.currentTimeMillis();
-      } else {
-        return reader.outer.outer.clock.currentTimeMillis();
-      }
-    }
-
-    /**
-     * BLOCKING
-     * NACK all messages which have been read from Pubsub but not passed downstream.
-     * This way Pubsub will send them again promptly.
-     */
-    public void nackAll(PubsubReader<T> reader) throws IOException {
-      checkState(this.reader == null, "Cannot nackAll on persisting checkpoint");
-      List<String> batchYetToAckIds =
-          new ArrayList<>(Math.min(notYetReadIds.size(), ACK_BATCH_SIZE));
-      for (String ackId : notYetReadIds) {
-        batchYetToAckIds.add(ackId);
-        if (batchYetToAckIds.size() >= ACK_BATCH_SIZE) {
-          long nowMsSinceEpoch = now(reader);
-          reader.nackBatch(nowMsSinceEpoch, batchYetToAckIds);
-          batchYetToAckIds.clear();
-        }
-      }
-      if (!batchYetToAckIds.isEmpty()) {
-        long nowMsSinceEpoch = now(reader);
-        reader.nackBatch(nowMsSinceEpoch, batchYetToAckIds);
-      }
-    }
-  }
-
-  /** The coder for our checkpoints. */
-  private static class PubsubCheckpointCoder<T> extends AtomicCoder<PubsubCheckpoint<T>> {
-    private static final Coder<String> SUBSCRIPTION_PATH_CODER =
-        NullableCoder.of(StringUtf8Coder.of());
-    private static final Coder<List<String>> LIST_CODER = ListCoder.of(StringUtf8Coder.of());
-
-    @Override
-    public void encode(PubsubCheckpoint<T> value, OutputStream outStream, Context context)
-        throws IOException {
-      SUBSCRIPTION_PATH_CODER.encode(
-          value.subscriptionPath,
-          outStream,
-          context.nested());
-      LIST_CODER.encode(value.notYetReadIds, outStream, context);
-    }
-
-    @Override
-    public PubsubCheckpoint<T> decode(InputStream inStream, Context context) throws IOException {
-      String path = SUBSCRIPTION_PATH_CODER.decode(inStream, context.nested());
-      List<String> notYetReadIds = LIST_CODER.decode(inStream, context);
-      return new PubsubCheckpoint<>(path, null, null, notYetReadIds);
-    }
-  }
-
-  // ================================================================================
-  // Reader
-  // ================================================================================
-
-  /**
-   * A reader which keeps track of which messages have been received from Pubsub
-   * but not yet consumed downstream and/or ACKed back to Pubsub.
-   */
-  @VisibleForTesting
-  static class PubsubReader<T> extends UnboundedSource.UnboundedReader<T> {
-    /**
-     * For access to topic and checkpointCoder.
-     */
-    private final PubsubSource<T> outer;
-    @VisibleForTesting
-    final SubscriptionPath subscription;
-
-    private final SimpleFunction<PubsubIO.PubsubMessage, T> parseFn;
-
-    /**
-     * Client on which to talk to Pubsub. Contains a null value if the client has been closed.
-     */
-    private AtomicReference<PubsubClient> pubsubClient;
-
-    /**
-     * The closed state of this {@link PubsubReader}. If true, the reader has not yet been closed,
-     * and it will have a non-null value within {@link #pubsubClient}.
-     */
-    private AtomicBoolean active = new AtomicBoolean(true);
-
-    /**
-     * Ack timeout, in ms, as set on subscription when we first start reading. Not
-     * updated thereafter. -1 if not yet determined.
-     */
-    private int ackTimeoutMs;
-
-    /**
-     * ACK ids of messages we have delivered downstream but not yet ACKed.
-     */
-    private Set<String> safeToAckIds;
-
-    /**
-     * Messages we have received from Pubsub and not yet delivered downstream.
-     * We preserve their order.
-     */
-    private final Queue<PubsubClient.IncomingMessage> notYetRead;
-
-    private static class InFlightState {
-      /**
-       * When request which yielded message was issues.
-       */
-      long requestTimeMsSinceEpoch;
-
-      /**
-       * When Pubsub will consider this message's ACK to timeout and thus it needs to be
-       * extended.
-       */
-      long ackDeadlineMsSinceEpoch;
-
-      public InFlightState(long requestTimeMsSinceEpoch, long ackDeadlineMsSinceEpoch) {
-        this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch;
-        this.ackDeadlineMsSinceEpoch = ackDeadlineMsSinceEpoch;
-      }
-    }
-
-    /**
-     * Map from ACK ids of messages we have received from Pubsub but not yet ACKed to their
-     * in flight state. Ordered from earliest to latest ACK deadline.
-     */
-    private final LinkedHashMap<String, InFlightState> inFlight;
-
-    /**
-     * Batches of successfully ACKed ids which need to be pruned from the above.
-     * CAUTION: Accessed by both reader and checkpointing threads.
-     */
-    private final Queue<List<String>> ackedIds;
-
-    /**
-     * Byte size of undecoded elements in {@link #notYetRead}.
-     */
-    private long notYetReadBytes;
-
-    /**
-     * Bucketed map from received time (as system time, ms since epoch) to message
-     * timestamps (mssince epoch) of all received but not-yet read messages.
-     * Used to estimate watermark.
-     */
-    private BucketingFunction minUnreadTimestampMsSinceEpoch;
-
-    /**
-     * Minimum of timestamps (ms since epoch) of all recently read messages.
-     * Used to estimate watermark.
-     */
-    private MovingFunction minReadTimestampMsSinceEpoch;
-
-    /**
-     * System time (ms since epoch) we last received a message from Pubsub, or -1 if
-     * not yet received any messages.
-     */
-    private long lastReceivedMsSinceEpoch;
-
-    /**
-     * The last reported watermark (ms since epoch), or beginning of time if none yet reported.
-     */
-    private long lastWatermarkMsSinceEpoch;
-
-    /**
-     * The current message, or {@literal null} if none.
-     */
-    @Nullable
-    private PubsubClient.IncomingMessage current;
-
-    /**
-     * Stats only: System time (ms since epoch) we last logs stats, or -1 if never.
-     */
-    private long lastLogTimestampMsSinceEpoch;
-
-    /**
-     * Stats only: Total number of messages received.
-     */
-    private long numReceived;
-
-    /**
-     * Stats only: Number of messages which have recently been received.
-     */
-    private MovingFunction numReceivedRecently;
-
-    /**
-     * Stats only: Number of messages which have recently had their deadline extended.
-     */
-    private MovingFunction numExtendedDeadlines;
-
-    /**
-     * Stats only: Number of messages which have recenttly had their deadline extended even
-     * though it may be too late to do so.
-     */
-    private MovingFunction numLateDeadlines;
-
-
-    /**
-     * Stats only: Number of messages which have recently been ACKed.
-     */
-    private MovingFunction numAcked;
-
-    /**
-     * Stats only: Number of messages which have recently expired (ACKs were extended for too
-     * long).
-     */
-    private MovingFunction numExpired;
-
-    /**
-     * Stats only: Number of messages which have recently been NACKed.
-     */
-    private MovingFunction numNacked;
-
-    /**
-     * Stats only: Number of message bytes which have recently been read by downstream consumer.
-     */
-    private MovingFunction numReadBytes;
-
-    /**
-     * Stats only: Minimum of timestamp (ms since epoch) of all recently received messages.
-     * Used to estimate timestamp skew. Does not contribute to watermark estimator.
-     */
-    private MovingFunction minReceivedTimestampMsSinceEpoch;
-
-    /**
-     * Stats only: Maximum of timestamp (ms since epoch) of all recently received messages.
-     * Used to estimate timestamp skew.
-     */
-    private MovingFunction maxReceivedTimestampMsSinceEpoch;
-
-    /**
-     * Stats only: Minimum of recent estimated watermarks (ms since epoch).
-     */
-    private MovingFunction minWatermarkMsSinceEpoch;
-
-    /**
-     * Stats ony: Maximum of recent estimated watermarks (ms since epoch).
-     */
-    private MovingFunction maxWatermarkMsSinceEpoch;
-
-    /**
-     * Stats only: Number of messages with timestamps strictly behind the estimated watermark
-     * at the time they are received. These may be considered 'late' by downstream computations.
-     */
-    private MovingFunction numLateMessages;
-
-    /**
-     * Stats only: Current number of checkpoints in flight.
-     * CAUTION: Accessed by both checkpointing and reader threads.
-     */
-    private AtomicInteger numInFlightCheckpoints;
-
-    /**
-     * Stats only: Maximum number of checkpoints in flight at any time.
-     */
-    private int maxInFlightCheckpoints;
-
-    private static MovingFunction newFun(Combine.BinaryCombineLongFn function) {
-      return new MovingFunction(SAMPLE_PERIOD.getMillis(),
-                                SAMPLE_UPDATE.getMillis(),
-                                MIN_WATERMARK_SPREAD,
-                                MIN_WATERMARK_MESSAGES,
-                                function);
-    }
-
-    /**
-     * Construct a reader.
-     */
-    public PubsubReader(PubsubOptions options, PubsubSource<T> outer, SubscriptionPath subscription,
-                        SimpleFunction<PubsubIO.PubsubMessage, T> parseFn)
-        throws IOException, GeneralSecurityException {
-      this.outer = outer;
-      this.subscription = subscription;
-      this.parseFn = parseFn;
-      pubsubClient =
-          new AtomicReference<>(
-              outer.outer.pubsubFactory.newClient(
-                  outer.outer.timestampLabel, outer.outer.idLabel, options));
-      ackTimeoutMs = -1;
-      safeToAckIds = new HashSet<>();
-      notYetRead = new ArrayDeque<>();
-      inFlight = new LinkedHashMap<>();
-      ackedIds = new ConcurrentLinkedQueue<>();
-      notYetReadBytes = 0;
-      minUnreadTimestampMsSinceEpoch = new BucketingFunction(SAMPLE_UPDATE.getMillis(),
-                                                             MIN_WATERMARK_SPREAD,
-                                                             MIN_WATERMARK_MESSAGES,
-                                                             MIN);
-      minReadTimestampMsSinceEpoch = newFun(MIN);
-      lastReceivedMsSinceEpoch = -1;
-      lastWatermarkMsSinceEpoch = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
-      current = null;
-      lastLogTimestampMsSinceEpoch = -1;
-      numReceived = 0L;
-      numReceivedRecently = newFun(SUM);
-      numExtendedDeadlines = newFun(SUM);
-      numLateDeadlines = newFun(SUM);
-      numAcked = newFun(SUM);
-      numExpired = newFun(SUM);
-      numNacked = newFun(SUM);
-      numReadBytes = newFun(SUM);
-      minReceivedTimestampMsSinceEpoch = newFun(MIN);
-      maxReceivedTimestampMsSinceEpoch = newFun(MAX);
-      minWatermarkMsSinceEpoch = newFun(MIN);
-      maxWatermarkMsSinceEpoch = newFun(MAX);
-      numLateMessages = newFun(SUM);
-      numInFlightCheckpoints = new AtomicInteger();
-      maxInFlightCheckpoints = 0;
-    }
-
-    @VisibleForTesting
-    PubsubClient getPubsubClient() {
-      return pubsubClient.get();
-    }
-
-    /**
-     * Acks the provided {@code ackIds} back to Pubsub, blocking until all of the messages are
-     * ACKed.
-     *
-     * <p>CAUTION: May be invoked from a separate thread.
-     *
-     * <p>CAUTION: Retains {@code ackIds}.
-     */
-    void ackBatch(List<String> ackIds) throws IOException {
-      pubsubClient.get().acknowledge(subscription, ackIds);
-      ackedIds.add(ackIds);
-    }
-
-    /**
-     * BLOCKING
-     * NACK (ie request deadline extension of 0) receipt of messages from Pubsub
-     * with the given {@code ockIds}. Does not retain {@code ackIds}.
-     */
-    public void nackBatch(long nowMsSinceEpoch, List<String> ackIds) throws IOException {
-      pubsubClient.get().modifyAckDeadline(subscription, ackIds, 0);
-      numNacked.add(nowMsSinceEpoch, ackIds.size());
-    }
-
-    /**
-     * BLOCKING
-     * Extend the processing deadline for messages from Pubsub with the given {@code ackIds}.
-     * Does not retain {@code ackIds}.
-     */
-    private void extendBatch(long nowMsSinceEpoch, List<String> ackIds) throws IOException {
-      int extensionSec = (ackTimeoutMs * ACK_EXTENSION_PCT) / (100 * 1000);
-      pubsubClient.get().modifyAckDeadline(subscription, ackIds, extensionSec);
-      numExtendedDeadlines.add(nowMsSinceEpoch, ackIds.size());
-    }
-
-    /**
-     * Return the current time, in ms since epoch.
-     */
-    private long now() {
-      if (outer.outer.clock == null) {
-        return System.currentTimeMillis();
-      } else {
-        return outer.outer.clock.currentTimeMillis();
-      }
-    }
-
-    /**
-     * Messages which have been ACKed (via the checkpoint finalize) are no longer in flight.
-     * This is only used for flow control and stats.
-     */
-    private void retire() throws IOException {
-      long nowMsSinceEpoch = now();
-      while (true) {
-        List<String> ackIds = ackedIds.poll();
-        if (ackIds == null) {
-          return;
-        }
-        numAcked.add(nowMsSinceEpoch, ackIds.size());
-        for (String ackId : ackIds) {
-          inFlight.remove(ackId);
-          safeToAckIds.remove(ackId);
-        }
-      }
-    }
-
-    /**
-     * BLOCKING
-     * Extend deadline for all messages which need it.
-     * CAUTION: If extensions can't keep up with wallclock then we'll never return.
-     */
-    private void extend() throws IOException {
-      while (true) {
-        long nowMsSinceEpoch = now();
-        List<String> assumeExpired = new ArrayList<>();
-        List<String> toBeExtended = new ArrayList<>();
-        List<String> toBeExpired = new ArrayList<>();
-        // Messages will be in increasing deadline order.
-        for (Map.Entry<String, InFlightState> entry : inFlight.entrySet()) {
-          if (entry.getValue().ackDeadlineMsSinceEpoch - (ackTimeoutMs * ACK_SAFETY_PCT) / 100
-              > nowMsSinceEpoch) {
-            // All remaining messages don't need their ACKs to be extended.
-            break;
-          }
-
-          if (entry.getValue().ackDeadlineMsSinceEpoch - ACK_TOO_LATE.getMillis()
-              < nowMsSinceEpoch) {
-            // Pubsub may have already considered this message to have expired.
-            // If so it will (eventually) be made available on a future pull request.
-            // If this message ends up being committed then it will be considered a duplicate
-            // when re-pulled.
-            assumeExpired.add(entry.getKey());
-            continue;
-          }
-
-          if (entry.getValue().requestTimeMsSinceEpoch + PROCESSING_TIMEOUT.getMillis()
-              < nowMsSinceEpoch) {
-            // This message has been in-flight for too long.
-            // Give up on it, otherwise we risk extending its ACK indefinitely.
-            toBeExpired.add(entry.getKey());
-            continue;
-          }
-
-          // Extend the ACK for this message.
-          toBeExtended.add(entry.getKey());
-          if (toBeExtended.size() >= ACK_BATCH_SIZE) {
-            // Enough for one batch.
-            break;
-          }
-        }
-
-        if (assumeExpired.isEmpty() && toBeExtended.isEmpty() && toBeExpired.isEmpty()) {
-          // Nothing to be done.
-          return;
-        }
-
-        if (!assumeExpired.isEmpty()) {
-          // If we didn't make the ACK deadline assume expired and no longer in flight.
-          numLateDeadlines.add(nowMsSinceEpoch, assumeExpired.size());
-          for (String ackId : assumeExpired) {
-            inFlight.remove(ackId);
-          }
-        }
-
-        if (!toBeExpired.isEmpty()) {
-          // Expired messages are no longer considered in flight.
-          numExpired.add(nowMsSinceEpoch, toBeExpired.size());
-          for (String ackId : toBeExpired) {
-            inFlight.remove(ackId);
-          }
-        }
-
-        if (!toBeExtended.isEmpty()) {
-          // Pubsub extends acks from it's notion of current time.
-          // We'll try to track that on our side, but note the deadlines won't necessarily agree.
-          long newDeadlineMsSinceEpoch = nowMsSinceEpoch + (ackTimeoutMs * ACK_EXTENSION_PCT) / 100;
-          for (String ackId : toBeExtended) {
-            // Maintain increasing ack deadline order.
-            InFlightState state = inFlight.remove(ackId);
-            inFlight.put(ackId,
-                         new InFlightState(state.requestTimeMsSinceEpoch, newDeadlineMsSinceEpoch));
-          }
-          // BLOCKs until extended.
-          extendBatch(nowMsSinceEpoch, toBeExtended);
-        }
-      }
-    }
-
-    /**
-     * BLOCKING
-     * Fetch another batch of messages from Pubsub.
-     */
-    private void pull() throws IOException {
-      if (inFlight.size() >= MAX_IN_FLIGHT) {
-        // Wait for checkpoint to be finalized before pulling anymore.
-        // There may be lag while checkpoints are persisted and the finalizeCheckpoint method
-        // is invoked. By limiting the in-flight messages we can ensure we don't end up consuming
-        // messages faster than we can checkpoint them.
-        return;
-      }
-
-      long requestTimeMsSinceEpoch = now();
-      long deadlineMsSinceEpoch = requestTimeMsSinceEpoch + ackTimeoutMs;
-
-      // Pull the next batch.
-      // BLOCKs until received.
-      Collection<PubsubClient.IncomingMessage> receivedMessages =
-          pubsubClient.get().pull(requestTimeMsSinceEpoch, subscription, PULL_BATCH_SIZE, true);
-      if (receivedMessages.isEmpty()) {
-        // Nothing available yet. Try again later.
-        return;
-      }
-
-      lastReceivedMsSinceEpoch = requestTimeMsSinceEpoch;
-
-      // Capture the received messages.
-      for (PubsubClient.IncomingMessage incomingMessage : receivedMessages) {
-        notYetRead.add(incomingMessage);
-        notYetReadBytes += incomingMessage.elementBytes.length;
-        inFlight.put(incomingMessage.ackId,
-                     new InFlightState(requestTimeMsSinceEpoch, deadlineMsSinceEpoch));
-        numReceived++;
-        numReceivedRecently.add(requestTimeMsSinceEpoch, 1L);
-        minReceivedTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch,
-                                             incomingMessage.timestampMsSinceEpoch);
-        maxReceivedTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch,
-                                             incomingMessage.timestampMsSinceEpoch);
-        minUnreadTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch,
-                                           incomingMessage.timestampMsSinceEpoch);
-      }
-    }
-
-    /**
-     * Log stats if time to do so.
-     */
-    private void stats() {
-      long nowMsSinceEpoch = now();
-      if (lastLogTimestampMsSinceEpoch < 0) {
-        lastLogTimestampMsSinceEpoch = nowMsSinceEpoch;
-        return;
-      }
-      long deltaMs = nowMsSinceEpoch - lastLogTimestampMsSinceEpoch;
-      if (deltaMs < LOG_PERIOD.getMillis()) {
-        return;
-      }
-
-      String messageSkew = "unknown";
-      long minTimestamp = minReceivedTimestampMsSinceEpoch.get(nowMsSinceEpoch);
-      long maxTimestamp = maxReceivedTimestampMsSinceEpoch.get(nowMsSinceEpoch);
-      if (minTimestamp < Long.MAX_VALUE && maxTimestamp > Long.MIN_VALUE) {
-        messageSkew = (maxTimestamp - minTimestamp) + "ms";
-      }
-
-      String watermarkSkew = "unknown";
-      long minWatermark = minWatermarkMsSinceEpoch.get(nowMsSinceEpoch);
-      long maxWatermark = maxWatermarkMsSinceEpoch.get(nowMsSinceEpoch);
-      if (minWatermark < Long.MAX_VALUE && maxWatermark > Long.MIN_VALUE) {
-        watermarkSkew = (maxWatermark - minWatermark) + "ms";
-      }
-
-      String oldestInFlight = "no";
-      String oldestAckId = Iterables.getFirst(inFlight.keySet(), null);
-      if (oldestAckId != null) {
-        oldestInFlight =
-            (nowMsSinceEpoch - inFlight.get(oldestAckId).requestTimeMsSinceEpoch) + "ms";
-      }
-
-      LOG.info("Pubsub {} has "
-               + "{} received messages, "
-               + "{} current unread messages, "
-               + "{} current unread bytes, "
-               + "{} current in-flight msgs, "
-               + "{} oldest in-flight, "
-               + "{} current in-flight checkpoints, "
-               + "{} max in-flight checkpoints, "
-               + "{}B/s recent read, "
-               + "{} recent received, "
-               + "{} recent extended, "
-               + "{} recent late extended, "
-               + "{} recent ACKed, "
-               + "{} recent NACKed, "
-               + "{} recent expired, "
-               + "{} recent message timestamp skew, "
-               + "{} recent watermark skew, "
-               + "{} recent late messages, "
-               + "{} last reported watermark",
-               subscription,
-               numReceived,
-               notYetRead.size(),
-               notYetReadBytes,
-               inFlight.size(),
-               oldestInFlight,
-               numInFlightCheckpoints.get(),
-               maxInFlightCheckpoints,
-               numReadBytes.get(nowMsSinceEpoch) / (SAMPLE_PERIOD.getMillis() / 1000L),
-               numReceivedRecently.get(nowMsSinceEpoch),
-               numExtendedDeadlines.get(nowMsSinceEpoch),
-               numLateDeadlines.get(nowMsSinceEpoch),
-               numAcked.get(nowMsSinceEpoch),
-               numNacked.get(nowMsSinceEpoch),
-               numExpired.get(nowMsSinceEpoch),
-               messageSkew,
-               watermarkSkew,
-               numLateMessages.get(nowMsSinceEpoch),
-               new Instant(lastWatermarkMsSinceEpoch));
-
-      lastLogTimestampMsSinceEpoch = nowMsSinceEpoch;
-    }
-
-    @Override
-    public boolean start() throws IOException {
-      // Determine the ack timeout.
-      ackTimeoutMs = pubsubClient.get().ackDeadlineSeconds(subscription) * 1000;
-      return advance();
-    }
-
-    /**
-     * BLOCKING
-     * Return {@literal true} if a Pubsub messaage is available, {@literal false} if
-     * none is available at this time or we are over-subscribed. May BLOCK while extending
-     * ACKs or fetching available messages. Will not block waiting for messages.
-     */
-    @Override
-    public boolean advance() throws IOException {
-      // Emit stats.
-      stats();
-
-      if (current != null) {
-        // Current is consumed. It can no longer contribute to holding back the watermark.
-        minUnreadTimestampMsSinceEpoch.remove(current.requestTimeMsSinceEpoch);
-        current = null;
-      }
-
-      // Retire state associated with ACKed messages.
-      retire();
-
-      // Extend all pressing deadlines.
-      // Will BLOCK until done.
-      // If the system is pulling messages only to let them sit in a downsteam queue then
-      // this will have the effect of slowing down the pull rate.
-      // However, if the system is genuinely taking longer to process each message then
-      // the work to extend ACKs would be better done in the background.
-      extend();
-
-      if (notYetRead.isEmpty()) {
-        // Pull another batch.
-        // Will BLOCK until fetch returns, but will not block until a message is available.
-        pull();
-      }
-
-      // Take one message from queue.
-      current = notYetRead.poll();
-      if (current == null) {
-        // Try again later.
-        return false;
-      }
-      notYetReadBytes -= current.elementBytes.length;
-      checkState(notYetReadBytes >= 0);
-      long nowMsSinceEpoch = now();
-      numReadBytes.add(nowMsSinceEpoch, current.elementBytes.length);
-      minReadTimestampMsSinceEpoch.add(nowMsSinceEpoch, current.timestampMsSinceEpoch);
-      if (current.timestampMsSinceEpoch < lastWatermarkMsSinceEpoch) {
-        numLateMessages.add(nowMsSinceEpoch, 1L);
-      }
-
-      // Current message can be considered 'read' and will be persisted by the next
-      // checkpoint. So it is now safe to ACK back to Pubsub.
-      safeToAckIds.add(current.ackId);
-      return true;
-    }
-
-    @Override
-    public T getCurrent() throws NoSuchElementException {
-      if (current == null) {
-        throw new NoSuchElementException();
-      }
-      try {
-        if (parseFn != null) {
-          return parseFn.apply(new PubsubIO.PubsubMessage(
-                  current.elementBytes, current.attributes));
-        } else {
-          return CoderUtils.decodeFromByteArray(outer.outer.elementCoder, current.elementBytes);
-        }
-      } catch (CoderException e) {
-        throw new RuntimeException("Unable to decode element from Pubsub message: ", e);
-      }
-    }
-
-    @Override
-    public Instant getCurrentTimestamp() throws NoSuchElementException {
-      if (current == null) {
-        throw new NoSuchElementException();
-      }
-      return new Instant(current.timestampMsSinceEpoch);
-    }
-
-    @Override
-    public byte[] getCurrentRecordId() throws NoSuchElementException {
-      if (current == null) {
-        throw new NoSuchElementException();
-      }
-      return current.recordId.getBytes(Charsets.UTF_8);
-    }
-
-    /**
-     * {@inheritDoc}.
-     *
-     * <p>Marks this {@link PubsubReader} as no longer active. The {@link PubsubClient}
-     * continue to exist and be active beyond the life of this call if there are any in-flight
-     * checkpoints. When no in-flight checkpoints remain, the reader will be closed.
-     */
-    @Override
-    public void close() throws IOException {
-      active.set(false);
-      maybeCloseClient();
-    }
-
-    /**
-     * Close this reader's underlying {@link PubsubClient} if the reader has been closed and there
-     * are no outstanding checkpoints.
-     */
-    private void maybeCloseClient() throws IOException {
-      if (!active.get() && numInFlightCheckpoints.get() == 0) {
-        // The reader has been closed and it has no more outstanding checkpoints. The client
-        // must be closed so it doesn't leak
-        PubsubClient client = pubsubClient.getAndSet(null);
-        if (client != null) {
-          client.close();
-        }
-      }
-    }
-
-    @Override
-    public PubsubSource<T> getCurrentSource() {
-      return outer;
-    }
-
-    @Override
-    public Instant getWatermark() {
-      if (pubsubClient.get().isEOF() && notYetRead.isEmpty()) {
-        // For testing only: Advance the watermark to the end of time to signal
-        // the test is complete.
-        return BoundedWindow.TIMESTAMP_MAX_VALUE;
-      }
-
-      // NOTE: We'll allow the watermark to go backwards. The underlying runner is responsible
-      // for aggregating all reported watermarks and ensuring the aggregate is latched.
-      // If we attempt to latch locally then it is possible a temporary starvation of one reader
-      // could cause its estimated watermark to fast forward to current system time. Then when
-      // the reader resumes its watermark would be unable to resume tracking.
-      // By letting the underlying runner latch we avoid any problems due to localized starvation.
-      long nowMsSinceEpoch = now();
-      long readMin = minReadTimestampMsSinceEpoch.get(nowMsSinceEpoch);
-      long unreadMin = minUnreadTimestampMsSinceEpoch.get();
-      if (readMin == Long.MAX_VALUE
-          && unreadMin == Long.MAX_VALUE
-          && lastReceivedMsSinceEpoch >= 0
-          && nowMsSinceEpoch > lastReceivedMsSinceEpoch + SAMPLE_PERIOD.getMillis()) {
-        // We don't currently have any unread messages pending, we have not had any messages
-        // read for a while, and we have not received any new messages from Pubsub for a while.
-        // Advance watermark to current time.
-        // TODO: Estimate a timestamp lag.
-        lastWatermarkMsSinceEpoch = nowMsSinceEpoch;
-      } else if (minReadTimestampMsSinceEpoch.isSignificant()
-                 || minUnreadTimestampMsSinceEpoch.isSignificant()) {
-        // Take minimum of the timestamps in all unread messages and recently read messages.
-        lastWatermarkMsSinceEpoch = Math.min(readMin, unreadMin);
-      }
-      // else: We're not confident enough to estimate a new watermark. Stick with the old one.
-      minWatermarkMsSinceEpoch.add(nowMsSinceEpoch, lastWatermarkMsSinceEpoch);
-      maxWatermarkMsSinceEpoch.add(nowMsSinceEpoch, lastWatermarkMsSinceEpoch);
-      return new Instant(lastWatermarkMsSinceEpoch);
-    }
-
-    @Override
-    public PubsubCheckpoint<T> getCheckpointMark() {
-      int cur = numInFlightCheckpoints.incrementAndGet();
-      maxInFlightCheckpoints = Math.max(maxInFlightCheckpoints, cur);
-      // It's possible for a checkpoint to be taken but never finalized.
-      // So we simply copy whatever safeToAckIds we currently have.
-      // It's possible a later checkpoint will be taken before an earlier one is finalized,
-      // in which case we'll double ACK messages to Pubsub. However Pubsub is fine with that.
-      List<String> snapshotSafeToAckIds = Lists.newArrayList(safeToAckIds);
-      List<String> snapshotNotYetReadIds = new ArrayList<>(notYetRead.size());
-      for (PubsubClient.IncomingMessage incomingMessage : notYetRead) {
-        snapshotNotYetReadIds.add(incomingMessage.ackId);
-      }
-      if (outer.subscriptionPath == null) {
-        // need to include the subscription in case we resume, as it's not stored in the source.
-        return new PubsubCheckpoint<>(
-            subscription.getPath(), this, snapshotSafeToAckIds, snapshotNotYetReadIds);
-      }
-      return new PubsubCheckpoint<>(null, this, snapshotSafeToAckIds, snapshotNotYetReadIds);
-    }
-
-    @Override
-    public long getSplitBacklogBytes() {
-      return notYetReadBytes;
-    }
-  }
-
-  // ================================================================================
-  // Source
-  // ================================================================================
-
-  @VisibleForTesting
-  static class PubsubSource<T> extends UnboundedSource<T, PubsubCheckpoint<T>> {
-    public final PubsubUnboundedSource<T> outer;
-    // The subscription to read from.
-    @VisibleForTesting
-    final SubscriptionPath subscriptionPath;
-
-    public PubsubSource(PubsubUnboundedSource<T> outer) {
-      this(outer, outer.getSubscription());
-    }
-
-    private PubsubSource(PubsubUnboundedSource<T> outer, SubscriptionPath subscriptionPath) {
-      this.outer = outer;
-      this.subscriptionPath = subscriptionPath;
-    }
-
-    @Override
-    public List<PubsubSource<T>> generateInitialSplits(
-        int desiredNumSplits, PipelineOptions options) throws Exception {
-      List<PubsubSource<T>> result = new ArrayList<>(desiredNumSplits);
-      PubsubSource<T> splitSource = this;
-      if (subscriptionPath == null) {
-        splitSource = new PubsubSource<>(outer, outer.createRandomSubscription(options));
-      }
-      for (int i = 0; i < desiredNumSplits * SCALE_OUT; i++) {
-        // Since the source is immutable and Pubsub automatically shards we simply
-        // replicate ourselves the requested number of times
-        result.add(splitSource);
-      }
-      return result;
-    }
-
-    @Override
-    public PubsubReader<T> createReader(
-        PipelineOptions options,
-        @Nullable PubsubCheckpoint<T> checkpoint) {
-      PubsubReader<T> reader;
-      SubscriptionPath subscription = subscriptionPath;
-      if (subscription == null) {
-        if (checkpoint == null) {
-          // This reader has never been started and there was no call to #splitIntoBundles; create
-          // a single random subscription, which will be kept in the checkpoint.
-          subscription = outer.createRandomSubscription(options);
-        } else {
-          subscription = checkpoint.getSubscription();
-        }
-      }
-      try {
-        reader = new PubsubReader<>(options.as(PubsubOptions.class), this, subscription,
-                outer.parseFn);
-      } catch (GeneralSecurityException | IOException e) {
-        throw new RuntimeException("Unable to subscribe to " + subscriptionPath + ": ", e);
-      }
-      if (checkpoint != null) {
-        // NACK all messages we may have lost.
-        try {
-          // Will BLOCK until NACKed.
-          checkpoint.nackAll(reader);
-        } catch (IOException e) {
-          LOG.error("Pubsub {} cannot have {} lost messages NACKed, ignoring: {}",
-                    subscriptionPath, checkpoint.notYetReadIds.size(), e);
-        }
-      }
-      return reader;
-    }
-
-    @Nullable
-    @Override
-    public Coder<PubsubCheckpoint<T>> getCheckpointMarkCoder() {
-      @SuppressWarnings("unchecked") PubsubCheckpointCoder<T> typedCoder =
-          (PubsubCheckpointCoder<T>) CHECKPOINT_CODER;
-      return typedCoder;
-    }
-
-    @Override
-    public Coder<T> getDefaultOutputCoder() {
-      return outer.elementCoder;
-    }
-
-    @Override
-    public void validate() {
-      // Nothing to validate.
-    }
-
-    @Override
-    public boolean requiresDeduping() {
-      // We cannot prevent re-offering already read messages after a restore from checkpoint.
-      return true;
-    }
-  }
-
-  // ================================================================================
-  // StatsFn
-  // ================================================================================
-
-  private static class StatsFn<T> extends DoFn<T, T> {
-    private final Counter elementCounter = Metrics.counter(StatsFn.class, "elements");
-
-    private final PubsubClientFactory pubsubFactory;
-    @Nullable
-    private final ValueProvider<SubscriptionPath> subscription;
-    @Nullable
-    private final ValueProvider<TopicPath> topic;
-    @Nullable
-    private final String timestampLabel;
-    @Nullable
-    private final String idLabel;
-
-    public StatsFn(
-        PubsubClientFactory pubsubFactory,
-        @Nullable ValueProvider<SubscriptionPath> subscription,
-        @Nullable ValueProvider<TopicPath> topic,
-        @Nullable String timestampLabel,
-        @Nullable String idLabel) {
-      checkArgument(pubsubFactory != null, "pubsubFactory should not be null");
-      this.pubsubFactory = pubsubFactory;
-      this.subscription = subscription;
-      this.topic = topic;
-      this.timestampLabel = timestampLabel;
-      this.idLabel = idLabel;
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext c) throws Exception {
-      elementCounter.inc();
-      c.output(c.element());
-    }
-
-    @Override
-    public void populateDisplayData(Builder builder) {
-      super.populateDisplayData(builder);
-      if (subscription != null) {
-        String subscriptionString = subscription.isAccessible()
-            ? subscription.get().getPath()
-            : subscription.toString();
-        builder.add(DisplayData.item("subscription", subscriptionString));
-      }
-      if (topic != null) {
-        String topicString = topic.isAccessible()
-            ? topic.get().getPath()
-            : topic.toString();
-        builder.add(DisplayData.item("topic", topicString));
-      }
-      builder.add(DisplayData.item("transport", pubsubFactory.getKind()));
-      builder.addIfNotNull(DisplayData.item("timestampLabel", timestampLabel));
-      builder.addIfNotNull(DisplayData.item("idLabel", idLabel));
-    }
-  }
-
-  // ================================================================================
-  // PubsubUnboundedSource
-  // ================================================================================
-
-  /**
-   * For testing only: Clock to use for all timekeeping. If {@literal null} use system clock.
-   */
-  @Nullable
-  private Clock clock;
-
-  /**
-   * Factory for creating underlying Pubsub transport.
-   */
-  private final PubsubClientFactory pubsubFactory;
-
-  /**
-   * Project under which to create a subscription if only the {@link #topic} was given.
-   */
-  @Nullable
-  private final ValueProvider<ProjectPath> project;
-
-  /**
-   * Topic to read from. If {@literal null}, then {@link #subscription} must be given.
-   * Otherwise {@link #subscription} must be null.
-   */
-  @Nullable
-  private final ValueProvider<TopicPath> topic;
-
-  /**
-   * Subscription to read from. If {@literal null} then {@link #topic} must be given.
-   * Otherwise {@link #topic} must be null.
-   *
-   * <p>If no subscription is given a random one will be created when the transorm is
-   * applied. This field will be update with that subscription's path. The created
-   * subscription is never deleted.
-   */
-  @Nullable
-  private ValueProvider<SubscriptionPath> subscription;
-
-  /**
-   * Coder for elements. Elements are effectively double-encoded: first to a byte array
-   * using this checkpointCoder, then to a base-64 string to conform to Pubsub's payload
-   * conventions.
-   */
-  private final Coder<T> elementCoder;
-
-  /**
-   * Pubsub metadata field holding timestamp of each element, or {@literal null} if should use
-   * Pubsub message publish timestamp instead.
-   */
-  @Nullable
-  private final String timestampLabel;
-
-  /**
-   * Pubsub metadata field holding id for each element, or {@literal null} if need to generate
-   * a unique id ourselves.
-   */
-  @Nullable
-  private final String idLabel;
-
-  /**
-   * If not {@literal null}, the user is asking for PubSub attributes. This parse function will be
-   * used to parse {@link PubsubIO.PubsubMessage}s containing a payload and attributes.
-   */
-  @Nullable
-  SimpleFunction<PubsubMessage, T> parseFn;
-
-  @VisibleForTesting
-  PubsubUnboundedSource(
-      Clock clock,
-      PubsubClientFactory pubsubFactory,
-      @Nullable ValueProvider<ProjectPath> project,
-      @Nullable ValueProvider<TopicPath> topic,
-      @Nullable ValueProvider<SubscriptionPath> subscription,
-      Coder<T> elementCoder,
-      @Nullable String timestampLabel,
-      @Nullable String idLabel,
-      @Nullable SimpleFunction<PubsubIO.PubsubMessage, T> parseFn) {
-    checkArgument((topic == null) != (subscription == null),
-                  "Exactly one of topic and subscription must be given");
-    checkArgument((topic == null) == (project == null),
-                  "Project must be given if topic is given");
-    this.clock = clock;
-    this.pubsubFactory = checkNotNull(pubsubFactory);
-    this.project = project;
-    this.topic = topic;
-    this.subscription = subscription;
-    this.elementCoder = checkNotNull(elementCoder);
-    this.timestampLabel = timestampLabel;
-    this.idLabel = idLabel;
-    this.parseFn = parseFn;
-  }
-
-  /**
-   * Construct an unbounded source to consume from the Pubsub {@code subscription}.
-   */
-  public PubsubUnboundedSource(
-      PubsubClientFactory pubsubFactory,
-      @Nullable ValueProvider<ProjectPath> project,
-      @Nullable ValueProvider<TopicPath> topic,
-      @Nullable ValueProvider<SubscriptionPath> subscription,
-      Coder<T> elementCoder,
-      @Nullable String timestampLabel,
-      @Nullable String idLabel,
-      @Nullable SimpleFunction<PubsubIO.PubsubMessage, T> parseFn) {
-    this(null, pubsubFactory, project, topic, subscription, elementCoder, timestampLabel, idLabel,
-        parseFn);
-  }
-
-  /**
-   * Get the coder used for elements.
-   */
-  public Coder<T> getElementCoder() {
-    return elementCoder;
-  }
-
-  /**
-   * Get the project path.
-   */
-  @Nullable
-  public ProjectPath getProject() {
-    return project == null ? null : project.get();
-  }
-
-  /**
-   * Get the topic being read from.
-   */
-  @Nullable
-  public TopicPath getTopic() {
-    return topic == null ? null : topic.get();
-  }
-
-  /**
-   * Get the {@link ValueProvider} for the topic being read from.
-   */
-  @Nullable
-  public ValueProvider<TopicPath> getTopicProvider() {
-    return topic;
-  }
-
-  /**
-   * Get the subscription being read from.
-   */
-  @Nullable
-  public SubscriptionPath getSubscription() {
-    return subscription == null ? null : subscription.get();
-  }
-
-  /**
-   * Get the {@link ValueProvider} for the subscription being read from.
-   */
-  @Nullable
-  public ValueProvider<SubscriptionPath> getSubscriptionProvider() {
-    return subscription;
-  }
-
-  /**
-   * Get the timestamp label.
-   */
-  @Nullable
-  public String getTimestampLabel() {
-    return timestampLabel;
-  }
-
-  /**
-   * Get the id label.
-   */
-  @Nullable
-  public String getIdLabel() {
-    return idLabel;
-  }
-
-  /**
-   * Get the parsing function for PubSub attributes.
-   */
-  @Nullable
-  public SimpleFunction<PubsubIO.PubsubMessage, T> getWithAttributesParseFn() {
-    return parseFn;
-  }
-
-  @Override
-  public PCollection<T> expand(PBegin input) {
-    return input.getPipeline().begin()
-                .apply(Read.from(new PubsubSource<T>(this)))
-                .apply("PubsubUnboundedSource.Stats",
-                    ParDo.of(new StatsFn<T>(
-                        pubsubFactory, subscription, topic, timestampLabel, idLabel)));
-  }
-
-  private SubscriptionPath createRandomSubscription(PipelineOptions options) {
-    try {
-      try (PubsubClient pubsubClient =
-          pubsubFactory.newClient(timestampLabel, idLabel, options.as(PubsubOptions.class))) {
-        checkState(project.isAccessible(), "createRandomSubscription must be called at runtime.");
-        checkState(topic.isAccessible(), "createRandomSubscription must be called at runtime.");
-        SubscriptionPath subscriptionPath =
-            pubsubClient.createRandomSubscription(
-                project.get(), topic.get(), DEAULT_ACK_TIMEOUT_SEC);
-        LOG.warn(
-            "Created subscription {} to topic {}."
-                + " Note this subscription WILL NOT be deleted when the pipeline terminates",
-            subscriptionPath,
-            topic);
-        return subscriptionPath;
-      }
-    } catch (Exception e) {
-      throw new RuntimeException("Failed to create subscription: ", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
index d228dbb..cc92102 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
@@ -101,9 +101,7 @@ import org.apache.beam.sdk.values.PCollection.IsBounded;
  * The output {@code PCollection} will have the same {@link WindowFn}
  * as the input.
  *
- * <p>If the input {@code PCollection} contains late data (see
- * {@link org.apache.beam.sdk.io.PubsubIO.Read#timestampLabel}
- * for an example of how this can occur) or the
+ * <p>If the input {@code PCollection} contains late data or the
  * {@link Window#triggering requested TriggerFn} can fire before
  * the watermark, then there may be multiple elements
  * output by a {@code GroupByKey} that correspond to the same key and window.

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
index 7cd2601..fb60538 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
@@ -33,18 +33,16 @@ import org.joda.time.Instant;
  * lower-bound, sometimes heuristically established, on event times that have been fully processed
  * by the pipeline.
  *
- * <p>For sources that provide non-heuristic watermarks (e.g.
- * {@link org.apache.beam.sdk.io.PubsubIO} when using arrival times as event times), the
- * watermark is a strict guarantee that no data with an event time earlier than
+ * <p>For sources that provide non-heuristic watermarks (e.g. PubsubIO when using arrival times as
+ * event times), the watermark is a strict guarantee that no data with an event time earlier than
  * that watermark will ever be observed in the pipeline. In this case, it's safe to assume that any
  * pane triggered by an {@code AfterWatermark} trigger with a reference point at or beyond the end
  * of the window will be the last pane ever for that window.
  *
- * <p>For sources that provide heuristic watermarks (e.g.
- * {@link org.apache.beam.sdk.io.PubsubIO} when using user-supplied event times), the
- * watermark itself becomes an <i>estimate</i> that no data with an event time earlier than that
- * watermark (i.e. "late data") will ever be observed in the pipeline. These heuristics can
- * often be quite accurate, but the chance of seeing late data for any given window is non-zero.
+ * <p>For sources that provide heuristic watermarks (e.g. PubsubIO when using user-supplied event
+ * times), the watermark itself becomes an <i>estimate</i> that no data with an event time earlier
+ * than that watermark (i.e. "late data") will ever be observed in the pipeline. These heuristics
+ * can often be quite accurate, but the chance of seeing late data for any given window is non-zero.
  * Thus, if absolute correctness over time is important to your use case, you may want to consider
  * using a trigger that accounts for late data. The default trigger,
  * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}, which fires

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java
deleted file mode 100644
index fc84057..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java
+++ /dev/null
@@ -1,544 +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.checkState;
-
-import com.google.api.client.util.DateTime;
-import com.google.common.base.Objects;
-import com.google.common.base.Strings;
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.options.PubsubOptions;
-
-/**
- * An (abstract) helper class for talking to Pubsub via an underlying transport.
- */
-public abstract class PubsubClient implements Closeable {
-  /**
-   * Factory for creating clients.
-   */
-  public interface PubsubClientFactory extends Serializable {
-    /**
-     * Construct a new Pubsub client. It should be closed via {@link #close} in order
-     * to ensure tidy cleanup of underlying netty resources (or use the try-with-resources
-     * construct). Uses {@code options} to derive pubsub endpoints and application credentials.
-     * If non-{@literal null}, use {@code timestampLabel} and {@code idLabel} to store custom
-     * timestamps/ids within message metadata.
-     */
-    PubsubClient newClient(
-        @Nullable String timestampLabel,
-        @Nullable String idLabel,
-        PubsubOptions options) throws IOException;
-
-    /**
-     * Return the display name for this factory. Eg "Json", "gRPC".
-     */
-    String getKind();
-  }
-
-  /**
-   * Return timestamp as ms-since-unix-epoch corresponding to {@code timestamp}.
-   * Return {@literal null} if no timestamp could be found. Throw {@link IllegalArgumentException}
-   * if timestamp cannot be recognized.
-   */
-  @Nullable
-  private static Long asMsSinceEpoch(@Nullable String timestamp) {
-    if (Strings.isNullOrEmpty(timestamp)) {
-      return null;
-    }
-    try {
-      // Try parsing as milliseconds since epoch. Note there is no way to parse a
-      // string in RFC 3339 format here.
-      // Expected IllegalArgumentException if parsing fails; we use that to fall back
-      // to RFC 3339.
-      return Long.parseLong(timestamp);
-    } catch (IllegalArgumentException e1) {
-      // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an
-      // IllegalArgumentException if parsing fails, and the caller should handle.
-      return DateTime.parseRfc3339(timestamp).getValue();
-    }
-  }
-
-  /**
-   * Return the timestamp (in ms since unix epoch) to use for a Pubsub message with {@code
-   * attributes} and {@code pubsubTimestamp}.
-   *
-   * <p>If {@code timestampLabel} is non-{@literal null} then the message attributes must contain
-   * that label, and the value of that label will be taken as the timestamp.
-   * Otherwise the timestamp will be taken from the Pubsub publish timestamp {@code
-   * pubsubTimestamp}.
-   *
-   * @throws IllegalArgumentException if the timestamp cannot be recognized as a ms-since-unix-epoch
-   * or RFC3339 time.
-   */
-  protected static long extractTimestamp(
-      @Nullable String timestampLabel,
-      @Nullable String pubsubTimestamp,
-      @Nullable Map<String, String> attributes) {
-    Long timestampMsSinceEpoch;
-    if (Strings.isNullOrEmpty(timestampLabel)) {
-      timestampMsSinceEpoch = asMsSinceEpoch(pubsubTimestamp);
-      checkArgument(timestampMsSinceEpoch != null,
-                    "Cannot interpret PubSub publish timestamp: %s",
-                    pubsubTimestamp);
-    } else {
-      String value = attributes == null ? null : attributes.get(timestampLabel);
-      checkArgument(value != null,
-                    "PubSub message is missing a value for timestamp label %s",
-                    timestampLabel);
-      timestampMsSinceEpoch = asMsSinceEpoch(value);
-      checkArgument(timestampMsSinceEpoch != null,
-                    "Cannot interpret value of label %s as timestamp: %s",
-                    timestampLabel, value);
-    }
-    return timestampMsSinceEpoch;
-  }
-
-  /**
-   * Path representing a cloud project id.
-   */
-  public static class ProjectPath implements Serializable {
-    private final String projectId;
-
-    /**
-     * Creates a {@link ProjectPath} from a {@link String} representation, which
-     * must be of the form {@code "projects/" + projectId}.
-     */
-    ProjectPath(String path) {
-      String[] splits = path.split("/");
-      checkArgument(
-          splits.length == 2 && splits[0].equals("projects"),
-          "Malformed project path \"%s\": must be of the form \"projects/\" + <project id>",
-          path);
-      this.projectId = splits[1];
-    }
-
-    public String getPath() {
-      return String.format("projects/%s", projectId);
-    }
-
-    public String getId() {
-      return projectId;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      ProjectPath that = (ProjectPath) o;
-
-      return projectId.equals(that.projectId);
-    }
-
-    @Override
-    public int hashCode() {
-      return projectId.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return getPath();
-    }
-  }
-
-  public static ProjectPath projectPathFromPath(String path) {
-    return new ProjectPath(path);
-  }
-
-  public static ProjectPath projectPathFromId(String projectId) {
-    return new ProjectPath(String.format("projects/%s", projectId));
-  }
-
-  /**
-   * Path representing a Pubsub subscription.
-   */
-  public static class SubscriptionPath implements Serializable {
-    private final String projectId;
-    private final String subscriptionName;
-
-    SubscriptionPath(String path) {
-      String[] splits = path.split("/");
-      checkState(
-          splits.length == 4 && splits[0].equals("projects") && splits[2].equals("subscriptions"),
-          "Malformed subscription path %s: "
-          + "must be of the form \"projects/\" + <project id> + \"subscriptions\"", path);
-      this.projectId = splits[1];
-      this.subscriptionName = splits[3];
-    }
-
-    public String getPath() {
-      return String.format("projects/%s/subscriptions/%s", projectId, subscriptionName);
-    }
-
-    public String getName() {
-      return subscriptionName;
-    }
-
-    public String getV1Beta1Path() {
-      return String.format("/subscriptions/%s/%s", projectId, subscriptionName);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      SubscriptionPath that = (SubscriptionPath) o;
-      return this.subscriptionName.equals(that.subscriptionName)
-          && this.projectId.equals(that.projectId);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(projectId, subscriptionName);
-    }
-
-    @Override
-    public String toString() {
-      return getPath();
-    }
-  }
-
-  public static SubscriptionPath subscriptionPathFromPath(String path) {
-    return new SubscriptionPath(path);
-  }
-
-  public static SubscriptionPath subscriptionPathFromName(
-      String projectId, String subscriptionName) {
-    return new SubscriptionPath(String.format("projects/%s/subscriptions/%s",
-                                              projectId, subscriptionName));
-  }
-
-  /**
-   * Path representing a Pubsub topic.
-   */
-  public static class TopicPath implements Serializable {
-    private final String path;
-
-    TopicPath(String path) {
-      this.path = path;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    public String getName() {
-      String[] splits = path.split("/");
-      checkState(splits.length == 4, "Malformed topic path %s", path);
-      return splits[3];
-    }
-
-    public String getV1Beta1Path() {
-      String[] splits = path.split("/");
-      checkState(splits.length == 4, "Malformed topic path %s", path);
-      return String.format("/topics/%s/%s", splits[1], splits[3]);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      TopicPath topicPath = (TopicPath) o;
-      return path.equals(topicPath.path);
-    }
-
-    @Override
-    public int hashCode() {
-      return path.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return path;
-    }
-  }
-
-  public static TopicPath topicPathFromPath(String path) {
-    return new TopicPath(path);
-  }
-
-  public static TopicPath topicPathFromName(String projectId, String topicName) {
-    return new TopicPath(String.format("projects/%s/topics/%s", projectId, topicName));
-  }
-
-  /**
-   * A message to be sent to Pubsub.
-   *
-   * <p>NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}.
-   * Java serialization is never used for non-test clients.
-   */
-  public static class OutgoingMessage implements Serializable {
-    /**
-     * Underlying (encoded) element.
-     */
-    public final byte[] elementBytes;
-
-    public final Map<String, String> attributes;
-
-    /**
-     * Timestamp for element (ms since epoch).
-     */
-    public final long timestampMsSinceEpoch;
-
-    /**
-     * If using an id label, the record id to associate with this record's metadata so the receiver
-     * can reject duplicates. Otherwise {@literal null}.
-     */
-    @Nullable
-    public final String recordId;
-
-    public OutgoingMessage(byte[] elementBytes, Map<String, String> attributes,
-                           long timestampMsSinceEpoch, @Nullable String recordId) {
-      this.elementBytes = elementBytes;
-      this.attributes = attributes;
-      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
-      this.recordId = recordId;
-    }
-
-    @Override
-    public String toString() {
-      return String.format("OutgoingMessage(%db, %dms)",
-                           elementBytes.length, timestampMsSinceEpoch);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      OutgoingMessage that = (OutgoingMessage) o;
-
-      return timestampMsSinceEpoch == that.timestampMsSinceEpoch
-              && Arrays.equals(elementBytes, that.elementBytes)
-              && Objects.equal(attributes, that.attributes)
-              && Objects.equal(recordId, that.recordId);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch,
-              recordId);
-    }
-  }
-
-  /**
-   * A message received from Pubsub.
-   *
-   * <p>NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}.
-   * Java serialization is never used for non-test clients.
-   */
-  public static class IncomingMessage implements Serializable {
-    /**
-     * Underlying (encoded) element.
-     */
-    public final byte[] elementBytes;
-
-    public Map<String, String> attributes;
-
-    /**
-     * Timestamp for element (ms since epoch). Either Pubsub's processing time,
-     * or the custom timestamp associated with the message.
-     */
-    public final long timestampMsSinceEpoch;
-
-    /**
-     * Timestamp (in system time) at which we requested the message (ms since epoch).
-     */
-    public final long requestTimeMsSinceEpoch;
-
-    /**
-     * Id to pass back to Pubsub to acknowledge receipt of this message.
-     */
-    public final String ackId;
-
-    /**
-     * Id to pass to the runner to distinguish this message from all others.
-     */
-    public final String recordId;
-
-    public IncomingMessage(
-        byte[] elementBytes,
-        Map<String, String> attributes,
-        long timestampMsSinceEpoch,
-        long requestTimeMsSinceEpoch,
-        String ackId,
-        String recordId) {
-      this.elementBytes = elementBytes;
-      this.attributes = attributes;
-      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
-      this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch;
-      this.ackId = ackId;
-      this.recordId = recordId;
-    }
-
-    public IncomingMessage withRequestTime(long requestTimeMsSinceEpoch) {
-      return new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch,
-              requestTimeMsSinceEpoch, ackId, recordId);
-    }
-
-    @Override
-    public String toString() {
-      return String.format("IncomingMessage(%db, %dms)",
-                           elementBytes.length, timestampMsSinceEpoch);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      IncomingMessage that = (IncomingMessage) o;
-
-      return timestampMsSinceEpoch == that.timestampMsSinceEpoch
-             && requestTimeMsSinceEpoch == that.requestTimeMsSinceEpoch
-             && ackId.equals(that.ackId)
-             && recordId.equals(that.recordId)
-             && Arrays.equals(elementBytes, that.elementBytes)
-              && Objects.equal(attributes, that.attributes);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch,
-                              requestTimeMsSinceEpoch,
-                              ackId, recordId);
-    }
-  }
-
-  /**
-   * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages
-   * published.
-   */
-  public abstract int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages)
-      throws IOException;
-
-  /**
-   * Request the next batch of up to {@code batchSize} messages from {@code subscription}.
-   * Return the received messages, or empty collection if none were available. Does not
-   * wait for messages to arrive if {@code returnImmediately} is {@literal true}.
-   * Returned messages will record their request time as {@code requestTimeMsSinceEpoch}.
-   */
-  public abstract List<IncomingMessage> pull(
-      long requestTimeMsSinceEpoch,
-      SubscriptionPath subscription,
-      int batchSize,
-      boolean returnImmediately)
-      throws IOException;
-
-  /**
-   * Acknowldege messages from {@code subscription} with {@code ackIds}.
-   */
-  public abstract void acknowledge(SubscriptionPath subscription, List<String> ackIds)
-      throws IOException;
-
-  /**
-   * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to
-   * be {@code deadlineSeconds} from now.
-   */
-  public abstract void modifyAckDeadline(
-      SubscriptionPath subscription, List<String> ackIds,
-      int deadlineSeconds) throws IOException;
-
-  /**
-   * Create {@code topic}.
-   */
-  public abstract void createTopic(TopicPath topic) throws IOException;
-
-  /*
-   * Delete {@code topic}.
-   */
-  public abstract void deleteTopic(TopicPath topic) throws IOException;
-
-  /**
-   * Return a list of topics for {@code project}.
-   */
-  public abstract List<TopicPath> listTopics(ProjectPath project) throws IOException;
-
-  /**
-   * Create {@code subscription} to {@code topic}.
-   */
-  public abstract void createSubscription(
-      TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException;
-
-  /**
-   * Create a random subscription for {@code topic}. Return the {@link SubscriptionPath}. It
-   * is the responsibility of the caller to later delete the subscription.
-   */
-  public SubscriptionPath createRandomSubscription(
-      ProjectPath project, TopicPath topic, int ackDeadlineSeconds) throws IOException {
-    // Create a randomized subscription derived from the topic name.
-    String subscriptionName = topic.getName() + "_beam_" + ThreadLocalRandom.current().nextLong();
-    SubscriptionPath subscription =
-        PubsubClient.subscriptionPathFromName(project.getId(), subscriptionName);
-    createSubscription(topic, subscription, ackDeadlineSeconds);
-    return subscription;
-  }
-
-  /**
-   * Delete {@code subscription}.
-   */
-  public abstract void deleteSubscription(SubscriptionPath subscription) throws IOException;
-
-  /**
-   * Return a list of subscriptions for {@code topic} in {@code project}.
-   */
-  public abstract List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
-      throws IOException;
-
-  /**
-   * Return the ack deadline, in seconds, for {@code subscription}.
-   */
-  public abstract int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException;
-
-  /**
-   * Return {@literal true} if {@link #pull} will always return empty list. Actual clients
-   * will return {@literal false}. Test clients may return {@literal true} to signal that all
-   * expected messages have been pulled and the test may complete.
-   */
-  public abstract boolean isEOF();
-}


[3/8] beam git commit: [BEAM-1722] Move PubsubIO into the google-cloud-platform module

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
new file mode 100644
index 0000000..8fc1c19
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
@@ -0,0 +1,1014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.ProjectPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
+import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Read and Write {@link PTransform}s for Cloud Pub/Sub streams. These transforms create
+ * and consume unbounded {@link PCollection PCollections}.
+ *
+ * <h3>Permissions</h3>
+ *
+ * <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
+ * Beam pipeline. Please refer to the documentation of corresponding
+ * {@link PipelineRunner PipelineRunners} for more details.
+ */
+public class PubsubIO {
+
+  private static final Logger LOG = LoggerFactory.getLogger(PubsubIO.class);
+
+  /** Factory for creating pubsub client to manage transport. */
+  private static final PubsubClient.PubsubClientFactory FACTORY = PubsubJsonClient.FACTORY;
+
+  /**
+   * Project IDs must contain 6-63 lowercase letters, digits, or dashes.
+   * IDs must start with a letter and may not end with a dash.
+   * This regex isn't exact - this allows for patterns that would be rejected by
+   * the service, but this is sufficient for basic parsing of table references.
+   */
+  private static final Pattern PROJECT_ID_REGEXP =
+      Pattern.compile("[a-z][-a-z0-9:.]{4,61}[a-z0-9]");
+
+  private static final Pattern SUBSCRIPTION_REGEXP =
+      Pattern.compile("projects/([^/]+)/subscriptions/(.+)");
+
+  private static final Pattern TOPIC_REGEXP = Pattern.compile("projects/([^/]+)/topics/(.+)");
+
+  private static final Pattern V1BETA1_SUBSCRIPTION_REGEXP =
+      Pattern.compile("/subscriptions/([^/]+)/(.+)");
+
+  private static final Pattern V1BETA1_TOPIC_REGEXP = Pattern.compile("/topics/([^/]+)/(.+)");
+
+  private static final Pattern PUBSUB_NAME_REGEXP = Pattern.compile("[a-zA-Z][-._~%+a-zA-Z0-9]+");
+
+  private static final int PUBSUB_NAME_MIN_LENGTH = 3;
+  private static final int PUBSUB_NAME_MAX_LENGTH = 255;
+
+  private static final String SUBSCRIPTION_RANDOM_TEST_PREFIX = "_random/";
+  private static final String SUBSCRIPTION_STARTING_SIGNAL = "_starting_signal/";
+  private static final String TOPIC_DEV_NULL_TEST_NAME = "/topics/dev/null";
+
+  private static void validateProjectName(String project) {
+    Matcher match = PROJECT_ID_REGEXP.matcher(project);
+    if (!match.matches()) {
+      throw new IllegalArgumentException(
+          "Illegal project name specified in Pubsub subscription: " + project);
+    }
+  }
+
+  private static void validatePubsubName(String name) {
+    if (name.length() < PUBSUB_NAME_MIN_LENGTH) {
+      throw new IllegalArgumentException(
+          "Pubsub object name is shorter than 3 characters: " + name);
+    }
+    if (name.length() > PUBSUB_NAME_MAX_LENGTH) {
+      throw new IllegalArgumentException(
+          "Pubsub object name is longer than 255 characters: " + name);
+    }
+
+    if (name.startsWith("goog")) {
+      throw new IllegalArgumentException("Pubsub object name cannot start with goog: " + name);
+    }
+
+    Matcher match = PUBSUB_NAME_REGEXP.matcher(name);
+    if (!match.matches()) {
+      throw new IllegalArgumentException("Illegal Pubsub object name specified: " + name
+          + " Please see Javadoc for naming rules.");
+    }
+  }
+
+  /**
+   * Populate common {@link DisplayData} between Pubsub source and sink.
+   */
+  private static void populateCommonDisplayData(DisplayData.Builder builder,
+      String timestampLabel, String idLabel, ValueProvider<PubsubTopic> topic) {
+    builder
+        .addIfNotNull(DisplayData.item("timestampLabel", timestampLabel)
+            .withLabel("Timestamp Label Attribute"))
+        .addIfNotNull(DisplayData.item("idLabel", idLabel)
+            .withLabel("ID Label Attribute"));
+
+    if (topic != null) {
+      String topicString = topic.isAccessible() ? topic.get().asPath()
+          : topic.toString();
+      builder.add(DisplayData.item("topic", topicString)
+          .withLabel("Pubsub Topic"));
+    }
+  }
+
+  /**
+   * Class representing a Pub/Sub message. Each message contains a single message payload and
+   * a map of attached attributes.
+   */
+  public static class PubsubMessage {
+
+    private byte[] message;
+    private Map<String, String> attributes;
+
+    public PubsubMessage(byte[] message, Map<String, String> attributes) {
+      this.message = message;
+      this.attributes = attributes;
+    }
+
+    /**
+     * Returns the main PubSub message.
+     */
+    public byte[] getMessage() {
+      return message;
+    }
+
+    /**
+     * Returns the given attribute value. If not such attribute exists, returns null.
+     */
+    @Nullable
+    public String getAttribute(String attribute) {
+      checkNotNull(attribute, "attribute");
+      return attributes.get(attribute);
+    }
+
+    /**
+     * Returns the full map of attributes. This is an unmodifiable map.
+     */
+    public Map<String, String> getAttributeMap() {
+      return attributes;
+    }
+  }
+
+  /**
+   * Class representing a Cloud Pub/Sub Subscription.
+   */
+  public static class PubsubSubscription implements Serializable {
+
+    private enum Type {NORMAL, FAKE}
+
+    private final Type type;
+    private final String project;
+    private final String subscription;
+
+    private PubsubSubscription(Type type, String project, String subscription) {
+      this.type = type;
+      this.project = project;
+      this.subscription = subscription;
+    }
+
+    /**
+     * Creates a class representing a Pub/Sub subscription from the specified subscription path.
+     *
+     * <p>Cloud Pub/Sub subscription names should be of the form
+     * {@code projects/<project>/subscriptions/<subscription>}, where {@code <project>} is the name
+     * of the project the subscription belongs to. The {@code <subscription>} component must comply
+     * with the following requirements:
+     *
+     * <ul>
+     * <li>Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods
+     * ('.').</li>
+     * <li>Must be between 3 and 255 characters.</li>
+     * <li>Must begin with a letter.</li>
+     * <li>Must end with a letter or a number.</li>
+     * <li>Cannot begin with {@code 'goog'} prefix.</li>
+     * </ul>
+     */
+    public static PubsubSubscription fromPath(String path) {
+      if (path.startsWith(SUBSCRIPTION_RANDOM_TEST_PREFIX)
+          || path.startsWith(SUBSCRIPTION_STARTING_SIGNAL)) {
+        return new PubsubSubscription(Type.FAKE, "", path);
+      }
+
+      String projectName, subscriptionName;
+
+      Matcher v1beta1Match = V1BETA1_SUBSCRIPTION_REGEXP.matcher(path);
+      if (v1beta1Match.matches()) {
+        LOG.warn("Saw subscription in v1beta1 format. Subscriptions should be in the format "
+            + "projects/<project_id>/subscriptions/<subscription_name>");
+        projectName = v1beta1Match.group(1);
+        subscriptionName = v1beta1Match.group(2);
+      } else {
+        Matcher match = SUBSCRIPTION_REGEXP.matcher(path);
+        if (!match.matches()) {
+          throw new IllegalArgumentException("Pubsub subscription is not in "
+              + "projects/<project_id>/subscriptions/<subscription_name> format: " + path);
+        }
+        projectName = match.group(1);
+        subscriptionName = match.group(2);
+      }
+
+      validateProjectName(projectName);
+      validatePubsubName(subscriptionName);
+      return new PubsubSubscription(Type.NORMAL, projectName, subscriptionName);
+    }
+
+    /**
+     * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub
+     * v1beta1 API.
+     *
+     * @deprecated the v1beta1 API for Cloud Pub/Sub is deprecated.
+     */
+    @Deprecated
+    public String asV1Beta1Path() {
+      if (type == Type.NORMAL) {
+        return "/subscriptions/" + project + "/" + subscription;
+      } else {
+        return subscription;
+      }
+    }
+
+    /**
+     * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub
+     * v1beta2 API.
+     *
+     * @deprecated the v1beta2 API for Cloud Pub/Sub is deprecated.
+     */
+    @Deprecated
+    public String asV1Beta2Path() {
+      if (type == Type.NORMAL) {
+        return "projects/" + project + "/subscriptions/" + subscription;
+      } else {
+        return subscription;
+      }
+    }
+
+    /**
+     * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub
+     * API.
+     */
+    public String asPath() {
+      if (type == Type.NORMAL) {
+        return "projects/" + project + "/subscriptions/" + subscription;
+      } else {
+        return subscription;
+      }
+    }
+  }
+
+  /**
+   * Used to build a {@link ValueProvider} for {@link PubsubSubscription}.
+   */
+  private static class SubscriptionTranslator
+      implements SerializableFunction<String, PubsubSubscription> {
+
+    @Override
+    public PubsubSubscription apply(String from) {
+      return PubsubSubscription.fromPath(from);
+    }
+  }
+
+  /**
+   * Used to build a {@link ValueProvider} for {@link SubscriptionPath}.
+   */
+  private static class SubscriptionPathTranslator
+      implements SerializableFunction<PubsubSubscription, SubscriptionPath> {
+
+    @Override
+    public SubscriptionPath apply(PubsubSubscription from) {
+      return PubsubClient.subscriptionPathFromName(from.project, from.subscription);
+    }
+  }
+
+  /**
+   * Used to build a {@link ValueProvider} for {@link PubsubTopic}.
+   */
+  private static class TopicTranslator
+      implements SerializableFunction<String, PubsubTopic> {
+
+    @Override
+    public PubsubTopic apply(String from) {
+      return PubsubTopic.fromPath(from);
+    }
+  }
+
+  /**
+   * Used to build a {@link ValueProvider} for {@link TopicPath}.
+   */
+  private static class TopicPathTranslator
+      implements SerializableFunction<PubsubTopic, TopicPath> {
+
+    @Override
+    public TopicPath apply(PubsubTopic from) {
+      return PubsubClient.topicPathFromName(from.project, from.topic);
+    }
+  }
+
+  /**
+   * Used to build a {@link ValueProvider} for {@link ProjectPath}.
+   */
+  private static class ProjectPathTranslator
+      implements SerializableFunction<PubsubTopic, ProjectPath> {
+
+    @Override
+    public ProjectPath apply(PubsubTopic from) {
+      return PubsubClient.projectPathFromId(from.project);
+    }
+  }
+
+  /**
+   * Class representing a Cloud Pub/Sub Topic.
+   */
+  public static class PubsubTopic implements Serializable {
+
+    private enum Type {NORMAL, FAKE}
+
+    private final Type type;
+    private final String project;
+    private final String topic;
+
+    private PubsubTopic(Type type, String project, String topic) {
+      this.type = type;
+      this.project = project;
+      this.topic = topic;
+    }
+
+    /**
+     * Creates a class representing a Cloud Pub/Sub topic from the specified topic path.
+     *
+     * <p>Cloud Pub/Sub topic names should be of the form
+     * {@code /topics/<project>/<topic>}, where {@code <project>} is the name of
+     * the publishing project. The {@code <topic>} component must comply with
+     * the following requirements:
+     *
+     * <ul>
+     * <li>Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods
+     * ('.').</li>
+     * <li>Must be between 3 and 255 characters.</li>
+     * <li>Must begin with a letter.</li>
+     * <li>Must end with a letter or a number.</li>
+     * <li>Cannot begin with 'goog' prefix.</li>
+     * </ul>
+     */
+    public static PubsubTopic fromPath(String path) {
+      if (path.equals(TOPIC_DEV_NULL_TEST_NAME)) {
+        return new PubsubTopic(Type.FAKE, "", path);
+      }
+
+      String projectName, topicName;
+
+      Matcher v1beta1Match = V1BETA1_TOPIC_REGEXP.matcher(path);
+      if (v1beta1Match.matches()) {
+        LOG.warn("Saw topic in v1beta1 format.  Topics should be in the format "
+            + "projects/<project_id>/topics/<topic_name>");
+        projectName = v1beta1Match.group(1);
+        topicName = v1beta1Match.group(2);
+      } else {
+        Matcher match = TOPIC_REGEXP.matcher(path);
+        if (!match.matches()) {
+          throw new IllegalArgumentException(
+              "Pubsub topic is not in projects/<project_id>/topics/<topic_name> format: " + path);
+        }
+        projectName = match.group(1);
+        topicName = match.group(2);
+      }
+
+      validateProjectName(projectName);
+      validatePubsubName(topicName);
+      return new PubsubTopic(Type.NORMAL, projectName, topicName);
+    }
+
+    /**
+     * Returns the string representation of this topic as a path used in the Cloud Pub/Sub
+     * v1beta1 API.
+     *
+     * @deprecated the v1beta1 API for Cloud Pub/Sub is deprecated.
+     */
+    @Deprecated
+    public String asV1Beta1Path() {
+      if (type == Type.NORMAL) {
+        return "/topics/" + project + "/" + topic;
+      } else {
+        return topic;
+      }
+    }
+
+    /**
+     * Returns the string representation of this topic as a path used in the Cloud Pub/Sub
+     * v1beta2 API.
+     *
+     * @deprecated the v1beta2 API for Cloud Pub/Sub is deprecated.
+     */
+    @Deprecated
+    public String asV1Beta2Path() {
+      if (type == Type.NORMAL) {
+        return "projects/" + project + "/topics/" + topic;
+      } else {
+        return topic;
+      }
+    }
+
+    /**
+     * Returns the string representation of this topic as a path used in the Cloud Pub/Sub
+     * API.
+     */
+    public String asPath() {
+      if (type == Type.NORMAL) {
+        return "projects/" + project + "/topics/" + topic;
+      } else {
+        return topic;
+      }
+    }
+  }
+
+  public static <T> Read<T> read() {
+    return new Read<>();
+  }
+
+  public static <T> Write<T> write() {
+    return new Write<>();
+  }
+
+  /**
+   * A {@link PTransform} that continuously reads from a Google Cloud Pub/Sub stream and
+   * returns a {@link PCollection} of {@link String Strings} containing the items from
+   * the stream.
+   */
+  public static class Read<T> extends PTransform<PBegin, PCollection<T>> {
+
+    /** The Cloud Pub/Sub topic to read from. */
+    @Nullable
+    private final ValueProvider<PubsubTopic> topic;
+
+    /** The Cloud Pub/Sub subscription to read from. */
+    @Nullable
+    private final ValueProvider<PubsubSubscription> subscription;
+
+    /** The name of the message attribute to read timestamps from. */
+    @Nullable
+    private final String timestampLabel;
+
+    /** The name of the message attribute to read unique message IDs from. */
+    @Nullable
+    private final String idLabel;
+
+    /** The coder used to decode each record. */
+    @Nullable
+    private final Coder<T> coder;
+
+    /** User function for parsing PubsubMessage object. */
+    SimpleFunction<PubsubMessage, T> parseFn;
+
+    private Read() {
+      this(null, null, null, null, null, null, null);
+    }
+
+    private Read(String name, ValueProvider<PubsubSubscription> subscription,
+        ValueProvider<PubsubTopic> topic, String timestampLabel, Coder<T> coder,
+        String idLabel,
+        SimpleFunction<PubsubMessage, T> parseFn) {
+      super(name);
+      this.subscription = subscription;
+      this.topic = topic;
+      this.timestampLabel = timestampLabel;
+      this.coder = coder;
+      this.idLabel = idLabel;
+      this.parseFn = parseFn;
+    }
+
+    /**
+     * Returns a transform that's like this one but reading from the
+     * given subscription.
+     *
+     * <p>See {@link PubsubIO.PubsubSubscription#fromPath(String)} for more details on the format
+     * of the {@code subscription} string.
+     *
+     * <p>Multiple readers reading from the same subscription will each receive
+     * some arbitrary portion of the data.  Most likely, separate readers should
+     * use their own subscriptions.
+     *
+     * <p>Does not modify this object.
+     */
+    public Read<T> subscription(String subscription) {
+      return subscription(StaticValueProvider.of(subscription));
+    }
+
+    /**
+     * Like {@code subscription()} but with a {@link ValueProvider}.
+     */
+    public Read<T> subscription(ValueProvider<String> subscription) {
+      if (subscription.isAccessible()) {
+        // Validate.
+        PubsubSubscription.fromPath(subscription.get());
+      }
+      return new Read<>(
+          name, NestedValueProvider.of(subscription, new SubscriptionTranslator()),
+          null /* reset topic to null */, timestampLabel, coder, idLabel, parseFn);
+    }
+
+    /**
+     * Creates and returns a transform for reading from a Cloud Pub/Sub topic. Mutually exclusive
+     * with {@link #subscription(String)}.
+     *
+     * <p>See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format
+     * of the {@code topic} string.
+     *
+     * <p>The Beam runner will start reading data published on this topic from the time the
+     * pipeline is started. Any data published on the topic before the pipeline is started will
+     * not be read by the runner.
+     */
+    public Read<T> topic(String topic) {
+      return topic(StaticValueProvider.of(topic));
+    }
+
+    /**
+     * Like {@code topic()} but with a {@link ValueProvider}.
+     */
+    public Read<T> topic(ValueProvider<String> topic) {
+      if (topic.isAccessible()) {
+        // Validate.
+        PubsubTopic.fromPath(topic.get());
+      }
+      return new Read<>(name, null /* reset subscription to null */,
+          NestedValueProvider.of(topic, new TopicTranslator()),
+          timestampLabel, coder, idLabel, parseFn);
+    }
+
+    /**
+     * Creates and returns a transform reading from Cloud Pub/Sub where record timestamps are
+     * expected to be provided as Pub/Sub message attributes. The {@code timestampLabel}
+     * parameter specifies the name of the attribute that contains the timestamp.
+     *
+     * <p>The timestamp value is expected to be represented in the attribute as either:
+     *
+     * <ul>
+     * <li>a numerical value representing the number of milliseconds since the Unix epoch. For
+     * example, if using the Joda time classes, {@link Instant#getMillis()} returns the correct
+     * value for this attribute.
+     * <li>a String in RFC 3339 format. For example, {@code 2015-10-29T23:41:41.123Z}. The
+     * sub-second component of the timestamp is optional, and digits beyond the first three
+     * (i.e., time units smaller than milliseconds) will be ignored.
+     * </ul>
+     *
+     * <p>If {@code timestampLabel} is not provided, the system will generate record timestamps
+     * the first time it sees each record. All windowing will be done relative to these
+     * timestamps.
+     *
+     * <p>By default, windows are emitted based on an estimate of when this source is likely
+     * done producing data for a given timestamp (referred to as the Watermark; see
+     * {@link AfterWatermark} for more details). Any late data will be handled by the trigger
+     * specified with the windowing strategy &ndash; by default it will be output immediately.
+     *
+     * <p>Note that the system can guarantee that no late data will ever be seen when it assigns
+     * timestamps by arrival time (i.e. {@code timestampLabel} is not provided).
+     *
+     * @see <a href="https://www.ietf.org/rfc/rfc3339.txt">RFC 3339</a>
+     */
+    public Read<T> timestampLabel(String timestampLabel) {
+      return new Read<>(
+          name, subscription, topic, timestampLabel, coder, idLabel,
+          parseFn);
+    }
+
+    /**
+     * Creates and returns a transform for reading from Cloud Pub/Sub where unique record
+     * identifiers are expected to be provided as Pub/Sub message attributes. The {@code idLabel}
+     * parameter specifies the attribute name. The value of the attribute can be any string
+     * that uniquely identifies this record.
+     *
+     * <p>Pub/Sub cannot guarantee that no duplicate data will be delivered on the Pub/Sub stream.
+     * If {@code idLabel} is not provided, Beam cannot guarantee that no duplicate data will
+     * be delivered, and deduplication of the stream will be strictly best effort.
+     */
+    public Read<T> idLabel(String idLabel) {
+      return new Read<>(
+          name, subscription, topic, timestampLabel, coder, idLabel,
+          parseFn);
+    }
+
+    /**
+     * Returns a transform that's like this one but that uses the given
+     * {@link Coder} to decode each record into a value of type {@code T}.
+     *
+     * <p>Does not modify this object.
+     */
+    public Read<T> withCoder(Coder<T> coder) {
+      return new Read<>(
+          name, subscription, topic, timestampLabel, coder, idLabel,
+          parseFn);
+    }
+
+    /**
+     * Causes the source to return a PubsubMessage that includes Pubsub attributes.
+     * The user must supply a parsing function to transform the PubsubMessage into an output type.
+     * A Coder for the output type T must be registered or set on the output via
+     * {@link PCollection#setCoder(Coder)}.
+     */
+    public Read<T> withAttributes(SimpleFunction<PubsubMessage, T> parseFn) {
+      return new Read<T>(
+          name, subscription, topic, timestampLabel, coder, idLabel,
+          parseFn);
+    }
+
+    @Override
+    public PCollection<T> expand(PBegin input) {
+      if (topic == null && subscription == null) {
+        throw new IllegalStateException("Need to set either the topic or the subscription for "
+            + "a PubsubIO.Read transform");
+      }
+      if (topic != null && subscription != null) {
+        throw new IllegalStateException("Can't set both the topic and the subscription for "
+            + "a PubsubIO.Read transform");
+      }
+      if (coder == null) {
+        throw new IllegalStateException("PubsubIO.Read requires that a coder be set using "
+            + "the withCoder method.");
+      }
+
+      @Nullable ValueProvider<ProjectPath> projectPath =
+          topic == null ? null : NestedValueProvider.of(topic, new ProjectPathTranslator());
+      @Nullable ValueProvider<TopicPath> topicPath =
+          topic == null ? null : NestedValueProvider.of(topic, new TopicPathTranslator());
+      @Nullable ValueProvider<SubscriptionPath> subscriptionPath =
+          subscription == null
+              ? null
+              : NestedValueProvider.of(subscription, new SubscriptionPathTranslator());
+      PubsubUnboundedSource<T> source = new PubsubUnboundedSource<T>(
+              FACTORY, projectPath, topicPath, subscriptionPath,
+              coder, timestampLabel, idLabel, parseFn);
+      return input.getPipeline().apply(source);
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      super.populateDisplayData(builder);
+      populateCommonDisplayData(builder, timestampLabel, idLabel, topic);
+
+      if (subscription != null) {
+        String subscriptionString = subscription.isAccessible()
+            ? subscription.get().asPath() : subscription.toString();
+        builder.add(DisplayData.item("subscription", subscriptionString)
+            .withLabel("Pubsub Subscription"));
+      }
+    }
+
+    @Override
+    protected Coder<T> getDefaultOutputCoder() {
+      return coder;
+    }
+
+    /**
+     * Get the topic being read from.
+     */
+    @Nullable
+    public PubsubTopic getTopic() {
+      return topic == null ? null : topic.get();
+    }
+
+    /**
+     * Get the {@link ValueProvider} for the topic being read from.
+     */
+    public ValueProvider<PubsubTopic> getTopicProvider() {
+      return topic;
+    }
+
+    /**
+     * Get the subscription being read from.
+     */
+    @Nullable
+    public PubsubSubscription getSubscription() {
+      return subscription == null ? null : subscription.get();
+    }
+
+    /**
+     * Get the {@link ValueProvider} for the subscription being read from.
+     */
+    public ValueProvider<PubsubSubscription> getSubscriptionProvider() {
+      return subscription;
+    }
+
+    /**
+     * Get the timestamp label.
+     */
+    @Nullable
+    public String getTimestampLabel() {
+      return timestampLabel;
+    }
+
+    /**
+     * Get the id label.
+     */
+    @Nullable
+    public String getIdLabel() {
+      return idLabel;
+    }
+
+
+    /**
+     * Get the {@link Coder} used for the transform's output.
+     */
+    @Nullable
+    public Coder<T> getCoder() {
+      return coder;
+    }
+
+    /**
+     * Get the parse function used for PubSub attributes.
+     */
+    @Nullable
+    public SimpleFunction<PubsubMessage, T> getPubSubMessageParseFn() {
+      return parseFn;
+    }
+
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  /** Disallow construction of utility class. */
+  private PubsubIO() {}
+
+
+  /**
+   * A {@link PTransform} that writes an unbounded {@link PCollection} of {@link String Strings}
+   * to a Cloud Pub/Sub stream.
+   */
+  public static class Write<T> extends PTransform<PCollection<T>, PDone> {
+
+    /** The Cloud Pub/Sub topic to publish to. */
+    @Nullable
+    private final ValueProvider<PubsubTopic> topic;
+    /** The name of the message attribute to publish message timestamps in. */
+    @Nullable
+    private final String timestampLabel;
+    /** The name of the message attribute to publish unique message IDs in. */
+    @Nullable
+    private final String idLabel;
+    /** The input type Coder. */
+    private final Coder<T> coder;
+    /** The format function for input PubsubMessage objects. */
+    SimpleFunction<T, PubsubMessage> formatFn;
+
+    private Write() {
+      this(null, null, null, null, null, null);
+    }
+
+    private Write(
+        String name, ValueProvider<PubsubTopic> topic, String timestampLabel,
+        String idLabel, Coder<T> coder, SimpleFunction<T, PubsubMessage> formatFn) {
+      super(name);
+      this.topic = topic;
+      this.timestampLabel = timestampLabel;
+      this.idLabel = idLabel;
+      this.coder = coder;
+      this.formatFn = formatFn;
+    }
+
+    /**
+     * Creates a transform that publishes to the specified topic.
+     *
+     * <p>See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format of the
+     * {@code topic} string.
+     */
+    public Write<T> topic(String topic) {
+      return topic(StaticValueProvider.of(topic));
+    }
+
+    /**
+     * Like {@code topic()} but with a {@link ValueProvider}.
+     */
+    public Write<T> topic(ValueProvider<String> topic) {
+      return new Write<>(name, NestedValueProvider.of(topic, new TopicTranslator()),
+          timestampLabel, idLabel, coder, formatFn);
+    }
+
+    /**
+     * Creates a transform that writes to Pub/Sub, adds each record's timestamp to the published
+     * messages in an attribute with the specified name. The value of the attribute will be a number
+     * representing the number of milliseconds since the Unix epoch. For example, if using the Joda
+     * time classes, {@link Instant#Instant(long)} can be used to parse this value.
+     *
+     * <p>If the output from this sink is being read by another Beam pipeline, then
+     * {@link PubsubIO.Read#timestampLabel(String)} can be used to ensure the other source reads
+     * these timestamps from the appropriate attribute.
+     */
+    public Write<T> timestampLabel(String timestampLabel) {
+      return new Write<>(name, topic, timestampLabel, idLabel, coder, formatFn);
+    }
+
+    /**
+     * Creates a transform that writes to Pub/Sub, adding each record's unique identifier to the
+     * published messages in an attribute with the specified name. The value of the attribute is an
+     * opaque string.
+     *
+     * <p>If the the output from this sink is being read by another Beam pipeline, then
+     * {@link PubsubIO.Read#idLabel(String)} can be used to ensure that* the other source reads
+     * these unique identifiers from the appropriate attribute.
+     */
+    public Write<T> idLabel(String idLabel) {
+      return new Write<>(name, topic, timestampLabel, idLabel, coder, formatFn);
+    }
+
+    /**
+     * Returns a new transform that's like this one
+     * but that uses the given {@link Coder} to encode each of
+     * the elements of the input {@link PCollection} into an
+     * output record.
+     *
+     * <p>Does not modify this object.
+     */
+    public Write<T> withCoder(Coder<T> coder) {
+      return new Write<>(name, topic, timestampLabel, idLabel, coder, formatFn);
+    }
+
+    /**
+     * Used to write a PubSub message together with PubSub attributes. The user-supplied format
+     * function translates the input type T to a PubsubMessage object, which is used by the sink
+     * to separately set the PubSub message's payload and attributes.
+     */
+    public Write<T> withAttributes(SimpleFunction<T, PubsubMessage> formatFn) {
+      return new Write<T>(name, topic, timestampLabel, idLabel, coder, formatFn);
+    }
+
+    @Override
+    public PDone expand(PCollection<T> input) {
+      if (topic == null) {
+        throw new IllegalStateException("need to set the topic of a PubsubIO.Write transform");
+      }
+      switch (input.isBounded()) {
+        case BOUNDED:
+          input.apply(ParDo.of(new PubsubBoundedWriter()));
+          return PDone.in(input.getPipeline());
+        case UNBOUNDED:
+          return input.apply(new PubsubUnboundedSink<T>(
+              FACTORY,
+              NestedValueProvider.of(topic, new TopicPathTranslator()),
+              coder,
+              timestampLabel,
+              idLabel,
+              formatFn,
+              100 /* numShards */));
+      }
+      throw new RuntimeException(); // cases are exhaustive.
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      super.populateDisplayData(builder);
+      populateCommonDisplayData(builder, timestampLabel, idLabel, topic);
+    }
+
+    @Override
+    protected Coder<Void> getDefaultOutputCoder() {
+      return VoidCoder.of();
+    }
+
+    /**
+     * Returns the PubSub topic being written to.
+     */
+    @Nullable
+    public PubsubTopic getTopic() {
+      return (topic == null) ? null : topic.get();
+    }
+
+    /**
+     * Returns the {@link ValueProvider} for the topic being written to.
+     */
+    @Nullable
+    public ValueProvider<PubsubTopic> getTopicProvider() {
+      return topic;
+    }
+
+    /**
+     * Returns the timestamp label.
+     */
+    @Nullable
+    public String getTimestampLabel() {
+      return timestampLabel;
+    }
+
+    /**
+     * Returns the id label.
+     */
+    @Nullable
+    public String getIdLabel() {
+      return idLabel;
+    }
+
+    /**
+     * Returns the output coder.
+     */
+    @Nullable
+    public Coder<T> getCoder() {
+      return coder;
+    }
+
+    /**
+     * Returns the formatting function used if publishing attributes.
+     */
+    @Nullable
+    public SimpleFunction<T, PubsubMessage> getFormatFn() {
+      return formatFn;
+    }
+
+    /**
+     * Writer to Pubsub which batches messages from bounded collections.
+     *
+     * <p>Public so can be suppressed by runners.
+     */
+    public class PubsubBoundedWriter extends DoFn<T, Void> {
+
+      private static final int MAX_PUBLISH_BATCH_SIZE = 100;
+      private transient List<OutgoingMessage> output;
+      private transient PubsubClient pubsubClient;
+
+      @StartBundle
+      public void startBundle(Context c) throws IOException {
+        this.output = new ArrayList<>();
+        // NOTE: idLabel is ignored.
+        this.pubsubClient =
+            FACTORY.newClient(timestampLabel, null,
+                c.getPipelineOptions().as(PubsubOptions.class));
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext c) throws IOException {
+        byte[] payload = null;
+        Map<String, String> attributes = null;
+        if (formatFn != null) {
+          PubsubMessage message = formatFn.apply(c.element());
+          payload = message.getMessage();
+          attributes = message.getAttributeMap();
+        } else {
+          payload = CoderUtils.encodeToByteArray(getCoder(), c.element());
+        }
+        // NOTE: The record id is always null.
+        OutgoingMessage message =
+            new OutgoingMessage(payload, attributes, c.timestamp().getMillis(), null);
+        output.add(message);
+
+        if (output.size() >= MAX_PUBLISH_BATCH_SIZE) {
+          publish();
+        }
+      }
+
+      @FinishBundle
+      public void finishBundle(Context c) throws IOException {
+        if (!output.isEmpty()) {
+          publish();
+        }
+        output = null;
+        pubsubClient.close();
+        pubsubClient = null;
+      }
+
+      private void publish() throws IOException {
+        int n = pubsubClient.publish(
+            PubsubClient.topicPathFromName(getTopic().project, getTopic().topic),
+            output);
+        checkState(n == output.size());
+        output.clear();
+      }
+
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        super.populateDisplayData(builder);
+        builder.delegate(Write.this);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java
new file mode 100644
index 0000000..e290a6b
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java
@@ -0,0 +1,319 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.services.pubsub.Pubsub;
+import com.google.api.services.pubsub.Pubsub.Builder;
+import com.google.api.services.pubsub.model.AcknowledgeRequest;
+import com.google.api.services.pubsub.model.ListSubscriptionsResponse;
+import com.google.api.services.pubsub.model.ListTopicsResponse;
+import com.google.api.services.pubsub.model.ModifyAckDeadlineRequest;
+import com.google.api.services.pubsub.model.PublishRequest;
+import com.google.api.services.pubsub.model.PublishResponse;
+import com.google.api.services.pubsub.model.PubsubMessage;
+import com.google.api.services.pubsub.model.PullRequest;
+import com.google.api.services.pubsub.model.PullResponse;
+import com.google.api.services.pubsub.model.ReceivedMessage;
+import com.google.api.services.pubsub.model.Subscription;
+import com.google.api.services.pubsub.model.Topic;
+import com.google.auth.Credentials;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
+import org.apache.beam.sdk.util.Transport;
+
+/**
+ * A Pubsub client using JSON transport.
+ */
+class PubsubJsonClient extends PubsubClient {
+
+  private static class PubsubJsonClientFactory implements PubsubClientFactory {
+    private static HttpRequestInitializer chainHttpRequestInitializer(
+        Credentials credential, HttpRequestInitializer httpRequestInitializer) {
+      if (credential == null) {
+        return httpRequestInitializer;
+      } else {
+        return new ChainingHttpRequestInitializer(
+            new HttpCredentialsAdapter(credential),
+            httpRequestInitializer);
+      }
+    }
+
+    @Override
+    public PubsubClient newClient(
+        @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
+        throws IOException {
+      Pubsub pubsub = new Builder(
+          Transport.getTransport(),
+          Transport.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())
+          .build();
+      return new PubsubJsonClient(timestampLabel, idLabel, pubsub);
+    }
+
+    @Override
+    public String getKind() {
+      return "Json";
+    }
+  }
+
+  /**
+   * Factory for creating Pubsub clients using Json transport.
+   */
+  public static final PubsubClientFactory FACTORY = new PubsubJsonClientFactory();
+
+  /**
+   * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time
+   * instead.
+   */
+  @Nullable
+  private final String timestampLabel;
+
+  /**
+   * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids.
+   */
+  @Nullable
+  private final String idLabel;
+
+  /**
+   * Underlying JSON transport.
+   */
+  private Pubsub pubsub;
+
+  @VisibleForTesting
+  PubsubJsonClient(
+      @Nullable String timestampLabel,
+      @Nullable String idLabel,
+      Pubsub pubsub) {
+    this.timestampLabel = timestampLabel;
+    this.idLabel = idLabel;
+    this.pubsub = pubsub;
+  }
+
+  @Override
+  public void close() {
+    // Nothing to close.
+  }
+
+  @Override
+  public int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages)
+      throws IOException {
+    List<PubsubMessage> pubsubMessages = new ArrayList<>(outgoingMessages.size());
+    for (OutgoingMessage outgoingMessage : outgoingMessages) {
+      PubsubMessage pubsubMessage = new PubsubMessage().encodeData(outgoingMessage.elementBytes);
+
+      Map<String, String> attributes = outgoingMessage.attributes;
+      if ((timestampLabel != null || idLabel != null) && attributes == null) {
+        attributes = new TreeMap<>();
+      }
+      if (attributes != null) {
+        pubsubMessage.setAttributes(attributes);
+      }
+
+      if (timestampLabel != null) {
+        attributes.put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch));
+      }
+
+      if (idLabel != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) {
+        attributes.put(idLabel, outgoingMessage.recordId);
+      }
+
+      pubsubMessages.add(pubsubMessage);
+    }
+    PublishRequest request = new PublishRequest().setMessages(pubsubMessages);
+    PublishResponse response = pubsub.projects()
+                                     .topics()
+                                     .publish(topic.getPath(), request)
+                                     .execute();
+    return response.getMessageIds().size();
+  }
+
+  @Override
+  public List<IncomingMessage> pull(
+      long requestTimeMsSinceEpoch,
+      SubscriptionPath subscription,
+      int batchSize,
+      boolean returnImmediately) throws IOException {
+    PullRequest request = new PullRequest()
+        .setReturnImmediately(returnImmediately)
+        .setMaxMessages(batchSize);
+    PullResponse response = pubsub.projects()
+                                  .subscriptions()
+                                  .pull(subscription.getPath(), request)
+                                  .execute();
+    if (response.getReceivedMessages() == null || response.getReceivedMessages().size() == 0) {
+      return ImmutableList.of();
+    }
+    List<IncomingMessage> incomingMessages = new ArrayList<>(response.getReceivedMessages().size());
+    for (ReceivedMessage message : response.getReceivedMessages()) {
+      PubsubMessage pubsubMessage = message.getMessage();
+      @Nullable Map<String, String> attributes = pubsubMessage.getAttributes();
+
+      // Payload.
+      byte[] elementBytes = pubsubMessage.decodeData();
+
+      // Timestamp.
+      long timestampMsSinceEpoch =
+          extractTimestamp(timestampLabel, message.getMessage().getPublishTime(), attributes);
+
+      // Ack id.
+      String ackId = message.getAckId();
+      checkState(!Strings.isNullOrEmpty(ackId));
+
+      // Record id, if any.
+      @Nullable String recordId = null;
+      if (idLabel != null && attributes != null) {
+        recordId = attributes.get(idLabel);
+      }
+      if (Strings.isNullOrEmpty(recordId)) {
+        // Fall back to the Pubsub provided message id.
+        recordId = pubsubMessage.getMessageId();
+      }
+
+      incomingMessages.add(new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch,
+                                               requestTimeMsSinceEpoch, ackId, recordId));
+    }
+
+    return incomingMessages;
+  }
+
+  @Override
+  public void acknowledge(SubscriptionPath subscription, List<String> ackIds) throws IOException {
+    AcknowledgeRequest request = new AcknowledgeRequest().setAckIds(ackIds);
+    pubsub.projects()
+          .subscriptions()
+          .acknowledge(subscription.getPath(), request)
+          .execute(); // ignore Empty result.
+  }
+
+  @Override
+  public void modifyAckDeadline(
+      SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds)
+      throws IOException {
+    ModifyAckDeadlineRequest request =
+        new ModifyAckDeadlineRequest().setAckIds(ackIds)
+                                      .setAckDeadlineSeconds(deadlineSeconds);
+    pubsub.projects()
+          .subscriptions()
+          .modifyAckDeadline(subscription.getPath(), request)
+          .execute(); // ignore Empty result.
+  }
+
+  @Override
+  public void createTopic(TopicPath topic) throws IOException {
+    pubsub.projects()
+          .topics()
+          .create(topic.getPath(), new Topic())
+          .execute(); // ignore Topic result.
+  }
+
+  @Override
+  public void deleteTopic(TopicPath topic) throws IOException {
+    pubsub.projects()
+          .topics()
+          .delete(topic.getPath())
+          .execute(); // ignore Empty result.
+  }
+
+  @Override
+  public List<TopicPath> listTopics(ProjectPath project) throws IOException {
+    ListTopicsResponse response = pubsub.projects()
+                                        .topics()
+                                        .list(project.getPath())
+                                        .execute();
+    if (response.getTopics() == null || response.getTopics().isEmpty()) {
+      return ImmutableList.of();
+    }
+    List<TopicPath> topics = new ArrayList<>(response.getTopics().size());
+    for (Topic topic : response.getTopics()) {
+      topics.add(topicPathFromPath(topic.getName()));
+    }
+    return topics;
+  }
+
+  @Override
+  public void createSubscription(
+      TopicPath topic, SubscriptionPath subscription,
+      int ackDeadlineSeconds) throws IOException {
+    Subscription request = new Subscription()
+        .setTopic(topic.getPath())
+        .setAckDeadlineSeconds(ackDeadlineSeconds);
+    pubsub.projects()
+          .subscriptions()
+          .create(subscription.getPath(), request)
+          .execute(); // ignore Subscription result.
+  }
+
+  @Override
+  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
+    pubsub.projects()
+          .subscriptions()
+          .delete(subscription.getPath())
+          .execute(); // ignore Empty result.
+  }
+
+  @Override
+  public List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
+      throws IOException {
+    ListSubscriptionsResponse response = pubsub.projects()
+                                               .subscriptions()
+                                               .list(project.getPath())
+                                               .execute();
+    if (response.getSubscriptions() == null || response.getSubscriptions().isEmpty()) {
+      return ImmutableList.of();
+    }
+    List<SubscriptionPath> subscriptions = new ArrayList<>(response.getSubscriptions().size());
+    for (Subscription subscription : response.getSubscriptions()) {
+      if (subscription.getTopic().equals(topic.getPath())) {
+        subscriptions.add(subscriptionPathFromPath(subscription.getName()));
+      }
+    }
+    return subscriptions;
+  }
+
+  @Override
+  public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException {
+    Subscription response = pubsub.projects().subscriptions().get(subscription.getPath()).execute();
+    return response.getAckDeadlineSeconds();
+  }
+
+  @Override
+  public boolean isEOF() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java
new file mode 100644
index 0000000..c88576e
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java
@@ -0,0 +1,436 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.api.client.util.Clock;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.PubsubOptions;
+
+/**
+ * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for
+ * testing {@link #publish}, {@link #pull}, {@link #acknowledge} and {@link #modifyAckDeadline}
+ * methods. Relies on statics to mimic the Pubsub service, though we try to hide that.
+ */
+class PubsubTestClient extends PubsubClient implements Serializable {
+  /**
+   * Mimic the state of the simulated Pubsub 'service'.
+   *
+   * <p>Note that the {@link PubsubTestClientFactory} is serialized/deserialized even when running
+   * test pipelines. Meanwhile it is valid for multiple {@link PubsubTestClient}s to be created
+   * from the same client factory and run in parallel. Thus we can't enforce aliasing of the
+   * following data structures over all clients and must resort to a static.
+   */
+  private static class State {
+    /**
+     * True if has been primed for a test but not yet validated.
+     */
+    boolean isActive;
+
+    /**
+     * Publish mode only: Only publish calls for this topic are allowed.
+     */
+    @Nullable
+    TopicPath expectedTopic;
+
+    /**
+     * Publish mode only: Messages yet to seen in a {@link #publish} call.
+     */
+    @Nullable
+    Set<OutgoingMessage> remainingExpectedOutgoingMessages;
+
+    /**
+     * Publish mode only: Messages which should throw when first sent to simulate transient publish
+     * failure.
+     */
+    @Nullable
+    Set<OutgoingMessage> remainingFailingOutgoingMessages;
+
+    /**
+     * Pull mode only: Clock from which to get current time.
+     */
+    @Nullable
+    Clock clock;
+
+    /**
+     * Pull mode only: Only pull calls for this subscription are allowed.
+     */
+    @Nullable
+    SubscriptionPath expectedSubscription;
+
+    /**
+     * Pull mode only: Timeout to simulate.
+     */
+    int ackTimeoutSec;
+
+    /**
+     * Pull mode only: Messages waiting to be received by a {@link #pull} call.
+     */
+    @Nullable
+    List<IncomingMessage> remainingPendingIncomingMessages;
+
+    /**
+     * Pull mode only: Messages which have been returned from a {@link #pull} call and
+     * not yet ACKed by an {@link #acknowledge} call.
+     */
+    @Nullable
+    Map<String, IncomingMessage> pendingAckIncomingMessages;
+
+    /**
+     * Pull mode only: When above messages are due to have their ACK deadlines expire.
+     */
+    @Nullable
+    Map<String, Long> ackDeadline;
+  }
+
+  private static final State STATE = new State();
+
+  /** Closing the factory will validate all expected messages were processed. */
+  public interface PubsubTestClientFactory
+          extends PubsubClientFactory, Closeable, Serializable {
+  }
+
+  /**
+   * Return a factory for testing publishers. Only one factory may be in-flight at a time.
+   * The factory must be closed when the test is complete, at which point final validation will
+   * occur.
+   */
+  static PubsubTestClientFactory createFactoryForPublish(
+      final TopicPath expectedTopic,
+      final Iterable<OutgoingMessage> expectedOutgoingMessages,
+      final Iterable<OutgoingMessage> failingOutgoingMessages) {
+    synchronized (STATE) {
+      checkState(!STATE.isActive, "Test still in flight");
+      STATE.expectedTopic = expectedTopic;
+      STATE.remainingExpectedOutgoingMessages = Sets.newHashSet(expectedOutgoingMessages);
+      STATE.remainingFailingOutgoingMessages = Sets.newHashSet(failingOutgoingMessages);
+      STATE.isActive = true;
+    }
+    return new PubsubTestClientFactory() {
+      @Override
+      public PubsubClient newClient(
+          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
+          throws IOException {
+        return new PubsubTestClient();
+      }
+
+      @Override
+      public String getKind() {
+        return "PublishTest";
+      }
+
+      @Override
+      public void close() {
+        synchronized (STATE) {
+          checkState(STATE.isActive, "No test still in flight");
+          checkState(STATE.remainingExpectedOutgoingMessages.isEmpty(),
+                     "Still waiting for %s messages to be published",
+                     STATE.remainingExpectedOutgoingMessages.size());
+          STATE.isActive = false;
+          STATE.remainingExpectedOutgoingMessages = null;
+        }
+      }
+    };
+  }
+
+  /**
+   * Return a factory for testing subscribers. Only one factory may be in-flight at a time.
+   * The factory must be closed when the test in complete
+   */
+  public static PubsubTestClientFactory createFactoryForPull(
+      final Clock clock,
+      final SubscriptionPath expectedSubscription,
+      final int ackTimeoutSec,
+      final Iterable<IncomingMessage> expectedIncomingMessages) {
+    synchronized (STATE) {
+      checkState(!STATE.isActive, "Test still in flight");
+      STATE.clock = clock;
+      STATE.expectedSubscription = expectedSubscription;
+      STATE.ackTimeoutSec = ackTimeoutSec;
+      STATE.remainingPendingIncomingMessages = Lists.newArrayList(expectedIncomingMessages);
+      STATE.pendingAckIncomingMessages = new HashMap<>();
+      STATE.ackDeadline = new HashMap<>();
+      STATE.isActive = true;
+    }
+    return new PubsubTestClientFactory() {
+      @Override
+      public PubsubClient newClient(
+          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
+          throws IOException {
+        return new PubsubTestClient();
+      }
+
+      @Override
+      public String getKind() {
+        return "PullTest";
+      }
+
+      @Override
+      public void close() {
+        synchronized (STATE) {
+          checkState(STATE.isActive, "No test still in flight");
+          checkState(STATE.remainingPendingIncomingMessages.isEmpty(),
+                     "Still waiting for %s messages to be pulled",
+                     STATE.remainingPendingIncomingMessages.size());
+          checkState(STATE.pendingAckIncomingMessages.isEmpty(),
+                     "Still waiting for %s messages to be ACKed",
+                     STATE.pendingAckIncomingMessages.size());
+          checkState(STATE.ackDeadline.isEmpty(),
+                     "Still waiting for %s messages to be ACKed",
+                     STATE.ackDeadline.size());
+          STATE.isActive = false;
+          STATE.remainingPendingIncomingMessages = null;
+          STATE.pendingAckIncomingMessages = null;
+          STATE.ackDeadline = null;
+        }
+      }
+    };
+  }
+
+  public static PubsubTestClientFactory createFactoryForCreateSubscription() {
+    return new PubsubTestClientFactory() {
+      int numCalls = 0;
+
+      @Override
+      public void close() throws IOException {
+        checkState(
+            numCalls == 1, "Expected exactly one subscription to be created, got %s", numCalls);
+      }
+
+      @Override
+      public PubsubClient newClient(
+          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
+          throws IOException {
+        return new PubsubTestClient() {
+          @Override
+          public void createSubscription(
+              TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds)
+              throws IOException {
+            checkState(numCalls == 0, "Expected at most one subscription to be created");
+            numCalls++;
+          }
+        };
+      }
+
+      @Override
+      public String getKind() {
+        return "CreateSubscriptionTest";
+      }
+    };
+  }
+
+  /**
+   * Return true if in pull mode.
+   */
+  private boolean inPullMode() {
+    checkState(STATE.isActive, "No test is active");
+    return STATE.expectedSubscription != null;
+  }
+
+  /**
+   * Return true if in publish mode.
+   */
+  private boolean inPublishMode() {
+    checkState(STATE.isActive, "No test is active");
+    return STATE.expectedTopic != null;
+  }
+
+  /**
+   * For subscription mode only:
+   * Track progression of time according to the {@link Clock} passed . This will simulate Pubsub
+   * expiring
+   * outstanding ACKs.
+   */
+  public void advance() {
+    synchronized (STATE) {
+      checkState(inPullMode(), "Can only advance in pull mode");
+      // Any messages who's ACKs timed out are available for re-pulling.
+      Iterator<Map.Entry<String, Long>> deadlineItr = STATE.ackDeadline.entrySet().iterator();
+      while (deadlineItr.hasNext()) {
+        Map.Entry<String, Long> entry = deadlineItr.next();
+        if (entry.getValue() <= STATE.clock.currentTimeMillis()) {
+          STATE.remainingPendingIncomingMessages.add(
+              STATE.pendingAckIncomingMessages.remove(entry.getKey()));
+          deadlineItr.remove();
+        }
+      }
+    }
+  }
+
+  @Override
+  public void close() {
+  }
+
+  @Override
+  public int publish(
+      TopicPath topic, List<OutgoingMessage> outgoingMessages) throws IOException {
+    synchronized (STATE) {
+      checkState(inPublishMode(), "Can only publish in publish mode");
+      checkState(topic.equals(STATE.expectedTopic), "Topic %s does not match expected %s", topic,
+                 STATE.expectedTopic);
+      for (OutgoingMessage outgoingMessage : outgoingMessages) {
+        if (STATE.remainingFailingOutgoingMessages.remove(outgoingMessage)) {
+          throw new RuntimeException("Simulating failure for " + outgoingMessage);
+        }
+        checkState(STATE.remainingExpectedOutgoingMessages.remove(outgoingMessage),
+                   "Unexpected outgoing message %s", outgoingMessage);
+      }
+      return outgoingMessages.size();
+    }
+  }
+
+  @Override
+  public List<IncomingMessage> pull(
+      long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize,
+      boolean returnImmediately) throws IOException {
+    synchronized (STATE) {
+      checkState(inPullMode(), "Can only pull in pull mode");
+      long now = STATE.clock.currentTimeMillis();
+      checkState(requestTimeMsSinceEpoch == now,
+                 "Simulated time %s does not match request time %s", now, requestTimeMsSinceEpoch);
+      checkState(subscription.equals(STATE.expectedSubscription),
+                 "Subscription %s does not match expected %s", subscription,
+                 STATE.expectedSubscription);
+      checkState(returnImmediately, "Pull only supported if returning immediately");
+
+      List<IncomingMessage> incomingMessages = new ArrayList<>();
+      Iterator<IncomingMessage> pendItr = STATE.remainingPendingIncomingMessages.iterator();
+      while (pendItr.hasNext()) {
+        IncomingMessage incomingMessage = pendItr.next();
+        pendItr.remove();
+        IncomingMessage incomingMessageWithRequestTime =
+            incomingMessage.withRequestTime(requestTimeMsSinceEpoch);
+        incomingMessages.add(incomingMessageWithRequestTime);
+        STATE.pendingAckIncomingMessages.put(incomingMessageWithRequestTime.ackId,
+                                             incomingMessageWithRequestTime);
+        STATE.ackDeadline.put(incomingMessageWithRequestTime.ackId,
+                              requestTimeMsSinceEpoch + STATE.ackTimeoutSec * 1000);
+        if (incomingMessages.size() >= batchSize) {
+          break;
+        }
+      }
+      return incomingMessages;
+    }
+  }
+
+  @Override
+  public void acknowledge(
+      SubscriptionPath subscription,
+      List<String> ackIds) throws IOException {
+    synchronized (STATE) {
+      checkState(inPullMode(), "Can only acknowledge in pull mode");
+      checkState(subscription.equals(STATE.expectedSubscription),
+                 "Subscription %s does not match expected %s", subscription,
+                 STATE.expectedSubscription);
+
+      for (String ackId : ackIds) {
+        checkState(STATE.ackDeadline.remove(ackId) != null,
+                   "No message with ACK id %s is waiting for an ACK", ackId);
+        checkState(STATE.pendingAckIncomingMessages.remove(ackId) != null,
+                   "No message with ACK id %s is waiting for an ACK", ackId);
+      }
+    }
+  }
+
+  @Override
+  public void modifyAckDeadline(
+      SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds) throws IOException {
+    synchronized (STATE) {
+      checkState(inPullMode(), "Can only modify ack deadline in pull mode");
+      checkState(subscription.equals(STATE.expectedSubscription),
+                 "Subscription %s does not match expected %s", subscription,
+                 STATE.expectedSubscription);
+
+      for (String ackId : ackIds) {
+        if (deadlineSeconds > 0) {
+          checkState(STATE.ackDeadline.remove(ackId) != null,
+                     "No message with ACK id %s is waiting for an ACK", ackId);
+          checkState(STATE.pendingAckIncomingMessages.containsKey(ackId),
+                     "No message with ACK id %s is waiting for an ACK", ackId);
+          STATE.ackDeadline.put(ackId, STATE.clock.currentTimeMillis() + deadlineSeconds * 1000);
+        } else {
+          checkState(STATE.ackDeadline.remove(ackId) != null,
+                     "No message with ACK id %s is waiting for an ACK", ackId);
+          IncomingMessage message = STATE.pendingAckIncomingMessages.remove(ackId);
+          checkState(message != null, "No message with ACK id %s is waiting for an ACK", ackId);
+          STATE.remainingPendingIncomingMessages.add(message);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void createTopic(TopicPath topic) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void deleteTopic(TopicPath topic) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<TopicPath> listTopics(ProjectPath project) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void createSubscription(
+      TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<SubscriptionPath> listSubscriptions(
+      ProjectPath project, TopicPath topic) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException {
+    synchronized (STATE) {
+      return STATE.ackTimeoutSec;
+    }
+  }
+
+  @Override
+  public boolean isEOF() {
+    synchronized (STATE) {
+      checkState(inPullMode(), "Can only check EOF in pull mode");
+      return STATE.remainingPendingIncomingMessages.isEmpty();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java
new file mode 100644
index 0000000..3ce9224
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.hash.Hashing;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.PubsubClientFactory;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
+import org.apache.beam.sdk.transforms.windowing.AfterFirst;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.joda.time.Duration;
+
+/**
+ * A PTransform which streams messages to Pubsub.
+ * <ul>
+ * <li>The underlying implementation is just a {@link GroupByKey} followed by a {@link ParDo} which
+ * publishes as a side effect. (In the future we want to design and switch to a custom
+ * {@code UnboundedSink} implementation so as to gain access to system watermark and
+ * end-of-pipeline cleanup.)
+ * <li>We try to send messages in batches while also limiting send latency.
+ * <li>No stats are logged. Rather some counters are used to keep track of elements and batches.
+ * <li>Though some background threads are used by the underlying netty system all actual Pubsub
+ * calls are blocking. We rely on the underlying runner to allow multiple {@link DoFn} instances
+ * to execute concurrently and hide latency.
+ * <li>A failed bundle will cause messages to be resent. Thus we rely on the Pubsub consumer
+ * to dedup messages.
+ * </ul>
+ */
+public class PubsubUnboundedSink<T> extends PTransform<PCollection<T>, PDone> {
+  /**
+   * Default maximum number of messages per publish.
+   */
+  private static final int DEFAULT_PUBLISH_BATCH_SIZE = 1000;
+
+  /**
+   * Default maximum size of a publish batch, in bytes.
+   */
+  private static final int DEFAULT_PUBLISH_BATCH_BYTES = 400000;
+
+  /**
+   * Default longest delay between receiving a message and pushing it to Pubsub.
+   */
+  private static final Duration DEFAULT_MAX_LATENCY = Duration.standardSeconds(2);
+
+  /**
+   * Coder for conveying outgoing messages between internal stages.
+   */
+  private static class OutgoingMessageCoder extends AtomicCoder<OutgoingMessage> {
+    private static final NullableCoder<String> RECORD_ID_CODER =
+        NullableCoder.of(StringUtf8Coder.of());
+    private static final NullableCoder<Map<String, String>> ATTRIBUTES_CODER =
+            NullableCoder.of(MapCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
+
+    @Override
+    public void encode(
+        OutgoingMessage value, OutputStream outStream, Context context)
+        throws CoderException, IOException {
+      ByteArrayCoder.of().encode(value.elementBytes, outStream, context.nested());
+      ATTRIBUTES_CODER.encode(value.attributes, outStream, context.nested());
+      BigEndianLongCoder.of().encode(value.timestampMsSinceEpoch, outStream, context.nested());
+      RECORD_ID_CODER.encode(value.recordId, outStream, context.nested());
+    }
+
+    @Override
+    public OutgoingMessage decode(
+        InputStream inStream, Context context) throws CoderException, IOException {
+      byte[] elementBytes = ByteArrayCoder.of().decode(inStream, context.nested());
+      Map<String, String> attributes = ATTRIBUTES_CODER.decode(inStream, context.nested());
+      long timestampMsSinceEpoch = BigEndianLongCoder.of().decode(inStream, context.nested());
+      @Nullable String recordId = RECORD_ID_CODER.decode(inStream, context.nested());
+      return new OutgoingMessage(elementBytes, attributes, timestampMsSinceEpoch, recordId);
+    }
+  }
+
+  @VisibleForTesting
+  static final Coder<OutgoingMessage> CODER = new OutgoingMessageCoder();
+
+  // ================================================================================
+  // RecordIdMethod
+  // ================================================================================
+
+  /**
+   * Specify how record ids are to be generated.
+   */
+  @VisibleForTesting
+  enum RecordIdMethod {
+    /** Leave null. */
+    NONE,
+    /** Generate randomly. */
+    RANDOM,
+    /** Generate deterministically. For testing only. */
+    DETERMINISTIC
+  }
+
+  // ================================================================================
+  // ShardFn
+  // ================================================================================
+
+  /**
+   * Convert elements to messages and shard them.
+   */
+  private static class ShardFn<T> extends DoFn<T, KV<Integer, OutgoingMessage>> {
+    private final Counter elementCounter = Metrics.counter(ShardFn.class, "elements");
+    private final Coder<T> elementCoder;
+    private final int numShards;
+    private final RecordIdMethod recordIdMethod;
+    private final SimpleFunction<T, PubsubIO.PubsubMessage> formatFn;
+
+    ShardFn(Coder<T> elementCoder, int numShards,
+            SimpleFunction<T, PubsubIO.PubsubMessage> formatFn, RecordIdMethod recordIdMethod) {
+      this.elementCoder = elementCoder;
+      this.numShards = numShards;
+      this.formatFn = formatFn;
+      this.recordIdMethod = recordIdMethod;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      elementCounter.inc();
+      byte[] elementBytes = null;
+      Map<String, String> attributes = ImmutableMap.<String, String>of();
+      if (formatFn != null) {
+        PubsubIO.PubsubMessage message = formatFn.apply(c.element());
+        elementBytes = message.getMessage();
+        attributes = message.getAttributeMap();
+      } else {
+        elementBytes = CoderUtils.encodeToByteArray(elementCoder, c.element());
+      }
+
+      long timestampMsSinceEpoch = c.timestamp().getMillis();
+      @Nullable String recordId = null;
+      switch (recordIdMethod) {
+        case NONE:
+          break;
+        case DETERMINISTIC:
+          recordId = Hashing.murmur3_128().hashBytes(elementBytes).toString();
+          break;
+        case RANDOM:
+          // Since these elements go through a GroupByKey, any  failures while sending to
+          // Pubsub will be retried without falling back and generating a new record id.
+          // Thus even though we may send the same message to Pubsub twice, it is guaranteed
+          // to have the same record id.
+          recordId = UUID.randomUUID().toString();
+          break;
+      }
+      c.output(KV.of(ThreadLocalRandom.current().nextInt(numShards),
+                     new OutgoingMessage(elementBytes, attributes, timestampMsSinceEpoch,
+                             recordId)));
+    }
+
+    @Override
+    public void populateDisplayData(Builder builder) {
+      super.populateDisplayData(builder);
+      builder.add(DisplayData.item("numShards", numShards));
+    }
+  }
+
+  // ================================================================================
+  // WriterFn
+  // ================================================================================
+
+  /**
+   * Publish messages to Pubsub in batches.
+   */
+  private static class WriterFn
+      extends DoFn<KV<Integer, Iterable<OutgoingMessage>>, Void> {
+    private final PubsubClientFactory pubsubFactory;
+    private final ValueProvider<TopicPath> topic;
+    private final String timestampLabel;
+    private final String idLabel;
+    private final int publishBatchSize;
+    private final int publishBatchBytes;
+
+    /**
+     * Client on which to talk to Pubsub. Null until created by {@link #startBundle}.
+     */
+    @Nullable
+    private transient PubsubClient pubsubClient;
+
+    private final Counter batchCounter = Metrics.counter(WriterFn.class, "batches");
+    private final Counter elementCounter = Metrics.counter(WriterFn.class, "elements");
+    private final Counter byteCounter = Metrics.counter(WriterFn.class, "bytes");
+
+    WriterFn(
+        PubsubClientFactory pubsubFactory, ValueProvider<TopicPath> topic,
+        String timestampLabel, String idLabel, int publishBatchSize, int publishBatchBytes) {
+      this.pubsubFactory = pubsubFactory;
+      this.topic = topic;
+      this.timestampLabel = timestampLabel;
+      this.idLabel = idLabel;
+      this.publishBatchSize = publishBatchSize;
+      this.publishBatchBytes = publishBatchBytes;
+    }
+
+    /**
+     * BLOCKING
+     * Send {@code messages} as a batch to Pubsub.
+     */
+    private void publishBatch(List<OutgoingMessage> messages, int bytes)
+        throws IOException {
+      int n = pubsubClient.publish(topic.get(), messages);
+      checkState(n == messages.size(), "Attempted to publish %s messages but %s were successful",
+                 messages.size(), n);
+      batchCounter.inc();
+      elementCounter.inc(messages.size());
+      byteCounter.inc(bytes);
+    }
+
+    @StartBundle
+    public void startBundle(Context c) throws Exception {
+      checkState(pubsubClient == null, "startBundle invoked without prior finishBundle");
+      pubsubClient = pubsubFactory.newClient(timestampLabel, idLabel,
+                                             c.getPipelineOptions().as(PubsubOptions.class));
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      List<OutgoingMessage> pubsubMessages = new ArrayList<>(publishBatchSize);
+      int bytes = 0;
+      for (OutgoingMessage message : c.element().getValue()) {
+        if (!pubsubMessages.isEmpty()
+            && bytes + message.elementBytes.length > publishBatchBytes) {
+          // Break large (in bytes) batches into smaller.
+          // (We've already broken by batch size using the trigger below, though that may
+          // run slightly over the actual PUBLISH_BATCH_SIZE. We'll consider that ok since
+          // the hard limit from Pubsub is by bytes rather than number of messages.)
+          // BLOCKS until published.
+          publishBatch(pubsubMessages, bytes);
+          pubsubMessages.clear();
+          bytes = 0;
+        }
+        pubsubMessages.add(message);
+        bytes += message.elementBytes.length;
+      }
+      if (!pubsubMessages.isEmpty()) {
+        // BLOCKS until published.
+        publishBatch(pubsubMessages, bytes);
+      }
+    }
+
+    @FinishBundle
+    public void finishBundle(Context c) throws Exception {
+      pubsubClient.close();
+      pubsubClient = null;
+    }
+
+    @Override
+    public void populateDisplayData(Builder builder) {
+      super.populateDisplayData(builder);
+        String topicString =
+            topic == null ? null
+            : topic.isAccessible() ? topic.get().getPath()
+            : topic.toString();
+      builder.add(DisplayData.item("topic", topicString));
+      builder.add(DisplayData.item("transport", pubsubFactory.getKind()));
+      builder.addIfNotNull(DisplayData.item("timestampLabel", timestampLabel));
+      builder.addIfNotNull(DisplayData.item("idLabel", idLabel));
+    }
+  }
+
+  // ================================================================================
+  // PubsubUnboundedSink
+  // ================================================================================
+
+  /**
+   * Which factory to use for creating Pubsub transport.
+   */
+  private final PubsubClientFactory pubsubFactory;
+
+  /**
+   * Pubsub topic to publish to.
+   */
+  private final ValueProvider<TopicPath> topic;
+
+  /**
+   * Coder for elements. It is the responsibility of the underlying Pubsub transport to
+   * re-encode element bytes if necessary, eg as Base64 strings.
+   */
+  private final Coder<T> elementCoder;
+
+  /**
+   * Pubsub metadata field holding timestamp of each element, or {@literal null} if should use
+   * Pubsub message publish timestamp instead.
+   */
+  @Nullable
+  private final String timestampLabel;
+
+  /**
+   * Pubsub metadata field holding id for each element, or {@literal null} if need to generate
+   * a unique id ourselves.
+   */
+  @Nullable
+  private final String idLabel;
+
+  /**
+   * Number of 'shards' to use so that latency in Pubsub publish can be hidden. Generally this
+   * should be a small multiple of the number of available cores. Too smoll a number results
+   * in too much time lost to blocking Pubsub calls. To large a number results in too many
+   * single-element batches being sent to Pubsub with high per-batch overhead.
+   */
+  private final int numShards;
+
+  /**
+   * Maximum number of messages per publish.
+   */
+  private final int publishBatchSize;
+
+  /**
+   * Maximum size of a publish batch, in bytes.
+   */
+  private final int publishBatchBytes;
+
+  /**
+   * Longest delay between receiving a message and pushing it to Pubsub.
+   */
+  private final Duration maxLatency;
+
+  /**
+   * How record ids should be generated for each record (if {@link #idLabel} is non-{@literal
+   * null}).
+   */
+  private final RecordIdMethod recordIdMethod;
+
+  /**
+   * In order to publish attributes, a formatting function is used to format the output into
+   * a {@link PubsubIO.PubsubMessage}.
+   */
+  private final SimpleFunction<T, PubsubIO.PubsubMessage> formatFn;
+
+  @VisibleForTesting
+  PubsubUnboundedSink(
+      PubsubClientFactory pubsubFactory,
+      ValueProvider<TopicPath> topic,
+      Coder<T> elementCoder,
+      String timestampLabel,
+      String idLabel,
+      int numShards,
+      int publishBatchSize,
+      int publishBatchBytes,
+      Duration maxLatency,
+      SimpleFunction<T, PubsubIO.PubsubMessage> formatFn,
+      RecordIdMethod recordIdMethod) {
+    this.pubsubFactory = pubsubFactory;
+    this.topic = topic;
+    this.elementCoder = elementCoder;
+    this.timestampLabel = timestampLabel;
+    this.idLabel = idLabel;
+    this.numShards = numShards;
+    this.publishBatchSize = publishBatchSize;
+    this.publishBatchBytes = publishBatchBytes;
+    this.maxLatency = maxLatency;
+    this.formatFn = formatFn;
+    this.recordIdMethod = idLabel == null ? RecordIdMethod.NONE : recordIdMethod;
+  }
+
+  public PubsubUnboundedSink(
+      PubsubClientFactory pubsubFactory,
+      ValueProvider<TopicPath> topic,
+      Coder<T> elementCoder,
+      String timestampLabel,
+      String idLabel,
+      SimpleFunction<T, PubsubIO.PubsubMessage> formatFn,
+      int numShards) {
+    this(pubsubFactory, topic, elementCoder, timestampLabel, idLabel, numShards,
+         DEFAULT_PUBLISH_BATCH_SIZE, DEFAULT_PUBLISH_BATCH_BYTES, DEFAULT_MAX_LATENCY,
+         formatFn, RecordIdMethod.RANDOM);
+  }
+
+  /**
+   * Get the topic being written to.
+   */
+  public TopicPath getTopic() {
+    return topic.get();
+  }
+
+  /**
+   * Get the {@link ValueProvider} for the topic being written to.
+   */
+  public ValueProvider<TopicPath> getTopicProvider() {
+    return topic;
+  }
+
+  /**
+   * Get the timestamp label.
+   */
+  @Nullable
+  public String getTimestampLabel() {
+    return timestampLabel;
+  }
+
+  /**
+   * Get the id label.
+   */
+  @Nullable
+  public String getIdLabel() {
+    return idLabel;
+  }
+
+  /**
+   * Get the format function used for PubSub attributes.
+   */
+  @Nullable
+  public SimpleFunction<T, PubsubIO.PubsubMessage> getFormatFn() {
+    return formatFn;
+  }
+
+  /**
+   * Get the Coder used to encode output elements.
+   */
+  public Coder<T> getElementCoder() {
+    return elementCoder;
+  }
+
+  @Override
+  public PDone expand(PCollection<T> input) {
+    input.apply("PubsubUnboundedSink.Window", Window.<T>into(new GlobalWindows())
+        .triggering(
+            Repeatedly.forever(
+                AfterFirst.of(AfterPane.elementCountAtLeast(publishBatchSize),
+                    AfterProcessingTime.pastFirstElementInPane()
+                    .plusDelayOf(maxLatency))))
+            .discardingFiredPanes())
+         .apply("PubsubUnboundedSink.Shard",
+             ParDo.of(new ShardFn<T>(elementCoder, numShards, formatFn, recordIdMethod)))
+         .setCoder(KvCoder.of(VarIntCoder.of(), CODER))
+         .apply(GroupByKey.<Integer, OutgoingMessage>create())
+         .apply("PubsubUnboundedSink.Writer",
+             ParDo.of(new WriterFn(pubsubFactory, topic, timestampLabel, idLabel,
+                 publishBatchSize, publishBatchBytes)));
+    return PDone.in(input.getPipeline());
+  }
+}


[2/8] beam git commit: [BEAM-1722] Move PubsubIO into the google-cloud-platform module

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java
new file mode 100644
index 0000000..9d8763b
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java
@@ -0,0 +1,1463 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.api.client.util.Clock;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.security.GeneralSecurityException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.ProjectPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.PubsubClientFactory;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.BucketingFunction;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.MovingFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A PTransform which streams messages from Pubsub.
+ * <ul>
+ * <li>The underlying implementation in an {@link UnboundedSource} which receives messages
+ * in batches and hands them out one at a time.
+ * <li>The watermark (either in Pubsub processing time or custom timestamp time) is estimated
+ * by keeping track of the minimum of the last minutes worth of messages. This assumes Pubsub
+ * delivers the oldest (in Pubsub processing time) available message at least once a minute,
+ * and that custom timestamps are 'mostly' monotonic with Pubsub processing time. Unfortunately
+ * both of those assumptions are fragile. Thus the estimated watermark may get ahead of
+ * the 'true' watermark and cause some messages to be late.
+ * <li>Checkpoints are used both to ACK received messages back to Pubsub (so that they may
+ * be retired on the Pubsub end), and to NACK already consumed messages should a checkpoint
+ * need to be restored (so that Pubsub will resend those messages promptly).
+ * <li>The backlog is determined by each reader using the messages which have been pulled from
+ * Pubsub but not yet consumed downstream. The backlog does not take account of any messages queued
+ * by Pubsub for the subscription. Unfortunately there is currently no API to determine
+ * the size of the Pubsub queue's backlog.
+ * <li>The subscription must already exist.
+ * <li>The subscription timeout is read whenever a reader is started. However it is not
+ * checked thereafter despite the timeout being user-changeable on-the-fly.
+ * <li>We log vital stats every 30 seconds.
+ * <li>Though some background threads may be used by the underlying transport all Pubsub calls
+ * are blocking. We rely on the underlying runner to allow multiple
+ * {@link UnboundedSource.UnboundedReader} instances to execute concurrently and thus hide latency.
+ * </ul>
+ */
+public class PubsubUnboundedSource<T> extends PTransform<PBegin, PCollection<T>> {
+  private static final Logger LOG = LoggerFactory.getLogger(PubsubUnboundedSource.class);
+
+  /**
+   * Default ACK timeout for created subscriptions.
+   */
+  private static final int DEAULT_ACK_TIMEOUT_SEC = 60;
+
+  /**
+   * Coder for checkpoints.
+   */
+  private static final PubsubCheckpointCoder<?> CHECKPOINT_CODER = new PubsubCheckpointCoder<>();
+
+  /**
+   * Maximum number of messages per pull.
+   */
+  private static final int PULL_BATCH_SIZE = 1000;
+
+  /**
+   * Maximum number of ACK ids per ACK or ACK extension call.
+   */
+  private static final int ACK_BATCH_SIZE = 2000;
+
+  /**
+   * Maximum number of messages in flight.
+   */
+  private static final int MAX_IN_FLIGHT = 20000;
+
+  /**
+   * Timeout for round trip from receiving a message to finally ACKing it back to Pubsub.
+   */
+  private static final Duration PROCESSING_TIMEOUT = Duration.standardSeconds(120);
+
+  /**
+   * Percentage of ack timeout by which to extend acks when they are near timeout.
+   */
+  private static final int ACK_EXTENSION_PCT = 50;
+
+  /**
+   * Percentage of ack timeout we should use as a safety margin. We'll try to extend acks
+   * by this margin before the ack actually expires.
+   */
+  private static final int ACK_SAFETY_PCT = 20;
+
+  /**
+   * For stats only: How close we can get to an ack deadline before we risk it being already
+   * considered passed by Pubsub.
+   */
+  private static final Duration ACK_TOO_LATE = Duration.standardSeconds(2);
+
+  /**
+   * Period of samples to determine watermark and other stats.
+   */
+  private static final Duration SAMPLE_PERIOD = Duration.standardMinutes(1);
+
+  /**
+   * Period of updates to determine watermark and other stats.
+   */
+  private static final Duration SAMPLE_UPDATE = Duration.standardSeconds(5);
+
+  /**
+   * Period for logging stats.
+   */
+  private static final Duration LOG_PERIOD = Duration.standardSeconds(30);
+
+  /**
+   * Minimum number of unread messages required before considering updating watermark.
+   */
+  private static final int MIN_WATERMARK_MESSAGES = 10;
+
+  /**
+   * Minimum number of SAMPLE_UPDATE periods over which unread messages shoud be spread
+   * before considering updating watermark.
+   */
+  private static final int MIN_WATERMARK_SPREAD = 2;
+
+  /**
+   * Additional sharding so that we can hide read message latency.
+   */
+  private static final int SCALE_OUT = 4;
+
+  // TODO: Would prefer to use MinLongFn but it is a BinaryCombineFn<Long> rather
+  // than a BinaryCombineLongFn. [BEAM-285]
+  private static final Combine.BinaryCombineLongFn MIN =
+      new Combine.BinaryCombineLongFn() {
+        @Override
+        public long apply(long left, long right) {
+          return Math.min(left, right);
+        }
+
+        @Override
+        public long identity() {
+          return Long.MAX_VALUE;
+        }
+      };
+
+  private static final Combine.BinaryCombineLongFn MAX =
+      new Combine.BinaryCombineLongFn() {
+        @Override
+        public long apply(long left, long right) {
+          return Math.max(left, right);
+        }
+
+        @Override
+        public long identity() {
+          return Long.MIN_VALUE;
+        }
+      };
+
+  private static final Combine.BinaryCombineLongFn SUM = Sum.ofLongs();
+
+  // ================================================================================
+  // Checkpoint
+  // ================================================================================
+
+  /**
+   * Which messages have been durably committed and thus can now be ACKed.
+   * Which messages have been read but not yet committed, in which case they should be NACKed if
+   * we need to restore.
+   */
+  @VisibleForTesting
+  static class PubsubCheckpoint<T> implements UnboundedSource.CheckpointMark {
+    /**
+     * The {@link SubscriptionPath} to the subscription the reader is reading from. May be
+     * {@code null} if the {@link PubsubUnboundedSource} contains the subscription.
+     */
+    @VisibleForTesting
+    @Nullable String subscriptionPath;
+
+    /**
+     * If the checkpoint is for persisting: the reader who's snapshotted state we are persisting.
+     * If the checkpoint is for restoring: {@literal null}.
+     * Not persisted in durable checkpoint.
+     * CAUTION: Between a checkpoint being taken and {@link #finalizeCheckpoint()} being called
+     * the 'true' active reader may have changed.
+     */
+    @Nullable
+    private PubsubReader<T> reader;
+
+    /**
+     * If the checkpoint is for persisting: The ACK ids of messages which have been passed
+     * downstream since the last checkpoint.
+     * If the checkpoint is for restoring: {@literal null}.
+     * Not persisted in durable checkpoint.
+     */
+    @Nullable
+    private List<String> safeToAckIds;
+
+    /**
+     * If the checkpoint is for persisting: The ACK ids of messages which have been received
+     * from Pubsub but not yet passed downstream at the time of the snapshot.
+     * If the checkpoint is for restoring: Same, but recovered from durable storage.
+     */
+    @VisibleForTesting
+    final List<String> notYetReadIds;
+
+    public PubsubCheckpoint(
+        @Nullable String subscriptionPath,
+        @Nullable PubsubReader<T> reader,
+        @Nullable List<String> safeToAckIds,
+        List<String> notYetReadIds) {
+      this.subscriptionPath = subscriptionPath;
+      this.reader = reader;
+      this.safeToAckIds = safeToAckIds;
+      this.notYetReadIds = notYetReadIds;
+    }
+
+    @Nullable
+    private SubscriptionPath getSubscription() {
+      return subscriptionPath == null
+          ? null
+          : PubsubClient.subscriptionPathFromPath(subscriptionPath);
+    }
+
+    /**
+     * BLOCKING
+     * All messages which have been passed downstream have now been durably committed.
+     * We can ACK them upstream.
+     * CAUTION: This may never be called.
+     */
+    @Override
+    public void finalizeCheckpoint() throws IOException {
+      checkState(reader != null && safeToAckIds != null, "Cannot finalize a restored checkpoint");
+      // Even if the 'true' active reader has changed since the checkpoint was taken we are
+      // fine:
+      // - The underlying Pubsub topic will not have changed, so the following ACKs will still
+      // go to the right place.
+      // - We'll delete the ACK ids from the readers in-flight state, but that only effects
+      // flow control and stats, neither of which are relevant anymore.
+      try {
+        int n = safeToAckIds.size();
+        List<String> batchSafeToAckIds = new ArrayList<>(Math.min(n, ACK_BATCH_SIZE));
+        for (String ackId : safeToAckIds) {
+          batchSafeToAckIds.add(ackId);
+          if (batchSafeToAckIds.size() >= ACK_BATCH_SIZE) {
+            reader.ackBatch(batchSafeToAckIds);
+            n -= batchSafeToAckIds.size();
+            // CAUTION: Don't reuse the same list since ackBatch holds on to it.
+            batchSafeToAckIds = new ArrayList<>(Math.min(n, ACK_BATCH_SIZE));
+          }
+        }
+        if (!batchSafeToAckIds.isEmpty()) {
+          reader.ackBatch(batchSafeToAckIds);
+        }
+      } finally {
+        int remainingInFlight = reader.numInFlightCheckpoints.decrementAndGet();
+        checkState(remainingInFlight >= 0,
+                   "Miscounted in-flight checkpoints");
+        reader.maybeCloseClient();
+        reader = null;
+        safeToAckIds = null;
+      }
+    }
+
+    /**
+     * Return current time according to {@code reader}.
+     */
+    private static long now(PubsubReader<?> reader) {
+      if (reader.outer.outer.clock == null) {
+        return System.currentTimeMillis();
+      } else {
+        return reader.outer.outer.clock.currentTimeMillis();
+      }
+    }
+
+    /**
+     * BLOCKING
+     * NACK all messages which have been read from Pubsub but not passed downstream.
+     * This way Pubsub will send them again promptly.
+     */
+    public void nackAll(PubsubReader<T> reader) throws IOException {
+      checkState(this.reader == null, "Cannot nackAll on persisting checkpoint");
+      List<String> batchYetToAckIds =
+          new ArrayList<>(Math.min(notYetReadIds.size(), ACK_BATCH_SIZE));
+      for (String ackId : notYetReadIds) {
+        batchYetToAckIds.add(ackId);
+        if (batchYetToAckIds.size() >= ACK_BATCH_SIZE) {
+          long nowMsSinceEpoch = now(reader);
+          reader.nackBatch(nowMsSinceEpoch, batchYetToAckIds);
+          batchYetToAckIds.clear();
+        }
+      }
+      if (!batchYetToAckIds.isEmpty()) {
+        long nowMsSinceEpoch = now(reader);
+        reader.nackBatch(nowMsSinceEpoch, batchYetToAckIds);
+      }
+    }
+  }
+
+  /** The coder for our checkpoints. */
+  private static class PubsubCheckpointCoder<T> extends AtomicCoder<PubsubCheckpoint<T>> {
+    private static final Coder<String> SUBSCRIPTION_PATH_CODER =
+        NullableCoder.of(StringUtf8Coder.of());
+    private static final Coder<List<String>> LIST_CODER = ListCoder.of(StringUtf8Coder.of());
+
+    @Override
+    public void encode(PubsubCheckpoint<T> value, OutputStream outStream, Context context)
+        throws IOException {
+      SUBSCRIPTION_PATH_CODER.encode(
+          value.subscriptionPath,
+          outStream,
+          context.nested());
+      LIST_CODER.encode(value.notYetReadIds, outStream, context);
+    }
+
+    @Override
+    public PubsubCheckpoint<T> decode(InputStream inStream, Context context) throws IOException {
+      String path = SUBSCRIPTION_PATH_CODER.decode(inStream, context.nested());
+      List<String> notYetReadIds = LIST_CODER.decode(inStream, context);
+      return new PubsubCheckpoint<>(path, null, null, notYetReadIds);
+    }
+  }
+
+  // ================================================================================
+  // Reader
+  // ================================================================================
+
+  /**
+   * A reader which keeps track of which messages have been received from Pubsub
+   * but not yet consumed downstream and/or ACKed back to Pubsub.
+   */
+  @VisibleForTesting
+  static class PubsubReader<T> extends UnboundedSource.UnboundedReader<T> {
+    /**
+     * For access to topic and checkpointCoder.
+     */
+    private final PubsubSource<T> outer;
+    @VisibleForTesting
+    final SubscriptionPath subscription;
+
+    private final SimpleFunction<PubsubIO.PubsubMessage, T> parseFn;
+
+    /**
+     * Client on which to talk to Pubsub. Contains a null value if the client has been closed.
+     */
+    private AtomicReference<PubsubClient> pubsubClient;
+
+    /**
+     * The closed state of this {@link PubsubReader}. If true, the reader has not yet been closed,
+     * and it will have a non-null value within {@link #pubsubClient}.
+     */
+    private AtomicBoolean active = new AtomicBoolean(true);
+
+    /**
+     * Ack timeout, in ms, as set on subscription when we first start reading. Not
+     * updated thereafter. -1 if not yet determined.
+     */
+    private int ackTimeoutMs;
+
+    /**
+     * ACK ids of messages we have delivered downstream but not yet ACKed.
+     */
+    private Set<String> safeToAckIds;
+
+    /**
+     * Messages we have received from Pubsub and not yet delivered downstream.
+     * We preserve their order.
+     */
+    private final Queue<PubsubClient.IncomingMessage> notYetRead;
+
+    private static class InFlightState {
+      /**
+       * When request which yielded message was issues.
+       */
+      long requestTimeMsSinceEpoch;
+
+      /**
+       * When Pubsub will consider this message's ACK to timeout and thus it needs to be
+       * extended.
+       */
+      long ackDeadlineMsSinceEpoch;
+
+      public InFlightState(long requestTimeMsSinceEpoch, long ackDeadlineMsSinceEpoch) {
+        this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch;
+        this.ackDeadlineMsSinceEpoch = ackDeadlineMsSinceEpoch;
+      }
+    }
+
+    /**
+     * Map from ACK ids of messages we have received from Pubsub but not yet ACKed to their
+     * in flight state. Ordered from earliest to latest ACK deadline.
+     */
+    private final LinkedHashMap<String, InFlightState> inFlight;
+
+    /**
+     * Batches of successfully ACKed ids which need to be pruned from the above.
+     * CAUTION: Accessed by both reader and checkpointing threads.
+     */
+    private final Queue<List<String>> ackedIds;
+
+    /**
+     * Byte size of undecoded elements in {@link #notYetRead}.
+     */
+    private long notYetReadBytes;
+
+    /**
+     * Bucketed map from received time (as system time, ms since epoch) to message
+     * timestamps (mssince epoch) of all received but not-yet read messages.
+     * Used to estimate watermark.
+     */
+    private BucketingFunction minUnreadTimestampMsSinceEpoch;
+
+    /**
+     * Minimum of timestamps (ms since epoch) of all recently read messages.
+     * Used to estimate watermark.
+     */
+    private MovingFunction minReadTimestampMsSinceEpoch;
+
+    /**
+     * System time (ms since epoch) we last received a message from Pubsub, or -1 if
+     * not yet received any messages.
+     */
+    private long lastReceivedMsSinceEpoch;
+
+    /**
+     * The last reported watermark (ms since epoch), or beginning of time if none yet reported.
+     */
+    private long lastWatermarkMsSinceEpoch;
+
+    /**
+     * The current message, or {@literal null} if none.
+     */
+    @Nullable
+    private PubsubClient.IncomingMessage current;
+
+    /**
+     * Stats only: System time (ms since epoch) we last logs stats, or -1 if never.
+     */
+    private long lastLogTimestampMsSinceEpoch;
+
+    /**
+     * Stats only: Total number of messages received.
+     */
+    private long numReceived;
+
+    /**
+     * Stats only: Number of messages which have recently been received.
+     */
+    private MovingFunction numReceivedRecently;
+
+    /**
+     * Stats only: Number of messages which have recently had their deadline extended.
+     */
+    private MovingFunction numExtendedDeadlines;
+
+    /**
+     * Stats only: Number of messages which have recenttly had their deadline extended even
+     * though it may be too late to do so.
+     */
+    private MovingFunction numLateDeadlines;
+
+
+    /**
+     * Stats only: Number of messages which have recently been ACKed.
+     */
+    private MovingFunction numAcked;
+
+    /**
+     * Stats only: Number of messages which have recently expired (ACKs were extended for too
+     * long).
+     */
+    private MovingFunction numExpired;
+
+    /**
+     * Stats only: Number of messages which have recently been NACKed.
+     */
+    private MovingFunction numNacked;
+
+    /**
+     * Stats only: Number of message bytes which have recently been read by downstream consumer.
+     */
+    private MovingFunction numReadBytes;
+
+    /**
+     * Stats only: Minimum of timestamp (ms since epoch) of all recently received messages.
+     * Used to estimate timestamp skew. Does not contribute to watermark estimator.
+     */
+    private MovingFunction minReceivedTimestampMsSinceEpoch;
+
+    /**
+     * Stats only: Maximum of timestamp (ms since epoch) of all recently received messages.
+     * Used to estimate timestamp skew.
+     */
+    private MovingFunction maxReceivedTimestampMsSinceEpoch;
+
+    /**
+     * Stats only: Minimum of recent estimated watermarks (ms since epoch).
+     */
+    private MovingFunction minWatermarkMsSinceEpoch;
+
+    /**
+     * Stats ony: Maximum of recent estimated watermarks (ms since epoch).
+     */
+    private MovingFunction maxWatermarkMsSinceEpoch;
+
+    /**
+     * Stats only: Number of messages with timestamps strictly behind the estimated watermark
+     * at the time they are received. These may be considered 'late' by downstream computations.
+     */
+    private MovingFunction numLateMessages;
+
+    /**
+     * Stats only: Current number of checkpoints in flight.
+     * CAUTION: Accessed by both checkpointing and reader threads.
+     */
+    private AtomicInteger numInFlightCheckpoints;
+
+    /**
+     * Stats only: Maximum number of checkpoints in flight at any time.
+     */
+    private int maxInFlightCheckpoints;
+
+    private static MovingFunction newFun(Combine.BinaryCombineLongFn function) {
+      return new MovingFunction(SAMPLE_PERIOD.getMillis(),
+                                SAMPLE_UPDATE.getMillis(),
+                                MIN_WATERMARK_SPREAD,
+                                MIN_WATERMARK_MESSAGES,
+                                function);
+    }
+
+    /**
+     * Construct a reader.
+     */
+    public PubsubReader(PubsubOptions options, PubsubSource<T> outer, SubscriptionPath subscription,
+                        SimpleFunction<PubsubIO.PubsubMessage, T> parseFn)
+        throws IOException, GeneralSecurityException {
+      this.outer = outer;
+      this.subscription = subscription;
+      this.parseFn = parseFn;
+      pubsubClient =
+          new AtomicReference<>(
+              outer.outer.pubsubFactory.newClient(
+                  outer.outer.timestampLabel, outer.outer.idLabel, options));
+      ackTimeoutMs = -1;
+      safeToAckIds = new HashSet<>();
+      notYetRead = new ArrayDeque<>();
+      inFlight = new LinkedHashMap<>();
+      ackedIds = new ConcurrentLinkedQueue<>();
+      notYetReadBytes = 0;
+      minUnreadTimestampMsSinceEpoch = new BucketingFunction(SAMPLE_UPDATE.getMillis(),
+                                                             MIN_WATERMARK_SPREAD,
+                                                             MIN_WATERMARK_MESSAGES,
+                                                             MIN);
+      minReadTimestampMsSinceEpoch = newFun(MIN);
+      lastReceivedMsSinceEpoch = -1;
+      lastWatermarkMsSinceEpoch = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
+      current = null;
+      lastLogTimestampMsSinceEpoch = -1;
+      numReceived = 0L;
+      numReceivedRecently = newFun(SUM);
+      numExtendedDeadlines = newFun(SUM);
+      numLateDeadlines = newFun(SUM);
+      numAcked = newFun(SUM);
+      numExpired = newFun(SUM);
+      numNacked = newFun(SUM);
+      numReadBytes = newFun(SUM);
+      minReceivedTimestampMsSinceEpoch = newFun(MIN);
+      maxReceivedTimestampMsSinceEpoch = newFun(MAX);
+      minWatermarkMsSinceEpoch = newFun(MIN);
+      maxWatermarkMsSinceEpoch = newFun(MAX);
+      numLateMessages = newFun(SUM);
+      numInFlightCheckpoints = new AtomicInteger();
+      maxInFlightCheckpoints = 0;
+    }
+
+    @VisibleForTesting
+    PubsubClient getPubsubClient() {
+      return pubsubClient.get();
+    }
+
+    /**
+     * Acks the provided {@code ackIds} back to Pubsub, blocking until all of the messages are
+     * ACKed.
+     *
+     * <p>CAUTION: May be invoked from a separate thread.
+     *
+     * <p>CAUTION: Retains {@code ackIds}.
+     */
+    void ackBatch(List<String> ackIds) throws IOException {
+      pubsubClient.get().acknowledge(subscription, ackIds);
+      ackedIds.add(ackIds);
+    }
+
+    /**
+     * BLOCKING
+     * NACK (ie request deadline extension of 0) receipt of messages from Pubsub
+     * with the given {@code ockIds}. Does not retain {@code ackIds}.
+     */
+    public void nackBatch(long nowMsSinceEpoch, List<String> ackIds) throws IOException {
+      pubsubClient.get().modifyAckDeadline(subscription, ackIds, 0);
+      numNacked.add(nowMsSinceEpoch, ackIds.size());
+    }
+
+    /**
+     * BLOCKING
+     * Extend the processing deadline for messages from Pubsub with the given {@code ackIds}.
+     * Does not retain {@code ackIds}.
+     */
+    private void extendBatch(long nowMsSinceEpoch, List<String> ackIds) throws IOException {
+      int extensionSec = (ackTimeoutMs * ACK_EXTENSION_PCT) / (100 * 1000);
+      pubsubClient.get().modifyAckDeadline(subscription, ackIds, extensionSec);
+      numExtendedDeadlines.add(nowMsSinceEpoch, ackIds.size());
+    }
+
+    /**
+     * Return the current time, in ms since epoch.
+     */
+    private long now() {
+      if (outer.outer.clock == null) {
+        return System.currentTimeMillis();
+      } else {
+        return outer.outer.clock.currentTimeMillis();
+      }
+    }
+
+    /**
+     * Messages which have been ACKed (via the checkpoint finalize) are no longer in flight.
+     * This is only used for flow control and stats.
+     */
+    private void retire() throws IOException {
+      long nowMsSinceEpoch = now();
+      while (true) {
+        List<String> ackIds = ackedIds.poll();
+        if (ackIds == null) {
+          return;
+        }
+        numAcked.add(nowMsSinceEpoch, ackIds.size());
+        for (String ackId : ackIds) {
+          inFlight.remove(ackId);
+          safeToAckIds.remove(ackId);
+        }
+      }
+    }
+
+    /**
+     * BLOCKING
+     * Extend deadline for all messages which need it.
+     * CAUTION: If extensions can't keep up with wallclock then we'll never return.
+     */
+    private void extend() throws IOException {
+      while (true) {
+        long nowMsSinceEpoch = now();
+        List<String> assumeExpired = new ArrayList<>();
+        List<String> toBeExtended = new ArrayList<>();
+        List<String> toBeExpired = new ArrayList<>();
+        // Messages will be in increasing deadline order.
+        for (Map.Entry<String, InFlightState> entry : inFlight.entrySet()) {
+          if (entry.getValue().ackDeadlineMsSinceEpoch - (ackTimeoutMs * ACK_SAFETY_PCT) / 100
+              > nowMsSinceEpoch) {
+            // All remaining messages don't need their ACKs to be extended.
+            break;
+          }
+
+          if (entry.getValue().ackDeadlineMsSinceEpoch - ACK_TOO_LATE.getMillis()
+              < nowMsSinceEpoch) {
+            // Pubsub may have already considered this message to have expired.
+            // If so it will (eventually) be made available on a future pull request.
+            // If this message ends up being committed then it will be considered a duplicate
+            // when re-pulled.
+            assumeExpired.add(entry.getKey());
+            continue;
+          }
+
+          if (entry.getValue().requestTimeMsSinceEpoch + PROCESSING_TIMEOUT.getMillis()
+              < nowMsSinceEpoch) {
+            // This message has been in-flight for too long.
+            // Give up on it, otherwise we risk extending its ACK indefinitely.
+            toBeExpired.add(entry.getKey());
+            continue;
+          }
+
+          // Extend the ACK for this message.
+          toBeExtended.add(entry.getKey());
+          if (toBeExtended.size() >= ACK_BATCH_SIZE) {
+            // Enough for one batch.
+            break;
+          }
+        }
+
+        if (assumeExpired.isEmpty() && toBeExtended.isEmpty() && toBeExpired.isEmpty()) {
+          // Nothing to be done.
+          return;
+        }
+
+        if (!assumeExpired.isEmpty()) {
+          // If we didn't make the ACK deadline assume expired and no longer in flight.
+          numLateDeadlines.add(nowMsSinceEpoch, assumeExpired.size());
+          for (String ackId : assumeExpired) {
+            inFlight.remove(ackId);
+          }
+        }
+
+        if (!toBeExpired.isEmpty()) {
+          // Expired messages are no longer considered in flight.
+          numExpired.add(nowMsSinceEpoch, toBeExpired.size());
+          for (String ackId : toBeExpired) {
+            inFlight.remove(ackId);
+          }
+        }
+
+        if (!toBeExtended.isEmpty()) {
+          // Pubsub extends acks from it's notion of current time.
+          // We'll try to track that on our side, but note the deadlines won't necessarily agree.
+          long newDeadlineMsSinceEpoch = nowMsSinceEpoch + (ackTimeoutMs * ACK_EXTENSION_PCT) / 100;
+          for (String ackId : toBeExtended) {
+            // Maintain increasing ack deadline order.
+            InFlightState state = inFlight.remove(ackId);
+            inFlight.put(ackId,
+                         new InFlightState(state.requestTimeMsSinceEpoch, newDeadlineMsSinceEpoch));
+          }
+          // BLOCKs until extended.
+          extendBatch(nowMsSinceEpoch, toBeExtended);
+        }
+      }
+    }
+
+    /**
+     * BLOCKING
+     * Fetch another batch of messages from Pubsub.
+     */
+    private void pull() throws IOException {
+      if (inFlight.size() >= MAX_IN_FLIGHT) {
+        // Wait for checkpoint to be finalized before pulling anymore.
+        // There may be lag while checkpoints are persisted and the finalizeCheckpoint method
+        // is invoked. By limiting the in-flight messages we can ensure we don't end up consuming
+        // messages faster than we can checkpoint them.
+        return;
+      }
+
+      long requestTimeMsSinceEpoch = now();
+      long deadlineMsSinceEpoch = requestTimeMsSinceEpoch + ackTimeoutMs;
+
+      // Pull the next batch.
+      // BLOCKs until received.
+      Collection<PubsubClient.IncomingMessage> receivedMessages =
+          pubsubClient.get().pull(requestTimeMsSinceEpoch, subscription, PULL_BATCH_SIZE, true);
+      if (receivedMessages.isEmpty()) {
+        // Nothing available yet. Try again later.
+        return;
+      }
+
+      lastReceivedMsSinceEpoch = requestTimeMsSinceEpoch;
+
+      // Capture the received messages.
+      for (PubsubClient.IncomingMessage incomingMessage : receivedMessages) {
+        notYetRead.add(incomingMessage);
+        notYetReadBytes += incomingMessage.elementBytes.length;
+        inFlight.put(incomingMessage.ackId,
+                     new InFlightState(requestTimeMsSinceEpoch, deadlineMsSinceEpoch));
+        numReceived++;
+        numReceivedRecently.add(requestTimeMsSinceEpoch, 1L);
+        minReceivedTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch,
+                                             incomingMessage.timestampMsSinceEpoch);
+        maxReceivedTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch,
+                                             incomingMessage.timestampMsSinceEpoch);
+        minUnreadTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch,
+                                           incomingMessage.timestampMsSinceEpoch);
+      }
+    }
+
+    /**
+     * Log stats if time to do so.
+     */
+    private void stats() {
+      long nowMsSinceEpoch = now();
+      if (lastLogTimestampMsSinceEpoch < 0) {
+        lastLogTimestampMsSinceEpoch = nowMsSinceEpoch;
+        return;
+      }
+      long deltaMs = nowMsSinceEpoch - lastLogTimestampMsSinceEpoch;
+      if (deltaMs < LOG_PERIOD.getMillis()) {
+        return;
+      }
+
+      String messageSkew = "unknown";
+      long minTimestamp = minReceivedTimestampMsSinceEpoch.get(nowMsSinceEpoch);
+      long maxTimestamp = maxReceivedTimestampMsSinceEpoch.get(nowMsSinceEpoch);
+      if (minTimestamp < Long.MAX_VALUE && maxTimestamp > Long.MIN_VALUE) {
+        messageSkew = (maxTimestamp - minTimestamp) + "ms";
+      }
+
+      String watermarkSkew = "unknown";
+      long minWatermark = minWatermarkMsSinceEpoch.get(nowMsSinceEpoch);
+      long maxWatermark = maxWatermarkMsSinceEpoch.get(nowMsSinceEpoch);
+      if (minWatermark < Long.MAX_VALUE && maxWatermark > Long.MIN_VALUE) {
+        watermarkSkew = (maxWatermark - minWatermark) + "ms";
+      }
+
+      String oldestInFlight = "no";
+      String oldestAckId = Iterables.getFirst(inFlight.keySet(), null);
+      if (oldestAckId != null) {
+        oldestInFlight =
+            (nowMsSinceEpoch - inFlight.get(oldestAckId).requestTimeMsSinceEpoch) + "ms";
+      }
+
+      LOG.info("Pubsub {} has "
+               + "{} received messages, "
+               + "{} current unread messages, "
+               + "{} current unread bytes, "
+               + "{} current in-flight msgs, "
+               + "{} oldest in-flight, "
+               + "{} current in-flight checkpoints, "
+               + "{} max in-flight checkpoints, "
+               + "{}B/s recent read, "
+               + "{} recent received, "
+               + "{} recent extended, "
+               + "{} recent late extended, "
+               + "{} recent ACKed, "
+               + "{} recent NACKed, "
+               + "{} recent expired, "
+               + "{} recent message timestamp skew, "
+               + "{} recent watermark skew, "
+               + "{} recent late messages, "
+               + "{} last reported watermark",
+               subscription,
+               numReceived,
+               notYetRead.size(),
+               notYetReadBytes,
+               inFlight.size(),
+               oldestInFlight,
+               numInFlightCheckpoints.get(),
+               maxInFlightCheckpoints,
+               numReadBytes.get(nowMsSinceEpoch) / (SAMPLE_PERIOD.getMillis() / 1000L),
+               numReceivedRecently.get(nowMsSinceEpoch),
+               numExtendedDeadlines.get(nowMsSinceEpoch),
+               numLateDeadlines.get(nowMsSinceEpoch),
+               numAcked.get(nowMsSinceEpoch),
+               numNacked.get(nowMsSinceEpoch),
+               numExpired.get(nowMsSinceEpoch),
+               messageSkew,
+               watermarkSkew,
+               numLateMessages.get(nowMsSinceEpoch),
+               new Instant(lastWatermarkMsSinceEpoch));
+
+      lastLogTimestampMsSinceEpoch = nowMsSinceEpoch;
+    }
+
+    @Override
+    public boolean start() throws IOException {
+      // Determine the ack timeout.
+      ackTimeoutMs = pubsubClient.get().ackDeadlineSeconds(subscription) * 1000;
+      return advance();
+    }
+
+    /**
+     * BLOCKING
+     * Return {@literal true} if a Pubsub messaage is available, {@literal false} if
+     * none is available at this time or we are over-subscribed. May BLOCK while extending
+     * ACKs or fetching available messages. Will not block waiting for messages.
+     */
+    @Override
+    public boolean advance() throws IOException {
+      // Emit stats.
+      stats();
+
+      if (current != null) {
+        // Current is consumed. It can no longer contribute to holding back the watermark.
+        minUnreadTimestampMsSinceEpoch.remove(current.requestTimeMsSinceEpoch);
+        current = null;
+      }
+
+      // Retire state associated with ACKed messages.
+      retire();
+
+      // Extend all pressing deadlines.
+      // Will BLOCK until done.
+      // If the system is pulling messages only to let them sit in a downsteam queue then
+      // this will have the effect of slowing down the pull rate.
+      // However, if the system is genuinely taking longer to process each message then
+      // the work to extend ACKs would be better done in the background.
+      extend();
+
+      if (notYetRead.isEmpty()) {
+        // Pull another batch.
+        // Will BLOCK until fetch returns, but will not block until a message is available.
+        pull();
+      }
+
+      // Take one message from queue.
+      current = notYetRead.poll();
+      if (current == null) {
+        // Try again later.
+        return false;
+      }
+      notYetReadBytes -= current.elementBytes.length;
+      checkState(notYetReadBytes >= 0);
+      long nowMsSinceEpoch = now();
+      numReadBytes.add(nowMsSinceEpoch, current.elementBytes.length);
+      minReadTimestampMsSinceEpoch.add(nowMsSinceEpoch, current.timestampMsSinceEpoch);
+      if (current.timestampMsSinceEpoch < lastWatermarkMsSinceEpoch) {
+        numLateMessages.add(nowMsSinceEpoch, 1L);
+      }
+
+      // Current message can be considered 'read' and will be persisted by the next
+      // checkpoint. So it is now safe to ACK back to Pubsub.
+      safeToAckIds.add(current.ackId);
+      return true;
+    }
+
+    @Override
+    public T getCurrent() throws NoSuchElementException {
+      if (current == null) {
+        throw new NoSuchElementException();
+      }
+      try {
+        if (parseFn != null) {
+          return parseFn.apply(new PubsubIO.PubsubMessage(
+                  current.elementBytes, current.attributes));
+        } else {
+          return CoderUtils.decodeFromByteArray(outer.outer.elementCoder, current.elementBytes);
+        }
+      } catch (CoderException e) {
+        throw new RuntimeException("Unable to decode element from Pubsub message: ", e);
+      }
+    }
+
+    @Override
+    public Instant getCurrentTimestamp() throws NoSuchElementException {
+      if (current == null) {
+        throw new NoSuchElementException();
+      }
+      return new Instant(current.timestampMsSinceEpoch);
+    }
+
+    @Override
+    public byte[] getCurrentRecordId() throws NoSuchElementException {
+      if (current == null) {
+        throw new NoSuchElementException();
+      }
+      return current.recordId.getBytes(Charsets.UTF_8);
+    }
+
+    /**
+     * {@inheritDoc}.
+     *
+     * <p>Marks this {@link PubsubReader} as no longer active. The {@link PubsubClient}
+     * continue to exist and be active beyond the life of this call if there are any in-flight
+     * checkpoints. When no in-flight checkpoints remain, the reader will be closed.
+     */
+    @Override
+    public void close() throws IOException {
+      active.set(false);
+      maybeCloseClient();
+    }
+
+    /**
+     * Close this reader's underlying {@link PubsubClient} if the reader has been closed and there
+     * are no outstanding checkpoints.
+     */
+    private void maybeCloseClient() throws IOException {
+      if (!active.get() && numInFlightCheckpoints.get() == 0) {
+        // The reader has been closed and it has no more outstanding checkpoints. The client
+        // must be closed so it doesn't leak
+        PubsubClient client = pubsubClient.getAndSet(null);
+        if (client != null) {
+          client.close();
+        }
+      }
+    }
+
+    @Override
+    public PubsubSource<T> getCurrentSource() {
+      return outer;
+    }
+
+    @Override
+    public Instant getWatermark() {
+      if (pubsubClient.get().isEOF() && notYetRead.isEmpty()) {
+        // For testing only: Advance the watermark to the end of time to signal
+        // the test is complete.
+        return BoundedWindow.TIMESTAMP_MAX_VALUE;
+      }
+
+      // NOTE: We'll allow the watermark to go backwards. The underlying runner is responsible
+      // for aggregating all reported watermarks and ensuring the aggregate is latched.
+      // If we attempt to latch locally then it is possible a temporary starvation of one reader
+      // could cause its estimated watermark to fast forward to current system time. Then when
+      // the reader resumes its watermark would be unable to resume tracking.
+      // By letting the underlying runner latch we avoid any problems due to localized starvation.
+      long nowMsSinceEpoch = now();
+      long readMin = minReadTimestampMsSinceEpoch.get(nowMsSinceEpoch);
+      long unreadMin = minUnreadTimestampMsSinceEpoch.get();
+      if (readMin == Long.MAX_VALUE
+          && unreadMin == Long.MAX_VALUE
+          && lastReceivedMsSinceEpoch >= 0
+          && nowMsSinceEpoch > lastReceivedMsSinceEpoch + SAMPLE_PERIOD.getMillis()) {
+        // We don't currently have any unread messages pending, we have not had any messages
+        // read for a while, and we have not received any new messages from Pubsub for a while.
+        // Advance watermark to current time.
+        // TODO: Estimate a timestamp lag.
+        lastWatermarkMsSinceEpoch = nowMsSinceEpoch;
+      } else if (minReadTimestampMsSinceEpoch.isSignificant()
+                 || minUnreadTimestampMsSinceEpoch.isSignificant()) {
+        // Take minimum of the timestamps in all unread messages and recently read messages.
+        lastWatermarkMsSinceEpoch = Math.min(readMin, unreadMin);
+      }
+      // else: We're not confident enough to estimate a new watermark. Stick with the old one.
+      minWatermarkMsSinceEpoch.add(nowMsSinceEpoch, lastWatermarkMsSinceEpoch);
+      maxWatermarkMsSinceEpoch.add(nowMsSinceEpoch, lastWatermarkMsSinceEpoch);
+      return new Instant(lastWatermarkMsSinceEpoch);
+    }
+
+    @Override
+    public PubsubCheckpoint<T> getCheckpointMark() {
+      int cur = numInFlightCheckpoints.incrementAndGet();
+      maxInFlightCheckpoints = Math.max(maxInFlightCheckpoints, cur);
+      // It's possible for a checkpoint to be taken but never finalized.
+      // So we simply copy whatever safeToAckIds we currently have.
+      // It's possible a later checkpoint will be taken before an earlier one is finalized,
+      // in which case we'll double ACK messages to Pubsub. However Pubsub is fine with that.
+      List<String> snapshotSafeToAckIds = Lists.newArrayList(safeToAckIds);
+      List<String> snapshotNotYetReadIds = new ArrayList<>(notYetRead.size());
+      for (PubsubClient.IncomingMessage incomingMessage : notYetRead) {
+        snapshotNotYetReadIds.add(incomingMessage.ackId);
+      }
+      if (outer.subscriptionPath == null) {
+        // need to include the subscription in case we resume, as it's not stored in the source.
+        return new PubsubCheckpoint<>(
+            subscription.getPath(), this, snapshotSafeToAckIds, snapshotNotYetReadIds);
+      }
+      return new PubsubCheckpoint<>(null, this, snapshotSafeToAckIds, snapshotNotYetReadIds);
+    }
+
+    @Override
+    public long getSplitBacklogBytes() {
+      return notYetReadBytes;
+    }
+  }
+
+  // ================================================================================
+  // Source
+  // ================================================================================
+
+  @VisibleForTesting
+  static class PubsubSource<T> extends UnboundedSource<T, PubsubCheckpoint<T>> {
+    public final PubsubUnboundedSource<T> outer;
+    // The subscription to read from.
+    @VisibleForTesting
+    final SubscriptionPath subscriptionPath;
+
+    public PubsubSource(PubsubUnboundedSource<T> outer) {
+      this(outer, outer.getSubscription());
+    }
+
+    private PubsubSource(PubsubUnboundedSource<T> outer, SubscriptionPath subscriptionPath) {
+      this.outer = outer;
+      this.subscriptionPath = subscriptionPath;
+    }
+
+    @Override
+    public List<PubsubSource<T>> generateInitialSplits(
+        int desiredNumSplits, PipelineOptions options) throws Exception {
+      List<PubsubSource<T>> result = new ArrayList<>(desiredNumSplits);
+      PubsubSource<T> splitSource = this;
+      if (subscriptionPath == null) {
+        splitSource = new PubsubSource<>(outer, outer.createRandomSubscription(options));
+      }
+      for (int i = 0; i < desiredNumSplits * SCALE_OUT; i++) {
+        // Since the source is immutable and Pubsub automatically shards we simply
+        // replicate ourselves the requested number of times
+        result.add(splitSource);
+      }
+      return result;
+    }
+
+    @Override
+    public PubsubReader<T> createReader(
+        PipelineOptions options,
+        @Nullable PubsubCheckpoint<T> checkpoint) {
+      PubsubReader<T> reader;
+      SubscriptionPath subscription = subscriptionPath;
+      if (subscription == null) {
+        if (checkpoint == null) {
+          // This reader has never been started and there was no call to #splitIntoBundles; create
+          // a single random subscription, which will be kept in the checkpoint.
+          subscription = outer.createRandomSubscription(options);
+        } else {
+          subscription = checkpoint.getSubscription();
+        }
+      }
+      try {
+        reader = new PubsubReader<>(options.as(PubsubOptions.class), this, subscription,
+                outer.parseFn);
+      } catch (GeneralSecurityException | IOException e) {
+        throw new RuntimeException("Unable to subscribe to " + subscriptionPath + ": ", e);
+      }
+      if (checkpoint != null) {
+        // NACK all messages we may have lost.
+        try {
+          // Will BLOCK until NACKed.
+          checkpoint.nackAll(reader);
+        } catch (IOException e) {
+          LOG.error("Pubsub {} cannot have {} lost messages NACKed, ignoring: {}",
+                    subscriptionPath, checkpoint.notYetReadIds.size(), e);
+        }
+      }
+      return reader;
+    }
+
+    @Nullable
+    @Override
+    public Coder<PubsubCheckpoint<T>> getCheckpointMarkCoder() {
+      @SuppressWarnings("unchecked") PubsubCheckpointCoder<T> typedCoder =
+          (PubsubCheckpointCoder<T>) CHECKPOINT_CODER;
+      return typedCoder;
+    }
+
+    @Override
+    public Coder<T> getDefaultOutputCoder() {
+      return outer.elementCoder;
+    }
+
+    @Override
+    public void validate() {
+      // Nothing to validate.
+    }
+
+    @Override
+    public boolean requiresDeduping() {
+      // We cannot prevent re-offering already read messages after a restore from checkpoint.
+      return true;
+    }
+  }
+
+  // ================================================================================
+  // StatsFn
+  // ================================================================================
+
+  private static class StatsFn<T> extends DoFn<T, T> {
+    private final Counter elementCounter = Metrics.counter(StatsFn.class, "elements");
+
+    private final PubsubClientFactory pubsubFactory;
+    @Nullable
+    private final ValueProvider<SubscriptionPath> subscription;
+    @Nullable
+    private final ValueProvider<TopicPath> topic;
+    @Nullable
+    private final String timestampLabel;
+    @Nullable
+    private final String idLabel;
+
+    public StatsFn(
+        PubsubClientFactory pubsubFactory,
+        @Nullable ValueProvider<SubscriptionPath> subscription,
+        @Nullable ValueProvider<TopicPath> topic,
+        @Nullable String timestampLabel,
+        @Nullable String idLabel) {
+      checkArgument(pubsubFactory != null, "pubsubFactory should not be null");
+      this.pubsubFactory = pubsubFactory;
+      this.subscription = subscription;
+      this.topic = topic;
+      this.timestampLabel = timestampLabel;
+      this.idLabel = idLabel;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      elementCounter.inc();
+      c.output(c.element());
+    }
+
+    @Override
+    public void populateDisplayData(Builder builder) {
+      super.populateDisplayData(builder);
+      if (subscription != null) {
+        String subscriptionString = subscription.isAccessible()
+            ? subscription.get().getPath()
+            : subscription.toString();
+        builder.add(DisplayData.item("subscription", subscriptionString));
+      }
+      if (topic != null) {
+        String topicString = topic.isAccessible()
+            ? topic.get().getPath()
+            : topic.toString();
+        builder.add(DisplayData.item("topic", topicString));
+      }
+      builder.add(DisplayData.item("transport", pubsubFactory.getKind()));
+      builder.addIfNotNull(DisplayData.item("timestampLabel", timestampLabel));
+      builder.addIfNotNull(DisplayData.item("idLabel", idLabel));
+    }
+  }
+
+  // ================================================================================
+  // PubsubUnboundedSource
+  // ================================================================================
+
+  /**
+   * For testing only: Clock to use for all timekeeping. If {@literal null} use system clock.
+   */
+  @Nullable
+  private Clock clock;
+
+  /**
+   * Factory for creating underlying Pubsub transport.
+   */
+  private final PubsubClientFactory pubsubFactory;
+
+  /**
+   * Project under which to create a subscription if only the {@link #topic} was given.
+   */
+  @Nullable
+  private final ValueProvider<ProjectPath> project;
+
+  /**
+   * Topic to read from. If {@literal null}, then {@link #subscription} must be given.
+   * Otherwise {@link #subscription} must be null.
+   */
+  @Nullable
+  private final ValueProvider<TopicPath> topic;
+
+  /**
+   * Subscription to read from. If {@literal null} then {@link #topic} must be given.
+   * Otherwise {@link #topic} must be null.
+   *
+   * <p>If no subscription is given a random one will be created when the transorm is
+   * applied. This field will be update with that subscription's path. The created
+   * subscription is never deleted.
+   */
+  @Nullable
+  private ValueProvider<SubscriptionPath> subscription;
+
+  /**
+   * Coder for elements. Elements are effectively double-encoded: first to a byte array
+   * using this checkpointCoder, then to a base-64 string to conform to Pubsub's payload
+   * conventions.
+   */
+  private final Coder<T> elementCoder;
+
+  /**
+   * Pubsub metadata field holding timestamp of each element, or {@literal null} if should use
+   * Pubsub message publish timestamp instead.
+   */
+  @Nullable
+  private final String timestampLabel;
+
+  /**
+   * Pubsub metadata field holding id for each element, or {@literal null} if need to generate
+   * a unique id ourselves.
+   */
+  @Nullable
+  private final String idLabel;
+
+  /**
+   * If not {@literal null}, the user is asking for PubSub attributes. This parse function will be
+   * used to parse {@link PubsubIO.PubsubMessage}s containing a payload and attributes.
+   */
+  @Nullable
+  SimpleFunction<PubsubIO.PubsubMessage, T> parseFn;
+
+  @VisibleForTesting
+  PubsubUnboundedSource(
+      Clock clock,
+      PubsubClientFactory pubsubFactory,
+      @Nullable ValueProvider<ProjectPath> project,
+      @Nullable ValueProvider<TopicPath> topic,
+      @Nullable ValueProvider<SubscriptionPath> subscription,
+      Coder<T> elementCoder,
+      @Nullable String timestampLabel,
+      @Nullable String idLabel,
+      @Nullable SimpleFunction<PubsubIO.PubsubMessage, T> parseFn) {
+    checkArgument((topic == null) != (subscription == null),
+                  "Exactly one of topic and subscription must be given");
+    checkArgument((topic == null) == (project == null),
+                  "Project must be given if topic is given");
+    this.clock = clock;
+    this.pubsubFactory = checkNotNull(pubsubFactory);
+    this.project = project;
+    this.topic = topic;
+    this.subscription = subscription;
+    this.elementCoder = checkNotNull(elementCoder);
+    this.timestampLabel = timestampLabel;
+    this.idLabel = idLabel;
+    this.parseFn = parseFn;
+  }
+
+  /**
+   * Construct an unbounded source to consume from the Pubsub {@code subscription}.
+   */
+  public PubsubUnboundedSource(
+      PubsubClientFactory pubsubFactory,
+      @Nullable ValueProvider<ProjectPath> project,
+      @Nullable ValueProvider<TopicPath> topic,
+      @Nullable ValueProvider<SubscriptionPath> subscription,
+      Coder<T> elementCoder,
+      @Nullable String timestampLabel,
+      @Nullable String idLabel,
+      @Nullable SimpleFunction<PubsubIO.PubsubMessage, T> parseFn) {
+    this(null, pubsubFactory, project, topic, subscription, elementCoder, timestampLabel, idLabel,
+        parseFn);
+  }
+
+  /**
+   * Get the coder used for elements.
+   */
+  public Coder<T> getElementCoder() {
+    return elementCoder;
+  }
+
+  /**
+   * Get the project path.
+   */
+  @Nullable
+  public ProjectPath getProject() {
+    return project == null ? null : project.get();
+  }
+
+  /**
+   * Get the topic being read from.
+   */
+  @Nullable
+  public TopicPath getTopic() {
+    return topic == null ? null : topic.get();
+  }
+
+  /**
+   * Get the {@link ValueProvider} for the topic being read from.
+   */
+  @Nullable
+  public ValueProvider<TopicPath> getTopicProvider() {
+    return topic;
+  }
+
+  /**
+   * Get the subscription being read from.
+   */
+  @Nullable
+  public SubscriptionPath getSubscription() {
+    return subscription == null ? null : subscription.get();
+  }
+
+  /**
+   * Get the {@link ValueProvider} for the subscription being read from.
+   */
+  @Nullable
+  public ValueProvider<SubscriptionPath> getSubscriptionProvider() {
+    return subscription;
+  }
+
+  /**
+   * Get the timestamp label.
+   */
+  @Nullable
+  public String getTimestampLabel() {
+    return timestampLabel;
+  }
+
+  /**
+   * Get the id label.
+   */
+  @Nullable
+  public String getIdLabel() {
+    return idLabel;
+  }
+
+  /**
+   * Get the parsing function for PubSub attributes.
+   */
+  @Nullable
+  public SimpleFunction<PubsubIO.PubsubMessage, T> getWithAttributesParseFn() {
+    return parseFn;
+  }
+
+  @Override
+  public PCollection<T> expand(PBegin input) {
+    return input.getPipeline().begin()
+                .apply(Read.from(new PubsubSource<T>(this)))
+                .apply("PubsubUnboundedSource.Stats",
+                    ParDo.of(new StatsFn<T>(
+                        pubsubFactory, subscription, topic, timestampLabel, idLabel)));
+  }
+
+  private SubscriptionPath createRandomSubscription(PipelineOptions options) {
+    try {
+      try (PubsubClient pubsubClient =
+          pubsubFactory.newClient(timestampLabel, idLabel, options.as(PubsubOptions.class))) {
+        checkState(project.isAccessible(), "createRandomSubscription must be called at runtime.");
+        checkState(topic.isAccessible(), "createRandomSubscription must be called at runtime.");
+        SubscriptionPath subscriptionPath =
+            pubsubClient.createRandomSubscription(
+                project.get(), topic.get(), DEAULT_ACK_TIMEOUT_SEC);
+        LOG.warn(
+            "Created subscription {} to topic {}."
+                + " Note this subscription WILL NOT be deleted when the pipeline terminates",
+            subscriptionPath,
+            topic);
+        return subscriptionPath;
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create subscription: ", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/package-info.java
new file mode 100644
index 0000000..55befba
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Defines transforms for reading and writing from
+ * <a href="https://cloud.google.com/pubsub/">Google Cloud Pub/Sub</a>.
+ * @see org.apache.beam.sdk.io.gcp.pubsub.PubsubIO
+ */
+package org.apache.beam.sdk.io.gcp.pubsub;

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java
index 0987140..f468ec0 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java
@@ -44,11 +44,14 @@ public class GcpApiSurfaceTest {
         ApiSurface.ofPackage(thisPackage, thisClassLoader)
             .pruningPattern("org[.]apache[.]beam[.].*Test.*")
             .pruningPattern("org[.]apache[.]beam[.].*IT")
-            .pruningPattern("java[.]lang.*");
+            .pruningPattern("java[.]lang.*")
+            .pruningPattern("java[.]util.*");
 
     @SuppressWarnings("unchecked")
     final Set<Matcher<Class<?>>> allowedClasses =
         ImmutableSet.of(
+            classesInPackage("com.google.api.client.googleapis"),
+            classesInPackage("com.google.api.client.http"),
             classesInPackage("com.google.api.client.json"),
             classesInPackage("com.google.api.client.util"),
             classesInPackage("com.google.api.services.bigquery.model"),

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java
new file mode 100644
index 0000000..14c36f9
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.ProjectPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for helper classes and methods in PubsubClient.
+ */
+@RunWith(JUnit4.class)
+public class PubsubClientTest {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  //
+  // Timestamp handling
+  //
+
+  private long parse(String timestamp) {
+    Map<String, String> map = ImmutableMap.of("myLabel", timestamp);
+    return PubsubClient.extractTimestamp("myLabel", null, map);
+  }
+
+  private void roundTripRfc339(String timestamp) {
+    assertEquals(Instant.parse(timestamp).getMillis(), parse(timestamp));
+  }
+
+  private void truncatedRfc339(String timestamp, String truncatedTimestmap) {
+    assertEquals(Instant.parse(truncatedTimestmap).getMillis(), parse(timestamp));
+  }
+
+  @Test
+  public void noTimestampLabelReturnsPubsubPublish() {
+    final long time = 987654321L;
+    long timestamp = PubsubClient.extractTimestamp(null, String.valueOf(time), null);
+    assertEquals(time, timestamp);
+  }
+
+  @Test
+  public void noTimestampLabelAndInvalidPubsubPublishThrowsError() {
+    thrown.expect(NumberFormatException.class);
+    PubsubClient.extractTimestamp(null, "not-a-date", null);
+  }
+
+  @Test
+  public void timestampLabelWithNullAttributesThrowsError() {
+    thrown.expect(RuntimeException.class);
+    thrown.expectMessage("PubSub message is missing a value for timestamp label myLabel");
+    PubsubClient.extractTimestamp("myLabel", null, null);
+  }
+
+  @Test
+  public void timestampLabelSetWithMissingAttributeThrowsError() {
+    thrown.expect(RuntimeException.class);
+    thrown.expectMessage("PubSub message is missing a value for timestamp label myLabel");
+    Map<String, String> map = ImmutableMap.of("otherLabel", "whatever");
+    PubsubClient.extractTimestamp("myLabel", null, map);
+  }
+
+  @Test
+  public void timestampLabelParsesMillisecondsSinceEpoch() {
+    long time = 1446162101123L;
+    Map<String, String> map = ImmutableMap.of("myLabel", String.valueOf(time));
+    long timestamp = PubsubClient.extractTimestamp("myLabel", null, map);
+    assertEquals(time, timestamp);
+  }
+
+  @Test
+  public void timestampLabelParsesRfc3339Seconds() {
+    roundTripRfc339("2015-10-29T23:41:41Z");
+  }
+
+  @Test
+  public void timestampLabelParsesRfc3339Tenths() {
+    roundTripRfc339("2015-10-29T23:41:41.1Z");
+  }
+
+  @Test
+  public void timestampLabelParsesRfc3339Hundredths() {
+    roundTripRfc339("2015-10-29T23:41:41.12Z");
+  }
+
+  @Test
+  public void timestampLabelParsesRfc3339Millis() {
+    roundTripRfc339("2015-10-29T23:41:41.123Z");
+  }
+
+  @Test
+  public void timestampLabelParsesRfc3339Micros() {
+    // Note: micros part 456/1000 is dropped.
+    truncatedRfc339("2015-10-29T23:41:41.123456Z", "2015-10-29T23:41:41.123Z");
+  }
+
+  @Test
+  public void timestampLabelParsesRfc3339MicrosRounding() {
+    // Note: micros part 999/1000 is dropped, not rounded up.
+    truncatedRfc339("2015-10-29T23:41:41.123999Z", "2015-10-29T23:41:41.123Z");
+  }
+
+  @Test
+  public void timestampLabelWithInvalidFormatThrowsError() {
+    thrown.expect(NumberFormatException.class);
+    parse("not-a-timestamp");
+  }
+
+  @Test
+  public void timestampLabelWithInvalidFormat2ThrowsError() {
+    thrown.expect(NumberFormatException.class);
+    parse("null");
+  }
+
+  @Test
+  public void timestampLabelWithInvalidFormat3ThrowsError() {
+    thrown.expect(NumberFormatException.class);
+    parse("2015-10");
+  }
+
+  @Test
+  public void timestampLabelParsesRfc3339WithSmallYear() {
+    // Google and JodaTime agree on dates after 1582-10-15, when the Gregorian Calendar was adopted
+    // This is therefore a "small year" until this difference is reconciled.
+    roundTripRfc339("1582-10-15T01:23:45.123Z");
+  }
+
+  @Test
+  public void timestampLabelParsesRfc3339WithLargeYear() {
+    // Year 9999 in range.
+    roundTripRfc339("9999-10-29T23:41:41.123999Z");
+  }
+
+  @Test
+  public void timestampLabelRfc3339WithTooLargeYearThrowsError() {
+    thrown.expect(NumberFormatException.class);
+    // Year 10000 out of range.
+    parse("10000-10-29T23:41:41.123999Z");
+  }
+
+  //
+  // Paths
+  //
+
+  @Test
+  public void projectPathFromIdWellFormed() {
+    ProjectPath path = PubsubClient.projectPathFromId("test");
+    assertEquals("projects/test", path.getPath());
+  }
+
+  @Test
+  public void subscriptionPathFromNameWellFormed() {
+    SubscriptionPath path = PubsubClient.subscriptionPathFromName("test", "something");
+    assertEquals("projects/test/subscriptions/something", path.getPath());
+    assertEquals("/subscriptions/test/something", path.getV1Beta1Path());
+  }
+
+  @Test
+  public void topicPathFromNameWellFormed() {
+    TopicPath path = PubsubClient.topicPathFromName("test", "something");
+    assertEquals("projects/test/topics/something", path.getPath());
+    assertEquals("/topics/test/something", path.getV1Beta1Path());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java
new file mode 100644
index 0000000..28e07e2
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.auth.Credentials;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Timestamp;
+import com.google.pubsub.v1.PublishRequest;
+import com.google.pubsub.v1.PublishResponse;
+import com.google.pubsub.v1.PublisherGrpc.PublisherImplBase;
+import com.google.pubsub.v1.PubsubMessage;
+import com.google.pubsub.v1.PullRequest;
+import com.google.pubsub.v1.PullResponse;
+import com.google.pubsub.v1.ReceivedMessage;
+import com.google.pubsub.v1.SubscriberGrpc.SubscriberImplBase;
+import io.grpc.ManagedChannel;
+import io.grpc.Server;
+import io.grpc.inprocess.InProcessChannelBuilder;
+import io.grpc.inprocess.InProcessServerBuilder;
+import io.grpc.stub.StreamObserver;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.IncomingMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
+import org.apache.beam.sdk.util.TestCredential;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for PubsubGrpcClient.
+ */
+@RunWith(JUnit4.class)
+public class PubsubGrpcClientTest {
+  private ManagedChannel inProcessChannel;
+  private Credentials testCredentials;
+
+  private PubsubClient client;
+  private String channelName;
+
+  private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic");
+  private static final SubscriptionPath SUBSCRIPTION =
+      PubsubClient.subscriptionPathFromName("testProject", "testSubscription");
+  private static final long REQ_TIME = 1234L;
+  private static final long PUB_TIME = 3456L;
+  private static final long MESSAGE_TIME = 6789L;
+  private static final String TIMESTAMP_LABEL = "timestamp";
+  private static final String ID_LABEL = "id";
+  private static final String MESSAGE_ID = "testMessageId";
+  private static final String DATA = "testData";
+  private static final String RECORD_ID = "testRecordId";
+  private static final String ACK_ID = "testAckId";
+  private static final Map<String, String> ATTRIBUTES =
+          ImmutableMap.<String, String>builder().put("a", "b").put("c", "d").build();
+
+  @Before
+  public void setup() {
+    channelName = String.format("%s-%s",
+        PubsubGrpcClientTest.class.getName(), ThreadLocalRandom.current().nextInt());
+    inProcessChannel = InProcessChannelBuilder.forName(channelName).directExecutor().build();
+    testCredentials = new TestCredential();
+    client = new PubsubGrpcClient(TIMESTAMP_LABEL, ID_LABEL, 10, inProcessChannel, testCredentials);
+  }
+
+  @After
+  public void teardown() throws IOException {
+    client.close();
+    inProcessChannel.shutdownNow();
+  }
+
+  @Test
+  public void pullOneMessage() throws IOException {
+    String expectedSubscription = SUBSCRIPTION.getPath();
+    final PullRequest expectedRequest =
+        PullRequest.newBuilder()
+                   .setSubscription(expectedSubscription)
+                   .setReturnImmediately(true)
+                   .setMaxMessages(10)
+                   .build();
+    Timestamp timestamp = Timestamp.newBuilder()
+                                   .setSeconds(PUB_TIME / 1000)
+                                   .setNanos((int) (PUB_TIME % 1000) * 1000)
+                                   .build();
+    PubsubMessage expectedPubsubMessage =
+        PubsubMessage.newBuilder()
+                     .setMessageId(MESSAGE_ID)
+                     .setData(
+                         ByteString.copyFrom(DATA.getBytes()))
+                     .setPublishTime(timestamp)
+                     .putAllAttributes(ATTRIBUTES)
+                     .putAllAttributes(
+                         ImmutableMap.of(TIMESTAMP_LABEL,
+                                         String.valueOf(MESSAGE_TIME),
+                                         ID_LABEL, RECORD_ID))
+                     .build();
+    ReceivedMessage expectedReceivedMessage =
+        ReceivedMessage.newBuilder()
+                       .setMessage(expectedPubsubMessage)
+                       .setAckId(ACK_ID)
+                       .build();
+    final PullResponse response =
+        PullResponse.newBuilder()
+                    .addAllReceivedMessages(ImmutableList.of(expectedReceivedMessage))
+                    .build();
+
+    final List<PullRequest> requestsReceived = new ArrayList<>();
+    SubscriberImplBase subscriberImplBase = new SubscriberImplBase() {
+      @Override
+      public void pull(PullRequest request, StreamObserver<PullResponse> responseObserver) {
+        requestsReceived.add(request);
+        responseObserver.onNext(response);
+        responseObserver.onCompleted();
+      }
+    };
+    Server server = InProcessServerBuilder.forName(channelName)
+        .addService(subscriberImplBase)
+        .build()
+        .start();
+    try {
+      List<IncomingMessage> acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true);
+      assertEquals(1, acutalMessages.size());
+      IncomingMessage actualMessage = acutalMessages.get(0);
+      assertEquals(ACK_ID, actualMessage.ackId);
+      assertEquals(DATA, new String(actualMessage.elementBytes));
+      assertEquals(RECORD_ID, actualMessage.recordId);
+      assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch);
+      assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch);
+      assertEquals(expectedRequest, Iterables.getOnlyElement(requestsReceived));
+    } finally {
+      server.shutdownNow();
+    }
+  }
+
+  @Test
+  public void publishOneMessage() throws IOException {
+    String expectedTopic = TOPIC.getPath();
+    PubsubMessage expectedPubsubMessage =
+        PubsubMessage.newBuilder()
+                     .setData(ByteString.copyFrom(DATA.getBytes()))
+                     .putAllAttributes(ATTRIBUTES)
+                     .putAllAttributes(
+                         ImmutableMap.of(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME),
+                                         ID_LABEL, RECORD_ID))
+                     .build();
+    final PublishRequest expectedRequest =
+        PublishRequest.newBuilder()
+                      .setTopic(expectedTopic)
+                      .addAllMessages(
+                          ImmutableList.of(expectedPubsubMessage))
+                      .build();
+    final PublishResponse response =
+        PublishResponse.newBuilder()
+                       .addAllMessageIds(ImmutableList.of(MESSAGE_ID))
+                       .build();
+
+    final List<PublishRequest> requestsReceived = new ArrayList<>();
+    PublisherImplBase publisherImplBase = new PublisherImplBase() {
+      @Override
+      public void publish(
+          PublishRequest request, StreamObserver<PublishResponse> responseObserver) {
+        requestsReceived.add(request);
+        responseObserver.onNext(response);
+        responseObserver.onCompleted();
+      }
+    };
+    Server server = InProcessServerBuilder.forName(channelName)
+        .addService(publisherImplBase)
+        .build()
+        .start();
+    try {
+      OutgoingMessage actualMessage = new OutgoingMessage(
+              DATA.getBytes(), ATTRIBUTES, MESSAGE_TIME, RECORD_ID);
+      int n = client.publish(TOPIC, ImmutableList.of(actualMessage));
+      assertEquals(1, n);
+      assertEquals(expectedRequest, Iterables.getOnlyElement(requestsReceived));
+    } finally {
+      server.shutdownNow();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/5bcb8c57/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java
new file mode 100644
index 0000000..6e9922c
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.gcp.pubsub;
+
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+import static org.hamcrest.Matchers.hasItem;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+
+import java.util.Set;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.testing.UsesUnboundedPCollections;
+import org.apache.beam.sdk.testing.ValidatesRunner;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
+import org.joda.time.Duration;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for PubsubIO Read and Write transforms.
+ */
+@RunWith(JUnit4.class)
+public class PubsubIOTest {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testPubsubIOGetName() {
+    assertEquals("PubsubIO.Read",
+        PubsubIO.<String>read().topic("projects/myproject/topics/mytopic").getName());
+    assertEquals("PubsubIO.Write",
+        PubsubIO.<String>write().topic("projects/myproject/topics/mytopic").getName());
+  }
+
+  @Test
+  public void testTopicValidationSuccess() throws Exception {
+    PubsubIO.<String>read().topic("projects/my-project/topics/abc");
+    PubsubIO.<String>read().topic("projects/my-project/topics/ABC");
+    PubsubIO.<String>read().topic("projects/my-project/topics/AbC-DeF");
+    PubsubIO.<String>read().topic("projects/my-project/topics/AbC-1234");
+    PubsubIO.<String>read().topic("projects/my-project/topics/AbC-1234-_.~%+-_.~%+-_.~%+-abc");
+    PubsubIO.<String>read().topic(new StringBuilder()
+        .append("projects/my-project/topics/A-really-long-one-")
+        .append("111111111111111111111111111111111111111111111111111111111111111111111111111111111")
+        .append("111111111111111111111111111111111111111111111111111111111111111111111111111111111")
+        .append("11111111111111111111111111111111111111111111111111111111111111111111111111")
+        .toString());
+  }
+
+  @Test
+  public void testTopicValidationBadCharacter() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
+    PubsubIO.<String>read().topic("projects/my-project/topics/abc-*-abc");
+  }
+
+  @Test
+  public void testTopicValidationTooLong() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
+    PubsubIO.<String>read().topic(new StringBuilder().append
+        ("projects/my-project/topics/A-really-long-one-")
+        .append("111111111111111111111111111111111111111111111111111111111111111111111111111111111")
+        .append("111111111111111111111111111111111111111111111111111111111111111111111111111111111")
+        .append("1111111111111111111111111111111111111111111111111111111111111111111111111111")
+        .toString());
+  }
+
+  @Test
+  public void testReadTopicDisplayData() {
+    String topic = "projects/project/topics/topic";
+    String subscription = "projects/project/subscriptions/subscription";
+    Duration maxReadTime = Duration.standardMinutes(5);
+    PubsubIO.Read<String> read = PubsubIO.<String>read()
+        .topic(StaticValueProvider.of(topic))
+        .timestampLabel("myTimestamp")
+        .idLabel("myId");
+
+    DisplayData displayData = DisplayData.from(read);
+
+    assertThat(displayData, hasDisplayItem("topic", topic));
+    assertThat(displayData, hasDisplayItem("timestampLabel", "myTimestamp"));
+    assertThat(displayData, hasDisplayItem("idLabel", "myId"));
+  }
+
+  @Test
+  public void testReadSubscriptionDisplayData() {
+    String topic = "projects/project/topics/topic";
+    String subscription = "projects/project/subscriptions/subscription";
+    Duration maxReadTime = Duration.standardMinutes(5);
+    PubsubIO.Read<String> read = PubsubIO.<String>read()
+        .subscription(StaticValueProvider.of(subscription))
+        .timestampLabel("myTimestamp")
+        .idLabel("myId");
+
+    DisplayData displayData = DisplayData.from(read);
+
+    assertThat(displayData, hasDisplayItem("subscription", subscription));
+    assertThat(displayData, hasDisplayItem("timestampLabel", "myTimestamp"));
+    assertThat(displayData, hasDisplayItem("idLabel", "myId"));
+  }
+
+  @Test
+  public void testNullTopic() {
+    String subscription = "projects/project/subscriptions/subscription";
+    PubsubIO.Read<String> read = PubsubIO.<String>read()
+        .subscription(StaticValueProvider.of(subscription));
+    assertNull(read.getTopic());
+    assertNotNull(read.getSubscription());
+    assertNotNull(DisplayData.from(read));
+  }
+
+  @Test
+  public void testNullSubscription() {
+    String topic = "projects/project/topics/topic";
+    PubsubIO.Read<String> read = PubsubIO.<String>read()
+        .topic(StaticValueProvider.of(topic));
+    assertNotNull(read.getTopic());
+    assertNull(read.getSubscription());
+    assertNotNull(DisplayData.from(read));
+  }
+
+  @Test
+  @Category({ValidatesRunner.class, UsesUnboundedPCollections.class})
+  public void testPrimitiveReadDisplayData() {
+    DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
+    Set<DisplayData> displayData;
+    PubsubIO.Read<String> read = PubsubIO.<String>read().withCoder(StringUtf8Coder.of());
+
+    // Reading from a subscription.
+    read = read.subscription("projects/project/subscriptions/subscription");
+    displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
+    assertThat("PubsubIO.Read should include the subscription in its primitive display data",
+        displayData, hasItem(hasDisplayItem("subscription")));
+
+    // Reading from a topic.
+    read = read.topic("projects/project/topics/topic");
+    displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
+    assertThat("PubsubIO.Read should include the topic in its primitive display data",
+        displayData, hasItem(hasDisplayItem("topic")));
+  }
+
+  @Test
+  public void testWriteDisplayData() {
+    String topic = "projects/project/topics/topic";
+    PubsubIO.Write<?> write = PubsubIO.<String>write()
+        .topic(topic)
+        .timestampLabel("myTimestamp")
+        .idLabel("myId");
+
+    DisplayData displayData = DisplayData.from(write);
+
+    assertThat(displayData, hasDisplayItem("topic", topic));
+    assertThat(displayData, hasDisplayItem("timestampLabel", "myTimestamp"));
+    assertThat(displayData, hasDisplayItem("idLabel", "myId"));
+  }
+
+  @Test
+  @Category(ValidatesRunner.class)
+  public void testPrimitiveWriteDisplayData() {
+    DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
+    PubsubIO.Write<?> write = PubsubIO.<String>write().topic("projects/project/topics/topic");
+
+    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(write);
+    assertThat("PubsubIO.Write should include the topic in its primitive display data",
+        displayData, hasItem(hasDisplayItem("topic")));
+  }
+}