You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by rm...@apache.org on 2019/07/05 12:46:15 UTC

[flink] branch master updated (b596ec4 -> f2494ce)

This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git.


    from b596ec4  [FLINK-12817][docs] Fix imports in Processing function example
     new 0937601  [FLINK-9311] [pubsub] Add PubSubSource and PubSubSink connectors
     new a962770  [FLINK-9311] [pubsub] Add unit and integration tests for PubSub connectors
     new e75c33f  [FLINK-9311] [pubsub] Add documentation of pubsub connectors
     new cc101de  [FLINK-9311] [pubsub] Clean up / add documentation and style issues in the PubSub connector
     new f2494ce  [FLINK-9311] [pubsub] Improvements to builders + minor improvement to PubSubSink flush logic

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docs/dev/connectors/guarantees.md                  |   5 +
 docs/dev/connectors/index.md                       |   1 +
 docs/dev/connectors/pubsub.md                      | 158 ++++++++++
 .../flink-connector-gcp-pubsub/pom.xml             |  89 ++++++
 .../gcp/pubsub/BlockingGrpcPubSubSubscriber.java   | 138 +++++++++
 .../gcp/pubsub/DefaultPubSubSubscriberFactory.java |  67 ++++
 .../gcp/pubsub/DeserializationSchemaWrapper.java   |  51 +++
 .../connectors/gcp/pubsub/PubSubSink.java          | 335 ++++++++++++++++++++
 .../connectors/gcp/pubsub/PubSubSource.java        | 344 +++++++++++++++++++++
 .../pubsub/PubSubSubscriberFactoryForEmulator.java |  66 ++++
 .../pubsub/common/AcknowledgeIdsForCheckpoint.java |  52 ++++
 .../gcp/pubsub/common/AcknowledgeOnCheckpoint.java | 110 +++++++
 .../connectors/gcp/pubsub/common/Acknowledger.java |  28 ++
 .../pubsub/common/PubSubDeserializationSchema.java |  52 ++++
 .../gcp/pubsub/common/PubSubSubscriber.java        |  37 +++
 .../gcp/pubsub/common/PubSubSubscriberFactory.java |  38 +++
 .../pubsub/DeserializationSchemaWrapperTest.java   |  84 +++++
 .../connectors/gcp/pubsub/PubSubSourceTest.java    | 206 ++++++++++++
 .../pubsub/common/AcknowledgeOnCheckpointTest.java | 141 +++++++++
 .../src/test/resources/log4j-test.properties       |  24 ++
 flink-connectors/pom.xml                           |   1 +
 .../pom.xml                                        | 128 ++++++++
 .../gcp/pubsub/CheckPubSubEmulatorTest.java        | 131 ++++++++
 .../gcp/pubsub/EmulatedPubSubSinkTest.java         | 146 +++++++++
 .../gcp/pubsub/EmulatedPubSubSourceTest.java       | 144 +++++++++
 .../gcp/pubsub/emulator/GCloudEmulatorManager.java | 264 ++++++++++++++++
 .../gcp/pubsub/emulator/GCloudUnitTestBase.java    |  80 +++++
 .../gcp/pubsub/emulator/PubsubHelper.java          | 221 +++++++++++++
 .../src/test/resources/log4j-test.properties       |  24 ++
 flink-end-to-end-tests/pom.xml                     |   1 +
 flink-end-to-end-tests/run-nightly-tests.sh        |   2 +
 .../test-scripts/test_streaming_gcp_pubsub.sh      |  22 ++
 .../flink-examples-streaming-gcp-pubsub/pom.xml    | 108 +++++++
 .../examples/gcp/pubsub/IntegerSerializer.java     |  54 ++++
 .../examples/gcp/pubsub/PubSubExample.java         |  84 +++++
 .../examples/gcp/pubsub/PubSubPublisher.java       |  67 ++++
 flink-examples/flink-examples-build-helper/pom.xml |   1 +
 flink-examples/flink-examples-streaming/pom.xml    |   1 +
 38 files changed, 3505 insertions(+)
 create mode 100644 docs/dev/connectors/pubsub.md
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/pom.xml
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/BlockingGrpcPubSubSubscriber.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/DefaultPubSubSubscriberFactory.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/DeserializationSchemaWrapper.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSink.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSource.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSubscriberFactoryForEmulator.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeIdsForCheckpoint.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeOnCheckpoint.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/Acknowledger.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubDeserializationSchema.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubSubscriber.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubSubscriberFactory.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/DeserializationSchemaWrapperTest.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSourceTest.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeOnCheckpointTest.java
 create mode 100644 flink-connectors/flink-connector-gcp-pubsub/src/test/resources/log4j-test.properties
 create mode 100644 flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml
 create mode 100644 flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/CheckPubSubEmulatorTest.java
 create mode 100644 flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSinkTest.java
 create mode 100644 flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSourceTest.java
 create mode 100644 flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/GCloudEmulatorManager.java
 create mode 100644 flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/GCloudUnitTestBase.java
 create mode 100644 flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/PubsubHelper.java
 create mode 100644 flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/resources/log4j-test.properties
 create mode 100755 flink-end-to-end-tests/test-scripts/test_streaming_gcp_pubsub.sh
 create mode 100644 flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml
 create mode 100644 flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/IntegerSerializer.java
 create mode 100644 flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubExample.java
 create mode 100644 flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubPublisher.java


[flink] 05/05: [FLINK-9311] [pubsub] Improvements to builders + minor improvement to PubSubSink flush logic

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit f2494cec219ad48f64993a04bec5d20464c11c94
Author: Jiangjie (Becket) Qin <be...@gmail.com>
AuthorDate: Fri Jul 5 01:05:41 2019 +0800

    [FLINK-9311] [pubsub] Improvements to builders + minor improvement to PubSubSink flush logic
---
 docs/dev/connectors/pubsub.md                      |  8 +--
 .../connectors/gcp/pubsub/PubSubSink.java          | 60 ++++++++++++++--------
 .../connectors/gcp/pubsub/PubSubSource.java        | 29 +++++------
 .../gcp/pubsub/EmulatedPubSubSinkTest.java         |  4 +-
 .../gcp/pubsub/EmulatedPubSubSourceTest.java       |  2 +-
 .../examples/gcp/pubsub/PubSubExample.java         |  4 +-
 6 files changed, 61 insertions(+), 46 deletions(-)

diff --git a/docs/dev/connectors/pubsub.md b/docs/dev/connectors/pubsub.md
index bfa6f4a..7c14cd5 100644
--- a/docs/dev/connectors/pubsub.md
+++ b/docs/dev/connectors/pubsub.md
@@ -62,7 +62,7 @@ Example:
 StreamExecutionEnvironment streamExecEnv = StreamExecutionEnvironment.getExecutionEnvironment();
 
 DeserializationSchema<SomeObject> deserializer = (...);
-SourceFunction<SomeObject> pubsubSource = PubSubSource.newBuilder(SomeObject.class)
+SourceFunction<SomeObject> pubsubSource = PubSubSource.newBuilder()
                                                       .withDeserializationSchema(deserializer)
                                                       .withProjectName("project")
                                                       .withSubscriptionName("subscription")
@@ -89,7 +89,7 @@ Example:
 DataStream<SomeObject> dataStream = (...);
 
 SerializationSchema<SomeObject> serializationSchema = (...);
-SinkFunction<SomeObject> pubsubSink = PubSubSink.newBuilder(SomeObject.class)
+SinkFunction<SomeObject> pubsubSink = PubSubSink.newBuilder()
                                                 .withDeserializationSchema(deserializer)
                                                 .withProjectName("project")
                                                 .withSubscriptionName("subscription")
@@ -117,13 +117,13 @@ The following example shows how you would create a source to read messages from
 <div data-lang="java" markdown="1">
 {% highlight java %}
 DeserializationSchema<SomeObject> deserializationSchema = (...);
-SourceFunction<SomeObject> pubsubSource = PubSubSource.newBuilder(SomeObject.class)
+SourceFunction<SomeObject> pubsubSource = PubSubSource.newBuilder()
                                                       .withDeserializationSchema(deserializationSchema)
                                                       .withProjectName("my-fake-project")
                                                       .withSubscriptionName("subscription")
                                                       .withPubSubSubscriberFactory(new PubSubSubscriberFactoryForEmulator("localhost:1234", "my-fake-project", "subscription", 10, Duration.ofSeconds(15), 100))
                                                       .build();
-SinkFunction<SomeObject> pubsubSink = PubSubSink.newBuilder(SomeObject.class)
+SinkFunction<SomeObject> pubsubSink = PubSubSink.newBuilder()
                                                 .withDeserializationSchema(deserializationSchema)
                                                 .withProjectName("my-fake-project")
                                                 .withSubscriptionName("subscription")
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSink.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSink.java
index d280dc9..f314d3e 100644
--- a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSink.java
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSink.java
@@ -46,8 +46,8 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
 import static com.google.cloud.pubsub.v1.SubscriptionAdminSettings.defaultCredentialsProviderBuilder;
@@ -63,7 +63,7 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> implements Checkpointed
 
 	private final AtomicReference<Exception> exceptionAtomicReference;
 	private final ApiFutureCallback<String> failureHandler;
-	private final ConcurrentLinkedQueue<ApiFuture<String>> outstandingFutures;
+	private final AtomicInteger numPendingFutures;
 	private final Credentials credentials;
 	private final SerializationSchema<IN> serializationSchema;
 	private final String projectName;
@@ -81,7 +81,7 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> implements Checkpointed
 		String hostAndPortForEmulator) {
 		this.exceptionAtomicReference = new AtomicReference<>();
 		this.failureHandler = new FailureHandler();
-		this.outstandingFutures = new ConcurrentLinkedQueue<>();
+		this.numPendingFutures = new AtomicInteger(0);
 		this.credentials = credentials;
 		this.serializationSchema = serializationSchema;
 		this.projectName = projectName;
@@ -162,18 +162,17 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> implements Checkpointed
 			.build();
 
 		ApiFuture<String> future = publisher.publish(pubsubMessage);
-		outstandingFutures.add(future);
+		numPendingFutures.incrementAndGet();
 		ApiFutures.addCallback(future, failureHandler, directExecutor());
 	}
 
 	/**
 	 * Create a builder for a new PubSubSink.
 	 *
-	 * @param <IN> The generic of the type that is to be written into the sink.
 	 * @return a new PubSubSinkBuilder instance
 	 */
-	public static <IN> SerializationSchemaBuilder<IN> newBuilder(Class<IN> clazz) {
-		return new PubSubSinkBuilder<>();
+	public static SerializationSchemaBuilder newBuilder() {
+		return new SerializationSchemaBuilder();
 	}
 
 	@Override
@@ -181,6 +180,8 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> implements Checkpointed
 		//before checkpoints make sure all the batched / buffered pubsub messages have actually been sent
 		publisher.publishAllOutstanding();
 
+		// At this point, no new messages will be published because this thread has successfully acquired
+		// the checkpoint lock. So we just wait for all the pending futures to complete.
 		waitForFuturesToComplete();
 		if (exceptionAtomicReference.get() != null) {
 			throw exceptionAtomicReference.get();
@@ -188,8 +189,17 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> implements Checkpointed
 	}
 
 	private void waitForFuturesToComplete() {
-		while (isRunning && !outstandingFutures.isEmpty()) {
-			outstandingFutures.removeIf(ApiFuture::isDone);
+		// We have to synchronize on numPendingFutures here to ensure the notification won't be missed.
+		synchronized (numPendingFutures) {
+			while (isRunning && numPendingFutures.get() > 0) {
+				try {
+					numPendingFutures.wait();
+				} catch (InterruptedException e) {
+					// Simply cache the interrupted exception. Supposedly the thread will exit the loop
+					// gracefully when it checks the isRunning flag.
+					LOG.info("Interrupted when waiting for futures to complete");
+				}
+			}
 		}
 	}
 
@@ -202,7 +212,7 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> implements Checkpointed
 	 *
 	 * @param <IN> Type of PubSubSink to create.
 	 */
-	public static class PubSubSinkBuilder<IN> implements SerializationSchemaBuilder<IN>, ProjectNameBuilder<IN>, TopicNameBuilder<IN> {
+	public static class PubSubSinkBuilder<IN> implements ProjectNameBuilder<IN>, TopicNameBuilder<IN> {
 		private SerializationSchema<IN> serializationSchema;
 		private String projectName;
 		private String topicName;
@@ -210,7 +220,9 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> implements Checkpointed
 		private Credentials credentials;
 		private String hostAndPort;
 
-		private PubSubSinkBuilder() { }
+		private PubSubSinkBuilder(SerializationSchema<IN> serializationSchema) {
+			this.serializationSchema = serializationSchema;
+		}
 
 		/**
 		 * Set the credentials.
@@ -225,13 +237,6 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> implements Checkpointed
 		}
 
 		@Override
-		public ProjectNameBuilder<IN> withSerializationSchema(SerializationSchema<IN> serializationSchema) {
-			Preconditions.checkNotNull(serializationSchema);
-			this.serializationSchema = serializationSchema;
-			return this;
-		}
-
-		@Override
 		public TopicNameBuilder<IN> withProjectName(String projectName) {
 			Preconditions.checkNotNull(projectName);
 			this.projectName = projectName;
@@ -275,11 +280,13 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> implements Checkpointed
 	/**
 	 * Part of {@link PubSubSinkBuilder} to set required fields.
 	 */
-	public interface SerializationSchemaBuilder<IN> {
+	public static class SerializationSchemaBuilder {
 		/**
 		 * Set the SerializationSchema used to Serialize objects to be added as payloads of PubSubMessages.
 		 */
-		ProjectNameBuilder<IN> withSerializationSchema(SerializationSchema<IN> deserializationSchema);
+		public <IN> ProjectNameBuilder<IN> withSerializationSchema(SerializationSchema<IN> deserializationSchema) {
+			return new PubSubSinkBuilder<>(deserializationSchema);
+		}
 	}
 
 	/**
@@ -305,13 +312,24 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> implements Checkpointed
 	private class FailureHandler implements ApiFutureCallback<String>, Serializable {
 		@Override
 		public void onFailure(Throwable t) {
+			ackAndMaybeNotifyNoPendingFutures();
 			exceptionAtomicReference.set(new RuntimeException("Failed trying to publish message", t));
 		}
 
 		@Override
 		public void onSuccess(String result) {
-			//do nothing on success
+			ackAndMaybeNotifyNoPendingFutures();
 			LOG.debug("Successfully published message with id: {}", result);
 		}
+
+		private void ackAndMaybeNotifyNoPendingFutures() {
+			// When there are no pending futures anymore, notify the thread that is waiting for
+			// all the pending futures to be completed.
+			if (numPendingFutures.decrementAndGet() == 0) {
+				synchronized (numPendingFutures) {
+					numPendingFutures.notify();
+				}
+			}
+		}
 	}
 }
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSource.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSource.java
index d093e0a..4ddd816 100644
--- a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSource.java
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSource.java
@@ -155,8 +155,8 @@ public class PubSubSource<OUT> extends RichSourceFunction<OUT>
 		return deserializationSchema.getProducedType();
 	}
 
-	public static <OUT> DeserializationSchemaBuilder<OUT> newBuilder(Class<OUT> clazz) {
-		return new PubSubSourceBuilder<>();
+	public static DeserializationSchemaBuilder newBuilder() {
+		return new DeserializationSchemaBuilder();
 	}
 
 	@Override
@@ -189,7 +189,7 @@ public class PubSubSource<OUT> extends RichSourceFunction<OUT>
 	 *
 	 * @param <OUT> The type of objects which will be read
 	 */
-	public static class PubSubSourceBuilder<OUT> implements DeserializationSchemaBuilder<OUT>, ProjectNameBuilder<OUT>, SubscriptionNameBuilder<OUT> {
+	public static class PubSubSourceBuilder<OUT> implements ProjectNameBuilder<OUT>, SubscriptionNameBuilder<OUT> {
 		private PubSubDeserializationSchema<OUT> deserializationSchema;
 		private String projectName;
 		private String subscriptionName;
@@ -198,25 +198,18 @@ public class PubSubSource<OUT> extends RichSourceFunction<OUT>
 		private Credentials credentials;
 		private int maxMessageToAcknowledge = 10000;
 
-		protected PubSubSourceBuilder() {
-		}
-
-		@Override
-		public ProjectNameBuilder withDeserializationSchema(DeserializationSchema<OUT> deserializationSchema) {
+		private PubSubSourceBuilder(DeserializationSchema<OUT> deserializationSchema) {
 			Preconditions.checkNotNull(deserializationSchema);
 			this.deserializationSchema = new DeserializationSchemaWrapper<>(deserializationSchema);
-			return this;
 		}
 
-		@Override
-		public ProjectNameBuilder withDeserializationSchema(PubSubDeserializationSchema deserializationSchema) {
+		private PubSubSourceBuilder(PubSubDeserializationSchema<OUT> deserializationSchema) {
 			Preconditions.checkNotNull(deserializationSchema);
 			this.deserializationSchema = deserializationSchema;
-			return this;
 		}
 
 		@Override
-		public SubscriptionNameBuilder withProjectName(String projectName) {
+		public SubscriptionNameBuilder<OUT> withProjectName(String projectName) {
 			Preconditions.checkNotNull(projectName);
 			this.projectName = projectName;
 			return this;
@@ -306,17 +299,21 @@ public class PubSubSource<OUT> extends RichSourceFunction<OUT>
 	/**
 	 * Part of {@link PubSubSourceBuilder} to set required fields.
 	 */
-	public interface DeserializationSchemaBuilder<OUT> {
+	public static class DeserializationSchemaBuilder {
 		/**
 		 * Set the DeserializationSchema used to deserialize incoming PubSubMessages.
 		 * If you want access to meta data of a PubSubMessage use the overloaded withDeserializationSchema({@link PubSubDeserializationSchema}) method instead.
 		 */
-		ProjectNameBuilder<OUT> withDeserializationSchema(DeserializationSchema<OUT> deserializationSchema);
+		public <OUT> ProjectNameBuilder<OUT> withDeserializationSchema(DeserializationSchema<OUT> deserializationSchema) {
+			return new PubSubSourceBuilder<>(deserializationSchema);
+		}
 
 		/**
 		 * Set the DeserializationSchema used to deserialize incoming PubSubMessages.
 		 */
-		ProjectNameBuilder<OUT> withDeserializationSchema(PubSubDeserializationSchema<OUT> deserializationSchema);
+		public <OUT> ProjectNameBuilder<OUT> withDeserializationSchema(PubSubDeserializationSchema<OUT> deserializationSchema) {
+			return new PubSubSourceBuilder<>(deserializationSchema);
+		}
 	}
 
 	/**
diff --git a/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSinkTest.java b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSinkTest.java
index 0566e01..5c0c3b1 100644
--- a/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSinkTest.java
+++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSinkTest.java
@@ -77,7 +77,7 @@ public class EmulatedPubSubSinkTest extends GCloudUnitTestBase {
 
 		// Sink into pubsub
 		theData
-			.addSink(PubSubSink.newBuilder(String.class)
+			.addSink(PubSubSink.newBuilder()
 								.withSerializationSchema(new SimpleStringSchema())
 								.withProjectName(PROJECT_NAME)
 								.withTopicName(TOPIC_NAME)
@@ -116,7 +116,7 @@ public class EmulatedPubSubSinkTest extends GCloudUnitTestBase {
 		env.addSource(new SingleInputSourceFunction())
 
 			.map((MapFunction<String, String>) StringUtils::reverse)
-			.addSink(PubSubSink.newBuilder(String.class)
+			.addSink(PubSubSink.newBuilder()
 								.withSerializationSchema(new SimpleStringSchema())
 								.withProjectName(PROJECT_NAME)
 								.withTopicName(TOPIC_NAME)
diff --git a/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSourceTest.java b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSourceTest.java
index f3f9b0a..b28569c 100644
--- a/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSourceTest.java
+++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSourceTest.java
@@ -96,7 +96,7 @@ public class EmulatedPubSubSourceTest extends GCloudUnitTestBase {
 		env.setRestartStrategy(RestartStrategies.noRestart());
 
 		DataStream<String> fromPubSub = env
-			.addSource(PubSubSource.newBuilder(String.class)
+			.addSource(PubSubSource.newBuilder()
 								.withDeserializationSchema(new BoundedStringDeserializer(10))
 								.withProjectName(PROJECT_NAME)
 								.withSubscriptionName(SUBSCRIPTION_NAME)
diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubExample.java b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubExample.java
index 7b66577..a960176 100644
--- a/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubExample.java
+++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubExample.java
@@ -63,13 +63,13 @@ public class PubSubExample {
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.enableCheckpointing(1000L);
 
-		env.addSource(PubSubSource.newBuilder(Integer.class)
+		env.addSource(PubSubSource.newBuilder()
 								.withDeserializationSchema(new IntegerSerializer())
 								.withProjectName(projectName)
 								.withSubscriptionName(subscriptionName)
 								.build())
 			.map(PubSubExample::printAndReturn).disableChaining()
-			.addSink(PubSubSink.newBuilder(Integer.class)
+			.addSink(PubSubSink.newBuilder()
 								.withSerializationSchema(new IntegerSerializer())
 								.withProjectName(projectName)
 								.withTopicName(outputTopicName).build());


[flink] 03/05: [FLINK-9311] [pubsub] Add documentation of pubsub connectors

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e75c33f05ccdb7506ffd8a8e0ad778aa32d00226
Author: Richard Deurwaarder <ri...@xeli.eu>
AuthorDate: Mon Aug 20 17:58:28 2018 +0200

    [FLINK-9311] [pubsub] Add documentation of pubsub connectors
---
 docs/dev/connectors/guarantees.md |   5 ++
 docs/dev/connectors/index.md      |   1 +
 docs/dev/connectors/pubsub.md     | 106 ++++++++++++++++++++++++++++++++++++++
 3 files changed, 112 insertions(+)

diff --git a/docs/dev/connectors/guarantees.md b/docs/dev/connectors/guarantees.md
index 732b8b2..56eac12 100644
--- a/docs/dev/connectors/guarantees.md
+++ b/docs/dev/connectors/guarantees.md
@@ -62,6 +62,11 @@ Please read the documentation of each connector to understand the details of the
             <td></td>
         </tr>
         <tr>
+            <td>Google PubSub</td>
+            <td>at least once</td>
+            <td></td>
+        </tr>
+        <tr>
             <td>Collections</td>
             <td>exactly once</td>
             <td></td>
diff --git a/docs/dev/connectors/index.md b/docs/dev/connectors/index.md
index b5405d4..6132014 100644
--- a/docs/dev/connectors/index.md
+++ b/docs/dev/connectors/index.md
@@ -47,6 +47,7 @@ Connectors provide code for interfacing with various third-party systems. Curren
  * [RabbitMQ](rabbitmq.html) (source/sink)
  * [Apache NiFi](nifi.html) (source/sink)
  * [Twitter Streaming API](twitter.html) (source)
+ * [Google PubSub](pubsub.html) (source/sink)
 
 Keep in mind that to use one of these connectors in an application, additional third party
 components are usually required, e.g. servers for the data stores or message queues.
diff --git a/docs/dev/connectors/pubsub.md b/docs/dev/connectors/pubsub.md
new file mode 100644
index 0000000..6eb3c24
--- /dev/null
+++ b/docs/dev/connectors/pubsub.md
@@ -0,0 +1,106 @@
+---
+title: "Google PubSub"
+nav-title: PubSub
+nav-parent_id: connectors
+nav-pos: 7
+---
+<!--
+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.
+-->
+
+This connector provides a Source and Sink that can read from and write to
+[Google PubSub](https://cloud.google.com/pubsub). To use this connector, add the
+following dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-pubsub{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+Note that the streaming connectors are currently not part of the binary
+distribution. See
+[here]({{site.baseurl}}/dev/linking.html)
+for information about how to package the program with the libraries for
+cluster execution.
+
+#### PubSub Source
+
+The connector provides a Source for reading data from Google PubSub to Apache Flink. PubSub has an Atleast-Once guarantee and as such.
+
+The class `PubSubSource(…)` has a builder to create PubSubsources. `PubSubSource.newBuilder()`
+
+There are several optional methods to alter how the PubSubSource is created, the bare minimum is to provide a google project and pubsub subscription and a way to deserialize the PubSubMessages.
+Example:
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+StreamExecutionEnvironment streamExecEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+
+DeserializationSchema<SomeObject> deserializationSchema = (...);
+SourceFunction<SomeObject> pubsubSource = PubSubSource.<SomeObject>newBuilder()
+                                                      .withDeserializationSchema(deserializationSchema)
+                                                      .withProjectSubscriptionName("google-project-name", "pubsub-subscription")
+                                                      .build();
+
+streamExecEnv.addSource(pubsubSource);
+{% endhighlight %}
+</div>
+</div>
+
+#### PubSub Sink
+
+The connector provides a Sink for writing data to PubSub.
+
+The class `PubSubSource(…)` has a builder to create PubSubsources. `PubSubSource.newBuilder()`
+
+This builder works in a similar way to the PubSubSource.
+Example:
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+StreamExecutionEnvironment streamExecEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+
+SerializationSchema<SomeObject> serializationSchema = (...);
+SourceFunction<SomeObject> pubsubSink = PubSubSink.<SomeObject>newBuilder()
+                                                  .withSerializationSchema(serializationSchema)
+                                                  .withTopicName("pubsub-topic-name")
+                                                  .withProjectName("google-project-name")
+                                                  .build()
+
+streamExecEnv.addSink(pubsubSink);
+{% endhighlight %}
+</div>
+</div>
+
+#### Google Credentials
+
+Google uses [Credentials](https://cloud.google.com/docs/authentication/production) to authenticate and authorize applications so that they can use Google cloud resources such as PubSub. Both builders allow several ways to provide these credentials.
+
+By default the connectors will look for an environment variable: [GOOGLE_APPLICATION_CREDENTIALS](https://cloud.google.com/docs/authentication/production#obtaining_and_providing_service_account_credentials_manually) which should point to a file containing the credentials.
+
+It is also possible to provide a Credentials object directly. For instance if you read the Credentials yourself from an external system. In this case you can use `PubSubSource.newBuilder().withCredentials(...)`
+
+#### Integration testing
+
+When using integration tests you might not want to connect to PubSub directly but use a docker container to read and write to. This is possible by using `PubSubSource.newBuilder().withHostAndPort("localhost:1234")`.
+{% top %}


[flink] 01/05: [FLINK-9311] [pubsub] Add PubSubSource and PubSubSink connectors

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 0937601972674cc548446616796566ad2eab9b81
Author: Richard Deurwaarder <ri...@xeli.eu>
AuthorDate: Sat May 26 14:59:32 2018 +0200

    [FLINK-9311] [pubsub] Add PubSubSource and PubSubSink connectors
    
    This closes #6594
---
 flink-connectors/flink-connector-pubsub/pom.xml    | 119 ++++++++++
 .../flink/streaming/connectors/pubsub/Bound.java   |  99 ++++++++
 .../connectors/pubsub/BoundedPubSubSource.java     |  79 +++++++
 .../streaming/connectors/pubsub/PubSubSink.java    | 241 +++++++++++++++++++
 .../streaming/connectors/pubsub/PubSubSource.java  | 257 +++++++++++++++++++++
 .../connectors/pubsub/SubscriberWrapper.java       | 104 +++++++++
 .../common/SerializableCredentialsProvider.java    |  67 ++++++
 .../streaming/connectors/pubsub/BoundTest.java     | 112 +++++++++
 .../connectors/pubsub/BoundedPubSubSourceTest.java |  70 ++++++
 .../connectors/pubsub/PubSubSourceTest.java        | 148 ++++++++++++
 .../connectors/pubsub/SubscriberWrapperTest.java   |  57 +++++
 flink-connectors/pom.xml                           |   1 +
 flink-examples/flink-examples-streaming/pom.xml    |  12 +
 .../examples/pubsub/IntegerSerializer.java         |  48 ++++
 .../streaming/examples/pubsub/PubSubExample.java   |  84 +++++++
 .../streaming/examples/pubsub/PubSubPublisher.java |  64 +++++
 16 files changed, 1562 insertions(+)

diff --git a/flink-connectors/flink-connector-pubsub/pom.xml b/flink-connectors/flink-connector-pubsub/pom.xml
new file mode 100644
index 0000000..a6e8d72
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/pom.xml
@@ -0,0 +1,119 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.7-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-pubsub_${scala.binary.version}</artifactId>
+	<name>flink-connector-pubsub</name>
+
+	<packaging>jar</packaging>
+
+	<properties>
+		<pubsub.version>1.37.1</pubsub.version>
+	</properties>
+
+	<dependencies>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>com.google.cloud</groupId>
+			<artifactId>google-cloud-pubsub</artifactId>
+			<version>${pubsub.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+	</dependencies>
+
+    <build>
+        <plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<shadeTestJar>false</shadeTestJar>
+							<artifactSet>
+								<includes>
+									<include>*:*</include>
+								</includes>
+							</artifactSet>
+							<relocations>
+								<relocation>
+									<pattern>com.google.guava</pattern>
+									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.com.google.guava</shadedPattern>
+								</relocation>
+								<relocation>
+									<pattern>com.google.common.base</pattern>
+									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.com.google.common.base</shadedPattern>
+								</relocation>
+								<relocation>
+									<pattern>io.grpc.auth</pattern>
+									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.io.grpc.auth</shadedPattern>
+								</relocation>
+								<relocation>
+									<pattern>io.grpc.protobuf</pattern>
+									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.io.grpc.protobuf</shadedPattern>
+								</relocation>
+							</relocations>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+        </plugins>
+    </build>
+
+</project>
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java
new file mode 100644
index 0000000..b37cc45
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java
@@ -0,0 +1,99 @@
+package org.apache.flink.streaming.connectors.pubsub;
+
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.Timer;
+import java.util.TimerTask;
+
+class Bound<OUT> implements Serializable {
+	private static final Logger LOG = LoggerFactory.getLogger(Bound.class);
+
+	private final Bound.Mode mode;
+	private final long maxMessagedReceived;
+	private final long maxTimeBetweenMessages;
+
+	private SourceFunction<OUT> sourceFunction;
+	private transient Timer timer;
+	private long messagesReceived;
+	private long lastReceivedMessage;
+	private boolean cancelled = false;
+
+	private Bound(Bound.Mode mode, long maxMessagedReceived, long maxTimeBetweenMessages) {
+		this.mode = mode;
+		this.maxMessagedReceived = maxMessagedReceived;
+		this.maxTimeBetweenMessages = maxTimeBetweenMessages;
+		this.messagesReceived = 0L;
+	}
+
+	static <OUT> Bound<OUT> boundByAmountOfMessages(long maxMessagedReceived) {
+		return new Bound<>(Mode.COUNTER, maxMessagedReceived, 0L);
+	}
+
+	static <OUT> Bound<OUT> boundByTimeSinceLastMessage(long maxTimeBetweenMessages) {
+		return new Bound<>(Mode.TIMER, 0L, maxTimeBetweenMessages);
+	}
+
+	static <OUT> Bound<OUT> boundByAmountOfMessagesOrTimeSinceLastMessage(long maxMessagedReceived, long maxTimeBetweenMessages) {
+		return new Bound<>(Mode.COUNTER_OR_TIMER, maxMessagedReceived, maxTimeBetweenMessages);
+	}
+
+	private TimerTask shutdownPubSubSource() {
+		return new TimerTask() {
+			@Override
+			public void run() {
+				if (maxTimeBetweenMessagesElapsed()) {
+					cancelPubSubSource("BoundedSourceFunction: Idle timeout --> canceling source");
+					timer.cancel();
+				}
+			}
+		};
+	}
+
+	private synchronized boolean maxTimeBetweenMessagesElapsed() {
+		return System.currentTimeMillis() - lastReceivedMessage > maxTimeBetweenMessages;
+	}
+
+	private synchronized void cancelPubSubSource(String logMessage) {
+		if (!cancelled) {
+			cancelled = true;
+			sourceFunction.cancel();
+			LOG.info(logMessage);
+		}
+	}
+
+	void start(SourceFunction<OUT> sourceFunction) {
+		if (this.sourceFunction != null) {
+			throw new IllegalStateException("start() already called");
+		}
+
+		this.sourceFunction = sourceFunction;
+		messagesReceived = 0;
+
+		if (mode == Mode.TIMER || mode == Mode.COUNTER_OR_TIMER) {
+			lastReceivedMessage = System.currentTimeMillis();
+			timer = new Timer();
+			timer.schedule(shutdownPubSubSource(), 0, 100);
+		}
+	}
+
+	synchronized void receivedMessage() {
+		if (sourceFunction == null) {
+			throw new IllegalStateException("start() not called");
+		}
+
+		lastReceivedMessage = System.currentTimeMillis();
+		messagesReceived++;
+
+		if ((mode == Mode.COUNTER || mode == Mode.COUNTER_OR_TIMER) && messagesReceived >= maxMessagedReceived) {
+			cancelPubSubSource("BoundedSourceFunction: Max received messages --> canceling source");
+		}
+	}
+
+	private enum Mode {
+		COUNTER, TIMER, COUNTER_OR_TIMER
+	}
+}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSource.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSource.java
new file mode 100644
index 0000000..5f829ae
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSource.java
@@ -0,0 +1,79 @@
+package org.apache.flink.streaming.connectors.pubsub;
+
+import com.google.cloud.pubsub.v1.AckReplyConsumer;
+import com.google.pubsub.v1.PubsubMessage;
+
+import java.io.IOException;
+
+class BoundedPubSubSource<OUT> extends PubSubSource<OUT> {
+	private Bound<OUT> bound;
+
+	private BoundedPubSubSource() {
+		super();
+	}
+
+	protected void setBound(Bound<OUT> bound) {
+		this.bound = bound;
+	}
+
+	@Override
+	public void run(SourceContext<OUT> sourceContext) {
+		bound.start(this);
+		super.run(sourceContext);
+	}
+
+	@Override
+	public void receiveMessage(PubsubMessage message, AckReplyConsumer consumer) {
+		super.receiveMessage(message, consumer);
+		bound.receivedMessage();
+	}
+
+	@SuppressWarnings("unchecked")
+	public static <OUT> BoundedPubSubSourceBuilder<OUT, ? extends PubSubSource, ? extends BoundedPubSubSourceBuilder> newBuilder() {
+		return new BoundedPubSubSourceBuilder<>(new BoundedPubSubSource<OUT>());
+	}
+
+	@SuppressWarnings("unchecked")
+	public static class BoundedPubSubSourceBuilder<OUT, PSS extends BoundedPubSubSource<OUT>, BUILDER extends BoundedPubSubSourceBuilder<OUT, PSS, BUILDER>> extends PubSubSourceBuilder<OUT, PSS, BUILDER> {
+		private Long boundedByAmountOfMessages;
+		private Long boundedByTimeSinceLastMessage;
+
+		BoundedPubSubSourceBuilder(PSS sourceUnderConstruction) {
+			super(sourceUnderConstruction);
+		}
+
+		public BUILDER boundedByAmountOfMessages(long maxAmountOfMessages) {
+			boundedByAmountOfMessages = maxAmountOfMessages;
+			return (BUILDER) this;
+		}
+
+		public BUILDER boundedByTimeSinceLastMessage(long timeSinceLastMessage) {
+			boundedByTimeSinceLastMessage = timeSinceLastMessage;
+			return (BUILDER) this;
+		}
+
+		private Bound <OUT> createBound() {
+			if (boundedByAmountOfMessages != null && boundedByTimeSinceLastMessage != null) {
+				return Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(boundedByAmountOfMessages, boundedByTimeSinceLastMessage);
+			}
+
+			if (boundedByAmountOfMessages != null) {
+				return Bound.boundByAmountOfMessages(boundedByAmountOfMessages);
+			}
+
+			if (boundedByTimeSinceLastMessage != null) {
+				return Bound.boundByTimeSinceLastMessage(boundedByTimeSinceLastMessage);
+			}
+
+			// This is functionally speaking no bound.
+			return Bound.boundByAmountOfMessages(Long.MAX_VALUE);
+		}
+
+		@Override
+		public PSS build() throws IOException {
+			sourceUnderConstruction.setBound(createBound());
+			return super.build();
+		}
+	}
+
+}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSink.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSink.java
new file mode 100644
index 0000000..92c9c6c
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSink.java
@@ -0,0 +1,241 @@
+/*
+ * 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.flink.streaming.connectors.pubsub;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.connectors.pubsub.common.SerializableCredentialsProvider;
+
+import com.google.api.gax.core.CredentialsProvider;
+import com.google.api.gax.grpc.GrpcTransportChannel;
+import com.google.api.gax.rpc.FixedTransportChannelProvider;
+import com.google.api.gax.rpc.TransportChannel;
+import com.google.auth.Credentials;
+import com.google.cloud.pubsub.v1.Publisher;
+import com.google.protobuf.ByteString;
+import com.google.pubsub.v1.ProjectTopicName;
+import com.google.pubsub.v1.PubsubMessage;
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+
+import java.io.IOException;
+
+/**
+ * A sink function that outputs to PubSub.
+ * @param <IN> type of PubSubSink messages to write
+ */
+public class PubSubSink<IN> extends RichSinkFunction<IN> {
+
+	private final SerializableCredentialsProvider serializableCredentialsProvider;
+	private final SerializationSchema<IN>         serializationSchema;
+	private final String                          projectName;
+	private final String                          topicName;
+	private       String                          hostAndPort = null;
+
+	private transient Publisher publisher;
+
+	public PubSubSink(SerializableCredentialsProvider serializableCredentialsProvider, SerializationSchema<IN> serializationSchema, String projectName, String topicName) {
+		this.serializableCredentialsProvider = serializableCredentialsProvider;
+		this.serializationSchema = serializationSchema;
+		this.projectName = projectName;
+		this.topicName = topicName;
+	}
+
+	/**
+	 * Set the custom hostname/port combination of PubSub.
+	 * The ONLY reason to use this is during tests with the emulator provided by Google.
+	 *
+	 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
+	 * @return The current instance
+	 */
+	public PubSubSink<IN> withHostAndPort(String hostAndPort) {
+		this.hostAndPort = hostAndPort;
+		return this;
+	}
+
+	private transient ManagedChannel   managedChannel = null;
+	private transient TransportChannel channel        = null;
+
+	@Override
+	public void open(Configuration configuration) throws Exception {
+		Publisher.Builder builder = Publisher
+			.newBuilder(ProjectTopicName.of(projectName, topicName))
+			.setCredentialsProvider(serializableCredentialsProvider);
+
+		if (hostAndPort != null) {
+			managedChannel = ManagedChannelBuilder
+				.forTarget(hostAndPort)
+				.usePlaintext(true) // This is 'Ok' because this is ONLY used for testing.
+				.build();
+			channel = GrpcTransportChannel.newBuilder().setManagedChannel(managedChannel).build();
+			builder.setChannelProvider(FixedTransportChannelProvider.create(channel));
+		}
+
+		publisher = builder.build();
+	}
+
+	@Override
+	public void close() throws Exception {
+		super.close();
+		publisher.shutdown();
+		if (channel != null) {
+			channel.close();
+			managedChannel.shutdownNow();
+		}
+	}
+
+	@Override
+	public void invoke(IN message, SinkFunction.Context context) {
+		PubsubMessage pubsubMessage = PubsubMessage
+			.newBuilder()
+			.setData(ByteString.copyFrom(serializationSchema.serialize(message)))
+			.build();
+		publisher.publish(pubsubMessage);
+		publisher.publishAllOutstanding();
+	}
+
+	/**
+	 * Create a builder for a new PubSubSink.
+	 * @param <IN> The generic of the type that is to be written into the sink.
+	 * @return a new PubSubSinkBuilder instance
+	 */
+	public static <IN> PubSubSinkBuilder<IN> newBuilder() {
+		return new PubSubSinkBuilder<>();
+	}
+
+	/**
+	 * PubSubSinkBuilder to create a PubSubSink.
+	 * @param <IN> Type of PubSubSink to create.
+	 */
+	public static class PubSubSinkBuilder<IN> {
+		private SerializableCredentialsProvider serializableCredentialsProvider = null;
+		private SerializationSchema<IN>         serializationSchema             = null;
+		private String                          projectName                     = null;
+		private String                          topicName                       = null;
+		private String                          hostAndPort                     = null;
+
+		private PubSubSinkBuilder() {
+		}
+
+		/**
+		 * Set the credentials.
+		 * If this is not used then the credentials are picked up from the environment variables.
+		 * @param credentials the Credentials needed to connect.
+		 * @return The current PubSubSinkBuilder instance
+		 */
+		public PubSubSinkBuilder<IN> withCredentials(Credentials credentials) {
+			this.serializableCredentialsProvider = new SerializableCredentialsProvider(credentials);
+			return this;
+		}
+
+		/**
+		 * Set the CredentialsProvider.
+		 * If this is not used then the credentials are picked up from the environment variables.
+		 * @param credentialsProvider the custom SerializableCredentialsProvider instance.
+		 * @return The current PubSubSinkBuilder instance
+		 */
+		public PubSubSinkBuilder<IN> withCredentialsProvider(CredentialsProvider credentialsProvider) throws IOException {
+			return withCredentials(credentialsProvider.getCredentials());
+		}
+
+		/**
+		 * Set the credentials to be absent.
+		 * This means that no credentials are to be used at all.
+		 * @return The current PubSubSinkBuilder instance
+		 */
+		public PubSubSinkBuilder<IN> withoutCredentials() {
+			this.serializableCredentialsProvider = SerializableCredentialsProvider.withoutCredentials();
+			return this;
+		}
+
+		/**
+		 * @param serializationSchema Instance of a SerializationSchema that converts the IN into a byte[]
+		 * @return The current PubSubSinkBuilder instance
+		 */
+		public PubSubSinkBuilder<IN> withSerializationSchema(SerializationSchema<IN> serializationSchema) {
+			this.serializationSchema = serializationSchema;
+			return this;
+		}
+
+		/**
+		 * @param projectName The name of the project in PubSub
+		 * @return The current PubSubSinkBuilder instance
+		 */
+		public PubSubSinkBuilder<IN> withProjectName (String projectName) {
+			this.projectName = projectName;
+			return this;
+		}
+
+		/**
+		 * @param topicName The name of the topic in PubSub
+		 * @return The current PubSubSinkBuilder instance
+		 */
+		public PubSubSinkBuilder<IN> withTopicName (String topicName) {
+			this.topicName = topicName;
+			return this;
+		}
+
+
+		/**
+		 * Set the custom hostname/port combination of PubSub.
+		 * The ONLY reason to use this is during tests with the emulator provided by Google.
+		 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
+		 * @return The current PubSubSinkBuilder instance
+		 */
+		public PubSubSinkBuilder<IN> withHostAndPort(String hostAndPort) {
+			this.hostAndPort = hostAndPort;
+			return this;
+		}
+
+		/**
+		 * Actually builder the desired instance of the PubSubSink.
+		 * @return a brand new PubSubSink
+		 * @throws IOException incase of a problem getting the credentials
+		 * @throws IllegalArgumentException incase required fields were not specified.
+		 */
+		public PubSubSink<IN> build() throws IOException {
+			if (serializableCredentialsProvider == null) {
+				serializableCredentialsProvider = SerializableCredentialsProvider.credentialsProviderFromEnvironmentVariables();
+			}
+			if (serializationSchema == null) {
+				throw new IllegalArgumentException("The serializationSchema has not been specified.");
+			}
+			if (projectName == null) {
+				throw new IllegalArgumentException("The projectName has not been specified.");
+			}
+			if (topicName == null) {
+				throw new IllegalArgumentException("The topicName has not been specified.");
+			}
+
+			PubSubSink<IN> pubSubSink = new PubSubSink<>(
+				serializableCredentialsProvider,
+				serializationSchema,
+				projectName, topicName);
+
+			if (hostAndPort != null) {
+				pubSubSink.withHostAndPort(hostAndPort);
+			}
+
+			return pubSubSink;
+		}
+	}
+
+}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java
new file mode 100644
index 0000000..8f8c689
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java
@@ -0,0 +1,257 @@
+/*
+ * 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.flink.streaming.connectors.pubsub;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.MultipleIdsMessageAcknowledgingSourceBase;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.connectors.pubsub.common.SerializableCredentialsProvider;
+
+import com.google.api.gax.core.CredentialsProvider;
+import com.google.auth.Credentials;
+import com.google.cloud.pubsub.v1.AckReplyConsumer;
+import com.google.cloud.pubsub.v1.MessageReceiver;
+import com.google.pubsub.v1.ProjectSubscriptionName;
+import com.google.pubsub.v1.PubsubMessage;
+
+import java.io.IOException;
+import java.util.List;
+
+
+/**
+ * PubSub Source, this Source will consume PubSub messages from a subscription and Acknowledge them as soon as they have been received.
+ */
+public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase<OUT, String, AckReplyConsumer> implements MessageReceiver, ResultTypeQueryable<OUT>, ParallelSourceFunction<OUT> {
+	private DeserializationSchema<OUT> deserializationSchema;
+	private SubscriberWrapper          subscriberWrapper;
+
+	protected transient SourceContext<OUT> sourceContext = null;
+
+	protected PubSubSource() {
+		super(String.class);
+	}
+
+	protected void setDeserializationSchema(DeserializationSchema<OUT> deserializationSchema) {
+		this.deserializationSchema = deserializationSchema;
+	}
+
+	protected void setSubscriberWrapper(SubscriberWrapper subscriberWrapper) {
+		this.subscriberWrapper = subscriberWrapper;
+	}
+
+	@Override
+	public void open(Configuration configuration) throws Exception {
+		super.open(configuration);
+		subscriberWrapper.initialize(this);
+		if (hasNoCheckpointingEnabled(getRuntimeContext())) {
+			throw new IllegalArgumentException("Checkpointing needs to be enabled to support: PubSub ATLEAST_ONCE");
+		}
+	}
+
+	private boolean hasNoCheckpointingEnabled(RuntimeContext runtimeContext) {
+		return !(runtimeContext instanceof StreamingRuntimeContext && ((StreamingRuntimeContext) runtimeContext).isCheckpointingEnabled());
+	}
+
+	@Override
+	protected void acknowledgeSessionIDs(List<AckReplyConsumer> ackReplyConsumers) {
+		ackReplyConsumers.forEach(AckReplyConsumer::ack);
+	}
+
+	@Override
+	public void run(SourceContext<OUT> sourceContext) {
+		this.sourceContext = sourceContext;
+		subscriberWrapper.startBlocking();
+	}
+
+	@Override
+	public void receiveMessage(PubsubMessage message, AckReplyConsumer consumer) {
+		if (sourceContext == null) {
+			consumer.nack();
+			return;
+		}
+
+		processMessage(message, consumer);
+	}
+
+	private void processMessage(PubsubMessage message, AckReplyConsumer ackReplyConsumer) {
+		synchronized (sourceContext.getCheckpointLock()) {
+			boolean alreadyProcessed = !addId(message.getMessageId());
+			if (alreadyProcessed) {
+				return;
+			}
+
+			sessionIds.add(ackReplyConsumer);
+			sourceContext.collect(deserializeMessage(message));
+		}
+	}
+
+	@Override
+	public void cancel() {
+		subscriberWrapper.stop();
+	}
+
+	private OUT deserializeMessage(PubsubMessage message) {
+		try {
+			return deserializationSchema.deserialize(message.getData().toByteArray());
+		} catch (IOException e) {
+			throw new RuntimeException(e);
+		}
+	}
+
+	@Override
+	public TypeInformation<OUT> getProducedType() {
+		return deserializationSchema.getProducedType();
+	}
+
+	@SuppressWarnings("unchecked")
+	public static <OUT> PubSubSourceBuilder<OUT, ? extends PubSubSource, ? extends PubSubSourceBuilder> newBuilder() {
+		return new PubSubSourceBuilder<>(new PubSubSource<OUT>());
+	}
+
+	/**
+	 * Builder to create PubSubSource.
+	 * @param <OUT> The type of objects which will be read
+	 * @param <PSS> The type of PubSubSource
+	 * @param <BUILDER> The type of Builder to create the PubSubSource
+	 */
+	public static class PubSubSourceBuilder<OUT, PSS extends PubSubSource<OUT>, BUILDER extends PubSubSourceBuilder<OUT, PSS, BUILDER>> {
+		protected PSS 							sourceUnderConstruction;
+
+		private SubscriberWrapper               subscriberWrapper = null;
+		private SerializableCredentialsProvider serializableCredentialsProvider;
+		private DeserializationSchema<OUT>      deserializationSchema;
+		private String                          projectName;
+		private String                          subscriptionName;
+		private String                          hostAndPort;
+
+		protected PubSubSourceBuilder(PSS sourceUnderConstruction) {
+			this.sourceUnderConstruction = sourceUnderConstruction;
+		}
+
+		/**
+		 * Set the credentials.
+		 * If this is not used then the credentials are picked up from the environment variables.
+		 * @param credentials the Credentials needed to connect.
+		 * @return The current PubSubSourceBuilder instance
+		 */
+		public BUILDER withCredentials(Credentials credentials) {
+			this.serializableCredentialsProvider = new SerializableCredentialsProvider(credentials);
+			return (BUILDER) this;
+		}
+
+		/**
+		 * Set the CredentialsProvider.
+		 * If this is not used then the credentials are picked up from the environment variables.
+		 * @param credentialsProvider the custom SerializableCredentialsProvider instance.
+		 * @return The current PubSubSourceBuilder instance
+		 */
+		public BUILDER withCredentialsProvider(CredentialsProvider credentialsProvider) throws IOException {
+			return withCredentials(credentialsProvider.getCredentials());
+		}
+
+		/**
+		 * Set the credentials to be absent.
+		 * This means that no credentials are to be used at all.
+		 * @return The current PubSubSourceBuilder instance
+		 */
+		public BUILDER withoutCredentials() {
+			this.serializableCredentialsProvider = SerializableCredentialsProvider.withoutCredentials();
+			return (BUILDER) this;
+		}
+
+		/**
+		 * @param deserializationSchema Instance of a DeserializationSchema that converts the OUT into a byte[]
+		 * @return The current PubSubSourceBuilder instance
+		 */
+		public BUILDER withDeserializationSchema(DeserializationSchema <OUT> deserializationSchema) {
+			this.deserializationSchema = deserializationSchema;
+			return (BUILDER) this;
+		}
+
+		/**
+		 * @param projectName The name of the project in GoogleCloudPlatform
+		 * @param subscriptionName The name of the subscription in PubSub
+		 * @return The current PubSubSourceBuilder instance
+		 */
+		public BUILDER withProjectSubscriptionName(String projectName, String subscriptionName) {
+			this.projectName = projectName;
+			this.subscriptionName = subscriptionName;
+			return (BUILDER) this;
+		}
+
+		/**
+		 * Set the custom hostname/port combination of PubSub.
+		 * The ONLY reason to use this is during tests with the emulator provided by Google.
+		 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
+		 * @return The current PubSubSourceBuilder instance
+		 */
+		public BUILDER withHostAndPort(String hostAndPort) {
+			this.hostAndPort = hostAndPort;
+			return (BUILDER) this;
+		}
+
+		/**
+		 * Set a complete SubscriberWrapper.
+		 * The ONLY reason to use this is during tests.
+		 * @param subscriberWrapper The fully instantiated SubscriberWrapper
+		 * @return The current PubSubSourceBuilder instance
+		 */
+		public BUILDER withSubscriberWrapper(SubscriberWrapper subscriberWrapper) {
+			this.subscriberWrapper = subscriberWrapper;
+			return (BUILDER) this;
+		}
+
+		/**
+		 * Actually build the desired instance of the PubSubSourceBuilder.
+		 * @return a brand new SourceFunction
+		 * @throws IOException incase of a problem getting the credentials
+		 * @throws IllegalArgumentException incase required fields were not specified.
+		 */
+		public PSS build() throws IOException {
+			if (serializableCredentialsProvider == null) {
+				serializableCredentialsProvider = SerializableCredentialsProvider.credentialsProviderFromEnvironmentVariables();
+			}
+			if (deserializationSchema == null) {
+				throw new IllegalArgumentException("The deserializationSchema has not been specified.");
+			}
+
+			if (subscriberWrapper == null) {
+				if (projectName == null || subscriptionName == null) {
+					throw new IllegalArgumentException("The ProjectName And SubscriptionName have not been specified.");
+				}
+
+				subscriberWrapper =
+					new SubscriberWrapper(serializableCredentialsProvider, ProjectSubscriptionName.of(projectName, subscriptionName));
+
+				if (hostAndPort != null) {
+					subscriberWrapper.withHostAndPort(hostAndPort);
+				}
+			}
+
+			sourceUnderConstruction.setSubscriberWrapper(subscriberWrapper);
+			sourceUnderConstruction.setDeserializationSchema(deserializationSchema);
+
+			return sourceUnderConstruction;
+		}
+	}
+}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapper.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapper.java
new file mode 100644
index 0000000..fb75f43
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapper.java
@@ -0,0 +1,104 @@
+/*
+ * 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.flink.streaming.connectors.pubsub;
+
+import org.apache.flink.streaming.connectors.pubsub.common.SerializableCredentialsProvider;
+
+import com.google.api.core.ApiService;
+import com.google.api.gax.grpc.GrpcTransportChannel;
+import com.google.api.gax.rpc.FixedTransportChannelProvider;
+import com.google.api.gax.rpc.TransportChannel;
+import com.google.cloud.pubsub.v1.MessageReceiver;
+import com.google.cloud.pubsub.v1.Subscriber;
+import com.google.pubsub.v1.ProjectSubscriptionName;
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+
+import java.io.Serializable;
+
+class SubscriberWrapper implements Serializable {
+	private final SerializableCredentialsProvider serializableCredentialsProvider;
+	private final String                          projectId;
+	private final String                          subscriptionId;
+	private       String                          hostAndPort = null;
+
+	private transient Subscriber       subscriber;
+	private transient ManagedChannel   managedChannel = null;
+	private transient TransportChannel channel        = null;
+
+	SubscriberWrapper(SerializableCredentialsProvider serializableCredentialsProvider, ProjectSubscriptionName projectSubscriptionName) {
+		this.serializableCredentialsProvider = serializableCredentialsProvider;
+		this.projectId = projectSubscriptionName.getProject();
+		this.subscriptionId = projectSubscriptionName.getSubscription();
+	}
+
+	void initialize(MessageReceiver messageReceiver) {
+		Subscriber.Builder builder = Subscriber
+				.newBuilder(ProjectSubscriptionName.of(projectId, subscriptionId), messageReceiver)
+				.setCredentialsProvider(serializableCredentialsProvider);
+
+		if (hostAndPort != null) {
+			managedChannel = ManagedChannelBuilder
+					.forTarget(hostAndPort)
+					.usePlaintext(true) // This is 'Ok' because this is ONLY used for testing.
+					.build();
+			channel = GrpcTransportChannel.newBuilder().setManagedChannel(managedChannel).build();
+			builder.setChannelProvider(FixedTransportChannelProvider.create(channel));
+		}
+
+		this.subscriber = builder.build();
+	}
+
+	/**
+	 * Set the custom hostname/port combination of PubSub.
+	 * The ONLY reason to use this is during tests with the emulator provided by Google.
+	 *
+	 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
+	 * @return The current instance
+	 */
+	public SubscriberWrapper withHostAndPort(String hostAndPort) {
+		this.hostAndPort = hostAndPort;
+		return this;
+	}
+
+	void startBlocking() {
+		ApiService apiService = subscriber.startAsync();
+		apiService.awaitRunning();
+
+		if (apiService.state() != ApiService.State.RUNNING) {
+			throw new IllegalStateException("Could not start PubSubSubscriber, ApiService.State: " + apiService.state());
+		}
+		apiService.awaitTerminated();
+	}
+
+	void stop() {
+		subscriber.stopAsync().awaitTerminated();
+		if (channel != null) {
+			try {
+				channel.close();
+				managedChannel.shutdownNow();
+			} catch (Exception e) {
+				// Ignore
+			}
+		}
+	}
+
+	Subscriber getSubscriber() {
+		return subscriber;
+	}
+}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/common/SerializableCredentialsProvider.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/common/SerializableCredentialsProvider.java
new file mode 100644
index 0000000..bd04058
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/common/SerializableCredentialsProvider.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.pubsub.common;
+
+import com.google.api.gax.core.CredentialsProvider;
+import com.google.auth.Credentials;
+import com.google.cloud.NoCredentials;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+import static com.google.cloud.pubsub.v1.SubscriptionAdminSettings.defaultCredentialsProviderBuilder;
+
+/**
+ * Wrapper class for CredentialsProvider to make it Serializable. This can be used to pass on Credentials to SourceFunctions
+ */
+public class SerializableCredentialsProvider implements CredentialsProvider, Serializable {
+	private final Credentials credentials;
+
+	/**
+	 * @param credentials The google {@link Credentials} needed to connect to PubSub
+	 */
+	public SerializableCredentialsProvider(Credentials credentials) {
+		this.credentials = credentials;
+	}
+
+	/**
+	 * Creates a SerializableCredentialsProvider for a PubSubSubscription based on environment variables. {@link com.google.cloud.pubsub.v1.SubscriptionAdminSettings}
+	 * @return serializableCredentialsProvider
+	 * @throws IOException thrown by {@link Credentials}
+	 */
+	public static SerializableCredentialsProvider credentialsProviderFromEnvironmentVariables() throws IOException {
+		Credentials credentials = defaultCredentialsProviderBuilder().build().getCredentials();
+		return new SerializableCredentialsProvider(credentials);
+	}
+
+	/**
+	 * Creates a SerializableCredentialsProvider for a PubSubSubscription without any credentials. {@link com.google.cloud.pubsub.v1.SubscriptionAdminSettings}
+	 * This is ONLY useful when running tests locally against Mockito or the Google PubSub emulator
+	 * @see <a href="https://cloud.google.com/pubsub/docs/emulator" target="_top">https://cloud.google.com/pubsub/docs/emulator</a>.
+	 * @return serializableCredentialsProvider
+	 * @throws IOException thrown by {@link Credentials}
+	 */
+	public static SerializableCredentialsProvider withoutCredentials() {
+		return new SerializableCredentialsProvider(NoCredentials.getInstance());
+	}
+
+	@Override
+	public Credentials getCredentials() {
+		return credentials;
+	}
+}
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundTest.java
new file mode 100644
index 0000000..f98731b
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundTest.java
@@ -0,0 +1,112 @@
+package org.apache.flink.streaming.connectors.pubsub;
+
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+
+/**
+ * Test for {@link Bound}.
+ */
+public class BoundTest {
+	private SourceFunction<Object> sourceFunction = mock(SourceFunction.class);
+
+	@Test
+	public void testNoShutdownBeforeCounterLimit() {
+		Bound<Object> bound = Bound.boundByAmountOfMessages(10);
+		bound.start(sourceFunction);
+		sleep(150L);
+
+		bound.receivedMessage();
+		verifyZeroInteractions(sourceFunction);
+	}
+
+	@Test
+	public void testShutdownOnCounterLimit() {
+		Bound<Object> bound = Bound.boundByAmountOfMessages(3);
+		bound.start(sourceFunction);
+
+		bound.receivedMessage();
+		bound.receivedMessage();
+		bound.receivedMessage();
+
+		verify(sourceFunction, times(1)).cancel();
+	}
+
+	@Test
+	public void testNoShutdownBeforeTimerLimit() {
+		Bound<Object> bound = Bound.boundByTimeSinceLastMessage(1000L);
+		bound.start(sourceFunction);
+		for (int i = 0; i < 10; i++) {
+			bound.receivedMessage();
+		}
+
+		verifyZeroInteractions(sourceFunction);
+	}
+
+	@Test
+	public void testShutdownAfterTimerLimitNoMessageReceived() {
+		Bound<Object> bound = Bound.boundByTimeSinceLastMessage(100L);
+		bound.start(sourceFunction);
+		sleep(250L);
+		verify(sourceFunction, times(1)).cancel();
+	}
+
+	@Test
+	public void testShutdownAfterTimerLimitAfterMessageReceived() {
+		Bound<Object> bound = Bound.boundByTimeSinceLastMessage(100L);
+		bound.start(sourceFunction);
+		sleep(50L);
+
+		bound.receivedMessage();
+		sleep(50L);
+		verifyZeroInteractions(sourceFunction);
+
+		sleep(200L);
+		verify(sourceFunction, times(1)).cancel();
+	}
+
+	@Test
+	public void testCounterOrTimerMaxMessages() {
+		Bound<Object> bound = Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(3, 1000L);
+		bound.start(sourceFunction);
+
+		bound.receivedMessage();
+		bound.receivedMessage();
+		bound.receivedMessage();
+
+		verify(sourceFunction, times(1)).cancel();
+	}
+
+	@Test
+	public void testCounterOrTimerTimerElapsed() {
+		Bound<Object> bound = Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(1L, 100L);
+		bound.start(sourceFunction);
+		sleep(200L);
+		verify(sourceFunction, times(1)).cancel();
+	}
+
+	@Test(expected = IllegalStateException.class)
+	public void testExceptionThrownIfStartNotCalled() {
+		Bound<Object> bound = Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(1L, 100L);
+		bound.receivedMessage();
+	}
+
+	@Test(expected = IllegalStateException.class)
+	public void testExceptionThrownIfStartCalledTwice() {
+		Bound<Object> bound = Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(1L, 100L);
+		bound.start(sourceFunction);
+		bound.start(sourceFunction);
+	}
+
+	private void sleep(long sleepTime) {
+		try {
+			Thread.sleep(sleepTime);
+		} catch (InterruptedException e) {
+		}
+	}
+}
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSourceTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSourceTest.java
new file mode 100644
index 0000000..805f823
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSourceTest.java
@@ -0,0 +1,70 @@
+package org.apache.flink.streaming.connectors.pubsub;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+
+import com.google.cloud.pubsub.v1.AckReplyConsumer;
+import com.google.protobuf.ByteString;
+import com.google.pubsub.v1.PubsubMessage;
+import org.junit.Test;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for {@link BoundedPubSubSource}.
+ */
+public class BoundedPubSubSourceTest {
+	private final Bound<Object> bound = mock(Bound.class);
+	private final SubscriberWrapper subscriberWrapper = mock(SubscriberWrapper.class);
+	private final SourceFunction.SourceContext<Object> sourceContext = mock(SourceFunction.SourceContext.class);
+	private final AckReplyConsumer ackReplyConsumer = mock(AckReplyConsumer.class);
+	private final DeserializationSchema<Object> deserializationSchema = mock(DeserializationSchema.class);
+
+	private FunctionInitializationContext functionInitializationContext = mock(FunctionInitializationContext.class);
+	private OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
+	private StreamingRuntimeContext streamingRuntimeContext = mock(StreamingRuntimeContext.class);
+
+	@Test
+	public void testBoundIsUsed() throws Exception {
+		BoundedPubSubSource<Object> boundedPubSubSource = createAndInitializeBoundedPubSubSource();
+		boundedPubSubSource.setBound(bound);
+
+		boundedPubSubSource.run(sourceContext);
+		verify(bound, times(1)).start(boundedPubSubSource);
+
+		boundedPubSubSource.receiveMessage(pubSubMessage(), ackReplyConsumer);
+		verify(bound, times(1)).receivedMessage();
+	}
+
+	private BoundedPubSubSource<Object> createAndInitializeBoundedPubSubSource() throws Exception {
+		when(sourceContext.getCheckpointLock()).thenReturn(new Object());
+		when(functionInitializationContext.getOperatorStateStore()).thenReturn(operatorStateStore);
+		when(operatorStateStore.getSerializableListState(any(String.class))).thenReturn(null);
+		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
+
+		BoundedPubSubSource<Object> boundedPubSubSource = BoundedPubSubSource.newBuilder()
+			.withoutCredentials()
+			.withSubscriberWrapper(subscriberWrapper)
+			.withDeserializationSchema(deserializationSchema)
+			.build();
+		boundedPubSubSource.initializeState(functionInitializationContext);
+		boundedPubSubSource.setRuntimeContext(streamingRuntimeContext);
+		boundedPubSubSource.open(null);
+
+		return boundedPubSubSource;
+	}
+
+	private PubsubMessage pubSubMessage() {
+		return PubsubMessage.newBuilder()
+				.setMessageId("message-id")
+				.setData(ByteString.copyFrom("some-message".getBytes()))
+				.build();
+	}
+}
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/PubSubSourceTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/PubSubSourceTest.java
new file mode 100644
index 0000000..73ca53b
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/PubSubSourceTest.java
@@ -0,0 +1,148 @@
+/*
+ * 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.flink.streaming.connectors.pubsub;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+
+import com.google.cloud.pubsub.v1.AckReplyConsumer;
+import com.google.protobuf.ByteString;
+import com.google.pubsub.v1.PubsubMessage;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
+
+
+/**
+ * Test for {@link SourceFunction}.
+ */
+@RunWith(MockitoJUnitRunner.class)
+public class PubSubSourceTest {
+	private static final String MESSAGE = "Message";
+	private static final byte[] SERIALIZED_MESSAGE = MESSAGE.getBytes();
+	@Mock
+	private SubscriberWrapper subscriberWrapper;
+	@Mock
+	private org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext<String> sourceContext;
+	@Mock
+	private DeserializationSchema<String> deserializationSchema;
+	@Mock
+	private AckReplyConsumer ackReplyConsumer;
+	@Mock
+	private StreamingRuntimeContext streamingRuntimeContext;
+	@Mock
+	private RuntimeContext runtimeContext;
+	@Mock
+	private OperatorStateStore operatorStateStore;
+	@Mock
+	private FunctionInitializationContext functionInitializationContext;
+
+	@Test
+	public void testOpenWithCheckpointing() throws Exception {
+		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
+
+		PubSubSource<String> pubSubSource = createTestSource();
+		pubSubSource.setRuntimeContext(streamingRuntimeContext);
+		pubSubSource.open(null);
+
+		verify(subscriberWrapper, times(1)).initialize(pubSubSource);
+	}
+
+	@Test
+	public void testRun() throws IOException {
+		PubSubSource<String> pubSubSource = createTestSource();
+		pubSubSource.run(sourceContext);
+
+		verify(subscriberWrapper, times(1)).startBlocking();
+	}
+
+	@Test
+	public void testWithCheckpoints() throws Exception {
+		when(deserializationSchema.deserialize(SERIALIZED_MESSAGE)).thenReturn(MESSAGE);
+		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
+		when(sourceContext.getCheckpointLock()).thenReturn("some object to lock on");
+		when(functionInitializationContext.getOperatorStateStore()).thenReturn(operatorStateStore);
+		when(operatorStateStore.getSerializableListState(any(String.class))).thenReturn(null);
+
+		PubSubSource<String> pubSubSource = createTestSource();
+		pubSubSource.initializeState(functionInitializationContext);
+		pubSubSource.setRuntimeContext(streamingRuntimeContext);
+		pubSubSource.open(null);
+		verify(subscriberWrapper, times(1)).initialize(pubSubSource);
+
+		pubSubSource.run(sourceContext);
+
+		pubSubSource.receiveMessage(pubSubMessage(), ackReplyConsumer);
+
+		verify(sourceContext, times(1)).getCheckpointLock();
+		verify(sourceContext, times(1)).collect(MESSAGE);
+		verifyZeroInteractions(ackReplyConsumer);
+	}
+
+	@Test
+	public void testMessagesAcknowledged() throws Exception {
+		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
+
+		PubSubSource<String> pubSubSource = createTestSource();
+		pubSubSource.setRuntimeContext(streamingRuntimeContext);
+		pubSubSource.open(null);
+
+		List<AckReplyConsumer> input = Collections.singletonList(ackReplyConsumer);
+
+		pubSubSource.acknowledgeSessionIDs(input);
+
+		verify(ackReplyConsumer, times(1)).ack();
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testOnceWithoutCheckpointing() throws Exception {
+		PubSubSource<String> pubSubSource = createTestSource();
+		pubSubSource.setRuntimeContext(runtimeContext);
+
+		pubSubSource.open(null);
+	}
+
+	private PubSubSource<String> createTestSource() throws IOException {
+		return PubSubSource.<String>newBuilder()
+			.withoutCredentials()
+			.withSubscriberWrapper(subscriberWrapper)
+			.withDeserializationSchema(deserializationSchema)
+			.build();
+	}
+
+	private PubsubMessage pubSubMessage() {
+		return PubsubMessage.newBuilder()
+							.setData(ByteString.copyFrom(SERIALIZED_MESSAGE))
+							.build();
+	}
+}
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapperTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapperTest.java
new file mode 100644
index 0000000..f5deb07
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapperTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.flink.streaming.connectors.pubsub;
+
+import org.apache.flink.streaming.connectors.pubsub.common.SerializableCredentialsProvider;
+
+import com.google.cloud.pubsub.v1.MessageReceiver;
+import com.google.pubsub.v1.ProjectSubscriptionName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import static org.apache.flink.api.java.ClosureCleaner.ensureSerializable;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+
+/**
+ * Tests for {@link SubscriberWrapper}.
+ */
+@RunWith(MockitoJUnitRunner.class)
+public class SubscriberWrapperTest {
+	@Mock
+	private SerializableCredentialsProvider credentialsProvider;
+
+	@Mock
+	private MessageReceiver messageReceiver;
+
+	@Test
+	public void testSerializedSubscriberBuilder() throws Exception {
+		SubscriberWrapper factory = new SubscriberWrapper(SerializableCredentialsProvider.withoutCredentials(), ProjectSubscriptionName.of("projectId", "subscriptionId"));
+		ensureSerializable(factory);
+	}
+
+	@Test
+	public void testInitialisation() {
+		SubscriberWrapper factory = new SubscriberWrapper(credentialsProvider, ProjectSubscriptionName.of("projectId", "subscriptionId"));
+		factory.initialize(messageReceiver);
+
+		assertThat(factory.getSubscriber().getSubscriptionNameString(), is(ProjectSubscriptionName.format("projectId", "subscriptionId")));
+	}
+}
diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml
index e6d601d..1c47830 100644
--- a/flink-connectors/pom.xml
+++ b/flink-connectors/pom.xml
@@ -56,6 +56,7 @@ under the License.
 		<module>flink-connector-cassandra</module>
 		<module>flink-connector-filesystem</module>
 		<module>flink-connector-kafka</module>
+		<module>flink-connector-pubsub</module>
 	</modules>
 
 	<!-- override these root dependencies as 'provided', so they don't end up
diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml
index c1b52a4..888399e 100644
--- a/flink-examples/flink-examples-streaming/pom.xml
+++ b/flink-examples/flink-examples-streaming/pom.xml
@@ -64,6 +64,17 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-pubsub_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>com.google.cloud</groupId>
+			<artifactId>google-cloud-pubsub</artifactId>
+			<version>1.31.0</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-shaded-jackson</artifactId>
 		</dependency>
 
@@ -364,6 +375,7 @@ under the License.
 							</includes>
 						</configuration>
 					</execution>
+
 				</executions>
 			</plugin>
 
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/IntegerSerializer.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/IntegerSerializer.java
new file mode 100644
index 0000000..6e9d1d5
--- /dev/null
+++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/IntegerSerializer.java
@@ -0,0 +1,48 @@
+/*
+ * 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.flink.streaming.examples.pubsub;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import java.io.IOException;
+import java.math.BigInteger;
+
+class IntegerSerializer implements DeserializationSchema<Integer>, SerializationSchema<Integer> {
+
+	@Override
+	public Integer deserialize(byte[] bytes) throws IOException {
+		return new BigInteger(bytes).intValue();
+	}
+
+	@Override
+	public boolean isEndOfStream(Integer integer) {
+		return false;
+	}
+
+	@Override
+	public TypeInformation<Integer> getProducedType() {
+		return TypeInformation.of(Integer.class);
+	}
+
+	@Override
+	public byte[] serialize(Integer integer) {
+		return BigInteger.valueOf(integer).toByteArray();
+	}
+}
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubExample.java
new file mode 100644
index 0000000..612da41
--- /dev/null
+++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubExample.java
@@ -0,0 +1,84 @@
+/*
+ * 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.flink.streaming.examples.pubsub;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pubsub.PubSubSink;
+import org.apache.flink.streaming.connectors.pubsub.PubSubSourceBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A simple PubSub example.
+ *
+ * <p>Before starting a flink job it will publish 10 messages on the input topic.
+ *
+ * Then a flink job is started to read these 10 messages from the input-subscription,
+ * it will print them to stdout
+ * and then write them to a the output-topic.</p>
+ */
+public class PubSubExample {
+	private static final Logger LOG = LoggerFactory.getLogger(PubSubExample.class);
+
+	public static void main(String[] args) throws Exception {
+		// parse input arguments
+		final ParameterTool parameterTool = ParameterTool.fromArgs(args);
+
+		if (parameterTool.getNumberOfParameters() < 3) {
+			System.out.println("Missing parameters!\n" +
+								"Usage: flink run PubSub.jar --input-subscription <subscription> --input-topicName <topic> --output-topicName " +
+								"--google-project <google project name> ");
+			//return;
+		}
+
+		String projectName = parameterTool.getRequired("google-project");
+		String inputTopicName = parameterTool.getRequired("input-topicName");
+		String subscriptionName = parameterTool.getRequired("input-subscription");
+		String outputTopicName = parameterTool.getRequired("output-topicName");
+
+		PubSubPublisher pubSubPublisher = new PubSubPublisher(projectName, inputTopicName);
+
+		pubSubPublisher.publish();
+		runFlinkJob(projectName, subscriptionName, outputTopicName);
+	}
+
+	private static void runFlinkJob(String projectName, String subscriptionName, String outputTopicName) throws Exception {
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		env.addSource(PubSubSourceBuilder.<Integer>builder()
+										.withProjectSubscriptionName(projectName, subscriptionName)
+										.withDeserializationSchema(new IntegerSerializer())
+										.withMode(PubSubSourceBuilder.Mode.NONE)
+										.build())
+			.map(PubSubExample::printAndReturn).disableChaining()
+			.addSink(PubSubSink.<Integer>newBuilder()
+							.withProjectName(projectName)
+							.withTopicName(outputTopicName)
+							.withSerializationSchema(new IntegerSerializer())
+							.build());
+
+		env.execute("Flink Streaming PubSubReader");
+	}
+
+	private static Integer printAndReturn(Integer i) {
+		LOG.info("Processed message with payload: " + i);
+		return i;
+	}
+}
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubPublisher.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubPublisher.java
new file mode 100644
index 0000000..7507945
--- /dev/null
+++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubPublisher.java
@@ -0,0 +1,64 @@
+/*
+ * 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.flink.streaming.examples.pubsub;
+
+import com.google.api.core.ApiFuture;
+import com.google.cloud.pubsub.v1.Publisher;
+import com.google.protobuf.ByteString;
+import com.google.pubsub.v1.ProjectTopicName;
+import com.google.pubsub.v1.PubsubMessage;
+
+import java.math.BigInteger;
+
+class PubSubPublisher {
+	private final String projectName;
+	private final String topicName;
+
+	PubSubPublisher(String projectName, String topicName) {
+		this.projectName = projectName;
+		this.topicName = topicName;
+	}
+
+	void publish() {
+		Publisher publisher = null;
+		try {
+			publisher = Publisher.newBuilder(ProjectTopicName.of(projectName, topicName)).build();
+			long counter = 0;
+			while (counter < 10) {
+				ByteString messageData = ByteString.copyFrom(BigInteger.valueOf(counter).toByteArray());
+				PubsubMessage message = PubsubMessage.newBuilder().setData(messageData).build();
+
+				ApiFuture<String> future = publisher.publish(message);
+				future.get();
+				System.out.println("Published message: " + counter);
+				Thread.sleep(100L);
+
+				counter++;
+			}
+		} catch (Exception e) {
+			throw new RuntimeException(e);
+		} finally {
+			try {
+				if (publisher != null) {
+					publisher.shutdown();
+				}
+			} catch (Exception e) {
+			}
+		}
+	}
+}


[flink] 02/05: [FLINK-9311] [pubsub] Add unit and integration tests for PubSub connectors

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit a96277090358d2237aab7426ebde7fdf1eaccbe4
Author: Niels Basjes <nb...@bol.com>
AuthorDate: Wed Aug 15 14:05:39 2018 +0200

    [FLINK-9311] [pubsub] Add unit and integration tests for PubSub connectors
---
 flink-connectors/flink-connector-pubsub/pom.xml    |  86 +++----
 .../flink/streaming/connectors/pubsub/Bound.java   |  17 ++
 .../connectors/pubsub/BoundedPubSubSource.java     |  33 ++-
 .../streaming/connectors/pubsub/PubSubSink.java    | 141 +++++------
 .../streaming/connectors/pubsub/PubSubSource.java  |  40 ++--
 .../connectors/pubsub/SubscriberWrapper.java       |  22 +-
 .../common/SerializableCredentialsProvider.java    |  11 +-
 .../streaming/connectors/pubsub/BoundTest.java     |  20 +-
 .../connectors/pubsub/BoundedPubSubSourceTest.java |  25 +-
 .../connectors/pubsub/PubSubSourceTest.java        |   5 +-
 .../src/test/resources/log4j-test.properties       |  24 ++
 .../flink-connector-pubsub-emulator-tests}/pom.xml | 112 +++++----
 .../connectors/pubsub/CheckPubSubEmulatorTest.java | 115 +++++++++
 .../connectors/pubsub/EmulatedPubSubSinkTest.java  | 109 +++++++++
 .../pubsub/EmulatedPubSubSourceTest.java           | 116 +++++++++
 .../pubsub/emulator/GCloudEmulatorManager.java     | 264 +++++++++++++++++++++
 .../pubsub/emulator/GCloudUnitTestBase.java        |  84 +++++++
 .../connectors/pubsub/emulator/PubsubHelper.java   | 232 ++++++++++++++++++
 .../src/test/resources/log4j-test.properties       |  24 ++
 flink-end-to-end-tests/pom.xml                     |   1 +
 flink-end-to-end-tests/run-nightly-tests.sh        |   2 +
 .../test-scripts/test_streaming_pubsub.sh          |  22 ++
 .../streaming/examples/pubsub/PubSubExample.java   |   7 +-
 23 files changed, 1309 insertions(+), 203 deletions(-)

diff --git a/flink-connectors/flink-connector-pubsub/pom.xml b/flink-connectors/flink-connector-pubsub/pom.xml
index a6e8d72..f50cbdd 100644
--- a/flink-connectors/flink-connector-pubsub/pom.xml
+++ b/flink-connectors/flink-connector-pubsub/pom.xml
@@ -35,12 +35,20 @@ under the License.
 
 	<packaging>jar</packaging>
 
-	<properties>
-		<pubsub.version>1.37.1</pubsub.version>
-	</properties>
+	<!-- This is the way we get a consistent set of versions of the Google tools -->
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>com.google.cloud</groupId>
+				<artifactId>google-cloud-bom</artifactId>
+				<version>0.53.0-alpha</version>
+				<type>pom</type>
+				<scope>import</scope>
+			</dependency>
+		</dependencies>
+	</dependencyManagement>
 
 	<dependencies>
-
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
@@ -51,7 +59,29 @@ under the License.
 		<dependency>
 			<groupId>com.google.cloud</groupId>
 			<artifactId>google-cloud-pubsub</artifactId>
-			<version>${pubsub.version}</version>
+			<!-- Version is pulled from google-cloud-bom -->
+			<exclusions>
+				<!-- Exclude an old version of guava that is being pulled
+                in by a transitive dependency of google-api-client -->
+				<exclusion>
+					<groupId>com.google.guava</groupId>
+					<artifactId>guava-jdk5</artifactId>
+				</exclusion>
+			</exclusions>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<version>${slf4j.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-log4j12</artifactId>
+			<version>${slf4j.version}</version>
+			<scope>test</scope>
 		</dependency>
 
 		<dependency>
@@ -69,51 +99,5 @@ under the License.
 			<type>test-jar</type>
 			<scope>test</scope>
 		</dependency>
-
 	</dependencies>
-
-    <build>
-        <plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>shade-flink</id>
-						<phase>package</phase>
-						<goals>
-							<goal>shade</goal>
-						</goals>
-						<configuration>
-							<shadeTestJar>false</shadeTestJar>
-							<artifactSet>
-								<includes>
-									<include>*:*</include>
-								</includes>
-							</artifactSet>
-							<relocations>
-								<relocation>
-									<pattern>com.google.guava</pattern>
-									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.com.google.guava</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>com.google.common.base</pattern>
-									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.com.google.common.base</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>io.grpc.auth</pattern>
-									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.io.grpc.auth</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>io.grpc.protobuf</pattern>
-									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.io.grpc.protobuf</shadedPattern>
-								</relocation>
-							</relocations>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-        </plugins>
-    </build>
-
 </project>
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java
index b37cc45..727f32e 100644
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.flink.streaming.connectors.pubsub;
 
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSource.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSource.java
index 5f829ae..83fc15e 100644
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSource.java
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSource.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.flink.streaming.connectors.pubsub;
 
 import com.google.cloud.pubsub.v1.AckReplyConsumer;
@@ -5,7 +22,11 @@ import com.google.pubsub.v1.PubsubMessage;
 
 import java.io.IOException;
 
-class BoundedPubSubSource<OUT> extends PubSubSource<OUT> {
+/**
+ * A bounded PubSub Source, similar to {@link PubSubSource} but this will stop at some point. For example after a period of idle or and after n amount of messages have been received.
+ *
+ */
+public class BoundedPubSubSource<OUT> extends PubSubSource<OUT> {
 	private Bound<OUT> bound;
 
 	private BoundedPubSubSource() {
@@ -28,11 +49,19 @@ class BoundedPubSubSource<OUT> extends PubSubSource<OUT> {
 		bound.receivedMessage();
 	}
 
+	/**
+	 * Creates a {@link BoundedPubSubSourceBuilder}.
+	 * @param <OUT> Type of Object which will be read by the produced {@link BoundedPubSubSource}
+	 */
 	@SuppressWarnings("unchecked")
 	public static <OUT> BoundedPubSubSourceBuilder<OUT, ? extends PubSubSource, ? extends BoundedPubSubSourceBuilder> newBuilder() {
 		return new BoundedPubSubSourceBuilder<>(new BoundedPubSubSource<OUT>());
 	}
 
+	/**
+	 * Builder to create BoundedPubSubSource.
+	 * @param <OUT> Type of Object which will be read by the BoundedPubSubSource
+	 */
 	@SuppressWarnings("unchecked")
 	public static class BoundedPubSubSourceBuilder<OUT, PSS extends BoundedPubSubSource<OUT>, BUILDER extends BoundedPubSubSourceBuilder<OUT, PSS, BUILDER>> extends PubSubSourceBuilder<OUT, PSS, BUILDER> {
 		private Long boundedByAmountOfMessages;
@@ -52,7 +81,7 @@ class BoundedPubSubSource<OUT> extends PubSubSource<OUT> {
 			return (BUILDER) this;
 		}
 
-		private Bound <OUT> createBound() {
+		private Bound<OUT> createBound() {
 			if (boundedByAmountOfMessages != null && boundedByTimeSinceLastMessage != null) {
 				return Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(boundedByAmountOfMessages, boundedByTimeSinceLastMessage);
 			}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSink.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSink.java
index 92c9c6c..e6ac53e 100644
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSink.java
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSink.java
@@ -17,7 +17,6 @@
 
 package org.apache.flink.streaming.connectors.pubsub;
 
-import com.google.api.gax.core.NoCredentialsProvider;
 import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
@@ -40,22 +39,35 @@ import java.io.IOException;
 
 /**
  * A sink function that outputs to PubSub.
+ *
  * @param <IN> type of PubSubSink messages to write
  */
 public class PubSubSink<IN> extends RichSinkFunction<IN> {
 
-	private final SerializableCredentialsProvider serializableCredentialsProvider;
-	private final SerializationSchema<IN>         serializationSchema;
-	private final String                          projectName;
-	private final String                          topicName;
-	private       String                          hostAndPort = null;
+	private SerializableCredentialsProvider serializableCredentialsProvider;
+	private SerializationSchema<IN> serializationSchema;
+	private String projectName;
+	private String topicName;
+	private String hostAndPort = null;
 
 	private transient Publisher publisher;
 
-	public PubSubSink(SerializableCredentialsProvider serializableCredentialsProvider, SerializationSchema<IN> serializationSchema, String projectName, String topicName) {
+	private PubSubSink() {
+	}
+
+	void setSerializableCredentialsProvider(SerializableCredentialsProvider serializableCredentialsProvider) {
 		this.serializableCredentialsProvider = serializableCredentialsProvider;
+	}
+
+	void setSerializationSchema(SerializationSchema<IN> serializationSchema) {
 		this.serializationSchema = serializationSchema;
+	}
+
+	void setProjectName(String projectName) {
 		this.projectName = projectName;
+	}
+
+	void setTopicName(String topicName) {
 		this.topicName = topicName;
 	}
 
@@ -64,15 +76,29 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> {
 	 * The ONLY reason to use this is during tests with the emulator provided by Google.
 	 *
 	 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
-	 * @return The current instance
 	 */
-	public PubSubSink<IN> withHostAndPort(String hostAndPort) {
+	void withHostAndPort(String hostAndPort) {
 		this.hostAndPort = hostAndPort;
-		return this;
 	}
 
-	private transient ManagedChannel   managedChannel = null;
-	private transient TransportChannel channel        = null;
+	void initialize() throws IOException {
+		if (serializableCredentialsProvider == null) {
+			serializableCredentialsProvider = SerializableCredentialsProvider.credentialsProviderFromEnvironmentVariables();
+		}
+		if (serializationSchema == null) {
+			throw new IllegalArgumentException("The serializationSchema has not been specified.");
+		}
+		if (projectName == null) {
+			throw new IllegalArgumentException("The projectName has not been specified.");
+		}
+		if (topicName == null) {
+			throw new IllegalArgumentException("The topicName has not been specified.");
+		}
+	}
+
+
+	private transient ManagedChannel managedChannel = null;
+	private transient TransportChannel channel = null;
 
 	@Override
 	public void open(Configuration configuration) throws Exception {
@@ -114,127 +140,110 @@ public class PubSubSink<IN> extends RichSinkFunction<IN> {
 
 	/**
 	 * Create a builder for a new PubSubSink.
+	 *
 	 * @param <IN> The generic of the type that is to be written into the sink.
 	 * @return a new PubSubSinkBuilder instance
 	 */
-	public static <IN> PubSubSinkBuilder<IN> newBuilder() {
-		return new PubSubSinkBuilder<>();
+	public static <IN> PubSubSinkBuilder<IN, ? extends PubSubSink<IN>, ? extends PubSubSinkBuilder<IN, ?, ?>> newBuilder() {
+		return new PubSubSinkBuilder<>(new PubSubSink<>());
 	}
 
 	/**
 	 * PubSubSinkBuilder to create a PubSubSink.
+	 *
 	 * @param <IN> Type of PubSubSink to create.
 	 */
-	public static class PubSubSinkBuilder<IN> {
-		private SerializableCredentialsProvider serializableCredentialsProvider = null;
-		private SerializationSchema<IN>         serializationSchema             = null;
-		private String                          projectName                     = null;
-		private String                          topicName                       = null;
-		private String                          hostAndPort                     = null;
-
-		private PubSubSinkBuilder() {
+	@SuppressWarnings("unchecked")
+	public static class PubSubSinkBuilder<IN, PSS extends PubSubSink<IN>, BUILDER extends PubSubSinkBuilder<IN, PSS, BUILDER>> {
+		protected PSS sinkUnderConstruction;
+
+		private PubSubSinkBuilder(PSS sinkUnderConstruction) {
+			this.sinkUnderConstruction = sinkUnderConstruction;
 		}
 
 		/**
 		 * Set the credentials.
 		 * If this is not used then the credentials are picked up from the environment variables.
+		 *
 		 * @param credentials the Credentials needed to connect.
 		 * @return The current PubSubSinkBuilder instance
 		 */
-		public PubSubSinkBuilder<IN> withCredentials(Credentials credentials) {
-			this.serializableCredentialsProvider = new SerializableCredentialsProvider(credentials);
-			return this;
+		public BUILDER withCredentials(Credentials credentials) {
+			sinkUnderConstruction.setSerializableCredentialsProvider(new SerializableCredentialsProvider(credentials));
+			return (BUILDER) this;
 		}
 
 		/**
 		 * Set the CredentialsProvider.
 		 * If this is not used then the credentials are picked up from the environment variables.
+		 *
 		 * @param credentialsProvider the custom SerializableCredentialsProvider instance.
 		 * @return The current PubSubSinkBuilder instance
 		 */
-		public PubSubSinkBuilder<IN> withCredentialsProvider(CredentialsProvider credentialsProvider) throws IOException {
+		public BUILDER withCredentialsProvider(CredentialsProvider credentialsProvider) throws IOException {
 			return withCredentials(credentialsProvider.getCredentials());
 		}
 
 		/**
 		 * Set the credentials to be absent.
 		 * This means that no credentials are to be used at all.
+		 *
 		 * @return The current PubSubSinkBuilder instance
 		 */
-		public PubSubSinkBuilder<IN> withoutCredentials() {
-			this.serializableCredentialsProvider = SerializableCredentialsProvider.withoutCredentials();
-			return this;
+		public BUILDER withoutCredentials() {
+			sinkUnderConstruction.setSerializableCredentialsProvider(SerializableCredentialsProvider.withoutCredentials());
+			return (BUILDER) this;
 		}
 
 		/**
 		 * @param serializationSchema Instance of a SerializationSchema that converts the IN into a byte[]
 		 * @return The current PubSubSinkBuilder instance
 		 */
-		public PubSubSinkBuilder<IN> withSerializationSchema(SerializationSchema<IN> serializationSchema) {
-			this.serializationSchema = serializationSchema;
-			return this;
+		public BUILDER withSerializationSchema(SerializationSchema<IN> serializationSchema) {
+			sinkUnderConstruction.setSerializationSchema(serializationSchema);
+			return (BUILDER) this;
 		}
 
 		/**
 		 * @param projectName The name of the project in PubSub
 		 * @return The current PubSubSinkBuilder instance
 		 */
-		public PubSubSinkBuilder<IN> withProjectName (String projectName) {
-			this.projectName = projectName;
-			return this;
+		public BUILDER withProjectName(String projectName) {
+			sinkUnderConstruction.setProjectName(projectName);
+			return (BUILDER) this;
 		}
 
 		/**
 		 * @param topicName The name of the topic in PubSub
 		 * @return The current PubSubSinkBuilder instance
 		 */
-		public PubSubSinkBuilder<IN> withTopicName (String topicName) {
-			this.topicName = topicName;
-			return this;
+		public BUILDER withTopicName(String topicName) {
+			sinkUnderConstruction.setTopicName(topicName);
+			return (BUILDER) this;
 		}
 
-
 		/**
 		 * Set the custom hostname/port combination of PubSub.
 		 * The ONLY reason to use this is during tests with the emulator provided by Google.
+		 *
 		 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
 		 * @return The current PubSubSinkBuilder instance
 		 */
-		public PubSubSinkBuilder<IN> withHostAndPort(String hostAndPort) {
-			this.hostAndPort = hostAndPort;
-			return this;
+		public BUILDER withHostAndPort(String hostAndPort) {
+			sinkUnderConstruction.withHostAndPort(hostAndPort);
+			return (BUILDER) this;
 		}
 
 		/**
 		 * Actually builder the desired instance of the PubSubSink.
+		 *
 		 * @return a brand new PubSubSink
-		 * @throws IOException incase of a problem getting the credentials
+		 * @throws IOException              incase of a problem getting the credentials
 		 * @throws IllegalArgumentException incase required fields were not specified.
 		 */
 		public PubSubSink<IN> build() throws IOException {
-			if (serializableCredentialsProvider == null) {
-				serializableCredentialsProvider = SerializableCredentialsProvider.credentialsProviderFromEnvironmentVariables();
-			}
-			if (serializationSchema == null) {
-				throw new IllegalArgumentException("The serializationSchema has not been specified.");
-			}
-			if (projectName == null) {
-				throw new IllegalArgumentException("The projectName has not been specified.");
-			}
-			if (topicName == null) {
-				throw new IllegalArgumentException("The topicName has not been specified.");
-			}
-
-			PubSubSink<IN> pubSubSink = new PubSubSink<>(
-				serializableCredentialsProvider,
-				serializationSchema,
-				projectName, topicName);
-
-			if (hostAndPort != null) {
-				pubSubSink.withHostAndPort(hostAndPort);
-			}
-
-			return pubSubSink;
+			sinkUnderConstruction.initialize();
+			return sinkUnderConstruction;
 		}
 	}
 
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java
index 8f8c689..2d93998 100644
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java
@@ -43,7 +43,7 @@ import java.util.List;
  */
 public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase<OUT, String, AckReplyConsumer> implements MessageReceiver, ResultTypeQueryable<OUT>, ParallelSourceFunction<OUT> {
 	private DeserializationSchema<OUT> deserializationSchema;
-	private SubscriberWrapper          subscriberWrapper;
+	private SubscriberWrapper subscriberWrapper;
 
 	protected transient SourceContext<OUT> sourceContext = null;
 
@@ -64,7 +64,8 @@ public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase
 		super.open(configuration);
 		subscriberWrapper.initialize(this);
 		if (hasNoCheckpointingEnabled(getRuntimeContext())) {
-			throw new IllegalArgumentException("Checkpointing needs to be enabled to support: PubSub ATLEAST_ONCE");
+			throw new IllegalArgumentException("The PubSubSource REQUIRES Checkpointing to be enabled and " +
+				"the checkpointing frequency must be MUCH lower than the PubSub timeout for it to retry a message.");
 		}
 	}
 
@@ -123,26 +124,27 @@ public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase
 		return deserializationSchema.getProducedType();
 	}
 
-	@SuppressWarnings("unchecked")
-	public static <OUT> PubSubSourceBuilder<OUT, ? extends PubSubSource, ? extends PubSubSourceBuilder> newBuilder() {
-		return new PubSubSourceBuilder<>(new PubSubSource<OUT>());
+	public static <OUT> PubSubSourceBuilder<OUT, ? extends PubSubSource<OUT>, ? extends PubSubSourceBuilder<OUT, ?, ?>> newBuilder() {
+		return new PubSubSourceBuilder<>(new PubSubSource<>());
 	}
 
 	/**
 	 * Builder to create PubSubSource.
-	 * @param <OUT> The type of objects which will be read
-	 * @param <PSS> The type of PubSubSource
+	 *
+	 * @param <OUT>     The type of objects which will be read
+	 * @param <PSS>     The type of PubSubSource
 	 * @param <BUILDER> The type of Builder to create the PubSubSource
 	 */
+	@SuppressWarnings("unchecked")
 	public static class PubSubSourceBuilder<OUT, PSS extends PubSubSource<OUT>, BUILDER extends PubSubSourceBuilder<OUT, PSS, BUILDER>> {
-		protected PSS 							sourceUnderConstruction;
+		protected PSS sourceUnderConstruction;
 
-		private SubscriberWrapper               subscriberWrapper = null;
+		private SubscriberWrapper subscriberWrapper = null;
 		private SerializableCredentialsProvider serializableCredentialsProvider;
-		private DeserializationSchema<OUT>      deserializationSchema;
-		private String                          projectName;
-		private String                          subscriptionName;
-		private String                          hostAndPort;
+		private DeserializationSchema<OUT> deserializationSchema;
+		private String projectName;
+		private String subscriptionName;
+		private String hostAndPort;
 
 		protected PubSubSourceBuilder(PSS sourceUnderConstruction) {
 			this.sourceUnderConstruction = sourceUnderConstruction;
@@ -151,6 +153,7 @@ public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase
 		/**
 		 * Set the credentials.
 		 * If this is not used then the credentials are picked up from the environment variables.
+		 *
 		 * @param credentials the Credentials needed to connect.
 		 * @return The current PubSubSourceBuilder instance
 		 */
@@ -162,6 +165,7 @@ public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase
 		/**
 		 * Set the CredentialsProvider.
 		 * If this is not used then the credentials are picked up from the environment variables.
+		 *
 		 * @param credentialsProvider the custom SerializableCredentialsProvider instance.
 		 * @return The current PubSubSourceBuilder instance
 		 */
@@ -172,6 +176,7 @@ public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase
 		/**
 		 * Set the credentials to be absent.
 		 * This means that no credentials are to be used at all.
+		 *
 		 * @return The current PubSubSourceBuilder instance
 		 */
 		public BUILDER withoutCredentials() {
@@ -183,13 +188,13 @@ public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase
 		 * @param deserializationSchema Instance of a DeserializationSchema that converts the OUT into a byte[]
 		 * @return The current PubSubSourceBuilder instance
 		 */
-		public BUILDER withDeserializationSchema(DeserializationSchema <OUT> deserializationSchema) {
+		public BUILDER withDeserializationSchema(DeserializationSchema<OUT> deserializationSchema) {
 			this.deserializationSchema = deserializationSchema;
 			return (BUILDER) this;
 		}
 
 		/**
-		 * @param projectName The name of the project in GoogleCloudPlatform
+		 * @param projectName      The name of the project in GoogleCloudPlatform
 		 * @param subscriptionName The name of the subscription in PubSub
 		 * @return The current PubSubSourceBuilder instance
 		 */
@@ -202,6 +207,7 @@ public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase
 		/**
 		 * Set the custom hostname/port combination of PubSub.
 		 * The ONLY reason to use this is during tests with the emulator provided by Google.
+		 *
 		 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
 		 * @return The current PubSubSourceBuilder instance
 		 */
@@ -213,6 +219,7 @@ public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase
 		/**
 		 * Set a complete SubscriberWrapper.
 		 * The ONLY reason to use this is during tests.
+		 *
 		 * @param subscriberWrapper The fully instantiated SubscriberWrapper
 		 * @return The current PubSubSourceBuilder instance
 		 */
@@ -223,8 +230,9 @@ public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase
 
 		/**
 		 * Actually build the desired instance of the PubSubSourceBuilder.
+		 *
 		 * @return a brand new SourceFunction
-		 * @throws IOException incase of a problem getting the credentials
+		 * @throws IOException              incase of a problem getting the credentials
 		 * @throws IllegalArgumentException incase required fields were not specified.
 		 */
 		public PSS build() throws IOException {
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapper.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapper.java
index fb75f43..0595877 100644
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapper.java
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapper.java
@@ -33,13 +33,13 @@ import java.io.Serializable;
 
 class SubscriberWrapper implements Serializable {
 	private final SerializableCredentialsProvider serializableCredentialsProvider;
-	private final String                          projectId;
-	private final String                          subscriptionId;
-	private       String                          hostAndPort = null;
+	private final String projectId;
+	private final String subscriptionId;
+	private String hostAndPort = null;
 
-	private transient Subscriber       subscriber;
-	private transient ManagedChannel   managedChannel = null;
-	private transient TransportChannel channel        = null;
+	private transient Subscriber subscriber;
+	private transient ManagedChannel managedChannel = null;
+	private transient TransportChannel channel = null;
 
 	SubscriberWrapper(SerializableCredentialsProvider serializableCredentialsProvider, ProjectSubscriptionName projectSubscriptionName) {
 		this.serializableCredentialsProvider = serializableCredentialsProvider;
@@ -49,14 +49,14 @@ class SubscriberWrapper implements Serializable {
 
 	void initialize(MessageReceiver messageReceiver) {
 		Subscriber.Builder builder = Subscriber
-				.newBuilder(ProjectSubscriptionName.of(projectId, subscriptionId), messageReceiver)
-				.setCredentialsProvider(serializableCredentialsProvider);
+			.newBuilder(ProjectSubscriptionName.of(projectId, subscriptionId), messageReceiver)
+			.setCredentialsProvider(serializableCredentialsProvider);
 
 		if (hostAndPort != null) {
 			managedChannel = ManagedChannelBuilder
-					.forTarget(hostAndPort)
-					.usePlaintext(true) // This is 'Ok' because this is ONLY used for testing.
-					.build();
+				.forTarget(hostAndPort)
+				.usePlaintext(true) // This is 'Ok' because this is ONLY used for testing.
+				.build();
 			channel = GrpcTransportChannel.newBuilder().setManagedChannel(managedChannel).build();
 			builder.setChannelProvider(FixedTransportChannelProvider.create(channel));
 		}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/common/SerializableCredentialsProvider.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/common/SerializableCredentialsProvider.java
index bd04058..44b1fa0 100644
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/common/SerializableCredentialsProvider.java
+++ b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/common/SerializableCredentialsProvider.java
@@ -40,7 +40,9 @@ public class SerializableCredentialsProvider implements CredentialsProvider, Ser
 	}
 
 	/**
-	 * Creates a SerializableCredentialsProvider for a PubSubSubscription based on environment variables. {@link com.google.cloud.pubsub.v1.SubscriptionAdminSettings}
+	 * Creates a SerializableCredentialsProvider for a PubSubSubscription based on environment variables.
+	 * {@link com.google.cloud.pubsub.v1.SubscriptionAdminSettings}
+	 *
 	 * @return serializableCredentialsProvider
 	 * @throws IOException thrown by {@link Credentials}
 	 */
@@ -50,11 +52,12 @@ public class SerializableCredentialsProvider implements CredentialsProvider, Ser
 	}
 
 	/**
-	 * Creates a SerializableCredentialsProvider for a PubSubSubscription without any credentials. {@link com.google.cloud.pubsub.v1.SubscriptionAdminSettings}
+	 * Creates a SerializableCredentialsProvider for a PubSubSubscription without any credentials.
+	 * {@link com.google.cloud.pubsub.v1.SubscriptionAdminSettings}
 	 * This is ONLY useful when running tests locally against Mockito or the Google PubSub emulator
-	 * @see <a href="https://cloud.google.com/pubsub/docs/emulator" target="_top">https://cloud.google.com/pubsub/docs/emulator</a>.
+	 * @see <a href="https://cloud.google.com/pubsub/docs/emulator" target="_top">https://cloud.google.com/pubsub/docs/emulator</a>
 	 * @return serializableCredentialsProvider
-	 * @throws IOException thrown by {@link Credentials}
+	 * @see <a href="https://cloud.google.com/pubsub/docs/emulator" target="_top">https://cloud.google.com/pubsub/docs/emulator</a>
 	 */
 	public static SerializableCredentialsProvider withoutCredentials() {
 		return new SerializableCredentialsProvider(NoCredentials.getInstance());
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundTest.java
index f98731b..a340ae9 100644
--- a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundTest.java
+++ b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundTest.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.flink.streaming.connectors.pubsub;
 
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
@@ -5,9 +22,9 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.junit.Test;
 
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
 
 /**
  * Test for {@link Bound}.
@@ -107,6 +124,7 @@ public class BoundTest {
 		try {
 			Thread.sleep(sleepTime);
 		} catch (InterruptedException e) {
+			// Ignore any exceptions
 		}
 	}
 }
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSourceTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSourceTest.java
index 805f823..5f938fd 100644
--- a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSourceTest.java
+++ b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSourceTest.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.flink.streaming.connectors.pubsub;
 
 import org.apache.flink.api.common.serialization.DeserializationSchema;
@@ -13,9 +30,9 @@ import org.junit.Test;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
 
 /**
  * Tests for {@link BoundedPubSubSource}.
@@ -63,8 +80,8 @@ public class BoundedPubSubSourceTest {
 
 	private PubsubMessage pubSubMessage() {
 		return PubsubMessage.newBuilder()
-				.setMessageId("message-id")
-				.setData(ByteString.copyFrom("some-message".getBytes()))
-				.build();
+			.setMessageId("message-id")
+			.setData(ByteString.copyFrom("some-message".getBytes()))
+			.build();
 	}
 }
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/PubSubSourceTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/PubSubSourceTest.java
index 73ca53b..9db5d7d 100644
--- a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/PubSubSourceTest.java
+++ b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/PubSubSourceTest.java
@@ -42,7 +42,6 @@ import static org.mockito.Mockito.verifyZeroInteractions;
 import static org.mockito.Mockito.when;
 import static org.mockito.internal.verification.VerificationModeFactory.times;
 
-
 /**
  * Test for {@link SourceFunction}.
  */
@@ -142,7 +141,7 @@ public class PubSubSourceTest {
 
 	private PubsubMessage pubSubMessage() {
 		return PubsubMessage.newBuilder()
-							.setData(ByteString.copyFrom(SERIALIZED_MESSAGE))
-							.build();
+			.setData(ByteString.copyFrom(SERIALIZED_MESSAGE))
+			.build();
 	}
 }
diff --git a/flink-connectors/flink-connector-pubsub/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-pubsub/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..b316a9a
--- /dev/null
+++ b/flink-connectors/flink-connector-pubsub/src/test/resources/log4j-test.properties
@@ -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.
+################################################################################
+log4j.rootLogger=INFO, testlogger
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target=System.out
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
diff --git a/flink-connectors/flink-connector-pubsub/pom.xml b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/pom.xml
similarity index 50%
copy from flink-connectors/flink-connector-pubsub/pom.xml
copy to flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/pom.xml
index a6e8d72..7dd0d15 100644
--- a/flink-connectors/flink-connector-pubsub/pom.xml
+++ b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/pom.xml
@@ -25,33 +25,75 @@ under the License.
 
 	<parent>
 		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-connectors</artifactId>
+		<artifactId>flink-end-to-end-tests</artifactId>
 		<version>1.7-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-pubsub_${scala.binary.version}</artifactId>
-	<name>flink-connector-pubsub</name>
+	<artifactId>flink-connector-pubsub-emulator-tests</artifactId>
+	<name>flink-connector-pubsub-emulator-tests</name>
 
 	<packaging>jar</packaging>
 
-	<properties>
-		<pubsub.version>1.37.1</pubsub.version>
-	</properties>
+	<!-- This is the way we get a consistent set of versions of the Google tools -->
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>com.google.cloud</groupId>
+				<artifactId>google-cloud-bom</artifactId>
+				<version>0.53.0-alpha</version>
+				<type>pom</type>
+				<scope>import</scope>
+			</dependency>
+		</dependencies>
+	</dependencyManagement>
 
 	<dependencies>
 
+		<!--All dependencies are   <scope>test</scope> -->
+
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
-			<scope>provided</scope>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-pubsub_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>com.google.cloud</groupId>
 			<artifactId>google-cloud-pubsub</artifactId>
-			<version>${pubsub.version}</version>
+			<!-- Version is pulled from google-cloud-bom -->
+			<exclusions>
+				<!-- Exclude an old version of guava that is being pulled
+                in by a transitive dependency of google-api-client -->
+				<exclusion>
+					<groupId>com.google.guava</groupId>
+					<artifactId>guava-jdk5</artifactId>
+				</exclusion>
+			</exclusions>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-log4j12</artifactId>
+			<version>${slf4j.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- This is used to run the local PubSub -->
+		<dependency>
+			<groupId>com.spotify</groupId>
+			<artifactId>docker-client</artifactId>
+			<version>8.11.7</version>
+			<scope>test</scope>
 		</dependency>
 
 		<dependency>
@@ -69,51 +111,39 @@ under the License.
 			<type>test-jar</type>
 			<scope>test</scope>
 		</dependency>
-
 	</dependencies>
 
-    <build>
-        <plugins>
+	<!-- ONLY run the tests when explicitly told to do so  -->
+	<properties>
+		<skipTests>true</skipTests>
+	</properties>
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<version>2.12.4</version>
+				<configuration>
+					<skipTests>${skipTests}</skipTests>
+				</configuration>
+			</plugin>
+			<!-- Disabling convergence check because there are multiple problems within the used pubsub dependencies -->
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
+				<artifactId>maven-enforcer-plugin</artifactId>
 				<executions>
 					<execution>
-						<id>shade-flink</id>
-						<phase>package</phase>
+						<id>dependency-convergence</id>
 						<goals>
-							<goal>shade</goal>
+							<goal>enforce</goal>
 						</goals>
 						<configuration>
-							<shadeTestJar>false</shadeTestJar>
-							<artifactSet>
-								<includes>
-									<include>*:*</include>
-								</includes>
-							</artifactSet>
-							<relocations>
-								<relocation>
-									<pattern>com.google.guava</pattern>
-									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.com.google.guava</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>com.google.common.base</pattern>
-									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.com.google.common.base</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>io.grpc.auth</pattern>
-									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.io.grpc.auth</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>io.grpc.protobuf</pattern>
-									<shadedPattern>org.apache.flink.streaming.connectors.pubsub.shaded.io.grpc.protobuf</shadedPattern>
-								</relocation>
-							</relocations>
+							<skip>true</skip>
 						</configuration>
 					</execution>
 				</executions>
 			</plugin>
-        </plugins>
-    </build>
+		</plugins>
+	</build>
 
 </project>
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/CheckPubSubEmulatorTest.java b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/CheckPubSubEmulatorTest.java
new file mode 100644
index 0000000..a54d47b
--- /dev/null
+++ b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/CheckPubSubEmulatorTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.flink.streaming.connectors.pubsub;
+
+import org.apache.flink.streaming.connectors.pubsub.emulator.GCloudUnitTestBase;
+import org.apache.flink.streaming.connectors.pubsub.emulator.PubsubHelper;
+
+import com.google.cloud.pubsub.v1.Publisher;
+import com.google.cloud.pubsub.v1.Subscriber;
+import com.google.protobuf.ByteString;
+import com.google.pubsub.v1.PubsubMessage;
+import com.google.pubsub.v1.ReceivedMessage;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests to ensure the docker image with PubSub is working correctly.
+ */
+public class CheckPubSubEmulatorTest extends GCloudUnitTestBase {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CheckPubSubEmulatorTest.class);
+
+	private static final String PROJECT_NAME = "Project";
+	private static final String TOPIC_NAME = "Topic";
+	private static final String SUBSCRIPTION_NAME = "Subscription";
+
+	private static PubsubHelper pubsubHelper = getPubsubHelper();
+
+	@BeforeClass
+	public static void setUp() throws Exception {
+		pubsubHelper.createTopic(PROJECT_NAME, TOPIC_NAME);
+		pubsubHelper.createSubscription(PROJECT_NAME, SUBSCRIPTION_NAME, PROJECT_NAME, TOPIC_NAME);
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		pubsubHelper.deleteSubscription(PROJECT_NAME, SUBSCRIPTION_NAME);
+		pubsubHelper.deleteTopic(PROJECT_NAME, TOPIC_NAME);
+	}
+
+	@Test
+	public void testPull() throws Exception {
+		Publisher publisher = pubsubHelper.createPublisher(PROJECT_NAME, TOPIC_NAME);
+		publisher
+			.publish(PubsubMessage
+				.newBuilder()
+				.setData(ByteString.copyFromUtf8("Hello World PULL"))
+				.build())
+			.get();
+
+		List<ReceivedMessage> receivedMessages = pubsubHelper.pullMessages(PROJECT_NAME, SUBSCRIPTION_NAME, 10);
+		assertEquals(1, receivedMessages.size());
+		assertEquals("Hello World PULL", receivedMessages.get(0).getMessage().getData().toStringUtf8());
+
+		publisher.shutdown();
+	}
+
+	@Test
+	public void testPush() throws Exception {
+		List<PubsubMessage> receivedMessages = new ArrayList<>();
+		Subscriber subscriber = pubsubHelper.
+			subscribeToSubscription(
+				PROJECT_NAME,
+				SUBSCRIPTION_NAME,
+				(message, consumer) -> receivedMessages.add(message)
+			);
+
+		Publisher publisher = pubsubHelper.createPublisher(PROJECT_NAME, TOPIC_NAME);
+		publisher
+			.publish(PubsubMessage
+				.newBuilder()
+				.setData(ByteString.copyFromUtf8("Hello World"))
+				.build())
+			.get();
+
+		LOG.info("Waiting a while to receive the message...");
+		Thread.sleep(1000);
+
+		assertEquals(1, receivedMessages.size());
+		assertEquals("Hello World", receivedMessages.get(0).getData().toStringUtf8());
+
+		try {
+			subscriber.stopAsync().awaitTerminated(100, MILLISECONDS);
+		} catch (TimeoutException tme) {
+			// Yeah, whatever. Don't care about clean shutdown here.
+		}
+		publisher.shutdown();
+	}
+
+}
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSinkTest.java b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSinkTest.java
new file mode 100644
index 0000000..165579f
--- /dev/null
+++ b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSinkTest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.flink.streaming.connectors.pubsub;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pubsub.emulator.GCloudUnitTestBase;
+import org.apache.flink.streaming.connectors.pubsub.emulator.PubsubHelper;
+
+import com.google.pubsub.v1.ReceivedMessage;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test of the PubSub SINK with the Google PubSub emulator.
+ */
+public class EmulatedPubSubSinkTest extends GCloudUnitTestBase {
+
+	private static final Logger LOG = LoggerFactory.getLogger(EmulatedPubSubSinkTest.class);
+
+	private static final String PROJECT_NAME = "FLProject";
+	private static final String TOPIC_NAME = "FLTopic";
+	private static final String SUBSCRIPTION_NAME = "FLSubscription";
+
+	private static PubsubHelper pubsubHelper;
+
+	@BeforeClass
+	public static void setUp() throws Exception {
+		pubsubHelper = getPubsubHelper();
+		pubsubHelper.createTopic(PROJECT_NAME, TOPIC_NAME);
+		pubsubHelper.createSubscription(PROJECT_NAME, SUBSCRIPTION_NAME, PROJECT_NAME, TOPIC_NAME);
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		pubsubHelper.deleteSubscription(PROJECT_NAME, SUBSCRIPTION_NAME);
+		pubsubHelper.deleteTopic(PROJECT_NAME, TOPIC_NAME);
+	}
+
+	@Test
+	public void testFlinkSink() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		List<String> input = Arrays.asList("One", "Two", "Three", "Four", "Five", "Six", "Seven", "Eigth", "Nine", "Ten");
+
+		// Create test stream
+		DataStream<String> theData = env
+			.fromCollection(input)
+			.name("Test input")
+			.map((MapFunction<String, String>) StringUtils::reverse);
+
+		// Sink into pubsub
+		theData
+			.addSink(PubSubSink.<String>newBuilder()
+				.withProjectName(PROJECT_NAME)
+				.withTopicName(TOPIC_NAME)
+				.withSerializationSchema(new SimpleStringSchema())
+				// Specific for emulator
+				.withCredentialsProvider(getPubsubHelper().getCredentialsProvider())
+				.withHostAndPort(getPubSubHostPort())
+				.build())
+			.name("PubSub sink");
+
+		// Run
+		env.execute();
+
+		// Now get the result from PubSub and verify if everything is there
+		List<ReceivedMessage> receivedMessages = pubsubHelper.pullMessages(PROJECT_NAME, SUBSCRIPTION_NAME, 100);
+
+		assertEquals("Wrong number of elements", input.size(), receivedMessages.size());
+
+		// Check output strings
+		List<String> output = new ArrayList<>();
+		receivedMessages.forEach(msg -> output.add(msg.getMessage().getData().toStringUtf8()));
+
+		for (String test : input) {
+			assertTrue("Missing " + test, output.contains(StringUtils.reverse(test)));
+		}
+	}
+
+}
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSourceTest.java b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSourceTest.java
new file mode 100644
index 0000000..aaa4fc3
--- /dev/null
+++ b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSourceTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.flink.streaming.connectors.pubsub;
+
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pubsub.emulator.GCloudUnitTestBase;
+import org.apache.flink.streaming.connectors.pubsub.emulator.PubsubHelper;
+
+import com.google.cloud.pubsub.v1.Publisher;
+import com.google.protobuf.ByteString;
+import com.google.pubsub.v1.PubsubMessage;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test of the PubSub SOURCE with the Google PubSub emulator.
+ */
+public class EmulatedPubSubSourceTest extends GCloudUnitTestBase {
+
+	private static final Logger LOG = LoggerFactory.getLogger(EmulatedPubSubSourceTest.class);
+
+	private static final String PROJECT_NAME = "FLProject";
+	private static final String TOPIC_NAME = "FLTopic";
+	private static final String SUBSCRIPTION_NAME = "FLSubscription";
+
+	private static PubsubHelper pubsubHelper;
+
+	@BeforeClass
+	public static void setUp() throws Exception {
+		pubsubHelper = getPubsubHelper();
+		pubsubHelper.createTopic(PROJECT_NAME, TOPIC_NAME);
+		pubsubHelper.createSubscription(PROJECT_NAME, SUBSCRIPTION_NAME, PROJECT_NAME, TOPIC_NAME);
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		pubsubHelper.deleteSubscription(PROJECT_NAME, SUBSCRIPTION_NAME);
+		pubsubHelper.deleteTopic(PROJECT_NAME, TOPIC_NAME);
+	}
+
+	@Test
+	public void testFlinkSource() throws Exception {
+		// Create some messages and put them into pubsub
+		List<String> input = Arrays.asList("One", "Two", "Three", "Four", "Five", "Six", "Seven", "Eigth", "Nine", "Ten");
+
+		// Publish the messages into PubSub
+		Publisher publisher = pubsubHelper.createPublisher(PROJECT_NAME, TOPIC_NAME);
+		input.forEach(s -> {
+			try {
+				publisher
+					.publish(PubsubMessage
+						.newBuilder()
+						.setData(ByteString.copyFromUtf8(s))
+						.build())
+					.get();
+			} catch (InterruptedException | ExecutionException e) {
+				e.printStackTrace();
+			}
+		});
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.enableCheckpointing(100);
+
+		DataStream<String> fromPubSub = env
+			.addSource(BoundedPubSubSource.<String>newBuilder()
+				.withDeserializationSchema(new SimpleStringSchema())
+				.withProjectSubscriptionName(PROJECT_NAME, SUBSCRIPTION_NAME)
+				// Specific for emulator
+				.withCredentialsProvider(getPubsubHelper().getCredentialsProvider())
+				.withHostAndPort(getPubSubHostPort())
+				// Make sure the test topology self terminates
+				.boundedByTimeSinceLastMessage(1000)
+				.build())
+			.name("PubSub source");
+
+		List<String> output = new ArrayList<>();
+		fromPubSub.writeUsingOutputFormat(new LocalCollectionOutputFormat<>(output));
+
+		env.execute();
+
+		assertEquals("Wrong number of elements", input.size(), output.size());
+		for (String test : input) {
+			assertTrue("Missing " + test, output.contains(test));
+		}
+	}
+
+}
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudEmulatorManager.java b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudEmulatorManager.java
new file mode 100644
index 0000000..27a658a
--- /dev/null
+++ b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudEmulatorManager.java
@@ -0,0 +1,264 @@
+/*
+ * 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.flink.streaming.connectors.pubsub.emulator;
+
+import com.spotify.docker.client.DefaultDockerClient;
+import com.spotify.docker.client.DockerClient;
+import com.spotify.docker.client.exceptions.ContainerNotFoundException;
+import com.spotify.docker.client.exceptions.DockerCertificateException;
+import com.spotify.docker.client.exceptions.DockerException;
+import com.spotify.docker.client.exceptions.ImageNotFoundException;
+import com.spotify.docker.client.messages.ContainerConfig;
+import com.spotify.docker.client.messages.ContainerCreation;
+import com.spotify.docker.client.messages.ContainerInfo;
+import com.spotify.docker.client.messages.HostConfig;
+import com.spotify.docker.client.messages.PortBinding;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * The class that handles the starting and stopping of the emulator docker image.
+ */
+public class GCloudEmulatorManager {
+
+	private static final Logger LOG = LoggerFactory.getLogger(GCloudEmulatorManager.class);
+
+	private static DockerClient docker;
+
+	private static String dockerIpAddress = "127.0.0.1";
+
+	public static final String INTERNAL_PUBSUB_PORT = "22222";
+	public static final String DOCKER_IMAGE_NAME = "google/cloud-sdk:latest";
+
+	private static String pubsubPort;
+
+	public static String getDockerIpAddress() {
+		if (dockerIpAddress == null) {
+			throw new IllegalStateException("The docker has not yet been started (yet) so you cannot get the IP address yet.");
+		}
+		return dockerIpAddress;
+	}
+
+	public static String getDockerPubSubPort() {
+		if (pubsubPort == null) {
+			throw new IllegalStateException("The docker has not yet been started (yet) so you cannot get the port information yet.");
+		}
+		return pubsubPort;
+	}
+
+	public static final String UNITTEST_PROJECT_ID = "running-from-junit-for-flink";
+	private static final String CONTAINER_NAME_JUNIT = (DOCKER_IMAGE_NAME + "_" + UNITTEST_PROJECT_ID).replaceAll("[^a-zA-Z0-9_]", "_");
+
+	public static void launchDocker() throws DockerException, InterruptedException, DockerCertificateException {
+		// Create a client based on DOCKER_HOST and DOCKER_CERT_PATH env vars
+		docker = DefaultDockerClient.fromEnv().build();
+
+		terminateAndDiscardAnyExistingContainers(true);
+
+		LOG.info("");
+		LOG.info("/===========================================");
+		LOG.info("| GCloud Emulator");
+
+		ContainerInfo containerInfo;
+		String id;
+
+		try {
+			docker.inspectImage(DOCKER_IMAGE_NAME);
+		} catch (ImageNotFoundException e) {
+			// No such image so we must download it first.
+			LOG.info("| - Getting docker image \"{}\"", DOCKER_IMAGE_NAME);
+			docker.pull(DOCKER_IMAGE_NAME, message -> {
+				if (message.id() != null && message.progress() != null) {
+					LOG.info("| - Downloading > {} : {}", message.id(), message.progress());
+				}
+			});
+		}
+
+		// No such container. Good, we create one!
+		LOG.info("| - Creating new container");
+
+		// Bind container ports to host ports
+		final Map<String, List<PortBinding>> portBindings = new HashMap<>();
+		portBindings.put(INTERNAL_PUBSUB_PORT, Collections.singletonList(PortBinding.randomPort("0.0.0.0")));
+
+		final HostConfig hostConfig = HostConfig.builder().portBindings(portBindings).build();
+
+		// Create new container with exposed ports
+		final ContainerConfig containerConfig = ContainerConfig.builder()
+			.hostConfig(hostConfig)
+			.exposedPorts(INTERNAL_PUBSUB_PORT)
+			.image(DOCKER_IMAGE_NAME)
+			.cmd("sh", "-c", "mkdir -p /opt/data/pubsub ; gcloud beta emulators pubsub start --data-dir=/opt/data/pubsub  --host-port=0.0.0.0:" + INTERNAL_PUBSUB_PORT)
+			.build();
+
+		final ContainerCreation creation = docker.createContainer(containerConfig, CONTAINER_NAME_JUNIT);
+		id = creation.id();
+
+		containerInfo = docker.inspectContainer(id);
+
+		if (!containerInfo.state().running()) {
+			LOG.warn("| - Starting it up ....");
+			docker.startContainer(id);
+			Thread.sleep(1000);
+		}
+
+		containerInfo = docker.inspectContainer(id);
+
+		dockerIpAddress = "127.0.0.1";
+
+		Map<String, List<PortBinding>> ports = containerInfo.networkSettings().ports();
+
+		assertNotNull("Unable to retrieve the ports where to connect to the emulators", ports);
+		assertEquals("We expect 1 port to be mapped", 1, ports.size());
+
+		pubsubPort = getPort(ports, INTERNAL_PUBSUB_PORT, "PubSub");
+
+		LOG.info("| Waiting for the emulators to be running");
+
+		// PubSub exposes an "Ok" at the root url when running.
+		if (!waitForOkStatus("PubSub", pubsubPort)) {
+			// Oops, we did not get an "Ok" within 10 seconds
+			startHasFailedKillEverything();
+		}
+		LOG.info("\\===========================================");
+		LOG.info("");
+	}
+
+	private static void startHasFailedKillEverything() throws DockerException, InterruptedException {
+		LOG.error("|");
+		LOG.error("| ==================== ");
+		LOG.error("| YOUR TESTS WILL FAIL ");
+		LOG.error("| ==================== ");
+		LOG.error("|");
+
+		// Kill this container and wipe all connection information
+		dockerIpAddress = null;
+		pubsubPort = null;
+		terminateAndDiscardAnyExistingContainers(false);
+	}
+
+	private static final long MAX_RETRY_TIMEOUT = 10000; // Milliseconds
+
+	private static boolean waitForOkStatus(String label, String port) {
+		long start = System.currentTimeMillis();
+		while (true) {
+			try {
+				URL url = new URL("http://" + dockerIpAddress + ":" + port + "/");
+				HttpURLConnection con = (HttpURLConnection) url.openConnection();
+				con.setRequestMethod("GET");
+				con.setConnectTimeout(50);
+				con.setReadTimeout(50);
+
+				BufferedReader in = new BufferedReader(new InputStreamReader(con.getInputStream()));
+				String inputLine;
+				StringBuilder content = new StringBuilder();
+				while ((inputLine = in.readLine()) != null) {
+					content.append(inputLine);
+				}
+				in.close();
+				con.disconnect();
+				if (content.toString().contains("Ok")) {
+					LOG.info("| - {} Emulator is running at {}:{}", label, dockerIpAddress, port);
+					return true;
+				}
+			} catch (IOException e) {
+				long now = System.currentTimeMillis();
+				if (now - start > MAX_RETRY_TIMEOUT) {
+					LOG.error("| - PubSub Emulator at {}:{} FAILED to return an Ok status within {} ms ", dockerIpAddress, port, MAX_RETRY_TIMEOUT);
+					return false;
+				}
+				try {
+					Thread.sleep(100); // Sleep a very short time
+				} catch (InterruptedException e1) {
+					// Ignore
+				}
+			}
+		}
+	}
+
+	private static String getPort(Map<String, List<PortBinding>> ports, String internalTCPPort, String label) {
+		List<PortBinding> portMappings = ports.get(internalTCPPort + "/tcp");
+		if (portMappings == null || portMappings.isEmpty()) {
+			LOG.info("| {} Emulator {} --> NOTHING CONNECTED TO {}", label, internalTCPPort + "/tcp");
+			return null;
+		}
+
+		return portMappings.get(0).hostPort();
+	}
+
+	private static void terminateAndDiscardAnyExistingContainers(boolean warnAboutExisting) throws DockerException, InterruptedException {
+		ContainerInfo containerInfo;
+		try {
+			containerInfo = docker.inspectContainer(CONTAINER_NAME_JUNIT);
+			// Already have this container running.
+
+			assertNotNull("We should either we get containerInfo or we get an exception", containerInfo);
+
+			LOG.info("");
+			LOG.info("/===========================================");
+			if (warnAboutExisting) {
+				LOG.warn("|    >>> FOUND OLD EMULATOR INSTANCE RUNNING <<< ");
+				LOG.warn("| Destroying that one to keep tests running smoothly.");
+			}
+			LOG.info("| Cleanup of GCloud Emulator");
+
+			// We REQUIRE 100% accurate side effect free unit tests
+			// So we completely discard this one.
+
+			String id = containerInfo.id();
+			// Kill container
+			if (containerInfo.state().running()) {
+				docker.killContainer(id);
+				LOG.info("| - Killed");
+			}
+
+			// Remove container
+			docker.removeContainer(id);
+
+			LOG.info("| - Removed");
+			LOG.info("\\===========================================");
+			LOG.info("");
+
+		} catch (ContainerNotFoundException cnfe) {
+			// No such container. Good !
+		}
+	}
+
+	public static void terminateDocker() throws DockerException, InterruptedException {
+		terminateAndDiscardAnyExistingContainers(false);
+
+		// Close the docker client
+		docker.close();
+	}
+
+	// ====================================================================================
+
+}
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudUnitTestBase.java b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudUnitTestBase.java
new file mode 100644
index 0000000..b6a011a
--- /dev/null
+++ b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudUnitTestBase.java
@@ -0,0 +1,84 @@
+/*
+ * 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.flink.streaming.connectors.pubsub.emulator;
+
+import com.google.api.gax.core.CredentialsProvider;
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.GrpcTransportChannel;
+import com.google.api.gax.rpc.FixedTransportChannelProvider;
+import com.google.api.gax.rpc.TransportChannelProvider;
+import com.spotify.docker.client.exceptions.DockerException;
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.Serializable;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.flink.streaming.connectors.pubsub.emulator.GCloudEmulatorManager.getDockerIpAddress;
+import static org.apache.flink.streaming.connectors.pubsub.emulator.GCloudEmulatorManager.getDockerPubSubPort;
+
+/**
+ * The base class from which unit tests should inherit if they need to use the Google cloud emulators.
+ */
+public class GCloudUnitTestBase implements Serializable {
+	@BeforeClass
+	public static void launchGCloudEmulator() throws Exception {
+		// Separated out into separate class so the entire test class to be serializable
+		GCloudEmulatorManager.launchDocker();
+	}
+
+	@AfterClass
+	public static void terminateGCloudEmulator() throws DockerException, InterruptedException {
+		GCloudEmulatorManager.terminateDocker();
+	}
+
+	// ====================================================================================
+	// Pubsub helpers
+
+	private static ManagedChannel channel = null;
+	private static TransportChannelProvider channelProvider = null;
+	private static CredentialsProvider credentialsProvider = null;
+
+	public static PubsubHelper getPubsubHelper() {
+		if (channel == null) {
+			//noinspection deprecation
+			channel = ManagedChannelBuilder
+				.forTarget(getPubSubHostPort())
+				.usePlaintext(true)
+				.build();
+			channelProvider = FixedTransportChannelProvider
+				.create(GrpcTransportChannel.create(channel));
+			credentialsProvider = NoCredentialsProvider.create();
+		}
+		return new PubsubHelper(channelProvider, credentialsProvider);
+	}
+
+	public static String getPubSubHostPort() {
+		return getDockerIpAddress() + ":" + getDockerPubSubPort();
+	}
+
+	@AfterClass
+	public static void cleanupPubsubChannel() throws InterruptedException {
+		if (channel != null) {
+			channel.shutdownNow().awaitTermination(1, SECONDS);
+			channel = null;
+		}
+	}
+}
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/PubsubHelper.java b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/PubsubHelper.java
new file mode 100644
index 0000000..f08576b
--- /dev/null
+++ b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/PubsubHelper.java
@@ -0,0 +1,232 @@
+/*
+ * 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.flink.streaming.connectors.pubsub.emulator;
+
+import com.google.api.gax.core.CredentialsProvider;
+import com.google.api.gax.rpc.NotFoundException;
+import com.google.api.gax.rpc.TransportChannelProvider;
+import com.google.cloud.pubsub.v1.MessageReceiver;
+import com.google.cloud.pubsub.v1.Publisher;
+import com.google.cloud.pubsub.v1.Subscriber;
+import com.google.cloud.pubsub.v1.SubscriptionAdminClient;
+import com.google.cloud.pubsub.v1.SubscriptionAdminSettings;
+import com.google.cloud.pubsub.v1.TopicAdminClient;
+import com.google.cloud.pubsub.v1.TopicAdminSettings;
+import com.google.cloud.pubsub.v1.stub.GrpcSubscriberStub;
+import com.google.cloud.pubsub.v1.stub.SubscriberStub;
+import com.google.cloud.pubsub.v1.stub.SubscriberStubSettings;
+import com.google.pubsub.v1.AcknowledgeRequest;
+import com.google.pubsub.v1.ProjectSubscriptionName;
+import com.google.pubsub.v1.ProjectTopicName;
+import com.google.pubsub.v1.PullRequest;
+import com.google.pubsub.v1.PullResponse;
+import com.google.pubsub.v1.PushConfig;
+import com.google.pubsub.v1.ReceivedMessage;
+import com.google.pubsub.v1.Topic;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A helper class to make managing the testing topics a bit easier.
+ */
+public class PubsubHelper {
+
+	private static final Logger LOG = LoggerFactory.getLogger(PubsubHelper.class);
+
+	private TransportChannelProvider channelProvider = null;
+	private CredentialsProvider credentialsProvider = null;
+
+	private TopicAdminClient topicClient;
+	private SubscriptionAdminClient subscriptionAdminClient;
+
+	public PubsubHelper() {
+		this(TopicAdminSettings.defaultTransportChannelProvider(),
+			TopicAdminSettings.defaultCredentialsProviderBuilder().build());
+	}
+
+	public PubsubHelper(TransportChannelProvider channelProvider, CredentialsProvider credentialsProvider) {
+		this.channelProvider = channelProvider;
+		this.credentialsProvider = credentialsProvider;
+	}
+
+	public TransportChannelProvider getChannelProvider() {
+		return channelProvider;
+	}
+
+	public CredentialsProvider getCredentialsProvider() {
+		return credentialsProvider;
+	}
+
+	public TopicAdminClient getTopicAdminClient() throws IOException {
+		if (topicClient == null) {
+			TopicAdminSettings topicAdminSettings = TopicAdminSettings.newBuilder()
+				.setTransportChannelProvider(channelProvider)
+				.setCredentialsProvider(credentialsProvider)
+				.build();
+			topicClient = TopicAdminClient.create(topicAdminSettings);
+		}
+		return topicClient;
+	}
+
+	public Topic createTopic(String project, String topic) throws IOException {
+		deleteTopic(project, topic);
+		ProjectTopicName topicName = ProjectTopicName.of(project, topic);
+		TopicAdminClient adminClient = getTopicAdminClient();
+		LOG.info("CreateTopic {}", topicName);
+		return adminClient.createTopic(topicName);
+	}
+
+	public void deleteTopic(String project, String topic) throws IOException {
+		deleteTopic(ProjectTopicName.of(project, topic));
+	}
+
+	public void deleteTopic(ProjectTopicName topicName) throws IOException {
+//        LOG.info("CreateTopic {}", topicName);
+		TopicAdminClient adminClient = getTopicAdminClient();
+		try {
+			Topic existingTopic = adminClient.getTopic(topicName);
+
+			// If it exists we delete all subscriptions and the topic itself.
+			LOG.info("DeleteTopic {} first delete old subscriptions.", topicName);
+			adminClient
+				.listTopicSubscriptions(topicName)
+				.iterateAllAsProjectSubscriptionName()
+				.forEach(subscriptionAdminClient::deleteSubscription);
+			LOG.info("DeleteTopic {}", topicName);
+			adminClient
+				.deleteTopic(topicName);
+		} catch (NotFoundException e) {
+			// Doesn't exist. Good.
+		}
+	}
+
+	public SubscriptionAdminClient getSubscriptionAdminClient() throws IOException {
+		if (subscriptionAdminClient == null) {
+			SubscriptionAdminSettings subscriptionAdminSettings =
+				SubscriptionAdminSettings
+					.newBuilder()
+					.setTransportChannelProvider(channelProvider)
+					.setCredentialsProvider(credentialsProvider)
+					.build();
+			subscriptionAdminClient = SubscriptionAdminClient.create(subscriptionAdminSettings);
+		}
+		return subscriptionAdminClient;
+	}
+
+	public void createSubscription(String subscriptionProject, String subscription, String topicProject, String topic) throws IOException {
+		ProjectSubscriptionName subscriptionName = ProjectSubscriptionName.newBuilder()
+			.setProject(subscriptionProject)
+			.setSubscription(subscription)
+			.build();
+
+		deleteSubscription(subscriptionName);
+
+		SubscriptionAdminClient adminClient = getSubscriptionAdminClient();
+
+		ProjectTopicName topicName = ProjectTopicName.of(topicProject, topic);
+
+		PushConfig pushConfig = PushConfig.getDefaultInstance();
+
+		LOG.info("CreateSubscription {}", subscriptionName);
+		getSubscriptionAdminClient().createSubscription(subscriptionName, topicName, pushConfig, 1);
+	}
+
+	public void deleteSubscription(String subscriptionProject, String subscription) throws IOException {
+		deleteSubscription(ProjectSubscriptionName
+			.newBuilder()
+			.setProject(subscriptionProject)
+			.setSubscription(subscription)
+			.build());
+	}
+
+	public void deleteSubscription(ProjectSubscriptionName subscriptionName) throws IOException {
+		SubscriptionAdminClient adminClient = getSubscriptionAdminClient();
+		try {
+			adminClient.getSubscription(subscriptionName);
+			// If it already exists we must first delete it.
+			LOG.info("DeleteSubscription {}", subscriptionName);
+			adminClient.deleteSubscription(subscriptionName);
+		} catch (NotFoundException e) {
+			// Doesn't exist. Good.
+		}
+	}
+
+	// Mostly copied from the example on https://cloud.google.com/pubsub/docs/pull
+	public List<ReceivedMessage> pullMessages(String projectId, String subscriptionId, int maxNumberOfMessages) throws Exception {
+		SubscriberStubSettings subscriberStubSettings =
+			SubscriberStubSettings.newBuilder()
+				.setTransportChannelProvider(channelProvider)
+				.setCredentialsProvider(credentialsProvider)
+				.build();
+		try (SubscriberStub subscriber = GrpcSubscriberStub.create(subscriberStubSettings)) {
+			// String projectId = "my-project-id";
+			// String subscriptionId = "my-subscription-id";
+			// int numOfMessages = 10;   // max number of messages to be pulled
+			String subscriptionName = ProjectSubscriptionName.format(projectId, subscriptionId);
+			PullRequest pullRequest =
+				PullRequest.newBuilder()
+					.setMaxMessages(maxNumberOfMessages)
+					.setReturnImmediately(false) // return immediately if messages are not available
+					.setSubscription(subscriptionName)
+					.build();
+
+			// use pullCallable().futureCall to asynchronously perform this operation
+			PullResponse pullResponse = subscriber.pullCallable().call(pullRequest);
+			List<String> ackIds = new ArrayList<>();
+			for (ReceivedMessage message : pullResponse.getReceivedMessagesList()) {
+				// handle received message
+				// ...
+				ackIds.add(message.getAckId());
+			}
+			// acknowledge received messages
+			AcknowledgeRequest acknowledgeRequest =
+				AcknowledgeRequest.newBuilder()
+					.setSubscription(subscriptionName)
+					.addAllAckIds(ackIds)
+					.build();
+			// use acknowledgeCallable().futureCall to asynchronously perform this operation
+			subscriber.acknowledgeCallable().call(acknowledgeRequest);
+			return pullResponse.getReceivedMessagesList();
+		}
+	}
+
+	public Subscriber subscribeToSubscription(String project, String subscription, MessageReceiver messageReceiver) {
+		ProjectSubscriptionName subscriptionName = ProjectSubscriptionName.of(project, subscription);
+		Subscriber subscriber =
+			Subscriber
+				.newBuilder(subscriptionName, messageReceiver)
+				.setChannelProvider(channelProvider)
+				.setCredentialsProvider(credentialsProvider)
+				.build();
+		subscriber.startAsync();
+		return subscriber;
+	}
+
+	public Publisher createPublisher(String project, String topic) throws IOException {
+		return Publisher
+			.newBuilder(ProjectTopicName.of(project, topic))
+			.setChannelProvider(channelProvider)
+			.setCredentialsProvider(credentialsProvider)
+			.build();
+	}
+
+}
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/resources/log4j-test.properties b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..b316a9a
--- /dev/null
+++ b/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/resources/log4j-test.properties
@@ -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.
+################################################################################
+log4j.rootLogger=INFO, testlogger
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target=System.out
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml
index 0950e2f..b17dc70 100644
--- a/flink-end-to-end-tests/pom.xml
+++ b/flink-end-to-end-tests/pom.xml
@@ -61,6 +61,7 @@ under the License.
 		<module>flink-metrics-availability-test</module>
 		<module>flink-metrics-reporter-prometheus-test</module>
 		<module>flink-heavy-deployment-stress-test</module>
+		<module>flink-connector-pubsub-emulator-tests</module>
 		<module>flink-streaming-kafka-test-base</module>
 		<module>flink-streaming-kafka-test</module>
 		<module>flink-streaming-kafka011-test</module>
diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh
index af21512..957b3c6 100755
--- a/flink-end-to-end-tests/run-nightly-tests.sh
+++ b/flink-end-to-end-tests/run-nightly-tests.sh
@@ -136,6 +136,8 @@ run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scr
 run_test "Quickstarts Java nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh java"
 run_test "Quickstarts Scala nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh scala"
 
+run_test "Test PubSub connector with Docker based Google PubSub Emulator" "$END_TO_END_DIR/test-scripts/test_streaming_pubsub.sh"
+
 run_test "Avro Confluent Schema Registry nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_confluent_schema_registry.sh"
 
 run_test "State TTL Heap backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh file"
diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_pubsub.sh b/flink-end-to-end-tests/test-scripts/test_streaming_pubsub.sh
new file mode 100755
index 0000000..8e08385
--- /dev/null
+++ b/flink-end-to-end-tests/test-scripts/test_streaming_pubsub.sh
@@ -0,0 +1,22 @@
+#!/usr/bin/env bash
+################################################################################
+# 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.
+################################################################################
+
+cd "${END_TO_END_DIR}/flink-connector-pubsub-emulator-tests"
+
+mvn test -DskipTests=false
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubExample.java
index 612da41..c5791bd 100644
--- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubExample.java
+++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubExample.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.examples.pubsub;
 import org.apache.flink.api.java.utils.ParameterTool;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.connectors.pubsub.PubSubSink;
-import org.apache.flink.streaming.connectors.pubsub.PubSubSourceBuilder;
+import org.apache.flink.streaming.connectors.pubsub.PubSubSource;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -45,7 +45,7 @@ public class PubSubExample {
 			System.out.println("Missing parameters!\n" +
 								"Usage: flink run PubSub.jar --input-subscription <subscription> --input-topicName <topic> --output-topicName " +
 								"--google-project <google project name> ");
-			//return;
+			return;
 		}
 
 		String projectName = parameterTool.getRequired("google-project");
@@ -62,10 +62,9 @@ public class PubSubExample {
 	private static void runFlinkJob(String projectName, String subscriptionName, String outputTopicName) throws Exception {
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-		env.addSource(PubSubSourceBuilder.<Integer>builder()
+		env.addSource(PubSubSource.<Integer>newBuilder()
 										.withProjectSubscriptionName(projectName, subscriptionName)
 										.withDeserializationSchema(new IntegerSerializer())
-										.withMode(PubSubSourceBuilder.Mode.NONE)
 										.build())
 			.map(PubSubExample::printAndReturn).disableChaining()
 			.addSink(PubSubSink.<Integer>newBuilder()


[flink] 04/05: [FLINK-9311] [pubsub] Clean up / add documentation and style issues in the PubSub connector

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit cc101de77fee1962907daf2b9f78c5c5a105bfc5
Author: Richard Deurwaarder <rd...@bol.com>
AuthorDate: Sat Aug 25 15:21:54 2018 +0200

    [FLINK-9311] [pubsub] Clean up / add documentation and style issues in the PubSub connector
---
 docs/dev/connectors/pubsub.md                      | 108 +++++--
 .../pom.xml                                        |  24 +-
 .../gcp/pubsub/BlockingGrpcPubSubSubscriber.java   | 138 ++++++++
 .../gcp/pubsub/DefaultPubSubSubscriberFactory.java |  67 ++++
 .../gcp/pubsub/DeserializationSchemaWrapper.java   |  51 +++
 .../connectors/gcp/pubsub/PubSubSink.java          | 317 +++++++++++++++++++
 .../connectors/gcp/pubsub/PubSubSource.java        | 347 +++++++++++++++++++++
 .../pubsub/PubSubSubscriberFactoryForEmulator.java |  66 ++++
 .../pubsub/common/AcknowledgeIdsForCheckpoint.java |  52 +++
 .../gcp/pubsub/common/AcknowledgeOnCheckpoint.java | 110 +++++++
 .../connectors/gcp/pubsub/common/Acknowledger.java |  28 ++
 .../pubsub/common/PubSubDeserializationSchema.java |  52 +++
 .../gcp/pubsub/common/PubSubSubscriber.java        |  37 +++
 .../gcp/pubsub/common/PubSubSubscriberFactory.java |  38 +++
 .../pubsub/DeserializationSchemaWrapperTest.java   |  84 +++++
 .../connectors/gcp/pubsub/PubSubSourceTest.java    | 206 ++++++++++++
 .../pubsub/common/AcknowledgeOnCheckpointTest.java | 141 +++++++++
 .../src/test/resources/log4j-test.properties       |   2 +-
 .../flink/streaming/connectors/pubsub/Bound.java   | 116 -------
 .../connectors/pubsub/BoundedPubSubSource.java     | 108 -------
 .../streaming/connectors/pubsub/PubSubSink.java    | 250 ---------------
 .../streaming/connectors/pubsub/PubSubSource.java  | 265 ----------------
 .../connectors/pubsub/SubscriberWrapper.java       | 104 ------
 .../common/SerializableCredentialsProvider.java    |  70 -----
 .../streaming/connectors/pubsub/BoundTest.java     | 130 --------
 .../connectors/pubsub/BoundedPubSubSourceTest.java |  87 ------
 .../connectors/pubsub/PubSubSourceTest.java        | 147 ---------
 .../connectors/pubsub/SubscriberWrapperTest.java   |  57 ----
 flink-connectors/pom.xml                           |   2 +-
 .../pom.xml                                        |  43 +--
 .../gcp}/pubsub/CheckPubSubEmulatorTest.java       |  26 +-
 .../gcp}/pubsub/EmulatedPubSubSinkTest.java        |  69 +++-
 .../gcp}/pubsub/EmulatedPubSubSourceTest.java      |  74 +++--
 .../pubsub/emulator/GCloudEmulatorManager.java     |   2 +-
 .../gcp}/pubsub/emulator/GCloudUnitTestBase.java   |  12 +-
 .../gcp}/pubsub/emulator/PubsubHelper.java         |  51 ++-
 .../src/test/resources/log4j-test.properties       |   2 +-
 flink-end-to-end-tests/pom.xml                     |   2 +-
 flink-end-to-end-tests/run-nightly-tests.sh        |   2 +-
 ...ming_pubsub.sh => test_streaming_gcp_pubsub.sh} |   2 +-
 .../flink-examples-streaming-gcp-pubsub/pom.xml    | 108 +++++++
 .../examples/gcp}/pubsub/IntegerSerializer.java    |  16 +-
 .../examples/gcp}/pubsub/PubSubExample.java        |  29 +-
 .../examples/gcp}/pubsub/PubSubPublisher.java      |  25 +-
 flink-examples/flink-examples-build-helper/pom.xml |   1 +
 flink-examples/flink-examples-streaming/pom.xml    |  11 -
 46 files changed, 2135 insertions(+), 1544 deletions(-)

diff --git a/docs/dev/connectors/pubsub.md b/docs/dev/connectors/pubsub.md
index 6eb3c24..bfa6f4a 100644
--- a/docs/dev/connectors/pubsub.md
+++ b/docs/dev/connectors/pubsub.md
@@ -30,24 +30,30 @@ following dependency to your project:
 {% highlight xml %}
 <dependency>
   <groupId>org.apache.flink</groupId>
-  <artifactId>flink-connector-pubsub{{ site.scala_version_suffix }}</artifactId>
-  <version>{{site.version }}</version>
+  <artifactId>flink-connector-gcp-pubsub{{ site.scala_version_suffix }}</artifactId>
+  <version>{{ site.version }}</version>
 </dependency>
 {% endhighlight %}
 
 Note that the streaming connectors are currently not part of the binary
 distribution. See
-[here]({{site.baseurl}}/dev/linking.html)
+[here]({{ site.baseurl }}/dev/projectsetup/dependencies.html)
 for information about how to package the program with the libraries for
 cluster execution.
+ 
+This pubsub connector has not received widespread testing yet.
 
-#### PubSub Source
+## Consuming or Producing PubSubMessages
 
-The connector provides a Source for reading data from Google PubSub to Apache Flink. PubSub has an Atleast-Once guarantee and as such.
+The connector provides a connectors for receiving and sending messages from and to Google PubSub.
+Google PubSub has an `at-least-once` guarantee and as such the connector delivers the same guarantees.
 
-The class `PubSubSource(…)` has a builder to create PubSubsources. `PubSubSource.newBuilder()`
+### PubSub SourceFunction
+
+The class `PubSubSource` has a builder to create PubSubsources: `PubSubSource.newBuilder(...)`
+
+There are several optional methods to alter how the PubSubSource is created, the bare minimum is to provide a Google project, Pubsub subscription and a way to deserialize the PubSubMessages.
 
-There are several optional methods to alter how the PubSubSource is created, the bare minimum is to provide a google project and pubsub subscription and a way to deserialize the PubSubMessages.
 Example:
 
 <div class="codetabs" markdown="1">
@@ -55,52 +61,98 @@ Example:
 {% highlight java %}
 StreamExecutionEnvironment streamExecEnv = StreamExecutionEnvironment.getExecutionEnvironment();
 
-DeserializationSchema<SomeObject> deserializationSchema = (...);
-SourceFunction<SomeObject> pubsubSource = PubSubSource.<SomeObject>newBuilder()
-                                                      .withDeserializationSchema(deserializationSchema)
-                                                      .withProjectSubscriptionName("google-project-name", "pubsub-subscription")
+DeserializationSchema<SomeObject> deserializer = (...);
+SourceFunction<SomeObject> pubsubSource = PubSubSource.newBuilder(SomeObject.class)
+                                                      .withDeserializationSchema(deserializer)
+                                                      .withProjectName("project")
+                                                      .withSubscriptionName("subscription")
                                                       .build();
 
-streamExecEnv.addSource(pubsubSource);
+streamExecEnv.addSource(source);
 {% endhighlight %}
 </div>
 </div>
 
-#### PubSub Sink
+Currently the source functions [pulls](https://cloud.google.com/pubsub/docs/pull) messages from PubSub, [push endpoints](https://cloud.google.com/pubsub/docs/push) are not supported.
 
-The connector provides a Sink for writing data to PubSub.
+### PubSub Sink
 
-The class `PubSubSource(…)` has a builder to create PubSubsources. `PubSubSource.newBuilder()`
+The class `PubSubSink` has a builder to create PubSubSinks. `PubSubSink.newBuilder(...)`
 
 This builder works in a similar way to the PubSubSource.
+
 Example:
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
 {% highlight java %}
-StreamExecutionEnvironment streamExecEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+DataStream<SomeObject> dataStream = (...);
 
 SerializationSchema<SomeObject> serializationSchema = (...);
-SourceFunction<SomeObject> pubsubSink = PubSubSink.<SomeObject>newBuilder()
-                                                  .withSerializationSchema(serializationSchema)
-                                                  .withTopicName("pubsub-topic-name")
-                                                  .withProjectName("google-project-name")
-                                                  .build()
+SinkFunction<SomeObject> pubsubSink = PubSubSink.newBuilder(SomeObject.class)
+                                                .withDeserializationSchema(deserializer)
+                                                .withProjectName("project")
+                                                .withSubscriptionName("subscription")
+                                                .build()
+
+dataStream.addSink(pubsubSink);
+{% endhighlight %}
+</div>
+</div>
+
+### Google Credentials
+
+Google uses [Credentials](https://cloud.google.com/docs/authentication/production) to authenticate and authorize applications so that they can use Google Cloud Platform resources (such as PubSub).
+
+Both builders allow you to provide these credentials but by default the connectors will look for an environment variable: [GOOGLE_APPLICATION_CREDENTIALS](https://cloud.google.com/docs/authentication/production#obtaining_and_providing_service_account_credentials_manually) which should point to a file containing the credentials.
 
-streamExecEnv.addSink(pubsubSink);
+If you want to provide Credentials manually, for instance if you read the Credentials yourself from an external system, you can use `PubSubSource.newBuilder(...).withCredentials(...)`.
+
+### Integration testing
+
+When running integration tests you might not want to connect to PubSub directly but use a docker container to read and write to. (See: [PubSub testing locally](https://cloud.google.com/pubsub/docs/emulator))
+
+The following example shows how you would create a source to read messages from the emulator and send them back:
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+DeserializationSchema<SomeObject> deserializationSchema = (...);
+SourceFunction<SomeObject> pubsubSource = PubSubSource.newBuilder(SomeObject.class)
+                                                      .withDeserializationSchema(deserializationSchema)
+                                                      .withProjectName("my-fake-project")
+                                                      .withSubscriptionName("subscription")
+                                                      .withPubSubSubscriberFactory(new PubSubSubscriberFactoryForEmulator("localhost:1234", "my-fake-project", "subscription", 10, Duration.ofSeconds(15), 100))
+                                                      .build();
+SinkFunction<SomeObject> pubsubSink = PubSubSink.newBuilder(SomeObject.class)
+                                                .withDeserializationSchema(deserializationSchema)
+                                                .withProjectName("my-fake-project")
+                                                .withSubscriptionName("subscription")
+                                                .withHostAndPortForEmulator(getPubSubHostPort())
+                                                .build()
+
+StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+env.addSource(pubsubSource)
+   .addSink(pubsubSink);
 {% endhighlight %}
 </div>
 </div>
 
-#### Google Credentials
+### Atleast once guarantee
+
+#### SourceFunction
+
+There are several reasons why a message might be send multiple times, such as failure scenarios on Google PubSub's side.
+
+Another reason is when the acknowledgement deadline has past. This is the time between receiving the message and between acknowledging the message. The PubSubSource will only acknowledge a message on successful checkpoints to guarantee Atleast-Once. This does mean if the time between successful checkpoints is larger than the acknowledgment deadline of your subscription messages will most likely be processed multiple times.
+
+For this reason it's recommended to have a (much) lower checkpoint interval than acknowledgement deadline.
 
-Google uses [Credentials](https://cloud.google.com/docs/authentication/production) to authenticate and authorize applications so that they can use Google cloud resources such as PubSub. Both builders allow several ways to provide these credentials.
+See [PubSub](https://cloud.google.com/pubsub/docs/subscriber) for details on how to increase the acknowledgment deadline of your subscription.
 
-By default the connectors will look for an environment variable: [GOOGLE_APPLICATION_CREDENTIALS](https://cloud.google.com/docs/authentication/production#obtaining_and_providing_service_account_credentials_manually) which should point to a file containing the credentials.
+Note: The metric `PubSubMessagesProcessedNotAcked` shows how many messages are waiting for the next checkpoint before they will be acknowledged.
 
-It is also possible to provide a Credentials object directly. For instance if you read the Credentials yourself from an external system. In this case you can use `PubSubSource.newBuilder().withCredentials(...)`
+#### SinkFunction
 
-#### Integration testing
+The sink function buffers messages that are to be send to PubSub for a short amount of time for performance reasons. Before each checkpoint this buffer is flushed and the checkpoint will not succeed unless the messages have been delivered to PubSub.
 
-When using integration tests you might not want to connect to PubSub directly but use a docker container to read and write to. This is possible by using `PubSubSource.newBuilder().withHostAndPort("localhost:1234")`.
 {% top %}
diff --git a/flink-connectors/flink-connector-pubsub/pom.xml b/flink-connectors/flink-connector-gcp-pubsub/pom.xml
similarity index 81%
rename from flink-connectors/flink-connector-pubsub/pom.xml
rename to flink-connectors/flink-connector-gcp-pubsub/pom.xml
index f50cbdd..11d12a7 100644
--- a/flink-connectors/flink-connector-pubsub/pom.xml
+++ b/flink-connectors/flink-connector-gcp-pubsub/pom.xml
@@ -26,22 +26,22 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.7-SNAPSHOT</version>
+		<version>1.9-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-pubsub_${scala.binary.version}</artifactId>
-	<name>flink-connector-pubsub</name>
+	<artifactId>flink-connector-gcp-pubsub_${scala.binary.version}</artifactId>
+	<name>flink-connector-gcp-pubsub</name>
 
 	<packaging>jar</packaging>
 
-	<!-- This is the way we get a consistent set of versions of the Google tools -->
 	<dependencyManagement>
 		<dependencies>
 			<dependency>
+				<!-- This is the way we get a consistent set of versions of the Google tools -->
 				<groupId>com.google.cloud</groupId>
 				<artifactId>google-cloud-bom</artifactId>
-				<version>0.53.0-alpha</version>
+				<version>0.80.0-alpha</version>
 				<type>pom</type>
 				<scope>import</scope>
 			</dependency>
@@ -68,20 +68,6 @@ under the License.
 					<artifactId>guava-jdk5</artifactId>
 				</exclusion>
 			</exclusions>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-api</artifactId>
-			<version>${slf4j.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-log4j12</artifactId>
-			<version>${slf4j.version}</version>
-			<scope>test</scope>
 		</dependency>
 
 		<dependency>
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/BlockingGrpcPubSubSubscriber.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/BlockingGrpcPubSubSubscriber.java
new file mode 100644
index 0000000..4b63912
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/BlockingGrpcPubSubSubscriber.java
@@ -0,0 +1,138 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriber;
+
+import com.google.pubsub.v1.AcknowledgeRequest;
+import com.google.pubsub.v1.PullRequest;
+import com.google.pubsub.v1.ReceivedMessage;
+import com.google.pubsub.v1.SubscriberGrpc;
+import io.grpc.ManagedChannel;
+import io.grpc.StatusRuntimeException;
+
+import java.time.Duration;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static java.util.Collections.emptyList;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+/**
+ * Implementation for {@link PubSubSubscriber}.
+ * This Grpc PubSubSubscriber allows for flexible timeouts and retries.
+ */
+public class BlockingGrpcPubSubSubscriber implements PubSubSubscriber {
+	private final String projectSubscriptionName;
+	private final ManagedChannel channel;
+	private final SubscriberGrpc.SubscriberBlockingStub stub;
+	private final int retries;
+	private final Duration timeout;
+	private final PullRequest pullRequest;
+
+	public BlockingGrpcPubSubSubscriber(String projectSubscriptionName,
+										ManagedChannel channel,
+										SubscriberGrpc.SubscriberBlockingStub stub,
+										PullRequest pullRequest,
+										int retries,
+										Duration timeout) {
+		this.projectSubscriptionName = projectSubscriptionName;
+		this.channel = channel;
+		this.stub = stub;
+		this.retries = retries;
+		this.timeout = timeout;
+		this.pullRequest = pullRequest;
+	}
+
+	@Override
+	public List<ReceivedMessage> pull() {
+		return pull(retries);
+	}
+
+	private List<ReceivedMessage> pull(int retriesRemaining) {
+		try {
+			return stub.withDeadlineAfter(timeout.toMillis(), TimeUnit.MILLISECONDS)
+					.pull(pullRequest)
+					.getReceivedMessagesList();
+		} catch (StatusRuntimeException e) {
+			if (retriesRemaining > 0) {
+				return pull(retriesRemaining - 1);
+			}
+
+			throw e;
+		}
+	}
+
+	@Override
+	public void acknowledge(List<String> acknowledgementIds) {
+		if (acknowledgementIds.isEmpty()) {
+			return;
+		}
+
+		//grpc servers won't accept acknowledge requests that are too large so we split the ackIds
+		Tuple2<List<String>, List<String>> splittedAckIds = splitAckIds(acknowledgementIds);
+		while (!splittedAckIds.f0.isEmpty()) {
+			AcknowledgeRequest acknowledgeRequest =
+					AcknowledgeRequest.newBuilder()
+									.setSubscription(projectSubscriptionName)
+									.addAllAckIds(splittedAckIds.f0)
+									.build();
+
+			stub.withDeadlineAfter(60, SECONDS).acknowledge(acknowledgeRequest);
+
+			splittedAckIds = splitAckIds(splittedAckIds.f1);
+		}
+	}
+
+	/* maxPayload is the maximum number of bytes to devote to actual ids in
+	 * acknowledgement or modifyAckDeadline requests. A serialized
+	 * AcknowledgeRequest grpc call has a small constant overhead, plus the size of the
+	 * subscription name, plus 3 bytes per ID (a tag byte and two size bytes). A
+	 * ModifyAckDeadlineRequest has an additional few bytes for the deadline. We
+	 * don't know the subscription name here, so we just assume the size exclusive
+	 * of ids is 100 bytes.
+
+	 * With gRPC there is no way for the client to know the server's max message size (it is
+	 * configurable on the server). We know from experience that it is 512K.
+	 * @return First list contains no more than 512k bytes, second list contains remaining ids
+	 */
+	private Tuple2<List<String>, List<String>> splitAckIds(List<String> ackIds) {
+		final int maxPayload = 500 * 1024; //little below 512k bytes to be on the safe side
+		final int fixedOverheadPerCall = 100;
+		final int overheadPerId = 3;
+
+		int totalBytes = fixedOverheadPerCall;
+
+		for (int i = 0; i < ackIds.size(); i++) {
+			totalBytes += ackIds.get(i).length() + overheadPerId;
+			if (totalBytes > maxPayload) {
+				return Tuple2.of(ackIds.subList(0, i), ackIds.subList(i, ackIds.size()));
+			}
+		}
+
+		return Tuple2.of(ackIds, emptyList());
+	}
+
+	@Override
+	public void close() throws Exception {
+		channel.shutdownNow();
+		channel.awaitTermination(20, SECONDS);
+	}
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/DefaultPubSubSubscriberFactory.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/DefaultPubSubSubscriberFactory.java
new file mode 100644
index 0000000..da39516
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/DefaultPubSubSubscriberFactory.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.gcp.pubsub;
+
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriber;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriberFactory;
+
+import com.google.auth.Credentials;
+import com.google.cloud.pubsub.v1.stub.SubscriberStubSettings;
+import com.google.pubsub.v1.PullRequest;
+import com.google.pubsub.v1.SubscriberGrpc;
+import io.grpc.ManagedChannel;
+import io.grpc.auth.MoreCallCredentials;
+import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts;
+import io.grpc.netty.shaded.io.grpc.netty.NegotiationType;
+import io.grpc.netty.shaded.io.grpc.netty.NettyChannelBuilder;
+
+import java.io.IOException;
+import java.time.Duration;
+
+class DefaultPubSubSubscriberFactory implements PubSubSubscriberFactory {
+	private final int retries;
+	private final Duration timeout;
+	private final int maxMessagesPerPull;
+	private final String projectSubscriptionName;
+
+	DefaultPubSubSubscriberFactory(String projectSubscriptionName, int retries, Duration pullTimeout, int maxMessagesPerPull) {
+		this.retries = retries;
+		this.timeout = pullTimeout;
+		this.maxMessagesPerPull = maxMessagesPerPull;
+		this.projectSubscriptionName = projectSubscriptionName;
+	}
+
+	@Override
+	public PubSubSubscriber getSubscriber(Credentials credentials) throws IOException {
+		ManagedChannel channel = NettyChannelBuilder.forTarget(SubscriberStubSettings.getDefaultEndpoint())
+													.negotiationType(NegotiationType.TLS)
+													.sslContext(GrpcSslContexts.forClient().ciphers(null).build())
+													.build();
+
+		PullRequest pullRequest = PullRequest.newBuilder()
+								.setMaxMessages(maxMessagesPerPull)
+								.setReturnImmediately(false)
+								.setSubscription(projectSubscriptionName)
+								.build();
+		SubscriberGrpc.SubscriberBlockingStub stub = SubscriberGrpc.newBlockingStub(channel)
+							.withCallCredentials(MoreCallCredentials.from(credentials));
+		return new BlockingGrpcPubSubSubscriber(projectSubscriptionName, channel, stub, pullRequest, retries, timeout);
+	}
+
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/DeserializationSchemaWrapper.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/DeserializationSchemaWrapper.java
new file mode 100644
index 0000000..68e25af
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/DeserializationSchemaWrapper.java
@@ -0,0 +1,51 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubDeserializationSchema;
+
+import com.google.pubsub.v1.PubsubMessage;
+
+/**
+ * This class wraps a {@link DeserializationSchema} so it can be used in a {@link PubSubSource} as a {@link PubSubDeserializationSchema}.
+ */
+class DeserializationSchemaWrapper<T> implements PubSubDeserializationSchema<T> {
+	private final DeserializationSchema<T> deserializationSchema;
+
+	DeserializationSchemaWrapper(DeserializationSchema<T> deserializationSchema) {
+		this.deserializationSchema = deserializationSchema;
+	}
+
+	@Override
+	public boolean isEndOfStream(T t) {
+		return deserializationSchema.isEndOfStream(t);
+	}
+
+	@Override
+	public T deserialize(PubsubMessage pubsubMessage) throws Exception {
+		return deserializationSchema.deserialize(pubsubMessage.getData().toByteArray());
+	}
+
+	@Override
+	public TypeInformation<T> getProducedType() {
+		return deserializationSchema.getProducedType();
+	}
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSink.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSink.java
new file mode 100644
index 0000000..d280dc9
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSink.java
@@ -0,0 +1,317 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.util.Preconditions;
+
+import com.google.api.core.ApiFuture;
+import com.google.api.core.ApiFutureCallback;
+import com.google.api.core.ApiFutures;
+import com.google.api.gax.core.FixedCredentialsProvider;
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.GrpcTransportChannel;
+import com.google.api.gax.rpc.FixedTransportChannelProvider;
+import com.google.api.gax.rpc.TransportChannel;
+import com.google.auth.Credentials;
+import com.google.cloud.pubsub.v1.Publisher;
+import com.google.protobuf.ByteString;
+import com.google.pubsub.v1.ProjectTopicName;
+import com.google.pubsub.v1.PubsubMessage;
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static com.google.cloud.pubsub.v1.SubscriptionAdminSettings.defaultCredentialsProviderBuilder;
+import static org.apache.flink.runtime.concurrent.Executors.directExecutor;
+
+/**
+ * A sink function that outputs to PubSub.
+ *
+ * @param <IN> type of PubSubSink messages to write
+ */
+public class PubSubSink<IN> extends RichSinkFunction<IN> implements CheckpointedFunction {
+	private static final Logger LOG = LoggerFactory.getLogger(PubSubSink.class);
+
+	private final AtomicReference<Exception> exceptionAtomicReference;
+	private final ApiFutureCallback<String> failureHandler;
+	private final ConcurrentLinkedQueue<ApiFuture<String>> outstandingFutures;
+	private final Credentials credentials;
+	private final SerializationSchema<IN> serializationSchema;
+	private final String projectName;
+	private final String topicName;
+	private final String hostAndPortForEmulator;
+
+	private transient Publisher publisher;
+	private volatile boolean isRunning;
+
+	private PubSubSink(
+		Credentials credentials,
+		SerializationSchema<IN> serializationSchema,
+		String projectName,
+		String topicName,
+		String hostAndPortForEmulator) {
+		this.exceptionAtomicReference = new AtomicReference<>();
+		this.failureHandler = new FailureHandler();
+		this.outstandingFutures = new ConcurrentLinkedQueue<>();
+		this.credentials = credentials;
+		this.serializationSchema = serializationSchema;
+		this.projectName = projectName;
+		this.topicName = topicName;
+		this.hostAndPortForEmulator = hostAndPortForEmulator;
+	}
+
+	private transient ManagedChannel managedChannel = null;
+	private transient TransportChannel channel = null;
+
+	@Override
+	public void open(Configuration configuration) throws Exception {
+		Publisher.Builder builder = Publisher
+			.newBuilder(ProjectTopicName.of(projectName, topicName))
+			.setCredentialsProvider(FixedCredentialsProvider.create(credentials));
+
+		if (hostAndPortForEmulator != null) {
+			managedChannel = ManagedChannelBuilder
+				.forTarget(hostAndPortForEmulator)
+				.usePlaintext(true) // This is 'Ok' because this is ONLY used for testing.
+				.build();
+			channel = GrpcTransportChannel.newBuilder().setManagedChannel(managedChannel).build();
+			builder.setChannelProvider(FixedTransportChannelProvider.create(channel))
+					.setCredentialsProvider(NoCredentialsProvider.create());
+		}
+
+		publisher = builder.build();
+		isRunning = true;
+	}
+
+	@Override
+	public void close() throws Exception {
+		super.close();
+		shutdownPublisher();
+		shutdownTransportChannel();
+		shutdownManagedChannel();
+		isRunning = false;
+	}
+
+	private void shutdownPublisher() {
+		try {
+			if (publisher != null) {
+				publisher.shutdown();
+			}
+		} catch (Exception e) {
+			LOG.info("Shutting down Publisher failed.", e);
+		}
+	}
+
+	private void shutdownTransportChannel() {
+		if (channel == null) {
+			return;
+		}
+		try {
+			channel.close();
+		} catch (Exception e) {
+			LOG.info("Shutting down TransportChannel failed.", e);
+		}
+	}
+
+	private void shutdownManagedChannel() {
+		if (managedChannel == null) {
+			return;
+		}
+		try {
+			managedChannel.shutdownNow();
+			managedChannel.awaitTermination(1000L, TimeUnit.MILLISECONDS);
+		} catch (Exception e) {
+			LOG.info("Shutting down ManagedChannel failed.", e);
+		}
+	}
+
+	@Override
+	public void invoke(IN message, SinkFunction.Context context) throws Exception {
+		PubsubMessage pubsubMessage = PubsubMessage
+			.newBuilder()
+			.setData(ByteString.copyFrom(serializationSchema.serialize(message)))
+			.build();
+
+		ApiFuture<String> future = publisher.publish(pubsubMessage);
+		outstandingFutures.add(future);
+		ApiFutures.addCallback(future, failureHandler, directExecutor());
+	}
+
+	/**
+	 * Create a builder for a new PubSubSink.
+	 *
+	 * @param <IN> The generic of the type that is to be written into the sink.
+	 * @return a new PubSubSinkBuilder instance
+	 */
+	public static <IN> SerializationSchemaBuilder<IN> newBuilder(Class<IN> clazz) {
+		return new PubSubSinkBuilder<>();
+	}
+
+	@Override
+	public void snapshotState(FunctionSnapshotContext context) throws Exception {
+		//before checkpoints make sure all the batched / buffered pubsub messages have actually been sent
+		publisher.publishAllOutstanding();
+
+		waitForFuturesToComplete();
+		if (exceptionAtomicReference.get() != null) {
+			throw exceptionAtomicReference.get();
+		}
+	}
+
+	private void waitForFuturesToComplete() {
+		while (isRunning && !outstandingFutures.isEmpty()) {
+			outstandingFutures.removeIf(ApiFuture::isDone);
+		}
+	}
+
+	@Override
+	public void initializeState(FunctionInitializationContext context) throws Exception {
+	}
+
+	/**
+	 * PubSubSinkBuilder to create a PubSubSink.
+	 *
+	 * @param <IN> Type of PubSubSink to create.
+	 */
+	public static class PubSubSinkBuilder<IN> implements SerializationSchemaBuilder<IN>, ProjectNameBuilder<IN>, TopicNameBuilder<IN> {
+		private SerializationSchema<IN> serializationSchema;
+		private String projectName;
+		private String topicName;
+
+		private Credentials credentials;
+		private String hostAndPort;
+
+		private PubSubSinkBuilder() { }
+
+		/**
+		 * Set the credentials.
+		 * If this is not used then the credentials are picked up from the environment variables.
+		 *
+		 * @param credentials the Credentials needed to connect.
+		 * @return The current PubSubSinkBuilder instance
+		 */
+		public PubSubSinkBuilder<IN> withCredentials(Credentials credentials) {
+			this.credentials = credentials;
+			return this;
+		}
+
+		@Override
+		public ProjectNameBuilder<IN> withSerializationSchema(SerializationSchema<IN> serializationSchema) {
+			Preconditions.checkNotNull(serializationSchema);
+			this.serializationSchema = serializationSchema;
+			return this;
+		}
+
+		@Override
+		public TopicNameBuilder<IN> withProjectName(String projectName) {
+			Preconditions.checkNotNull(projectName);
+			this.projectName = projectName;
+			return this;
+		}
+
+		@Override
+		public PubSubSinkBuilder<IN> withTopicName(String topicName) {
+			Preconditions.checkNotNull(topicName);
+			this.topicName = topicName;
+			return this;
+		}
+
+		/**
+		 * Set the custom hostname/port combination of PubSub.
+		 * The ONLY reason to use this is during tests with the emulator provided by Google.
+		 *
+		 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
+		 * @return The current PubSubSinkBuilder instance
+		 */
+		public PubSubSinkBuilder<IN> withHostAndPortForEmulator(String hostAndPort) {
+			this.hostAndPort = hostAndPort;
+			return this;
+		}
+
+		/**
+		 * Actually builder the desired instance of the PubSubSink.
+		 *
+		 * @return a brand new PubSubSink
+		 * @throws IOException              in case of a problem getting the credentials
+		 * @throws IllegalArgumentException in case required fields were not specified.
+		 */
+		public PubSubSink<IN> build() throws IOException {
+			if (credentials == null) {
+				credentials = defaultCredentialsProviderBuilder().build().getCredentials();
+			}
+			return new PubSubSink<>(credentials, serializationSchema, projectName, topicName, hostAndPort);
+		}
+	}
+
+	/**
+	 * Part of {@link PubSubSinkBuilder} to set required fields.
+	 */
+	public interface SerializationSchemaBuilder<IN> {
+		/**
+		 * Set the SerializationSchema used to Serialize objects to be added as payloads of PubSubMessages.
+		 */
+		ProjectNameBuilder<IN> withSerializationSchema(SerializationSchema<IN> deserializationSchema);
+	}
+
+	/**
+	 * Part of {@link PubSubSinkBuilder} to set required fields.
+	 */
+	public interface ProjectNameBuilder<IN> {
+		/**
+		 * Set the project name of the subscription to pull messages from.
+		 */
+		TopicNameBuilder<IN> withProjectName(String projectName);
+	}
+
+	/**
+	 * Part of {@link PubSubSinkBuilder} to set required fields.
+	 */
+	public interface TopicNameBuilder<IN> {
+		/**
+		 * Set the subscription name of the subscription to pull messages from.
+		 */
+		PubSubSinkBuilder<IN> withTopicName(String topicName);
+	}
+
+	private class FailureHandler implements ApiFutureCallback<String>, Serializable {
+		@Override
+		public void onFailure(Throwable t) {
+			exceptionAtomicReference.set(new RuntimeException("Failed trying to publish message", t));
+		}
+
+		@Override
+		public void onSuccess(String result) {
+			//do nothing on success
+			LOG.debug("Successfully published message with id: {}", result);
+		}
+	}
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSource.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSource.java
new file mode 100644
index 0000000..d093e0a
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSource.java
@@ -0,0 +1,347 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.AcknowledgeIdsForCheckpoint;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.AcknowledgeOnCheckpoint;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.Acknowledger;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubDeserializationSchema;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriber;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriberFactory;
+import org.apache.flink.util.Preconditions;
+
+import com.google.auth.Credentials;
+import com.google.cloud.pubsub.v1.Subscriber;
+import com.google.pubsub.v1.ProjectSubscriptionName;
+import com.google.pubsub.v1.PubsubMessage;
+import com.google.pubsub.v1.ReceivedMessage;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.List;
+import java.util.concurrent.CancellationException;
+
+import static com.google.cloud.pubsub.v1.SubscriptionAdminSettings.defaultCredentialsProviderBuilder;
+
+/**
+ * PubSub Source, this Source will consume PubSub messages from a subscription and Acknowledge them on the next checkpoint.
+ * This ensures every message will get acknowledged at least once.
+ */
+public class PubSubSource<OUT> extends RichSourceFunction<OUT>
+	implements ResultTypeQueryable<OUT>, ParallelSourceFunction<OUT>, CheckpointListener, ListCheckpointed<AcknowledgeIdsForCheckpoint<String>> {
+	public static final int NO_MAX_MESSAGES_TO_ACKNOWLEDGE_LIMIT = -1;
+	private static final Logger LOG = LoggerFactory.getLogger(PubSubSource.class);
+	protected final PubSubDeserializationSchema<OUT> deserializationSchema;
+	protected final PubSubSubscriberFactory pubSubSubscriberFactory;
+	protected final Credentials credentials;
+	protected final int maxMessagesToAcknowledge;
+	protected final AcknowledgeOnCheckpointFactory acknowledgeOnCheckpointFactory;
+
+	protected transient AcknowledgeOnCheckpoint<String> acknowledgeOnCheckpoint;
+	protected transient PubSubSubscriber subscriber;
+
+	protected transient volatile boolean isRunning;
+
+	PubSubSource(PubSubDeserializationSchema<OUT> deserializationSchema,
+				PubSubSubscriberFactory pubSubSubscriberFactory,
+				Credentials credentials,
+				int maxMessagesToAcknowledge,
+				AcknowledgeOnCheckpointFactory acknowledgeOnCheckpointFactory) {
+		this.deserializationSchema = deserializationSchema;
+		this.pubSubSubscriberFactory = pubSubSubscriberFactory;
+		this.credentials = credentials;
+		this.maxMessagesToAcknowledge = maxMessagesToAcknowledge;
+		this.acknowledgeOnCheckpointFactory = acknowledgeOnCheckpointFactory;
+	}
+
+	@Override
+	public void open(Configuration configuration) throws Exception {
+		super.open(configuration);
+		if (hasNoCheckpointingEnabled(getRuntimeContext())) {
+			throw new IllegalArgumentException("The PubSubSource REQUIRES Checkpointing to be enabled and " +
+				"the checkpointing frequency must be MUCH lower than the PubSub timeout for it to retry a message.");
+		}
+
+		getRuntimeContext().getMetricGroup().gauge("PubSubMessagesProcessedNotAcked", this::getOutstandingMessagesToAck);
+
+		createAndSetPubSubSubscriber();
+		this.isRunning = true;
+	}
+
+	private boolean hasNoCheckpointingEnabled(RuntimeContext runtimeContext) {
+		return !(runtimeContext instanceof StreamingRuntimeContext && ((StreamingRuntimeContext) runtimeContext).isCheckpointingEnabled());
+	}
+
+	@Override
+	public void run(SourceContext<OUT> sourceContext) throws Exception {
+		while (isRunning) {
+			try {
+				if (maxMessagesToAcknowledgeLimitReached()) {
+					LOG.debug("Sleeping because there are {} messages waiting to be ack'ed but limit is {}", getOutstandingMessagesToAck(), maxMessagesToAcknowledge);
+					Thread.sleep(100);
+					continue;
+				}
+
+				processMessage(sourceContext, subscriber.pull());
+			} catch (InterruptedException | CancellationException e) {
+				isRunning = false;
+			}
+		}
+		subscriber.close();
+	}
+
+	void processMessage(SourceContext<OUT> sourceContext, List<ReceivedMessage> messages) throws Exception {
+		synchronized (sourceContext.getCheckpointLock()) {
+			for (ReceivedMessage message : messages) {
+				acknowledgeOnCheckpoint.addAcknowledgeId(message.getAckId());
+
+				PubsubMessage pubsubMessage = message.getMessage();
+
+				OUT deserializedMessage = deserializationSchema.deserialize(pubsubMessage);
+				if (deserializationSchema.isEndOfStream(deserializedMessage)) {
+					cancel();
+					return;
+				}
+
+				sourceContext.collect(deserializedMessage);
+			}
+
+		}
+	}
+
+	private boolean maxMessagesToAcknowledgeLimitReached() throws Exception {
+		return maxMessagesToAcknowledge != NO_MAX_MESSAGES_TO_ACKNOWLEDGE_LIMIT && getOutstandingMessagesToAck() > maxMessagesToAcknowledge;
+	}
+
+	private Integer getOutstandingMessagesToAck() {
+		return acknowledgeOnCheckpoint.numberOfOutstandingAcknowledgements();
+	}
+
+	@Override
+	public void cancel() {
+		isRunning = false;
+	}
+
+	@Override
+	public TypeInformation<OUT> getProducedType() {
+		return deserializationSchema.getProducedType();
+	}
+
+	public static <OUT> DeserializationSchemaBuilder<OUT> newBuilder(Class<OUT> clazz) {
+		return new PubSubSourceBuilder<>();
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) throws Exception {
+		acknowledgeOnCheckpoint.notifyCheckpointComplete(checkpointId);
+	}
+
+	@Override
+	public List<AcknowledgeIdsForCheckpoint<String>> snapshotState(long checkpointId, long timestamp) throws Exception {
+		return acknowledgeOnCheckpoint.snapshotState(checkpointId, timestamp);
+	}
+
+	@Override
+	public void restoreState(List<AcknowledgeIdsForCheckpoint<String>> state) throws Exception {
+		createAndSetPubSubSubscriber();
+		acknowledgeOnCheckpoint.restoreState(state);
+	}
+
+	private void createAndSetPubSubSubscriber() throws Exception {
+		if (subscriber == null) {
+			this.subscriber = pubSubSubscriberFactory.getSubscriber(credentials);
+		}
+		if (acknowledgeOnCheckpoint == null) {
+			this.acknowledgeOnCheckpoint = acknowledgeOnCheckpointFactory.create(subscriber);
+		}
+	}
+
+	/**
+	 * Builder to create PubSubSource.
+	 *
+	 * @param <OUT> The type of objects which will be read
+	 */
+	public static class PubSubSourceBuilder<OUT> implements DeserializationSchemaBuilder<OUT>, ProjectNameBuilder<OUT>, SubscriptionNameBuilder<OUT> {
+		private PubSubDeserializationSchema<OUT> deserializationSchema;
+		private String projectName;
+		private String subscriptionName;
+
+		private PubSubSubscriberFactory pubSubSubscriberFactory;
+		private Credentials credentials;
+		private int maxMessageToAcknowledge = 10000;
+
+		protected PubSubSourceBuilder() {
+		}
+
+		@Override
+		public ProjectNameBuilder withDeserializationSchema(DeserializationSchema<OUT> deserializationSchema) {
+			Preconditions.checkNotNull(deserializationSchema);
+			this.deserializationSchema = new DeserializationSchemaWrapper<>(deserializationSchema);
+			return this;
+		}
+
+		@Override
+		public ProjectNameBuilder withDeserializationSchema(PubSubDeserializationSchema deserializationSchema) {
+			Preconditions.checkNotNull(deserializationSchema);
+			this.deserializationSchema = deserializationSchema;
+			return this;
+		}
+
+		@Override
+		public SubscriptionNameBuilder withProjectName(String projectName) {
+			Preconditions.checkNotNull(projectName);
+			this.projectName = projectName;
+			return this;
+		}
+
+		@Override
+		public PubSubSourceBuilder<OUT> withSubscriptionName(String subscriptionName) {
+			Preconditions.checkNotNull(subscriptionName);
+			this.subscriptionName = subscriptionName;
+			return this;
+		}
+
+		/**
+		 * Set the credentials.
+		 * If this is not used then the credentials are picked up from the environment variables.
+		 *
+		 * @param credentials the Credentials needed to connect.
+		 * @return The current PubSubSourceBuilder instance
+		 */
+		public PubSubSourceBuilder<OUT> withCredentials(Credentials credentials) {
+			this.credentials = credentials;
+			return this;
+		}
+
+		/**
+		 * Set a PubSubSubscriberFactory
+		 * This allows for custom Subscriber options to be set.
+		 * {@link DefaultPubSubSubscriberFactory} is the default.
+		 *
+		 * @param pubSubSubscriberFactory A factory to create a {@link Subscriber}
+		 * @return The current PubSubSourceBuilder instance
+		 */
+		public PubSubSourceBuilder<OUT> withPubSubSubscriberFactory(PubSubSubscriberFactory pubSubSubscriberFactory) {
+			this.pubSubSubscriberFactory = pubSubSubscriberFactory;
+			return this;
+		}
+
+		/**
+		 * There is a default PubSubSubscriber factory that uses gRPC to pull in PubSub messages. This method can be used to tune this default factory.
+         * Note this will not work in combination with a custom PubSubSubscriber factory.
+		 * @param maxMessagesPerPull the number of messages pulled per request. Default: 100
+		 * @param perRequestTimeout the timeout per request. Default: 15 seconds
+		 * @param retries the number of retries when requests fail
+		 * @return The current PubSubSourceBuilder instance
+		 */
+		public PubSubSourceBuilder<OUT> withPubSubSubscriberFactory(int maxMessagesPerPull, Duration perRequestTimeout, int retries) {
+			this.pubSubSubscriberFactory = new DefaultPubSubSubscriberFactory(ProjectSubscriptionName.format(projectName, subscriptionName),
+																			retries,
+																			perRequestTimeout,
+																			maxMessagesPerPull);
+			return this;
+		}
+
+		/**
+		 * Set a limit of the number of outstanding or to-be acknowledged messages.
+		 * default is 10000. Adjust this if you have high checkpoint intervals and / or run into memory issues
+		 * due to the amount of acknowledgement ids. Use {@link PubSubSource}.NO_MAX_MESSAGES_TO_ACKNOWLEDGE_LIMIT if you want to remove the limit.
+		 */
+		public PubSubSourceBuilder<OUT> withMaxMessageToAcknowledge(int maxMessageToAcknowledge) {
+			this.maxMessageToAcknowledge = maxMessageToAcknowledge;
+			return this;
+		}
+
+		/**
+		 * Actually build the desired instance of the PubSubSourceBuilder.
+		 *
+		 * @return a brand new SourceFunction
+		 * @throws IOException              in case of a problem getting the credentials
+		 * @throws IllegalArgumentException in case required fields were not specified.
+		 */
+		public PubSubSource<OUT> build() throws IOException {
+			if (credentials == null) {
+				credentials = defaultCredentialsProviderBuilder().build().getCredentials();
+			}
+
+			if (pubSubSubscriberFactory == null) {
+				pubSubSubscriberFactory = new DefaultPubSubSubscriberFactory(ProjectSubscriptionName.format(projectName, subscriptionName),
+																			3,
+																			Duration.ofSeconds(15),
+																			100);
+			}
+
+			return new PubSubSource<>(deserializationSchema, pubSubSubscriberFactory, credentials, maxMessageToAcknowledge, new AcknowledgeOnCheckpointFactory());
+		}
+	}
+
+	/**
+	 * Part of {@link PubSubSourceBuilder} to set required fields.
+	 */
+	public interface DeserializationSchemaBuilder<OUT> {
+		/**
+		 * Set the DeserializationSchema used to deserialize incoming PubSubMessages.
+		 * If you want access to meta data of a PubSubMessage use the overloaded withDeserializationSchema({@link PubSubDeserializationSchema}) method instead.
+		 */
+		ProjectNameBuilder<OUT> withDeserializationSchema(DeserializationSchema<OUT> deserializationSchema);
+
+		/**
+		 * Set the DeserializationSchema used to deserialize incoming PubSubMessages.
+		 */
+		ProjectNameBuilder<OUT> withDeserializationSchema(PubSubDeserializationSchema<OUT> deserializationSchema);
+	}
+
+	/**
+	 * Part of {@link PubSubSourceBuilder} to set required fields.
+	 */
+	public interface ProjectNameBuilder<OUT> {
+		/**
+		 * Set the project name of the subscription to pull messages from.
+		 */
+		SubscriptionNameBuilder<OUT> withProjectName(String projectName);
+	}
+
+	/**
+	 * Part of {@link PubSubSourceBuilder} to set required fields.
+	 */
+	public interface SubscriptionNameBuilder<OUT> {
+		/**
+		 * Set the subscription name of the subscription to pull messages from.
+		 */
+		PubSubSourceBuilder<OUT> withSubscriptionName(String subscriptionName);
+	}
+
+	static class AcknowledgeOnCheckpointFactory implements Serializable {
+		AcknowledgeOnCheckpoint<String> create(Acknowledger<String> acknowledger) {
+			return new AcknowledgeOnCheckpoint<>(acknowledger);
+		}
+	}
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSubscriberFactoryForEmulator.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSubscriberFactoryForEmulator.java
new file mode 100644
index 0000000..61c0623
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSubscriberFactoryForEmulator.java
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub;
+
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriber;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriberFactory;
+
+import com.google.auth.Credentials;
+import com.google.pubsub.v1.ProjectSubscriptionName;
+import com.google.pubsub.v1.PullRequest;
+import com.google.pubsub.v1.SubscriberGrpc;
+import io.grpc.ManagedChannel;
+import io.grpc.netty.shaded.io.grpc.netty.NettyChannelBuilder;
+
+import java.io.IOException;
+import java.time.Duration;
+
+/**
+ * A convenience PubSubSubscriberFactory that can be used to connect to a PubSub emulator.
+ * The PubSub emulators do not support SSL or Credentials and as such this SubscriberStub does not require or provide this.
+ */
+public class PubSubSubscriberFactoryForEmulator implements PubSubSubscriberFactory {
+	private final String hostAndPort;
+	private final String projectSubscriptionName;
+	private final int retries;
+	private final Duration timeout;
+	private final int maxMessagesPerPull;
+
+	public PubSubSubscriberFactoryForEmulator(String hostAndPort, String project, String subscription, int retries, Duration timeout, int maxMessagesPerPull) {
+		this.hostAndPort = hostAndPort;
+		this.retries = retries;
+		this.timeout = timeout;
+		this.maxMessagesPerPull = maxMessagesPerPull;
+		this.projectSubscriptionName = ProjectSubscriptionName.format(project, subscription);
+	}
+
+	@Override
+	public PubSubSubscriber getSubscriber(Credentials credentials) throws IOException {
+		ManagedChannel managedChannel = NettyChannelBuilder.forTarget(hostAndPort)
+														.usePlaintext() // This is 'Ok' because this is ONLY used for testing.
+														.build();
+
+		PullRequest pullRequest = PullRequest.newBuilder()
+											.setMaxMessages(maxMessagesPerPull)
+											.setReturnImmediately(false)
+											.setSubscription(projectSubscriptionName)
+											.build();
+		SubscriberGrpc.SubscriberBlockingStub stub = SubscriberGrpc.newBlockingStub(managedChannel);
+		return new BlockingGrpcPubSubSubscriber(projectSubscriptionName, managedChannel, stub, pullRequest, retries, timeout);
+	}
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeIdsForCheckpoint.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeIdsForCheckpoint.java
new file mode 100644
index 0000000..9de79f9
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeIdsForCheckpoint.java
@@ -0,0 +1,52 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub.common;
+
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * This class contains a checkpointId and a List of AcknowledgementIds.
+ * This class is used by {@link AcknowledgeOnCheckpoint} to keep track of acknowledgementIds
+ * @param <AcknowledgeId> Type of the Ids used for acknowledging.
+ */
+public class AcknowledgeIdsForCheckpoint<AcknowledgeId> implements Serializable {
+	private long checkpointId;
+	private List<AcknowledgeId> acknowledgeIds;
+
+	AcknowledgeIdsForCheckpoint(long checkpointId, List<AcknowledgeId> acknowledgeIds) {
+		this.checkpointId = checkpointId;
+		this.acknowledgeIds = acknowledgeIds;
+	}
+
+	public long getCheckpointId() {
+		return checkpointId;
+	}
+
+	public void setCheckpointId(long checkpointId) {
+		this.checkpointId = checkpointId;
+	}
+
+	public List<AcknowledgeId> getAcknowledgeIds() {
+		return acknowledgeIds;
+	}
+
+	public void setAcknowledgeIds(List<AcknowledgeId> acknowledgeIds) {
+		this.acknowledgeIds = acknowledgeIds;
+	}
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeOnCheckpoint.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeOnCheckpoint.java
new file mode 100644
index 0000000..f538b69
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeOnCheckpoint.java
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub.common;
+
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static java.util.stream.Collectors.toList;
+
+/**
+ * Helper class for SourceFunctions to acknowledge messages to external systems after a successful checkpoint.
+ *
+ * <p>The mechanism for this source assumes that messages are identified by a unique ID.
+ * When messages are taken from the message queue, the message must not be dropped immediately from the external system,
+ * but must be retained until acknowledged. Messages that are not acknowledged within a certain
+ * time interval will be served again (to a different connection, established by the recovered source).
+ *
+ * <p>Note that this source can give no guarantees about message order in the case of failures,
+ * because messages that were retrieved but not yet acknowledged will be returned later again, after
+ * a set of messages that was not retrieved before the failure.
+ *
+ * <p>Internally, this class gathers the IDs of elements it emits. Per checkpoint, the IDs are stored and
+ * acknowledged when the checkpoint is complete. That way, no message is acknowledged unless it is certain
+ * that it has been successfully processed throughout the topology and the updates to any state caused by
+ * that message are persistent.
+ *
+ * @param <ACKID> Type of Ids to acknowledge
+ */
+public class AcknowledgeOnCheckpoint<ACKID extends Serializable> implements CheckpointListener, ListCheckpointed<AcknowledgeIdsForCheckpoint<ACKID>> {
+	private final Acknowledger<ACKID> acknowledger;
+	private List<AcknowledgeIdsForCheckpoint<ACKID>> acknowledgeIdsPerCheckpoint;
+	private List<ACKID> acknowledgeIdsForPendingCheckpoint;
+	private AtomicInteger outstandingAcknowledgements;
+
+	public AcknowledgeOnCheckpoint(Acknowledger<ACKID> acknowledger) {
+		this.acknowledger = acknowledger;
+		this.acknowledgeIdsPerCheckpoint = new ArrayList<>();
+		this.acknowledgeIdsForPendingCheckpoint = new ArrayList<>();
+		this.outstandingAcknowledgements = new AtomicInteger(0);
+	}
+
+	public void addAcknowledgeId(ACKID id) {
+		acknowledgeIdsForPendingCheckpoint.add(id);
+		outstandingAcknowledgements.incrementAndGet();
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) throws Exception {
+		//get all acknowledgeIds of this and earlier checkpoints
+		List<ACKID> idsToAcknowledge = acknowledgeIdsPerCheckpoint
+			.stream()
+			.filter(acknowledgeIdsForCheckpoint -> acknowledgeIdsForCheckpoint.getCheckpointId() <= checkpointId)
+			.flatMap(acknowledgeIdsForCheckpoint -> acknowledgeIdsForCheckpoint.getAcknowledgeIds().stream())
+			.collect(toList());
+
+		acknowledger.acknowledge(idsToAcknowledge);
+
+		//only keep acknowledgeIds of newer checkpointIds
+		acknowledgeIdsPerCheckpoint = acknowledgeIdsPerCheckpoint.stream()
+			.filter(acknowledgeIdsForCheckpoint -> acknowledgeIdsForCheckpoint.getCheckpointId() > checkpointId)
+			.collect(toList());
+		outstandingAcknowledgements = new AtomicInteger(numberOfAcknowledgementIds(acknowledgeIdsPerCheckpoint));
+	}
+
+	@Override
+	public List<AcknowledgeIdsForCheckpoint<ACKID>> snapshotState(long checkpointId, long timestamp) throws Exception {
+		acknowledgeIdsPerCheckpoint.add(new AcknowledgeIdsForCheckpoint<>(checkpointId, acknowledgeIdsForPendingCheckpoint));
+		acknowledgeIdsForPendingCheckpoint = new ArrayList<>();
+
+		return acknowledgeIdsPerCheckpoint;
+	}
+
+	@Override
+	public void restoreState(List<AcknowledgeIdsForCheckpoint<ACKID>> state) throws Exception {
+		outstandingAcknowledgements = new AtomicInteger(numberOfAcknowledgementIds(state));
+		acknowledgeIdsPerCheckpoint = state;
+	}
+
+	private int numberOfAcknowledgementIds(List<AcknowledgeIdsForCheckpoint<ACKID>> acknowledgeIdsForCheckpoints) {
+		return acknowledgeIdsForCheckpoints
+			.stream()
+			.map(AcknowledgeIdsForCheckpoint::getAcknowledgeIds)
+			.mapToInt(List::size)
+			.sum();
+	}
+
+	public int numberOfOutstandingAcknowledgements() {
+		return outstandingAcknowledgements.get();
+	}
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/Acknowledger.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/Acknowledger.java
new file mode 100644
index 0000000..d7b741d
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/Acknowledger.java
@@ -0,0 +1,28 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub.common;
+
+import java.util.List;
+
+/**
+ * @param <AcknowledgeId>
+ */
+public interface Acknowledger<AcknowledgeId> {
+
+	void acknowledge(List<AcknowledgeId> ids);
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubDeserializationSchema.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubDeserializationSchema.java
new file mode 100644
index 0000000..ef0ada1
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubDeserializationSchema.java
@@ -0,0 +1,52 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub.common;
+
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+import com.google.pubsub.v1.PubsubMessage;
+
+import java.io.Serializable;
+
+/**
+ * The deserialization schema describes how to turn the PubsubMessages
+ * into data types (Java/Scala objects) that are processed by Flink.
+ *
+ * @param <T> The type created by the deserialization schema.
+ */
+public interface PubSubDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
+
+	/**
+	 * Method to decide whether the element signals the end of the stream. If
+	 * true is returned the element won't be emitted.
+	 *
+	 * @param nextElement The element to test for the end-of-stream signal.
+	 *
+	 * @return True, if the element signals end of stream, false otherwise.
+	 */
+	boolean isEndOfStream(T nextElement);
+
+	/**
+	 * Deserializes a PubsubMessage.
+	 *
+	 * @param message PubsubMessage to be deserialized.
+	 *
+	 * @return The deserialized message as an object (null if the message cannot be deserialized).
+	 */
+	T deserialize(PubsubMessage message) throws Exception;
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubSubscriber.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubSubscriber.java
new file mode 100644
index 0000000..8ff416d
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubSubscriber.java
@@ -0,0 +1,37 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub.common;
+
+import org.apache.flink.streaming.connectors.gcp.pubsub.PubSubSource;
+
+import com.google.pubsub.v1.ReceivedMessage;
+
+import java.util.List;
+
+/**
+ * Used by {@link PubSubSource} to pull and acknowledge messages.
+ */
+public interface PubSubSubscriber extends Acknowledger<String> {
+	List<ReceivedMessage> pull();
+
+	/**
+	 * After returning all resources should have been cleaned up.
+	 */
+	void close() throws Exception;
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubSubscriberFactory.java b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubSubscriberFactory.java
new file mode 100644
index 0000000..d67b0bb
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/PubSubSubscriberFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.gcp.pubsub.common;
+
+import com.google.auth.Credentials;
+import com.google.cloud.pubsub.v1.stub.SubscriberStub;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * A factory class to create a {@link SubscriberStub}.
+ * This allows for customized Subscribers with for instance tweaked configurations.
+ * Note: this class needs to be serializable.
+ */
+public interface PubSubSubscriberFactory extends Serializable {
+	/**
+	 * Creates a new SubscriberStub using the EventLoopGroup and credentials.
+	 * If the SubscriberStub uses a EventLoopGroup, as many Grpc classes do, this EventLoopGroup should be used.
+	 */
+	PubSubSubscriber getSubscriber(Credentials credentials) throws IOException;
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/DeserializationSchemaWrapperTest.java b/flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/DeserializationSchemaWrapperTest.java
new file mode 100644
index 0000000..386a9f9
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/DeserializationSchemaWrapperTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import com.google.protobuf.ByteString;
+import com.google.pubsub.v1.PubsubMessage;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Unit tests for {@link DeserializationSchema}.
+ */
+@RunWith(MockitoJUnitRunner.class)
+public class DeserializationSchemaWrapperTest {
+	@Mock
+	private DeserializationSchema<String> deserializationSchema;
+
+	@InjectMocks
+	private DeserializationSchemaWrapper<String> deserializationSchemaWrapper;
+
+	@Test
+	public void testProducedType() {
+		TypeInformation<String> typeInformation = TypeInformation.of(String.class);
+		when(deserializationSchema.getProducedType()).thenReturn(typeInformation);
+
+		assertThat(deserializationSchemaWrapper.getProducedType(), is(typeInformation));
+		verify(deserializationSchema, times(1)).getProducedType();
+	}
+
+	@Test
+	public void testEndOfStream() {
+		String input = "some-input";
+		when(deserializationSchema.isEndOfStream(any())).thenReturn(true);
+
+		assertThat(deserializationSchemaWrapper.isEndOfStream(input), is(true));
+		verify(deserializationSchema, times(1)).isEndOfStream(input);
+	}
+
+	@Test
+	public void testDeserialize() throws Exception {
+		String inputAsString = "some-input";
+		byte[] inputAsBytes = inputAsString.getBytes();
+
+		when(deserializationSchema.deserialize(any())).thenReturn(inputAsString);
+
+		assertThat(deserializationSchemaWrapper.deserialize(pubSubMessage(inputAsString)), is(inputAsString));
+		verify(deserializationSchema, times(1)).deserialize(inputAsBytes);
+	}
+
+	private PubsubMessage pubSubMessage(String message) {
+		return PubsubMessage.newBuilder()
+			.setMessageId("some id")
+			.setData(ByteString.copyFrom(message.getBytes()))
+			.build();
+	}
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSourceTest.java b/flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSourceTest.java
new file mode 100644
index 0000000..943f075
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSourceTest.java
@@ -0,0 +1,206 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.AcknowledgeIdsForCheckpoint;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.AcknowledgeOnCheckpoint;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubDeserializationSchema;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriber;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriberFactory;
+
+import com.google.auth.Credentials;
+import com.google.protobuf.ByteString;
+import com.google.pubsub.v1.PubsubMessage;
+import com.google.pubsub.v1.ReceivedMessage;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.refEq;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
+
+/**
+ * Test for {@link SourceFunction}.
+ */
+@RunWith(MockitoJUnitRunner.class)
+public class PubSubSourceTest {
+	private static final String FIRST_MESSAGE = "FirstMessage";
+	private static final String SECOND_MESSAGE = "SecondMessage";
+
+	@Mock
+	private SourceFunction.SourceContext<String> sourceContext;
+	@Mock
+	private PubSubDeserializationSchema<String> deserializationSchema;
+	@Mock
+	private PubSubSource.AcknowledgeOnCheckpointFactory acknowledgeOnCheckpointFactory;
+	@Mock
+	private AcknowledgeOnCheckpoint<String> acknowledgeOnCheckpoint;
+	@Mock
+	private StreamingRuntimeContext streamingRuntimeContext;
+	@Mock
+	private MetricGroup metricGroup;
+	@Mock
+	private PubSubSubscriberFactory pubSubSubscriberFactory;
+	@Mock
+	private Credentials credentials;
+	@Mock
+	private PubSubSubscriber pubsubSubscriber;
+
+	private PubSubSource<String> pubSubSource;
+
+	@Before
+	public void setup() throws Exception {
+		when(pubSubSubscriberFactory.getSubscriber(eq(credentials))).thenReturn(pubsubSubscriber);
+		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
+		when(streamingRuntimeContext.getMetricGroup()).thenReturn(metricGroup);
+		when(acknowledgeOnCheckpointFactory.create(any())).thenReturn(acknowledgeOnCheckpoint);
+
+		pubSubSource = new PubSubSource<>(deserializationSchema,
+			pubSubSubscriberFactory,
+			credentials,
+			100,
+			acknowledgeOnCheckpointFactory);
+		pubSubSource.setRuntimeContext(streamingRuntimeContext);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testOpenWithoutCheckpointing() throws Exception {
+		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(false);
+		pubSubSource.open(null);
+	}
+
+	@Test
+	public void testOpenWithCheckpointing() throws Exception {
+		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
+
+		pubSubSource.open(null);
+
+		verify(pubSubSubscriberFactory, times(1)).getSubscriber(eq(credentials));
+		verify(acknowledgeOnCheckpointFactory, times(1)).create(pubsubSubscriber);
+	}
+
+	@Test
+	public void testProcessMessage() throws Exception {
+		when(deserializationSchema.isEndOfStream(any())).thenReturn(false).thenReturn(false);
+		when(deserializationSchema.deserialize(pubSubMessage(FIRST_MESSAGE))).thenReturn(FIRST_MESSAGE);
+		when(deserializationSchema.deserialize(pubSubMessage(SECOND_MESSAGE))).thenReturn(SECOND_MESSAGE);
+		when(sourceContext.getCheckpointLock()).thenReturn("some object to lock on");
+
+		pubSubSource.open(null);
+		pubSubSource.processMessage(sourceContext, asList(receivedMessage("firstAckId", pubSubMessage(FIRST_MESSAGE)),
+															receivedMessage("secondAckId", pubSubMessage(SECOND_MESSAGE))));
+
+		//verify handling messages
+		verify(sourceContext, times(1)).getCheckpointLock();
+		verify(deserializationSchema, times(1)).isEndOfStream(FIRST_MESSAGE);
+		verify(deserializationSchema, times(1)).deserialize(pubSubMessage(FIRST_MESSAGE));
+		verify(sourceContext, times(1)).collect(FIRST_MESSAGE);
+		verify(acknowledgeOnCheckpoint, times(1)).addAcknowledgeId("firstAckId");
+
+		verify(deserializationSchema, times(1)).isEndOfStream(SECOND_MESSAGE);
+		verify(deserializationSchema, times(1)).deserialize(pubSubMessage(SECOND_MESSAGE));
+		verify(sourceContext, times(1)).collect(SECOND_MESSAGE);
+		verify(acknowledgeOnCheckpoint, times(1)).addAcknowledgeId("secondAckId");
+	}
+
+	@Test
+	public void testTypeInformationFromDeserializationSchema() {
+		TypeInformation<String> schemaTypeInformation = TypeInformation.of(String.class);
+		when(deserializationSchema.getProducedType()).thenReturn(schemaTypeInformation);
+
+		TypeInformation<String> actualTypeInformation = pubSubSource.getProducedType();
+
+		assertThat(actualTypeInformation, is(schemaTypeInformation));
+		verify(deserializationSchema, times(1)).getProducedType();
+	}
+
+	@Test
+	public void testStoppingConnectorWhenDeserializationSchemaIndicatesEndOfStream() throws Exception {
+		when(deserializationSchema.deserialize(pubSubMessage(FIRST_MESSAGE))).thenReturn(FIRST_MESSAGE);
+		when(sourceContext.getCheckpointLock()).thenReturn("some object to lock on");
+		pubSubSource.open(null);
+
+		when(deserializationSchema.isEndOfStream(FIRST_MESSAGE)).thenReturn(true);
+
+		ReceivedMessage message = receivedMessage("ackId", pubSubMessage(FIRST_MESSAGE));
+
+		//Process message
+		pubSubSource.processMessage(sourceContext, singletonList(message));
+		verify(deserializationSchema, times(1)).isEndOfStream(FIRST_MESSAGE);
+		verify(sourceContext, times(1)).getCheckpointLock();
+		verifyNoMoreInteractions(sourceContext);
+	}
+
+	@Test
+	public void testNotifyCheckpointComplete() throws Exception {
+		pubSubSource.open(null);
+		pubSubSource.notifyCheckpointComplete(45L);
+
+		verify(acknowledgeOnCheckpoint, times(1)).notifyCheckpointComplete(45L);
+	}
+
+	@Test
+	public void testRestoreState() throws Exception {
+		pubSubSource.open(null);
+
+		List<AcknowledgeIdsForCheckpoint<String>> input = new ArrayList<>();
+		pubSubSource.restoreState(input);
+
+		verify(acknowledgeOnCheckpoint, times(1)).restoreState(refEq(input));
+	}
+
+	@Test
+	public void testSnapshotState() throws Exception {
+		pubSubSource.open(null);
+		pubSubSource.snapshotState(1337L, 15000L);
+
+		verify(acknowledgeOnCheckpoint, times(1)).snapshotState(1337L, 15000L);
+	}
+
+	private ReceivedMessage receivedMessage(String ackId, PubsubMessage pubsubMessage) {
+		return ReceivedMessage.newBuilder()
+								.setAckId(ackId)
+								.setMessage(pubsubMessage)
+								.build();
+	}
+
+	private PubsubMessage pubSubMessage(String message) {
+		return PubsubMessage.newBuilder()
+			.setMessageId("some id")
+			.setData(ByteString.copyFrom(message.getBytes()))
+			.build();
+	}
+}
diff --git a/flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeOnCheckpointTest.java b/flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeOnCheckpointTest.java
new file mode 100644
index 0000000..80113cb
--- /dev/null
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/common/AcknowledgeOnCheckpointTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.flink.streaming.connectors.gcp.pubsub.common;
+
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.util.Arrays.asList;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Unit tests for {@link AcknowledgeOnCheckpoint}.
+ */
+public class AcknowledgeOnCheckpointTest {
+	private final Acknowledger<String> mockedAcknowledger = mock(Acknowledger.class);
+
+	@Test
+	public void testRestoreStateAndSnapshot() throws Exception {
+		List<AcknowledgeIdsForCheckpoint<String>> input = new ArrayList<>();
+		input.add(new AcknowledgeIdsForCheckpoint<>(0, asList("idsFor0", "moreIdsFor0")));
+		input.add(new AcknowledgeIdsForCheckpoint<>(1, asList("idsFor1", "moreIdsFor1")));
+
+		AcknowledgeOnCheckpoint<String> acknowledgeOnCheckpoint = new AcknowledgeOnCheckpoint<>(mockedAcknowledger);
+		acknowledgeOnCheckpoint.restoreState(input);
+
+		List<AcknowledgeIdsForCheckpoint<String>> actual = acknowledgeOnCheckpoint.snapshotState(2, 100);
+
+		assertThat(actual, hasSize(3));
+		assertThat(actual.get(0), equalTo(input.get(0)));
+		assertThat(actual.get(1), equalTo(input.get(1)));
+		assertThat(actual.get(2).getCheckpointId(), is(2L));
+		assertThat(actual.get(2).getAcknowledgeIds(), hasSize(0));
+
+		assertThat(acknowledgeOnCheckpoint.numberOfOutstandingAcknowledgements(), is(4));
+	}
+
+	@Test
+	public void testAddAcknowledgeIdOnEmptyState() throws Exception {
+		AcknowledgeOnCheckpoint<String> acknowledgeOnCheckpoint = new AcknowledgeOnCheckpoint<>(mockedAcknowledger);
+
+		acknowledgeOnCheckpoint.addAcknowledgeId("ackId");
+
+		List<AcknowledgeIdsForCheckpoint<String>> actual = acknowledgeOnCheckpoint.snapshotState(2, 100);
+
+		assertThat(actual.get(0).getCheckpointId(), is(2L));
+		assertThat(actual.get(0).getAcknowledgeIds(), containsInAnyOrder("ackId"));
+
+		assertThat(acknowledgeOnCheckpoint.numberOfOutstandingAcknowledgements(), is(1));
+	}
+
+	@Test
+	public void testAddAcknowledgeIdOnExistingState() throws Exception {
+		List<AcknowledgeIdsForCheckpoint<String>> input = new ArrayList<>();
+		input.add(new AcknowledgeIdsForCheckpoint<>(0, asList("idsFor0", "moreIdsFor0")));
+		input.add(new AcknowledgeIdsForCheckpoint<>(1, asList("idsFor1", "moreIdsFor1")));
+
+		AcknowledgeOnCheckpoint<String> acknowledgeOnCheckpoint = new AcknowledgeOnCheckpoint<>(mockedAcknowledger);
+
+		acknowledgeOnCheckpoint.restoreState(input);
+
+		acknowledgeOnCheckpoint.addAcknowledgeId("ackId");
+
+		List<AcknowledgeIdsForCheckpoint<String>> actual = acknowledgeOnCheckpoint.snapshotState(94, 100);
+
+		assertThat(actual.get(0), equalTo(input.get(0)));
+		assertThat(actual.get(1), equalTo(input.get(1)));
+		assertThat(actual.get(2).getCheckpointId(), is(94L));
+		assertThat(actual.get(2).getAcknowledgeIds(), containsInAnyOrder("ackId"));
+
+		assertThat(acknowledgeOnCheckpoint.numberOfOutstandingAcknowledgements(), is(5));
+	}
+
+	@Test
+	public void testAddMultipleAcknowledgeIds() throws Exception {
+		AcknowledgeOnCheckpoint<String> acknowledgeOnCheckpoint = new AcknowledgeOnCheckpoint<>(mockedAcknowledger);
+
+		acknowledgeOnCheckpoint.addAcknowledgeId("ackId");
+		acknowledgeOnCheckpoint.addAcknowledgeId("ackId2");
+
+		List<AcknowledgeIdsForCheckpoint<String>> actual = acknowledgeOnCheckpoint.snapshotState(2, 100);
+
+		assertThat(actual.get(0).getCheckpointId(), is(2L));
+		assertThat(actual.get(0).getAcknowledgeIds(), containsInAnyOrder("ackId", "ackId2"));
+
+		assertThat(acknowledgeOnCheckpoint.numberOfOutstandingAcknowledgements(), is(2));
+	}
+
+	@Test
+	public void testAcknowledgeIdsForCheckpoint() throws Exception {
+		List<AcknowledgeIdsForCheckpoint<String>> input = new ArrayList<>();
+		input.add(new AcknowledgeIdsForCheckpoint<>(0, asList("idsFor0", "moreIdsFor0")));
+		input.add(new AcknowledgeIdsForCheckpoint<>(1, asList("idsFor1", "moreIdsFor1")));
+		input.add(new AcknowledgeIdsForCheckpoint<>(2, asList("idsFor2", "moreIdsFor2")));
+		input.add(new AcknowledgeIdsForCheckpoint<>(3, asList("idsFor3", "moreIdsFor3")));
+
+		AcknowledgeOnCheckpoint<String> acknowledgeOnCheckpoint = new AcknowledgeOnCheckpoint<>(mockedAcknowledger);
+		acknowledgeOnCheckpoint.restoreState(input);
+
+		acknowledgeOnCheckpoint.notifyCheckpointComplete(2);
+
+		ArgumentCaptor<List<String>> argumentCaptor = ArgumentCaptor.forClass(List.class);
+		verify(mockedAcknowledger, times(1)).acknowledge(argumentCaptor.capture());
+		assertThat(argumentCaptor.getValue(), containsInAnyOrder(
+			"idsFor0", "moreIdsFor0",
+			"idsFor1", "moreIdsFor1",
+			"idsFor2", "moreIdsFor2"));
+
+		assertThat(acknowledgeOnCheckpoint.numberOfOutstandingAcknowledgements(), is(2));
+	}
+
+	@Test
+	public void testNumberOfOutstandingAcknowledgementsOnEmptyState() throws Exception {
+		AcknowledgeOnCheckpoint<String> acknowledgeOnCheckpoint = new AcknowledgeOnCheckpoint<>(mockedAcknowledger);
+		assertThat(acknowledgeOnCheckpoint.numberOfOutstandingAcknowledgements(), is(0));
+	}
+}
diff --git a/flink-connectors/flink-connector-pubsub/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-gcp-pubsub/src/test/resources/log4j-test.properties
similarity index 97%
rename from flink-connectors/flink-connector-pubsub/src/test/resources/log4j-test.properties
rename to flink-connectors/flink-connector-gcp-pubsub/src/test/resources/log4j-test.properties
index b316a9a..a6cdf55 100644
--- a/flink-connectors/flink-connector-pubsub/src/test/resources/log4j-test.properties
+++ b/flink-connectors/flink-connector-gcp-pubsub/src/test/resources/log4j-test.properties
@@ -15,7 +15,7 @@
 #  See the License for the specific language governing permissions and
 # limitations under the License.
 ################################################################################
-log4j.rootLogger=INFO, testlogger
+log4j.rootLogger=OFF, testlogger
 log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
 log4j.appender.testlogger.target=System.out
 log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java
deleted file mode 100644
index 727f32e..0000000
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java
+++ /dev/null
@@ -1,116 +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.flink.streaming.connectors.pubsub;
-
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-import java.util.Timer;
-import java.util.TimerTask;
-
-class Bound<OUT> implements Serializable {
-	private static final Logger LOG = LoggerFactory.getLogger(Bound.class);
-
-	private final Bound.Mode mode;
-	private final long maxMessagedReceived;
-	private final long maxTimeBetweenMessages;
-
-	private SourceFunction<OUT> sourceFunction;
-	private transient Timer timer;
-	private long messagesReceived;
-	private long lastReceivedMessage;
-	private boolean cancelled = false;
-
-	private Bound(Bound.Mode mode, long maxMessagedReceived, long maxTimeBetweenMessages) {
-		this.mode = mode;
-		this.maxMessagedReceived = maxMessagedReceived;
-		this.maxTimeBetweenMessages = maxTimeBetweenMessages;
-		this.messagesReceived = 0L;
-	}
-
-	static <OUT> Bound<OUT> boundByAmountOfMessages(long maxMessagedReceived) {
-		return new Bound<>(Mode.COUNTER, maxMessagedReceived, 0L);
-	}
-
-	static <OUT> Bound<OUT> boundByTimeSinceLastMessage(long maxTimeBetweenMessages) {
-		return new Bound<>(Mode.TIMER, 0L, maxTimeBetweenMessages);
-	}
-
-	static <OUT> Bound<OUT> boundByAmountOfMessagesOrTimeSinceLastMessage(long maxMessagedReceived, long maxTimeBetweenMessages) {
-		return new Bound<>(Mode.COUNTER_OR_TIMER, maxMessagedReceived, maxTimeBetweenMessages);
-	}
-
-	private TimerTask shutdownPubSubSource() {
-		return new TimerTask() {
-			@Override
-			public void run() {
-				if (maxTimeBetweenMessagesElapsed()) {
-					cancelPubSubSource("BoundedSourceFunction: Idle timeout --> canceling source");
-					timer.cancel();
-				}
-			}
-		};
-	}
-
-	private synchronized boolean maxTimeBetweenMessagesElapsed() {
-		return System.currentTimeMillis() - lastReceivedMessage > maxTimeBetweenMessages;
-	}
-
-	private synchronized void cancelPubSubSource(String logMessage) {
-		if (!cancelled) {
-			cancelled = true;
-			sourceFunction.cancel();
-			LOG.info(logMessage);
-		}
-	}
-
-	void start(SourceFunction<OUT> sourceFunction) {
-		if (this.sourceFunction != null) {
-			throw new IllegalStateException("start() already called");
-		}
-
-		this.sourceFunction = sourceFunction;
-		messagesReceived = 0;
-
-		if (mode == Mode.TIMER || mode == Mode.COUNTER_OR_TIMER) {
-			lastReceivedMessage = System.currentTimeMillis();
-			timer = new Timer();
-			timer.schedule(shutdownPubSubSource(), 0, 100);
-		}
-	}
-
-	synchronized void receivedMessage() {
-		if (sourceFunction == null) {
-			throw new IllegalStateException("start() not called");
-		}
-
-		lastReceivedMessage = System.currentTimeMillis();
-		messagesReceived++;
-
-		if ((mode == Mode.COUNTER || mode == Mode.COUNTER_OR_TIMER) && messagesReceived >= maxMessagedReceived) {
-			cancelPubSubSource("BoundedSourceFunction: Max received messages --> canceling source");
-		}
-	}
-
-	private enum Mode {
-		COUNTER, TIMER, COUNTER_OR_TIMER
-	}
-}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSource.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSource.java
deleted file mode 100644
index 83fc15e..0000000
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSource.java
+++ /dev/null
@@ -1,108 +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.flink.streaming.connectors.pubsub;
-
-import com.google.cloud.pubsub.v1.AckReplyConsumer;
-import com.google.pubsub.v1.PubsubMessage;
-
-import java.io.IOException;
-
-/**
- * A bounded PubSub Source, similar to {@link PubSubSource} but this will stop at some point. For example after a period of idle or and after n amount of messages have been received.
- *
- */
-public class BoundedPubSubSource<OUT> extends PubSubSource<OUT> {
-	private Bound<OUT> bound;
-
-	private BoundedPubSubSource() {
-		super();
-	}
-
-	protected void setBound(Bound<OUT> bound) {
-		this.bound = bound;
-	}
-
-	@Override
-	public void run(SourceContext<OUT> sourceContext) {
-		bound.start(this);
-		super.run(sourceContext);
-	}
-
-	@Override
-	public void receiveMessage(PubsubMessage message, AckReplyConsumer consumer) {
-		super.receiveMessage(message, consumer);
-		bound.receivedMessage();
-	}
-
-	/**
-	 * Creates a {@link BoundedPubSubSourceBuilder}.
-	 * @param <OUT> Type of Object which will be read by the produced {@link BoundedPubSubSource}
-	 */
-	@SuppressWarnings("unchecked")
-	public static <OUT> BoundedPubSubSourceBuilder<OUT, ? extends PubSubSource, ? extends BoundedPubSubSourceBuilder> newBuilder() {
-		return new BoundedPubSubSourceBuilder<>(new BoundedPubSubSource<OUT>());
-	}
-
-	/**
-	 * Builder to create BoundedPubSubSource.
-	 * @param <OUT> Type of Object which will be read by the BoundedPubSubSource
-	 */
-	@SuppressWarnings("unchecked")
-	public static class BoundedPubSubSourceBuilder<OUT, PSS extends BoundedPubSubSource<OUT>, BUILDER extends BoundedPubSubSourceBuilder<OUT, PSS, BUILDER>> extends PubSubSourceBuilder<OUT, PSS, BUILDER> {
-		private Long boundedByAmountOfMessages;
-		private Long boundedByTimeSinceLastMessage;
-
-		BoundedPubSubSourceBuilder(PSS sourceUnderConstruction) {
-			super(sourceUnderConstruction);
-		}
-
-		public BUILDER boundedByAmountOfMessages(long maxAmountOfMessages) {
-			boundedByAmountOfMessages = maxAmountOfMessages;
-			return (BUILDER) this;
-		}
-
-		public BUILDER boundedByTimeSinceLastMessage(long timeSinceLastMessage) {
-			boundedByTimeSinceLastMessage = timeSinceLastMessage;
-			return (BUILDER) this;
-		}
-
-		private Bound<OUT> createBound() {
-			if (boundedByAmountOfMessages != null && boundedByTimeSinceLastMessage != null) {
-				return Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(boundedByAmountOfMessages, boundedByTimeSinceLastMessage);
-			}
-
-			if (boundedByAmountOfMessages != null) {
-				return Bound.boundByAmountOfMessages(boundedByAmountOfMessages);
-			}
-
-			if (boundedByTimeSinceLastMessage != null) {
-				return Bound.boundByTimeSinceLastMessage(boundedByTimeSinceLastMessage);
-			}
-
-			// This is functionally speaking no bound.
-			return Bound.boundByAmountOfMessages(Long.MAX_VALUE);
-		}
-
-		@Override
-		public PSS build() throws IOException {
-			sourceUnderConstruction.setBound(createBound());
-			return super.build();
-		}
-	}
-
-}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSink.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSink.java
deleted file mode 100644
index e6ac53e..0000000
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSink.java
+++ /dev/null
@@ -1,250 +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.flink.streaming.connectors.pubsub;
-
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.connectors.pubsub.common.SerializableCredentialsProvider;
-
-import com.google.api.gax.core.CredentialsProvider;
-import com.google.api.gax.grpc.GrpcTransportChannel;
-import com.google.api.gax.rpc.FixedTransportChannelProvider;
-import com.google.api.gax.rpc.TransportChannel;
-import com.google.auth.Credentials;
-import com.google.cloud.pubsub.v1.Publisher;
-import com.google.protobuf.ByteString;
-import com.google.pubsub.v1.ProjectTopicName;
-import com.google.pubsub.v1.PubsubMessage;
-import io.grpc.ManagedChannel;
-import io.grpc.ManagedChannelBuilder;
-
-import java.io.IOException;
-
-/**
- * A sink function that outputs to PubSub.
- *
- * @param <IN> type of PubSubSink messages to write
- */
-public class PubSubSink<IN> extends RichSinkFunction<IN> {
-
-	private SerializableCredentialsProvider serializableCredentialsProvider;
-	private SerializationSchema<IN> serializationSchema;
-	private String projectName;
-	private String topicName;
-	private String hostAndPort = null;
-
-	private transient Publisher publisher;
-
-	private PubSubSink() {
-	}
-
-	void setSerializableCredentialsProvider(SerializableCredentialsProvider serializableCredentialsProvider) {
-		this.serializableCredentialsProvider = serializableCredentialsProvider;
-	}
-
-	void setSerializationSchema(SerializationSchema<IN> serializationSchema) {
-		this.serializationSchema = serializationSchema;
-	}
-
-	void setProjectName(String projectName) {
-		this.projectName = projectName;
-	}
-
-	void setTopicName(String topicName) {
-		this.topicName = topicName;
-	}
-
-	/**
-	 * Set the custom hostname/port combination of PubSub.
-	 * The ONLY reason to use this is during tests with the emulator provided by Google.
-	 *
-	 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
-	 */
-	void withHostAndPort(String hostAndPort) {
-		this.hostAndPort = hostAndPort;
-	}
-
-	void initialize() throws IOException {
-		if (serializableCredentialsProvider == null) {
-			serializableCredentialsProvider = SerializableCredentialsProvider.credentialsProviderFromEnvironmentVariables();
-		}
-		if (serializationSchema == null) {
-			throw new IllegalArgumentException("The serializationSchema has not been specified.");
-		}
-		if (projectName == null) {
-			throw new IllegalArgumentException("The projectName has not been specified.");
-		}
-		if (topicName == null) {
-			throw new IllegalArgumentException("The topicName has not been specified.");
-		}
-	}
-
-
-	private transient ManagedChannel managedChannel = null;
-	private transient TransportChannel channel = null;
-
-	@Override
-	public void open(Configuration configuration) throws Exception {
-		Publisher.Builder builder = Publisher
-			.newBuilder(ProjectTopicName.of(projectName, topicName))
-			.setCredentialsProvider(serializableCredentialsProvider);
-
-		if (hostAndPort != null) {
-			managedChannel = ManagedChannelBuilder
-				.forTarget(hostAndPort)
-				.usePlaintext(true) // This is 'Ok' because this is ONLY used for testing.
-				.build();
-			channel = GrpcTransportChannel.newBuilder().setManagedChannel(managedChannel).build();
-			builder.setChannelProvider(FixedTransportChannelProvider.create(channel));
-		}
-
-		publisher = builder.build();
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		publisher.shutdown();
-		if (channel != null) {
-			channel.close();
-			managedChannel.shutdownNow();
-		}
-	}
-
-	@Override
-	public void invoke(IN message, SinkFunction.Context context) {
-		PubsubMessage pubsubMessage = PubsubMessage
-			.newBuilder()
-			.setData(ByteString.copyFrom(serializationSchema.serialize(message)))
-			.build();
-		publisher.publish(pubsubMessage);
-		publisher.publishAllOutstanding();
-	}
-
-	/**
-	 * Create a builder for a new PubSubSink.
-	 *
-	 * @param <IN> The generic of the type that is to be written into the sink.
-	 * @return a new PubSubSinkBuilder instance
-	 */
-	public static <IN> PubSubSinkBuilder<IN, ? extends PubSubSink<IN>, ? extends PubSubSinkBuilder<IN, ?, ?>> newBuilder() {
-		return new PubSubSinkBuilder<>(new PubSubSink<>());
-	}
-
-	/**
-	 * PubSubSinkBuilder to create a PubSubSink.
-	 *
-	 * @param <IN> Type of PubSubSink to create.
-	 */
-	@SuppressWarnings("unchecked")
-	public static class PubSubSinkBuilder<IN, PSS extends PubSubSink<IN>, BUILDER extends PubSubSinkBuilder<IN, PSS, BUILDER>> {
-		protected PSS sinkUnderConstruction;
-
-		private PubSubSinkBuilder(PSS sinkUnderConstruction) {
-			this.sinkUnderConstruction = sinkUnderConstruction;
-		}
-
-		/**
-		 * Set the credentials.
-		 * If this is not used then the credentials are picked up from the environment variables.
-		 *
-		 * @param credentials the Credentials needed to connect.
-		 * @return The current PubSubSinkBuilder instance
-		 */
-		public BUILDER withCredentials(Credentials credentials) {
-			sinkUnderConstruction.setSerializableCredentialsProvider(new SerializableCredentialsProvider(credentials));
-			return (BUILDER) this;
-		}
-
-		/**
-		 * Set the CredentialsProvider.
-		 * If this is not used then the credentials are picked up from the environment variables.
-		 *
-		 * @param credentialsProvider the custom SerializableCredentialsProvider instance.
-		 * @return The current PubSubSinkBuilder instance
-		 */
-		public BUILDER withCredentialsProvider(CredentialsProvider credentialsProvider) throws IOException {
-			return withCredentials(credentialsProvider.getCredentials());
-		}
-
-		/**
-		 * Set the credentials to be absent.
-		 * This means that no credentials are to be used at all.
-		 *
-		 * @return The current PubSubSinkBuilder instance
-		 */
-		public BUILDER withoutCredentials() {
-			sinkUnderConstruction.setSerializableCredentialsProvider(SerializableCredentialsProvider.withoutCredentials());
-			return (BUILDER) this;
-		}
-
-		/**
-		 * @param serializationSchema Instance of a SerializationSchema that converts the IN into a byte[]
-		 * @return The current PubSubSinkBuilder instance
-		 */
-		public BUILDER withSerializationSchema(SerializationSchema<IN> serializationSchema) {
-			sinkUnderConstruction.setSerializationSchema(serializationSchema);
-			return (BUILDER) this;
-		}
-
-		/**
-		 * @param projectName The name of the project in PubSub
-		 * @return The current PubSubSinkBuilder instance
-		 */
-		public BUILDER withProjectName(String projectName) {
-			sinkUnderConstruction.setProjectName(projectName);
-			return (BUILDER) this;
-		}
-
-		/**
-		 * @param topicName The name of the topic in PubSub
-		 * @return The current PubSubSinkBuilder instance
-		 */
-		public BUILDER withTopicName(String topicName) {
-			sinkUnderConstruction.setTopicName(topicName);
-			return (BUILDER) this;
-		}
-
-		/**
-		 * Set the custom hostname/port combination of PubSub.
-		 * The ONLY reason to use this is during tests with the emulator provided by Google.
-		 *
-		 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
-		 * @return The current PubSubSinkBuilder instance
-		 */
-		public BUILDER withHostAndPort(String hostAndPort) {
-			sinkUnderConstruction.withHostAndPort(hostAndPort);
-			return (BUILDER) this;
-		}
-
-		/**
-		 * Actually builder the desired instance of the PubSubSink.
-		 *
-		 * @return a brand new PubSubSink
-		 * @throws IOException              incase of a problem getting the credentials
-		 * @throws IllegalArgumentException incase required fields were not specified.
-		 */
-		public PubSubSink<IN> build() throws IOException {
-			sinkUnderConstruction.initialize();
-			return sinkUnderConstruction;
-		}
-	}
-
-}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java
deleted file mode 100644
index 2d93998..0000000
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java
+++ /dev/null
@@ -1,265 +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.flink.streaming.connectors.pubsub;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.source.MultipleIdsMessageAcknowledgingSourceBase;
-import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.pubsub.common.SerializableCredentialsProvider;
-
-import com.google.api.gax.core.CredentialsProvider;
-import com.google.auth.Credentials;
-import com.google.cloud.pubsub.v1.AckReplyConsumer;
-import com.google.cloud.pubsub.v1.MessageReceiver;
-import com.google.pubsub.v1.ProjectSubscriptionName;
-import com.google.pubsub.v1.PubsubMessage;
-
-import java.io.IOException;
-import java.util.List;
-
-
-/**
- * PubSub Source, this Source will consume PubSub messages from a subscription and Acknowledge them as soon as they have been received.
- */
-public class PubSubSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase<OUT, String, AckReplyConsumer> implements MessageReceiver, ResultTypeQueryable<OUT>, ParallelSourceFunction<OUT> {
-	private DeserializationSchema<OUT> deserializationSchema;
-	private SubscriberWrapper subscriberWrapper;
-
-	protected transient SourceContext<OUT> sourceContext = null;
-
-	protected PubSubSource() {
-		super(String.class);
-	}
-
-	protected void setDeserializationSchema(DeserializationSchema<OUT> deserializationSchema) {
-		this.deserializationSchema = deserializationSchema;
-	}
-
-	protected void setSubscriberWrapper(SubscriberWrapper subscriberWrapper) {
-		this.subscriberWrapper = subscriberWrapper;
-	}
-
-	@Override
-	public void open(Configuration configuration) throws Exception {
-		super.open(configuration);
-		subscriberWrapper.initialize(this);
-		if (hasNoCheckpointingEnabled(getRuntimeContext())) {
-			throw new IllegalArgumentException("The PubSubSource REQUIRES Checkpointing to be enabled and " +
-				"the checkpointing frequency must be MUCH lower than the PubSub timeout for it to retry a message.");
-		}
-	}
-
-	private boolean hasNoCheckpointingEnabled(RuntimeContext runtimeContext) {
-		return !(runtimeContext instanceof StreamingRuntimeContext && ((StreamingRuntimeContext) runtimeContext).isCheckpointingEnabled());
-	}
-
-	@Override
-	protected void acknowledgeSessionIDs(List<AckReplyConsumer> ackReplyConsumers) {
-		ackReplyConsumers.forEach(AckReplyConsumer::ack);
-	}
-
-	@Override
-	public void run(SourceContext<OUT> sourceContext) {
-		this.sourceContext = sourceContext;
-		subscriberWrapper.startBlocking();
-	}
-
-	@Override
-	public void receiveMessage(PubsubMessage message, AckReplyConsumer consumer) {
-		if (sourceContext == null) {
-			consumer.nack();
-			return;
-		}
-
-		processMessage(message, consumer);
-	}
-
-	private void processMessage(PubsubMessage message, AckReplyConsumer ackReplyConsumer) {
-		synchronized (sourceContext.getCheckpointLock()) {
-			boolean alreadyProcessed = !addId(message.getMessageId());
-			if (alreadyProcessed) {
-				return;
-			}
-
-			sessionIds.add(ackReplyConsumer);
-			sourceContext.collect(deserializeMessage(message));
-		}
-	}
-
-	@Override
-	public void cancel() {
-		subscriberWrapper.stop();
-	}
-
-	private OUT deserializeMessage(PubsubMessage message) {
-		try {
-			return deserializationSchema.deserialize(message.getData().toByteArray());
-		} catch (IOException e) {
-			throw new RuntimeException(e);
-		}
-	}
-
-	@Override
-	public TypeInformation<OUT> getProducedType() {
-		return deserializationSchema.getProducedType();
-	}
-
-	public static <OUT> PubSubSourceBuilder<OUT, ? extends PubSubSource<OUT>, ? extends PubSubSourceBuilder<OUT, ?, ?>> newBuilder() {
-		return new PubSubSourceBuilder<>(new PubSubSource<>());
-	}
-
-	/**
-	 * Builder to create PubSubSource.
-	 *
-	 * @param <OUT>     The type of objects which will be read
-	 * @param <PSS>     The type of PubSubSource
-	 * @param <BUILDER> The type of Builder to create the PubSubSource
-	 */
-	@SuppressWarnings("unchecked")
-	public static class PubSubSourceBuilder<OUT, PSS extends PubSubSource<OUT>, BUILDER extends PubSubSourceBuilder<OUT, PSS, BUILDER>> {
-		protected PSS sourceUnderConstruction;
-
-		private SubscriberWrapper subscriberWrapper = null;
-		private SerializableCredentialsProvider serializableCredentialsProvider;
-		private DeserializationSchema<OUT> deserializationSchema;
-		private String projectName;
-		private String subscriptionName;
-		private String hostAndPort;
-
-		protected PubSubSourceBuilder(PSS sourceUnderConstruction) {
-			this.sourceUnderConstruction = sourceUnderConstruction;
-		}
-
-		/**
-		 * Set the credentials.
-		 * If this is not used then the credentials are picked up from the environment variables.
-		 *
-		 * @param credentials the Credentials needed to connect.
-		 * @return The current PubSubSourceBuilder instance
-		 */
-		public BUILDER withCredentials(Credentials credentials) {
-			this.serializableCredentialsProvider = new SerializableCredentialsProvider(credentials);
-			return (BUILDER) this;
-		}
-
-		/**
-		 * Set the CredentialsProvider.
-		 * If this is not used then the credentials are picked up from the environment variables.
-		 *
-		 * @param credentialsProvider the custom SerializableCredentialsProvider instance.
-		 * @return The current PubSubSourceBuilder instance
-		 */
-		public BUILDER withCredentialsProvider(CredentialsProvider credentialsProvider) throws IOException {
-			return withCredentials(credentialsProvider.getCredentials());
-		}
-
-		/**
-		 * Set the credentials to be absent.
-		 * This means that no credentials are to be used at all.
-		 *
-		 * @return The current PubSubSourceBuilder instance
-		 */
-		public BUILDER withoutCredentials() {
-			this.serializableCredentialsProvider = SerializableCredentialsProvider.withoutCredentials();
-			return (BUILDER) this;
-		}
-
-		/**
-		 * @param deserializationSchema Instance of a DeserializationSchema that converts the OUT into a byte[]
-		 * @return The current PubSubSourceBuilder instance
-		 */
-		public BUILDER withDeserializationSchema(DeserializationSchema<OUT> deserializationSchema) {
-			this.deserializationSchema = deserializationSchema;
-			return (BUILDER) this;
-		}
-
-		/**
-		 * @param projectName      The name of the project in GoogleCloudPlatform
-		 * @param subscriptionName The name of the subscription in PubSub
-		 * @return The current PubSubSourceBuilder instance
-		 */
-		public BUILDER withProjectSubscriptionName(String projectName, String subscriptionName) {
-			this.projectName = projectName;
-			this.subscriptionName = subscriptionName;
-			return (BUILDER) this;
-		}
-
-		/**
-		 * Set the custom hostname/port combination of PubSub.
-		 * The ONLY reason to use this is during tests with the emulator provided by Google.
-		 *
-		 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
-		 * @return The current PubSubSourceBuilder instance
-		 */
-		public BUILDER withHostAndPort(String hostAndPort) {
-			this.hostAndPort = hostAndPort;
-			return (BUILDER) this;
-		}
-
-		/**
-		 * Set a complete SubscriberWrapper.
-		 * The ONLY reason to use this is during tests.
-		 *
-		 * @param subscriberWrapper The fully instantiated SubscriberWrapper
-		 * @return The current PubSubSourceBuilder instance
-		 */
-		public BUILDER withSubscriberWrapper(SubscriberWrapper subscriberWrapper) {
-			this.subscriberWrapper = subscriberWrapper;
-			return (BUILDER) this;
-		}
-
-		/**
-		 * Actually build the desired instance of the PubSubSourceBuilder.
-		 *
-		 * @return a brand new SourceFunction
-		 * @throws IOException              incase of a problem getting the credentials
-		 * @throws IllegalArgumentException incase required fields were not specified.
-		 */
-		public PSS build() throws IOException {
-			if (serializableCredentialsProvider == null) {
-				serializableCredentialsProvider = SerializableCredentialsProvider.credentialsProviderFromEnvironmentVariables();
-			}
-			if (deserializationSchema == null) {
-				throw new IllegalArgumentException("The deserializationSchema has not been specified.");
-			}
-
-			if (subscriberWrapper == null) {
-				if (projectName == null || subscriptionName == null) {
-					throw new IllegalArgumentException("The ProjectName And SubscriptionName have not been specified.");
-				}
-
-				subscriberWrapper =
-					new SubscriberWrapper(serializableCredentialsProvider, ProjectSubscriptionName.of(projectName, subscriptionName));
-
-				if (hostAndPort != null) {
-					subscriberWrapper.withHostAndPort(hostAndPort);
-				}
-			}
-
-			sourceUnderConstruction.setSubscriberWrapper(subscriberWrapper);
-			sourceUnderConstruction.setDeserializationSchema(deserializationSchema);
-
-			return sourceUnderConstruction;
-		}
-	}
-}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapper.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapper.java
deleted file mode 100644
index 0595877..0000000
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapper.java
+++ /dev/null
@@ -1,104 +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.flink.streaming.connectors.pubsub;
-
-import org.apache.flink.streaming.connectors.pubsub.common.SerializableCredentialsProvider;
-
-import com.google.api.core.ApiService;
-import com.google.api.gax.grpc.GrpcTransportChannel;
-import com.google.api.gax.rpc.FixedTransportChannelProvider;
-import com.google.api.gax.rpc.TransportChannel;
-import com.google.cloud.pubsub.v1.MessageReceiver;
-import com.google.cloud.pubsub.v1.Subscriber;
-import com.google.pubsub.v1.ProjectSubscriptionName;
-import io.grpc.ManagedChannel;
-import io.grpc.ManagedChannelBuilder;
-
-import java.io.Serializable;
-
-class SubscriberWrapper implements Serializable {
-	private final SerializableCredentialsProvider serializableCredentialsProvider;
-	private final String projectId;
-	private final String subscriptionId;
-	private String hostAndPort = null;
-
-	private transient Subscriber subscriber;
-	private transient ManagedChannel managedChannel = null;
-	private transient TransportChannel channel = null;
-
-	SubscriberWrapper(SerializableCredentialsProvider serializableCredentialsProvider, ProjectSubscriptionName projectSubscriptionName) {
-		this.serializableCredentialsProvider = serializableCredentialsProvider;
-		this.projectId = projectSubscriptionName.getProject();
-		this.subscriptionId = projectSubscriptionName.getSubscription();
-	}
-
-	void initialize(MessageReceiver messageReceiver) {
-		Subscriber.Builder builder = Subscriber
-			.newBuilder(ProjectSubscriptionName.of(projectId, subscriptionId), messageReceiver)
-			.setCredentialsProvider(serializableCredentialsProvider);
-
-		if (hostAndPort != null) {
-			managedChannel = ManagedChannelBuilder
-				.forTarget(hostAndPort)
-				.usePlaintext(true) // This is 'Ok' because this is ONLY used for testing.
-				.build();
-			channel = GrpcTransportChannel.newBuilder().setManagedChannel(managedChannel).build();
-			builder.setChannelProvider(FixedTransportChannelProvider.create(channel));
-		}
-
-		this.subscriber = builder.build();
-	}
-
-	/**
-	 * Set the custom hostname/port combination of PubSub.
-	 * The ONLY reason to use this is during tests with the emulator provided by Google.
-	 *
-	 * @param hostAndPort The combination of hostname and port to connect to ("hostname:1234")
-	 * @return The current instance
-	 */
-	public SubscriberWrapper withHostAndPort(String hostAndPort) {
-		this.hostAndPort = hostAndPort;
-		return this;
-	}
-
-	void startBlocking() {
-		ApiService apiService = subscriber.startAsync();
-		apiService.awaitRunning();
-
-		if (apiService.state() != ApiService.State.RUNNING) {
-			throw new IllegalStateException("Could not start PubSubSubscriber, ApiService.State: " + apiService.state());
-		}
-		apiService.awaitTerminated();
-	}
-
-	void stop() {
-		subscriber.stopAsync().awaitTerminated();
-		if (channel != null) {
-			try {
-				channel.close();
-				managedChannel.shutdownNow();
-			} catch (Exception e) {
-				// Ignore
-			}
-		}
-	}
-
-	Subscriber getSubscriber() {
-		return subscriber;
-	}
-}
diff --git a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/common/SerializableCredentialsProvider.java b/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/common/SerializableCredentialsProvider.java
deleted file mode 100644
index 44b1fa0..0000000
--- a/flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/common/SerializableCredentialsProvider.java
+++ /dev/null
@@ -1,70 +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.flink.streaming.connectors.pubsub.common;
-
-import com.google.api.gax.core.CredentialsProvider;
-import com.google.auth.Credentials;
-import com.google.cloud.NoCredentials;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-import static com.google.cloud.pubsub.v1.SubscriptionAdminSettings.defaultCredentialsProviderBuilder;
-
-/**
- * Wrapper class for CredentialsProvider to make it Serializable. This can be used to pass on Credentials to SourceFunctions
- */
-public class SerializableCredentialsProvider implements CredentialsProvider, Serializable {
-	private final Credentials credentials;
-
-	/**
-	 * @param credentials The google {@link Credentials} needed to connect to PubSub
-	 */
-	public SerializableCredentialsProvider(Credentials credentials) {
-		this.credentials = credentials;
-	}
-
-	/**
-	 * Creates a SerializableCredentialsProvider for a PubSubSubscription based on environment variables.
-	 * {@link com.google.cloud.pubsub.v1.SubscriptionAdminSettings}
-	 *
-	 * @return serializableCredentialsProvider
-	 * @throws IOException thrown by {@link Credentials}
-	 */
-	public static SerializableCredentialsProvider credentialsProviderFromEnvironmentVariables() throws IOException {
-		Credentials credentials = defaultCredentialsProviderBuilder().build().getCredentials();
-		return new SerializableCredentialsProvider(credentials);
-	}
-
-	/**
-	 * Creates a SerializableCredentialsProvider for a PubSubSubscription without any credentials.
-	 * {@link com.google.cloud.pubsub.v1.SubscriptionAdminSettings}
-	 * This is ONLY useful when running tests locally against Mockito or the Google PubSub emulator
-	 * @see <a href="https://cloud.google.com/pubsub/docs/emulator" target="_top">https://cloud.google.com/pubsub/docs/emulator</a>
-	 * @return serializableCredentialsProvider
-	 * @see <a href="https://cloud.google.com/pubsub/docs/emulator" target="_top">https://cloud.google.com/pubsub/docs/emulator</a>
-	 */
-	public static SerializableCredentialsProvider withoutCredentials() {
-		return new SerializableCredentialsProvider(NoCredentials.getInstance());
-	}
-
-	@Override
-	public Credentials getCredentials() {
-		return credentials;
-	}
-}
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundTest.java
deleted file mode 100644
index a340ae9..0000000
--- a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundTest.java
+++ /dev/null
@@ -1,130 +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.flink.streaming.connectors.pubsub;
-
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-
-import org.junit.Test;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyZeroInteractions;
-import static org.mockito.internal.verification.VerificationModeFactory.times;
-
-/**
- * Test for {@link Bound}.
- */
-public class BoundTest {
-	private SourceFunction<Object> sourceFunction = mock(SourceFunction.class);
-
-	@Test
-	public void testNoShutdownBeforeCounterLimit() {
-		Bound<Object> bound = Bound.boundByAmountOfMessages(10);
-		bound.start(sourceFunction);
-		sleep(150L);
-
-		bound.receivedMessage();
-		verifyZeroInteractions(sourceFunction);
-	}
-
-	@Test
-	public void testShutdownOnCounterLimit() {
-		Bound<Object> bound = Bound.boundByAmountOfMessages(3);
-		bound.start(sourceFunction);
-
-		bound.receivedMessage();
-		bound.receivedMessage();
-		bound.receivedMessage();
-
-		verify(sourceFunction, times(1)).cancel();
-	}
-
-	@Test
-	public void testNoShutdownBeforeTimerLimit() {
-		Bound<Object> bound = Bound.boundByTimeSinceLastMessage(1000L);
-		bound.start(sourceFunction);
-		for (int i = 0; i < 10; i++) {
-			bound.receivedMessage();
-		}
-
-		verifyZeroInteractions(sourceFunction);
-	}
-
-	@Test
-	public void testShutdownAfterTimerLimitNoMessageReceived() {
-		Bound<Object> bound = Bound.boundByTimeSinceLastMessage(100L);
-		bound.start(sourceFunction);
-		sleep(250L);
-		verify(sourceFunction, times(1)).cancel();
-	}
-
-	@Test
-	public void testShutdownAfterTimerLimitAfterMessageReceived() {
-		Bound<Object> bound = Bound.boundByTimeSinceLastMessage(100L);
-		bound.start(sourceFunction);
-		sleep(50L);
-
-		bound.receivedMessage();
-		sleep(50L);
-		verifyZeroInteractions(sourceFunction);
-
-		sleep(200L);
-		verify(sourceFunction, times(1)).cancel();
-	}
-
-	@Test
-	public void testCounterOrTimerMaxMessages() {
-		Bound<Object> bound = Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(3, 1000L);
-		bound.start(sourceFunction);
-
-		bound.receivedMessage();
-		bound.receivedMessage();
-		bound.receivedMessage();
-
-		verify(sourceFunction, times(1)).cancel();
-	}
-
-	@Test
-	public void testCounterOrTimerTimerElapsed() {
-		Bound<Object> bound = Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(1L, 100L);
-		bound.start(sourceFunction);
-		sleep(200L);
-		verify(sourceFunction, times(1)).cancel();
-	}
-
-	@Test(expected = IllegalStateException.class)
-	public void testExceptionThrownIfStartNotCalled() {
-		Bound<Object> bound = Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(1L, 100L);
-		bound.receivedMessage();
-	}
-
-	@Test(expected = IllegalStateException.class)
-	public void testExceptionThrownIfStartCalledTwice() {
-		Bound<Object> bound = Bound.boundByAmountOfMessagesOrTimeSinceLastMessage(1L, 100L);
-		bound.start(sourceFunction);
-		bound.start(sourceFunction);
-	}
-
-	private void sleep(long sleepTime) {
-		try {
-			Thread.sleep(sleepTime);
-		} catch (InterruptedException e) {
-			// Ignore any exceptions
-		}
-	}
-}
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSourceTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSourceTest.java
deleted file mode 100644
index 5f938fd..0000000
--- a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/BoundedPubSubSourceTest.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.pubsub;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-
-import com.google.cloud.pubsub.v1.AckReplyConsumer;
-import com.google.protobuf.ByteString;
-import com.google.pubsub.v1.PubsubMessage;
-import org.junit.Test;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-import static org.mockito.internal.verification.VerificationModeFactory.times;
-
-/**
- * Tests for {@link BoundedPubSubSource}.
- */
-public class BoundedPubSubSourceTest {
-	private final Bound<Object> bound = mock(Bound.class);
-	private final SubscriberWrapper subscriberWrapper = mock(SubscriberWrapper.class);
-	private final SourceFunction.SourceContext<Object> sourceContext = mock(SourceFunction.SourceContext.class);
-	private final AckReplyConsumer ackReplyConsumer = mock(AckReplyConsumer.class);
-	private final DeserializationSchema<Object> deserializationSchema = mock(DeserializationSchema.class);
-
-	private FunctionInitializationContext functionInitializationContext = mock(FunctionInitializationContext.class);
-	private OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
-	private StreamingRuntimeContext streamingRuntimeContext = mock(StreamingRuntimeContext.class);
-
-	@Test
-	public void testBoundIsUsed() throws Exception {
-		BoundedPubSubSource<Object> boundedPubSubSource = createAndInitializeBoundedPubSubSource();
-		boundedPubSubSource.setBound(bound);
-
-		boundedPubSubSource.run(sourceContext);
-		verify(bound, times(1)).start(boundedPubSubSource);
-
-		boundedPubSubSource.receiveMessage(pubSubMessage(), ackReplyConsumer);
-		verify(bound, times(1)).receivedMessage();
-	}
-
-	private BoundedPubSubSource<Object> createAndInitializeBoundedPubSubSource() throws Exception {
-		when(sourceContext.getCheckpointLock()).thenReturn(new Object());
-		when(functionInitializationContext.getOperatorStateStore()).thenReturn(operatorStateStore);
-		when(operatorStateStore.getSerializableListState(any(String.class))).thenReturn(null);
-		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
-
-		BoundedPubSubSource<Object> boundedPubSubSource = BoundedPubSubSource.newBuilder()
-			.withoutCredentials()
-			.withSubscriberWrapper(subscriberWrapper)
-			.withDeserializationSchema(deserializationSchema)
-			.build();
-		boundedPubSubSource.initializeState(functionInitializationContext);
-		boundedPubSubSource.setRuntimeContext(streamingRuntimeContext);
-		boundedPubSubSource.open(null);
-
-		return boundedPubSubSource;
-	}
-
-	private PubsubMessage pubSubMessage() {
-		return PubsubMessage.newBuilder()
-			.setMessageId("message-id")
-			.setData(ByteString.copyFrom("some-message".getBytes()))
-			.build();
-	}
-}
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/PubSubSourceTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/PubSubSourceTest.java
deleted file mode 100644
index 9db5d7d..0000000
--- a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/PubSubSourceTest.java
+++ /dev/null
@@ -1,147 +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.flink.streaming.connectors.pubsub;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-
-import com.google.cloud.pubsub.v1.AckReplyConsumer;
-import com.google.protobuf.ByteString;
-import com.google.pubsub.v1.PubsubMessage;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Mock;
-import org.mockito.runners.MockitoJUnitRunner;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyZeroInteractions;
-import static org.mockito.Mockito.when;
-import static org.mockito.internal.verification.VerificationModeFactory.times;
-
-/**
- * Test for {@link SourceFunction}.
- */
-@RunWith(MockitoJUnitRunner.class)
-public class PubSubSourceTest {
-	private static final String MESSAGE = "Message";
-	private static final byte[] SERIALIZED_MESSAGE = MESSAGE.getBytes();
-	@Mock
-	private SubscriberWrapper subscriberWrapper;
-	@Mock
-	private org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext<String> sourceContext;
-	@Mock
-	private DeserializationSchema<String> deserializationSchema;
-	@Mock
-	private AckReplyConsumer ackReplyConsumer;
-	@Mock
-	private StreamingRuntimeContext streamingRuntimeContext;
-	@Mock
-	private RuntimeContext runtimeContext;
-	@Mock
-	private OperatorStateStore operatorStateStore;
-	@Mock
-	private FunctionInitializationContext functionInitializationContext;
-
-	@Test
-	public void testOpenWithCheckpointing() throws Exception {
-		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
-
-		PubSubSource<String> pubSubSource = createTestSource();
-		pubSubSource.setRuntimeContext(streamingRuntimeContext);
-		pubSubSource.open(null);
-
-		verify(subscriberWrapper, times(1)).initialize(pubSubSource);
-	}
-
-	@Test
-	public void testRun() throws IOException {
-		PubSubSource<String> pubSubSource = createTestSource();
-		pubSubSource.run(sourceContext);
-
-		verify(subscriberWrapper, times(1)).startBlocking();
-	}
-
-	@Test
-	public void testWithCheckpoints() throws Exception {
-		when(deserializationSchema.deserialize(SERIALIZED_MESSAGE)).thenReturn(MESSAGE);
-		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
-		when(sourceContext.getCheckpointLock()).thenReturn("some object to lock on");
-		when(functionInitializationContext.getOperatorStateStore()).thenReturn(operatorStateStore);
-		when(operatorStateStore.getSerializableListState(any(String.class))).thenReturn(null);
-
-		PubSubSource<String> pubSubSource = createTestSource();
-		pubSubSource.initializeState(functionInitializationContext);
-		pubSubSource.setRuntimeContext(streamingRuntimeContext);
-		pubSubSource.open(null);
-		verify(subscriberWrapper, times(1)).initialize(pubSubSource);
-
-		pubSubSource.run(sourceContext);
-
-		pubSubSource.receiveMessage(pubSubMessage(), ackReplyConsumer);
-
-		verify(sourceContext, times(1)).getCheckpointLock();
-		verify(sourceContext, times(1)).collect(MESSAGE);
-		verifyZeroInteractions(ackReplyConsumer);
-	}
-
-	@Test
-	public void testMessagesAcknowledged() throws Exception {
-		when(streamingRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
-
-		PubSubSource<String> pubSubSource = createTestSource();
-		pubSubSource.setRuntimeContext(streamingRuntimeContext);
-		pubSubSource.open(null);
-
-		List<AckReplyConsumer> input = Collections.singletonList(ackReplyConsumer);
-
-		pubSubSource.acknowledgeSessionIDs(input);
-
-		verify(ackReplyConsumer, times(1)).ack();
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testOnceWithoutCheckpointing() throws Exception {
-		PubSubSource<String> pubSubSource = createTestSource();
-		pubSubSource.setRuntimeContext(runtimeContext);
-
-		pubSubSource.open(null);
-	}
-
-	private PubSubSource<String> createTestSource() throws IOException {
-		return PubSubSource.<String>newBuilder()
-			.withoutCredentials()
-			.withSubscriberWrapper(subscriberWrapper)
-			.withDeserializationSchema(deserializationSchema)
-			.build();
-	}
-
-	private PubsubMessage pubSubMessage() {
-		return PubsubMessage.newBuilder()
-			.setData(ByteString.copyFrom(SERIALIZED_MESSAGE))
-			.build();
-	}
-}
diff --git a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapperTest.java b/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapperTest.java
deleted file mode 100644
index f5deb07..0000000
--- a/flink-connectors/flink-connector-pubsub/src/test/java/org/apache/flink/streaming/connectors/pubsub/SubscriberWrapperTest.java
+++ /dev/null
@@ -1,57 +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.flink.streaming.connectors.pubsub;
-
-import org.apache.flink.streaming.connectors.pubsub.common.SerializableCredentialsProvider;
-
-import com.google.cloud.pubsub.v1.MessageReceiver;
-import com.google.pubsub.v1.ProjectSubscriptionName;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Mock;
-import org.mockito.runners.MockitoJUnitRunner;
-
-import static org.apache.flink.api.java.ClosureCleaner.ensureSerializable;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.is;
-
-/**
- * Tests for {@link SubscriberWrapper}.
- */
-@RunWith(MockitoJUnitRunner.class)
-public class SubscriberWrapperTest {
-	@Mock
-	private SerializableCredentialsProvider credentialsProvider;
-
-	@Mock
-	private MessageReceiver messageReceiver;
-
-	@Test
-	public void testSerializedSubscriberBuilder() throws Exception {
-		SubscriberWrapper factory = new SubscriberWrapper(SerializableCredentialsProvider.withoutCredentials(), ProjectSubscriptionName.of("projectId", "subscriptionId"));
-		ensureSerializable(factory);
-	}
-
-	@Test
-	public void testInitialisation() {
-		SubscriberWrapper factory = new SubscriberWrapper(credentialsProvider, ProjectSubscriptionName.of("projectId", "subscriptionId"));
-		factory.initialize(messageReceiver);
-
-		assertThat(factory.getSubscriber().getSubscriptionNameString(), is(ProjectSubscriptionName.format("projectId", "subscriptionId")));
-	}
-}
diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml
index 1c47830..2586002 100644
--- a/flink-connectors/pom.xml
+++ b/flink-connectors/pom.xml
@@ -56,7 +56,7 @@ under the License.
 		<module>flink-connector-cassandra</module>
 		<module>flink-connector-filesystem</module>
 		<module>flink-connector-kafka</module>
-		<module>flink-connector-pubsub</module>
+		<module>flink-connector-gcp-pubsub</module>
 	</modules>
 
 	<!-- override these root dependencies as 'provided', so they don't end up
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/pom.xml b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml
similarity index 75%
rename from flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/pom.xml
rename to flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml
index 7dd0d15..d7f3fc6 100644
--- a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/pom.xml
+++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml
@@ -26,28 +26,15 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-end-to-end-tests</artifactId>
-		<version>1.7-SNAPSHOT</version>
+		<version>1.9-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-pubsub-emulator-tests</artifactId>
-	<name>flink-connector-pubsub-emulator-tests</name>
+	<artifactId>flink-connector-gcp-pubsub-emulator-tests</artifactId>
+	<name>flink-connector-gcp-pubsub-emulator-tests</name>
 
 	<packaging>jar</packaging>
 
-	<!-- This is the way we get a consistent set of versions of the Google tools -->
-	<dependencyManagement>
-		<dependencies>
-			<dependency>
-				<groupId>com.google.cloud</groupId>
-				<artifactId>google-cloud-bom</artifactId>
-				<version>0.53.0-alpha</version>
-				<type>pom</type>
-				<scope>import</scope>
-			</dependency>
-		</dependencies>
-	</dependencyManagement>
-
 	<dependencies>
 
 		<!--All dependencies are   <scope>test</scope> -->
@@ -61,27 +48,12 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-pubsub_${scala.binary.version}</artifactId>
+			<artifactId>flink-connector-gcp-pubsub_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
-			<groupId>com.google.cloud</groupId>
-			<artifactId>google-cloud-pubsub</artifactId>
-			<!-- Version is pulled from google-cloud-bom -->
-			<exclusions>
-				<!-- Exclude an old version of guava that is being pulled
-                in by a transitive dependency of google-api-client -->
-				<exclusion>
-					<groupId>com.google.guava</groupId>
-					<artifactId>guava-jdk5</artifactId>
-				</exclusion>
-			</exclusions>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
 			<groupId>org.slf4j</groupId>
 			<artifactId>slf4j-log4j12</artifactId>
 			<version>${slf4j.version}</version>
@@ -126,6 +98,13 @@ under the License.
 				<configuration>
 					<skipTests>${skipTests}</skipTests>
 				</configuration>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test</goal>
+						</goals>
+					</execution>
+				</executions>
 			</plugin>
 			<!-- Disabling convergence check because there are multiple problems within the used pubsub dependencies -->
 			<plugin>
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/CheckPubSubEmulatorTest.java b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/CheckPubSubEmulatorTest.java
similarity index 84%
rename from flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/CheckPubSubEmulatorTest.java
rename to flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/CheckPubSubEmulatorTest.java
index a54d47b..d5541c3 100644
--- a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/CheckPubSubEmulatorTest.java
+++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/CheckPubSubEmulatorTest.java
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.flink.streaming.connectors.pubsub;
+package org.apache.flink.streaming.connectors.gcp.pubsub;
 
-import org.apache.flink.streaming.connectors.pubsub.emulator.GCloudUnitTestBase;
-import org.apache.flink.streaming.connectors.pubsub.emulator.PubsubHelper;
+import org.apache.flink.streaming.connectors.gcp.pubsub.emulator.GCloudUnitTestBase;
+import org.apache.flink.streaming.connectors.gcp.pubsub.emulator.PubsubHelper;
 
 import com.google.cloud.pubsub.v1.Publisher;
 import com.google.cloud.pubsub.v1.Subscriber;
@@ -34,6 +34,7 @@ import org.slf4j.LoggerFactory;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.junit.Assert.assertEquals;
@@ -81,7 +82,7 @@ public class CheckPubSubEmulatorTest extends GCloudUnitTestBase {
 	}
 
 	@Test
-	public void testPush() throws Exception {
+	public void testPub() throws Exception {
 		List<PubsubMessage> receivedMessages = new ArrayList<>();
 		Subscriber subscriber = pubsubHelper.
 			subscribeToSubscription(
@@ -99,7 +100,8 @@ public class CheckPubSubEmulatorTest extends GCloudUnitTestBase {
 			.get();
 
 		LOG.info("Waiting a while to receive the message...");
-		Thread.sleep(1000);
+
+		waitUntill(() -> receivedMessages.size() > 0);
 
 		assertEquals(1, receivedMessages.size());
 		assertEquals("Hello World", receivedMessages.get(0).getData().toStringUtf8());
@@ -112,4 +114,18 @@ public class CheckPubSubEmulatorTest extends GCloudUnitTestBase {
 		publisher.shutdown();
 	}
 
+	/*
+	 * Returns when predicate returns true or if 10 seconds have passed
+	 */
+	private void waitUntill(Supplier<Boolean> predicate) {
+		int retries = 0;
+
+		while (!predicate.get() && retries < 100) {
+			retries++;
+			try {
+				Thread.sleep(10);
+			} catch (InterruptedException e) { }
+		}
+	}
+
 }
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSinkTest.java b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSinkTest.java
similarity index 62%
rename from flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSinkTest.java
rename to flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSinkTest.java
index 165579f..0566e01 100644
--- a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSinkTest.java
+++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSinkTest.java
@@ -15,22 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.flink.streaming.connectors.pubsub;
+package org.apache.flink.streaming.connectors.gcp.pubsub;
 
 import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.pubsub.emulator.GCloudUnitTestBase;
-import org.apache.flink.streaming.connectors.pubsub.emulator.PubsubHelper;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.gcp.pubsub.emulator.GCloudUnitTestBase;
+import org.apache.flink.streaming.connectors.gcp.pubsub.emulator.PubsubHelper;
 
+import com.google.cloud.NoCredentials;
 import com.google.pubsub.v1.ReceivedMessage;
 import org.apache.commons.lang3.StringUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -43,9 +44,6 @@ import static org.junit.Assert.assertTrue;
  * Test of the PubSub SINK with the Google PubSub emulator.
  */
 public class EmulatedPubSubSinkTest extends GCloudUnitTestBase {
-
-	private static final Logger LOG = LoggerFactory.getLogger(EmulatedPubSubSinkTest.class);
-
 	private static final String PROJECT_NAME = "FLProject";
 	private static final String TOPIC_NAME = "FLTopic";
 	private static final String SUBSCRIPTION_NAME = "FLSubscription";
@@ -79,14 +77,14 @@ public class EmulatedPubSubSinkTest extends GCloudUnitTestBase {
 
 		// Sink into pubsub
 		theData
-			.addSink(PubSubSink.<String>newBuilder()
-				.withProjectName(PROJECT_NAME)
-				.withTopicName(TOPIC_NAME)
-				.withSerializationSchema(new SimpleStringSchema())
-				// Specific for emulator
-				.withCredentialsProvider(getPubsubHelper().getCredentialsProvider())
-				.withHostAndPort(getPubSubHostPort())
-				.build())
+			.addSink(PubSubSink.newBuilder(String.class)
+								.withSerializationSchema(new SimpleStringSchema())
+								.withProjectName(PROJECT_NAME)
+								.withTopicName(TOPIC_NAME)
+							   // Specific for emulator
+							.withHostAndPortForEmulator(getPubSubHostPort())
+							.withCredentials(NoCredentials.getInstance())
+							.build())
 			.name("PubSub sink");
 
 		// Run
@@ -106,4 +104,43 @@ public class EmulatedPubSubSinkTest extends GCloudUnitTestBase {
 		}
 	}
 
+	@Test(expected = Exception.class)
+	public void testPubSubSinkThrowsExceptionOnFailure() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.enableCheckpointing(100);
+		env.setParallelism(1);
+		env.setRestartStrategy(RestartStrategies.noRestart());
+
+		// Create test stream
+		//use source function to prevent the job from shutting down before a checkpoint has been made
+		env.addSource(new SingleInputSourceFunction())
+
+			.map((MapFunction<String, String>) StringUtils::reverse)
+			.addSink(PubSubSink.newBuilder(String.class)
+								.withSerializationSchema(new SimpleStringSchema())
+								.withProjectName(PROJECT_NAME)
+								.withTopicName(TOPIC_NAME)
+								// Specific for emulator
+								.withHostAndPortForEmulator("unknown-host-to-force-sink-crash:1234")
+								.withCredentials(NoCredentials.getInstance())
+								.build()).name("PubSub sink");
+
+		// Run
+		env.execute();
+	}
+
+	private class SingleInputSourceFunction implements SourceFunction<String> {
+
+		@Override
+		public void run(SourceContext<String> ctx) throws Exception {
+			ctx.collect("input");
+			Thread.sleep(1000 * 60);
+		}
+
+		@Override
+		public void cancel() {
+
+		}
+	}
+
 }
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSourceTest.java b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSourceTest.java
similarity index 61%
rename from flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSourceTest.java
rename to flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSourceTest.java
index aaa4fc3..f3f9b0a 100644
--- a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/EmulatedPubSubSourceTest.java
+++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/EmulatedPubSubSourceTest.java
@@ -15,24 +15,28 @@
  * limitations under the License.
  */
 
-package org.apache.flink.streaming.connectors.pubsub;
+package org.apache.flink.streaming.connectors.gcp.pubsub;
 
-import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.pubsub.emulator.GCloudUnitTestBase;
-import org.apache.flink.streaming.connectors.pubsub.emulator.PubsubHelper;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubDeserializationSchema;
+import org.apache.flink.streaming.connectors.gcp.pubsub.emulator.GCloudUnitTestBase;
+import org.apache.flink.streaming.connectors.gcp.pubsub.emulator.PubsubHelper;
 
+import com.google.cloud.NoCredentials;
 import com.google.cloud.pubsub.v1.Publisher;
 import com.google.protobuf.ByteString;
 import com.google.pubsub.v1.PubsubMessage;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -45,9 +49,6 @@ import static org.junit.Assert.assertTrue;
  * Test of the PubSub SOURCE with the Google PubSub emulator.
  */
 public class EmulatedPubSubSourceTest extends GCloudUnitTestBase {
-
-	private static final Logger LOG = LoggerFactory.getLogger(EmulatedPubSubSourceTest.class);
-
 	private static final String PROJECT_NAME = "FLProject";
 	private static final String TOPIC_NAME = "FLTopic";
 	private static final String SUBSCRIPTION_NAME = "FLSubscription";
@@ -70,15 +71,17 @@ public class EmulatedPubSubSourceTest extends GCloudUnitTestBase {
 	@Test
 	public void testFlinkSource() throws Exception {
 		// Create some messages and put them into pubsub
-		List<String> input = Arrays.asList("One", "Two", "Three", "Four", "Five", "Six", "Seven", "Eigth", "Nine", "Ten");
+		List<String> input = Arrays.asList("One", "Two", "Three", "Four", "Five", "Six", "Seven", "Eight", "Nine", "Ten");
+
+		List<String> messagesToSend = new ArrayList<>(input);
+		messagesToSend.add("End");
 
 		// Publish the messages into PubSub
 		Publisher publisher = pubsubHelper.createPublisher(PROJECT_NAME, TOPIC_NAME);
-		input.forEach(s -> {
+		messagesToSend.forEach(s -> {
 			try {
 				publisher
-					.publish(PubsubMessage
-						.newBuilder()
+					.publish(PubsubMessage.newBuilder()
 						.setData(ByteString.copyFromUtf8(s))
 						.build())
 					.get();
@@ -88,18 +91,18 @@ public class EmulatedPubSubSourceTest extends GCloudUnitTestBase {
 		});
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.enableCheckpointing(100);
+		env.enableCheckpointing(1000);
+		env.setParallelism(1);
+		env.setRestartStrategy(RestartStrategies.noRestart());
 
 		DataStream<String> fromPubSub = env
-			.addSource(BoundedPubSubSource.<String>newBuilder()
-				.withDeserializationSchema(new SimpleStringSchema())
-				.withProjectSubscriptionName(PROJECT_NAME, SUBSCRIPTION_NAME)
-				// Specific for emulator
-				.withCredentialsProvider(getPubsubHelper().getCredentialsProvider())
-				.withHostAndPort(getPubSubHostPort())
-				// Make sure the test topology self terminates
-				.boundedByTimeSinceLastMessage(1000)
-				.build())
+			.addSource(PubSubSource.newBuilder(String.class)
+								.withDeserializationSchema(new BoundedStringDeserializer(10))
+								.withProjectName(PROJECT_NAME)
+								.withSubscriptionName(SUBSCRIPTION_NAME)
+								.withCredentials(NoCredentials.getInstance())
+								.withPubSubSubscriberFactory(new PubSubSubscriberFactoryForEmulator(getPubSubHostPort(), PROJECT_NAME, SUBSCRIPTION_NAME, 10, Duration.ofSeconds(15), 100))
+								.build())
 			.name("PubSub source");
 
 		List<String> output = new ArrayList<>();
@@ -113,4 +116,29 @@ public class EmulatedPubSubSourceTest extends GCloudUnitTestBase {
 		}
 	}
 
+	private static class BoundedStringDeserializer implements PubSubDeserializationSchema<String> {
+		private final int maxMessage;
+		private int counter;
+
+		private BoundedStringDeserializer(int maxMessages) {
+			this.maxMessage = maxMessages;
+			this.counter = 0;
+		}
+
+		@Override
+		public boolean isEndOfStream(String message) {
+			counter++;
+			return counter > maxMessage;
+		}
+
+		@Override
+		public String deserialize(PubsubMessage message) throws Exception {
+			return message.getData().toString(StandardCharsets.UTF_8);
+		}
+
+		@Override
+		public TypeInformation<String> getProducedType() {
+			return Types.STRING;
+		}
+	}
 }
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudEmulatorManager.java b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/GCloudEmulatorManager.java
similarity index 99%
rename from flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudEmulatorManager.java
rename to flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/GCloudEmulatorManager.java
index 27a658a..a0aad49 100644
--- a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudEmulatorManager.java
+++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/GCloudEmulatorManager.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.streaming.connectors.pubsub.emulator;
+package org.apache.flink.streaming.connectors.gcp.pubsub.emulator;
 
 import com.spotify.docker.client.DefaultDockerClient;
 import com.spotify.docker.client.DockerClient;
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudUnitTestBase.java b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/GCloudUnitTestBase.java
similarity index 81%
rename from flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudUnitTestBase.java
rename to flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/GCloudUnitTestBase.java
index b6a011a..6fdc0117 100644
--- a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/GCloudUnitTestBase.java
+++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/GCloudUnitTestBase.java
@@ -15,10 +15,8 @@
  * limitations under the License.
  */
 
-package org.apache.flink.streaming.connectors.pubsub.emulator;
+package org.apache.flink.streaming.connectors.gcp.pubsub.emulator;
 
-import com.google.api.gax.core.CredentialsProvider;
-import com.google.api.gax.core.NoCredentialsProvider;
 import com.google.api.gax.grpc.GrpcTransportChannel;
 import com.google.api.gax.rpc.FixedTransportChannelProvider;
 import com.google.api.gax.rpc.TransportChannelProvider;
@@ -31,8 +29,8 @@ import org.junit.BeforeClass;
 import java.io.Serializable;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.flink.streaming.connectors.pubsub.emulator.GCloudEmulatorManager.getDockerIpAddress;
-import static org.apache.flink.streaming.connectors.pubsub.emulator.GCloudEmulatorManager.getDockerPubSubPort;
+import static org.apache.flink.streaming.connectors.gcp.pubsub.emulator.GCloudEmulatorManager.getDockerIpAddress;
+import static org.apache.flink.streaming.connectors.gcp.pubsub.emulator.GCloudEmulatorManager.getDockerPubSubPort;
 
 /**
  * The base class from which unit tests should inherit if they need to use the Google cloud emulators.
@@ -54,7 +52,6 @@ public class GCloudUnitTestBase implements Serializable {
 
 	private static ManagedChannel channel = null;
 	private static TransportChannelProvider channelProvider = null;
-	private static CredentialsProvider credentialsProvider = null;
 
 	public static PubsubHelper getPubsubHelper() {
 		if (channel == null) {
@@ -65,9 +62,8 @@ public class GCloudUnitTestBase implements Serializable {
 				.build();
 			channelProvider = FixedTransportChannelProvider
 				.create(GrpcTransportChannel.create(channel));
-			credentialsProvider = NoCredentialsProvider.create();
 		}
-		return new PubsubHelper(channelProvider, credentialsProvider);
+		return new PubsubHelper(channelProvider);
 	}
 
 	public static String getPubSubHostPort() {
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/PubsubHelper.java b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/PubsubHelper.java
similarity index 85%
rename from flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/PubsubHelper.java
rename to flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/PubsubHelper.java
index f08576b..0b13e49 100644
--- a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/pubsub/emulator/PubsubHelper.java
+++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/java/org/apache/flink/streaming/connectors/gcp/pubsub/emulator/PubsubHelper.java
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.flink.streaming.connectors.pubsub.emulator;
+package org.apache.flink.streaming.connectors.gcp.pubsub.emulator;
 
-import com.google.api.gax.core.CredentialsProvider;
+import com.google.api.gax.core.NoCredentialsProvider;
 import com.google.api.gax.rpc.NotFoundException;
 import com.google.api.gax.rpc.TransportChannelProvider;
 import com.google.cloud.pubsub.v1.MessageReceiver;
@@ -53,34 +53,23 @@ public class PubsubHelper {
 	private static final Logger LOG = LoggerFactory.getLogger(PubsubHelper.class);
 
 	private TransportChannelProvider channelProvider = null;
-	private CredentialsProvider credentialsProvider = null;
 
 	private TopicAdminClient topicClient;
 	private SubscriptionAdminClient subscriptionAdminClient;
 
-	public PubsubHelper() {
-		this(TopicAdminSettings.defaultTransportChannelProvider(),
-			TopicAdminSettings.defaultCredentialsProviderBuilder().build());
-	}
-
-	public PubsubHelper(TransportChannelProvider channelProvider, CredentialsProvider credentialsProvider) {
+	public PubsubHelper(TransportChannelProvider channelProvider) {
 		this.channelProvider = channelProvider;
-		this.credentialsProvider = credentialsProvider;
 	}
 
 	public TransportChannelProvider getChannelProvider() {
 		return channelProvider;
 	}
 
-	public CredentialsProvider getCredentialsProvider() {
-		return credentialsProvider;
-	}
-
 	public TopicAdminClient getTopicAdminClient() throws IOException {
 		if (topicClient == null) {
 			TopicAdminSettings topicAdminSettings = TopicAdminSettings.newBuilder()
 				.setTransportChannelProvider(channelProvider)
-				.setCredentialsProvider(credentialsProvider)
+				.setCredentialsProvider(NoCredentialsProvider.create())
 				.build();
 			topicClient = TopicAdminClient.create(topicAdminSettings);
 		}
@@ -100,23 +89,23 @@ public class PubsubHelper {
 	}
 
 	public void deleteTopic(ProjectTopicName topicName) throws IOException {
-//        LOG.info("CreateTopic {}", topicName);
 		TopicAdminClient adminClient = getTopicAdminClient();
 		try {
-			Topic existingTopic = adminClient.getTopic(topicName);
-
-			// If it exists we delete all subscriptions and the topic itself.
-			LOG.info("DeleteTopic {} first delete old subscriptions.", topicName);
-			adminClient
-				.listTopicSubscriptions(topicName)
-				.iterateAllAsProjectSubscriptionName()
-				.forEach(subscriptionAdminClient::deleteSubscription);
-			LOG.info("DeleteTopic {}", topicName);
-			adminClient
-				.deleteTopic(topicName);
+			adminClient.getTopic(topicName);
 		} catch (NotFoundException e) {
 			// Doesn't exist. Good.
+			return;
 		}
+
+		// If it exists we delete all subscriptions and the topic itself.
+		LOG.info("DeleteTopic {} first delete old subscriptions.", topicName);
+		adminClient
+			.listTopicSubscriptions(topicName)
+			.iterateAllAsProjectSubscriptionName()
+			.forEach(subscriptionAdminClient::deleteSubscription);
+		LOG.info("DeleteTopic {}", topicName);
+		adminClient
+			.deleteTopic(topicName);
 	}
 
 	public SubscriptionAdminClient getSubscriptionAdminClient() throws IOException {
@@ -125,7 +114,7 @@ public class PubsubHelper {
 				SubscriptionAdminSettings
 					.newBuilder()
 					.setTransportChannelProvider(channelProvider)
-					.setCredentialsProvider(credentialsProvider)
+					.setCredentialsProvider(NoCredentialsProvider.create())
 					.build();
 			subscriptionAdminClient = SubscriptionAdminClient.create(subscriptionAdminSettings);
 		}
@@ -175,7 +164,7 @@ public class PubsubHelper {
 		SubscriberStubSettings subscriberStubSettings =
 			SubscriberStubSettings.newBuilder()
 				.setTransportChannelProvider(channelProvider)
-				.setCredentialsProvider(credentialsProvider)
+				.setCredentialsProvider(NoCredentialsProvider.create())
 				.build();
 		try (SubscriberStub subscriber = GrpcSubscriberStub.create(subscriberStubSettings)) {
 			// String projectId = "my-project-id";
@@ -215,7 +204,7 @@ public class PubsubHelper {
 			Subscriber
 				.newBuilder(subscriptionName, messageReceiver)
 				.setChannelProvider(channelProvider)
-				.setCredentialsProvider(credentialsProvider)
+				.setCredentialsProvider(NoCredentialsProvider.create())
 				.build();
 		subscriber.startAsync();
 		return subscriber;
@@ -225,7 +214,7 @@ public class PubsubHelper {
 		return Publisher
 			.newBuilder(ProjectTopicName.of(project, topic))
 			.setChannelProvider(channelProvider)
-			.setCredentialsProvider(credentialsProvider)
+			.setCredentialsProvider(NoCredentialsProvider.create())
 			.build();
 	}
 
diff --git a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/resources/log4j-test.properties b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/resources/log4j-test.properties
similarity index 97%
rename from flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/resources/log4j-test.properties
rename to flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/resources/log4j-test.properties
index b316a9a..a6cdf55 100644
--- a/flink-end-to-end-tests/flink-connector-pubsub-emulator-tests/src/test/resources/log4j-test.properties
+++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/src/test/resources/log4j-test.properties
@@ -15,7 +15,7 @@
 #  See the License for the specific language governing permissions and
 # limitations under the License.
 ################################################################################
-log4j.rootLogger=INFO, testlogger
+log4j.rootLogger=OFF, testlogger
 log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
 log4j.appender.testlogger.target=System.out
 log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml
index b17dc70..baea71d 100644
--- a/flink-end-to-end-tests/pom.xml
+++ b/flink-end-to-end-tests/pom.xml
@@ -61,7 +61,7 @@ under the License.
 		<module>flink-metrics-availability-test</module>
 		<module>flink-metrics-reporter-prometheus-test</module>
 		<module>flink-heavy-deployment-stress-test</module>
-		<module>flink-connector-pubsub-emulator-tests</module>
+		<module>flink-connector-gcp-pubsub-emulator-tests</module>
 		<module>flink-streaming-kafka-test-base</module>
 		<module>flink-streaming-kafka-test</module>
 		<module>flink-streaming-kafka011-test</module>
diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh
index 957b3c6..b2578fc 100755
--- a/flink-end-to-end-tests/run-nightly-tests.sh
+++ b/flink-end-to-end-tests/run-nightly-tests.sh
@@ -136,7 +136,7 @@ run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scr
 run_test "Quickstarts Java nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh java"
 run_test "Quickstarts Scala nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh scala"
 
-run_test "Test PubSub connector with Docker based Google PubSub Emulator" "$END_TO_END_DIR/test-scripts/test_streaming_pubsub.sh"
+run_test "Test PubSub connector with Docker based Google PubSub Emulator" "$END_TO_END_DIR/test-scripts/test_streaming_gcp_pubsub.sh"
 
 run_test "Avro Confluent Schema Registry nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_confluent_schema_registry.sh"
 
diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_pubsub.sh b/flink-end-to-end-tests/test-scripts/test_streaming_gcp_pubsub.sh
similarity index 93%
rename from flink-end-to-end-tests/test-scripts/test_streaming_pubsub.sh
rename to flink-end-to-end-tests/test-scripts/test_streaming_gcp_pubsub.sh
index 8e08385..976e764 100755
--- a/flink-end-to-end-tests/test-scripts/test_streaming_pubsub.sh
+++ b/flink-end-to-end-tests/test-scripts/test_streaming_gcp_pubsub.sh
@@ -17,6 +17,6 @@
 # limitations under the License.
 ################################################################################
 
-cd "${END_TO_END_DIR}/flink-connector-pubsub-emulator-tests"
+cd "${END_TO_END_DIR}/flink-connector-gcp-pubsub-emulator-tests"
 
 mvn test -DskipTests=false
diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml
new file mode 100644
index 0000000..ab1c91f
--- /dev/null
+++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml
@@ -0,0 +1,108 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-examples-build-helper</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.9-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-examples-streaming-gcp-pubsub_${scala.binary.version}</artifactId>
+	<name>flink-examples-streaming-gcp-pubsub</name>
+	<packaging>jar</packaging>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-gcp-pubsub_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-examples-streaming_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<finalName>PubSub</finalName>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-deploy-plugin</artifactId>
+				<configuration>
+					<skip>true</skip>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<shadeTestJar>false</shadeTestJar>
+							<shadedArtifactAttached>false</shadedArtifactAttached>
+							<createDependencyReducedPom>false</createDependencyReducedPom>
+							<transformers>
+								<transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+									<mainClass>org.apache.flink.streaming.examples.gcp.pubsub.PubSubExample</mainClass>
+								</transformer>
+							</transformers>
+							<artifactSet>
+								<includes>
+									<include>org.apache.flink:flink-connector-gcp-pubsub*</include>
+									<include>com.google.cloud:google-cloud-pubsub</include>
+									<include>com.google.*:*</include>
+									<include>org.threeten:*</include>
+									<include>io.grpc:*</include>
+									<include>io.opencensus:*</include>
+								</includes>
+							</artifactSet>
+							<relocations>
+								<relocation>
+									<pattern>com.google</pattern>
+									<shadedPattern>org.apache.flink.streaming.examples.gcp.pubsub.shaded.com.google</shadedPattern>
+								</relocation>
+							</relocations>
+
+							<filters>
+								<filter>
+									<artifact>org.apache.flink:flink-examples-streaming_*</artifact>
+									<includes>
+										<include>org/apache/flink/streaming/examples/gcp/pubsub/**</include>
+									</includes>
+								</filter>
+							</filters>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/IntegerSerializer.java b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/IntegerSerializer.java
similarity index 68%
rename from flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/IntegerSerializer.java
rename to flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/IntegerSerializer.java
index 6e9d1d5..3c1eab4 100644
--- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/IntegerSerializer.java
+++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/IntegerSerializer.java
@@ -15,20 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.flink.streaming.examples.pubsub;
+package org.apache.flink.streaming.examples.gcp.pubsub;
 
-import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubDeserializationSchema;
+
+import com.google.pubsub.v1.PubsubMessage;
 
 import java.io.IOException;
 import java.math.BigInteger;
 
-class IntegerSerializer implements DeserializationSchema<Integer>, SerializationSchema<Integer> {
+/**
+ * Deserialization schema to deserialize messages produced by {@link PubSubPublisher}.
+ * The byte[] received by this schema must contain a single Integer.
+ */
+class IntegerSerializer implements PubSubDeserializationSchema<Integer>, SerializationSchema<Integer> {
 
 	@Override
-	public Integer deserialize(byte[] bytes) throws IOException {
-		return new BigInteger(bytes).intValue();
+	public Integer deserialize(PubsubMessage message) throws IOException {
+		return new BigInteger(message.getData().toByteArray()).intValue();
 	}
 
 	@Override
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubExample.java b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubExample.java
similarity index 78%
rename from flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubExample.java
rename to flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubExample.java
index c5791bd..7b66577 100644
--- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubExample.java
+++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubExample.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.streaming.examples.pubsub;
+package org.apache.flink.streaming.examples.gcp.pubsub;
 
 import org.apache.flink.api.java.utils.ParameterTool;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.pubsub.PubSubSink;
-import org.apache.flink.streaming.connectors.pubsub.PubSubSource;
+import org.apache.flink.streaming.connectors.gcp.pubsub.PubSubSink;
+import org.apache.flink.streaming.connectors.gcp.pubsub.PubSubSource;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,7 +43,7 @@ public class PubSubExample {
 
 		if (parameterTool.getNumberOfParameters() < 3) {
 			System.out.println("Missing parameters!\n" +
-								"Usage: flink run PubSub.jar --input-subscription <subscription> --input-topicName <topic> --output-topicName " +
+								"Usage: flink run PubSub.jar --input-subscription <subscription> --input-topicName <topic> --output-topicName <output-topic> " +
 								"--google-project <google project name> ");
 			return;
 		}
@@ -54,24 +54,25 @@ public class PubSubExample {
 		String outputTopicName = parameterTool.getRequired("output-topicName");
 
 		PubSubPublisher pubSubPublisher = new PubSubPublisher(projectName, inputTopicName);
+		pubSubPublisher.publish(10);
 
-		pubSubPublisher.publish();
 		runFlinkJob(projectName, subscriptionName, outputTopicName);
 	}
 
 	private static void runFlinkJob(String projectName, String subscriptionName, String outputTopicName) throws Exception {
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.enableCheckpointing(1000L);
 
-		env.addSource(PubSubSource.<Integer>newBuilder()
-										.withProjectSubscriptionName(projectName, subscriptionName)
-										.withDeserializationSchema(new IntegerSerializer())
-										.build())
+		env.addSource(PubSubSource.newBuilder(Integer.class)
+								.withDeserializationSchema(new IntegerSerializer())
+								.withProjectName(projectName)
+								.withSubscriptionName(subscriptionName)
+								.build())
 			.map(PubSubExample::printAndReturn).disableChaining()
-			.addSink(PubSubSink.<Integer>newBuilder()
-							.withProjectName(projectName)
-							.withTopicName(outputTopicName)
-							.withSerializationSchema(new IntegerSerializer())
-							.build());
+			.addSink(PubSubSink.newBuilder(Integer.class)
+								.withSerializationSchema(new IntegerSerializer())
+								.withProjectName(projectName)
+								.withTopicName(outputTopicName).build());
 
 		env.execute("Flink Streaming PubSubReader");
 	}
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubPublisher.java b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubPublisher.java
similarity index 76%
rename from flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubPublisher.java
rename to flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubPublisher.java
index 7507945..8f7bfe6 100644
--- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/pubsub/PubSubPublisher.java
+++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/src/main/java/org/apache/flink/streaming/examples/gcp/pubsub/PubSubPublisher.java
@@ -15,9 +15,8 @@
  * limitations under the License.
  */
 
-package org.apache.flink.streaming.examples.pubsub;
+package org.apache.flink.streaming.examples.gcp.pubsub;
 
-import com.google.api.core.ApiFuture;
 import com.google.cloud.pubsub.v1.Publisher;
 import com.google.protobuf.ByteString;
 import com.google.pubsub.v1.ProjectTopicName;
@@ -25,6 +24,9 @@ import com.google.pubsub.v1.PubsubMessage;
 
 import java.math.BigInteger;
 
+/**
+ * Helper class to send PubSubMessages to a PubSub topic.
+ */
 class PubSubPublisher {
 	private final String projectName;
 	private final String topicName;
@@ -34,21 +36,22 @@ class PubSubPublisher {
 		this.topicName = topicName;
 	}
 
-	void publish() {
+	/**
+	 * Publish messages with as payload a single integer.
+	 * The integers inside the messages start from 0 and increase by one for each message send.
+	 * @param amountOfMessages amount of messages to send
+	 */
+	void publish(int amountOfMessages) {
 		Publisher publisher = null;
 		try {
 			publisher = Publisher.newBuilder(ProjectTopicName.of(projectName, topicName)).build();
-			long counter = 0;
-			while (counter < 10) {
-				ByteString messageData = ByteString.copyFrom(BigInteger.valueOf(counter).toByteArray());
+			for (int i = 0; i < amountOfMessages; i++) {
+				ByteString messageData = ByteString.copyFrom(BigInteger.valueOf(i).toByteArray());
 				PubsubMessage message = PubsubMessage.newBuilder().setData(messageData).build();
+				publisher.publish(message).get();
 
-				ApiFuture<String> future = publisher.publish(message);
-				future.get();
-				System.out.println("Published message: " + counter);
+				System.out.println("Published message: " + i);
 				Thread.sleep(100L);
-
-				counter++;
 			}
 		} catch (Exception e) {
 			throw new RuntimeException(e);
diff --git a/flink-examples/flink-examples-build-helper/pom.xml b/flink-examples/flink-examples-build-helper/pom.xml
index 6f89161..456de28 100644
--- a/flink-examples/flink-examples-build-helper/pom.xml
+++ b/flink-examples/flink-examples-build-helper/pom.xml
@@ -35,6 +35,7 @@ under the License.
 	<modules>
 		<module>flink-examples-streaming-twitter</module>
 		<module>flink-examples-streaming-state-machine</module>
+		<module>flink-examples-streaming-gcp-pubsub</module>
 	</modules>
 
 </project>
diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml
index 888399e..47055d4 100644
--- a/flink-examples/flink-examples-streaming/pom.xml
+++ b/flink-examples/flink-examples-streaming/pom.xml
@@ -64,17 +64,6 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-pubsub_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>com.google.cloud</groupId>
-			<artifactId>google-cloud-pubsub</artifactId>
-			<version>1.31.0</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-shaded-jackson</artifactId>
 		</dependency>