diff --git a/.gitignore b/.gitignore
new file mode 100644
index 00000000..86083d15
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1,32 @@
+# Compiled class file
+*.class
+
+# Log file
+*.log
+
+# BlueJ files
+*.ctxt
+
+# Mobile Tools for Java (J2ME)
+.mtj.tmp/
+
+# Package Files #
+*.jar
+*.war
+*.nar
+*.ear
+*.zip
+*.tar.gz
+*.rar
+
+# virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml
+hs_err_pid*
+
+# intellij project folder
+.idea/
+*.iml
+
+# maven build directory
+target/
+
+.flattened-pom.xml
\ No newline at end of file
diff --git a/.readme-partials.yaml b/.readme-partials.yaml
new file mode 100644
index 00000000..275ee9ae
--- /dev/null
+++ b/.readme-partials.yaml
@@ -0,0 +1,126 @@
+custom_content: |
+ ## Requirements
+
+ ### Enable the PubSub Lite API
+
+ Follow [these instructions](https://cloud.google.com/pubsub/lite/docs/quickstart#before-you-begin).
+
+ ### Create a new subscription or use existing subscription
+
+ Follow [the instruction](https://cloud.google.com/pubsub/lite/docs/quickstart#create_a_lite_subscription) to create a new
+ subscription or use existing subscription. If using existing subscription, the connector will read message from the
+ oldest unacknowledged.
+
+ ### Create a Google Cloud Dataproc cluster (Optional)
+
+ If you do not have an Apache Spark environment you can create a Cloud Dataproc cluster with pre-configured auth. The following examples assume you are using Cloud Dataproc, but you can use `spark-submit` on any cluster.
+
+ ```
+ MY_CLUSTER=...
+ gcloud dataproc clusters create "$MY_CLUSTER"
+ ```
+
+ ## Downloading and Using the Connector
+
+
+ The latest version connector of the connector (Scala 2.11) is publicly available in
+ gs://spark-lib/pubsublite/spark-pubsublite-latest.jar.
+
+
+ The connector is also available from the Maven Central
+ repository. It can be used using the `--packages` option or the
+ `spark.jars.packages` configuration property. Use the following value
+
+ | Scala version | Connector Artifact |
+ | --- | --- |
+ | Scala 2.11 | `com.google.cloud.pubsublite.spark:pubsublite-spark-sql-streaming-with-dependencies_2.11:0.1.0` |
+
+
+
+ ## Usage
+
+ ### Reading data from PubSub Lite
+
+ ```
+ df = spark.readStream \
+ .option("pubsublite.subscription", "projects/123456789/locations/us-central1-a/subscriptions/test-spark-subscription")
+ .format("pubsublite") \
+ .load
+ ```
+
+ Note that the connector supports both MicroBatch Processing and [Continuous Processing](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#continuous-processing).
+
+ ### Properties
+
+ The connector supports a number of options to configure the read:
+
+ | Option | Type | Required | Meaning |
+ | ------ | ---- | -------- | ------- |
+ | pubsublite.subscription | String | Y | Full subscription path that the connector will read from. |
+ | pubsublite.flowcontrol.byteoutstandingperpartition | Long | N | Max number of bytes per partition that will be cached in workers before Spark processes the messages. Default to 50000000 bytes. |
+ | pubsublite.flowcontrol.messageoutstandingperpartition | Long | N | Max number of messages per partition that will be cached in workers before Spark processes the messages. Default to Long.MAX_VALUE. |
+ | gcp.credentials.key | String | N | Service account JSON in base64. Default to [Application Default Credentials](https://cloud.google.com/docs/authentication/production#automatically). |
+
+ ### Data Schema
+
+ The connector has fixed data schema as follows:
+
+ | Data Field | Spark Data Type | Notes |
+ | ---------- | --------------- | ----- |
+ | subscription | StringType | Full subscription path |
+ | partition | LongType | |
+ | offset | LongType | |
+ | key | BinaryType | |
+ | data | BinaryType | |
+ | attributes | MapType\[StringType, ArrayType\[BinaryType\]\] | |
+ | publish_timestamp | TimestampType | |
+ | event_timestamp | TimestampType | Nullable |
+
+ ## Compiling with the connector
+
+ To include the connector in your project:
+
+ ### Maven
+
+ ```xml
+
+ com.google.cloud.pubsublite.spark
+ pubsublite-spark-sql-streaming-with-dependencies_2.11
+ 0.1.0
+
+ ```
+
+ ### SBT
+
+ ```sbt
+ libraryDependencies += "com.google.cloud.pubsublite.spark" %% "pubsublite-spark-sql-streaming-with-dependencies_2.11" % "0.1.0"
+ ```
+
+ ## Building the Connector
+
+ The connector is built using Maven. Following command creates a jar with shaded dependencies:
+
+ ```
+ mvn package
+ ```
+
+ ## FAQ
+
+ ### What is the Pricing for the PubSub Lite?
+
+ See the [PubSub Lite pricing documentation](https://cloud.google.com/pubsub/lite/pricing).
+
+ ### Can I configure the number of spark partitions?
+
+ No, the number of spark partitions is set to be the number of PubSub Lite partitions of the topic that the supplied subscription is for.
+
+ ### How do I authenticate outside GCE / Dataproc?
+
+ Use a service account JSON key and `GOOGLE_APPLICATION_CREDENTIALS` as described [here](https://cloud.google.com/docs/authentication/getting-started).
+
+ Credentials can be provided with `gcp.credentials.key` option, it needs be passed in as a base64-encoded string directly.
+
+ Example:
+ ```
+ spark.readStream.format("pubsublite").option("gcp.credentials.key", "")
+ ```
\ No newline at end of file
diff --git a/README.md b/README.md
deleted file mode 100644
index a486d6be..00000000
--- a/README.md
+++ /dev/null
@@ -1 +0,0 @@
-# java-pubsublite-spark
diff --git a/pom.xml b/pom.xml
new file mode 100644
index 00000000..a49376d8
--- /dev/null
+++ b/pom.xml
@@ -0,0 +1,272 @@
+
+
+
+ com.google.cloud
+ google-cloud-pubsublite-parent
+ 0.8.0
+
+ 4.0.0
+ com.google.cloud
+ pubsublite-spark-sql-streaming
+ 0.1.0-SNAPSHOT
+ jar
+ Pub/Sub Lite Spark SQL Streaming
+ https://github.com/googleapis/java-pubsublite-spark
+ Spark SQL Streaming Connector for Google Cloud Pub/Sub Lite
+
+ 1.8
+ 1.8
+ UTF-8
+ 2.11.12
+ 2.11
+ 2.4.7
+
+
+
+ org.apache.spark
+ spark-sql_${scala.version.short}
+ ${spark.version}
+ provided
+
+
+ org.apache.spark
+ spark-catalyst_${scala.version.short}
+ ${spark.version}
+ provided
+
+
+ org.apache.spark
+ spark-unsafe_${scala.version.short}
+ ${spark.version}
+ provided
+
+
+ com.google.cloud
+ google-cloud-pubsublite
+ 0.8.0
+
+
+ com.google.api.grpc
+ proto-google-cloud-pubsublite-v1
+ 0.8.0
+
+
+ com.google.guava
+ guava
+
+
+ com.google.api
+ gax
+
+
+ com.google.auto.value
+ auto-value-annotations
+
+
+ com.google.auto.service
+ auto-service-annotations
+
+
+ com.google.code.findbugs
+ jsr305
+
+
+ com.google.auth
+ google-auth-library-credentials
+
+
+ com.google.auth
+ google-auth-library-oauth2-http
+
+
+ com.google.http-client
+ google-http-client
+
+
+ com.google.protobuf
+ protobuf-java
+
+
+ com.google.protobuf
+ protobuf-java-util
+
+
+ com.google.flogger
+ google-extensions
+
+
+ com.google.api
+ api-common
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+ 2.12.1
+
+
+ com.github.ben-manes.caffeine
+ caffeine
+ 2.8.8
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ 2.12.1
+
+
+ org.scala-lang
+ scala-library
+ ${scala.version}
+ provided
+
+
+
+
+ junit
+ junit
+ 4.13.1
+ test
+
+
+ com.google.truth
+ truth
+ 1.1
+ test
+
+
+ org.mockito
+ mockito-core
+ test
+ 3.7.0
+
+
+ com.google.guava
+ guava-testlib
+ test
+
+
+
+
+ src/main/java
+ src/test/java
+ target/classes
+ target/test-classes
+
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+ 3.2.4
+
+
+ package
+
+ shade
+
+
+
+
+
+
+
+
+ *:*
+
+ META-INF/maven/**
+ META-INF/*.MF
+ META-INF/*.SF
+ META-INF/*.DSA
+ META-INF/*.RSA
+
+
+
+
+
+ com
+ repackaged.com
+
+ com.google.protobuf.**
+ com.google.common.**
+
+
+
+
+ io.grpc.netty.shaded
+
+ com.google.cloud.pubsublite.repackaged.io.grpc.netty.shaded
+
+
+
+ io
+ com.google.cloud.pubsublite.repackaged.io
+
+ io.grpc.**
+ io.opencensus.**
+ io.perfmark.**
+
+
+
+ META-INF/native/io_grpc_netty_shaded_
+
+ META-INF/native/com_google_cloud_pubsublite_repackaged_io_grpc_netty_shaded_
+
+
+
+ META-INF/native/libio_grpc_netty_shaded_
+
+ META-INF/native/libcom_google_cloud_pubsublite_repackaged_io_grpc_netty_shaded_
+
+
+
+ true
+ with-dependencies
+
+
+
+
+
+ org.sonatype.plugins
+ nexus-staging-maven-plugin
+
+
+ true
+
+
+
+ org.apache.maven.plugins
+ maven-enforcer-plugin
+ 3.0.0-M3
+
+
+ enforce
+
+
+
+ WARN
+
+ org.checkerframework:checker-compat-qual
+
+
+
+
+
+ org.apache.commons.logging.*
+ org.apache.commons.collections.*
+ org.apache.spark.unused.*
+ org.apache.hadoop.yarn.*
+ javax.ws.rs.*
+
+ true
+
+
+
+
+ enforce
+
+
+
+
+
+
+
diff --git a/src/main/java/com/google/cloud/pubsublite/spark/Constants.java b/src/main/java/com/google/cloud/pubsublite/spark/Constants.java
new file mode 100644
index 00000000..bdc8a1bd
--- /dev/null
+++ b/src/main/java/com/google/cloud/pubsublite/spark/Constants.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2020 Google LLC
+ *
+ * Licensed 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 com.google.cloud.pubsublite.spark;
+
+import com.google.cloud.pubsublite.internal.wire.PubsubContext;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+public class Constants {
+ public static long DEFAULT_BYTES_OUTSTANDING = 50_000_000;
+ public static long DEFAULT_MESSAGES_OUTSTANDING = Long.MAX_VALUE;
+ public static int DEFAULT_BATCH_OFFSET_RANGE = 100_000;
+ public static StructType DEFAULT_SCHEMA =
+ new StructType(
+ new StructField[] {
+ new StructField("subscription", DataTypes.StringType, false, Metadata.empty()),
+ new StructField("partition", DataTypes.LongType, false, Metadata.empty()),
+ new StructField("offset", DataTypes.LongType, false, Metadata.empty()),
+ new StructField("key", DataTypes.BinaryType, false, Metadata.empty()),
+ new StructField("data", DataTypes.BinaryType, false, Metadata.empty()),
+ new StructField("publish_timestamp", DataTypes.TimestampType, false, Metadata.empty()),
+ new StructField("event_timestamp", DataTypes.TimestampType, true, Metadata.empty()),
+ new StructField(
+ "attributes",
+ DataTypes.createMapType(
+ DataTypes.StringType, DataTypes.createArrayType(DataTypes.BinaryType)),
+ true,
+ Metadata.empty())
+ });
+
+ public static final PubsubContext.Framework FRAMEWORK = PubsubContext.Framework.of("SPARK");
+
+ public static String BYTES_OUTSTANDING_CONFIG_KEY =
+ "pubsublite.flowcontrol.byteoutstandingperpartition";
+ public static String MESSAGES_OUTSTANDING_CONFIG_KEY =
+ "pubsublite.flowcontrol.messageoutstandingperparition";
+ public static String SUBSCRIPTION_CONFIG_KEY = "pubsublite.subscription";
+ public static String CREDENTIALS_KEY_CONFIG_KEY = "gcp.credentials.key";
+}
diff --git a/src/main/java/com/google/cloud/pubsublite/spark/LimitingHeadOffsetReader.java b/src/main/java/com/google/cloud/pubsublite/spark/LimitingHeadOffsetReader.java
new file mode 100644
index 00000000..5954492f
--- /dev/null
+++ b/src/main/java/com/google/cloud/pubsublite/spark/LimitingHeadOffsetReader.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2020 Google LLC
+ *
+ * Licensed 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 com.google.cloud.pubsublite.spark;
+
+import com.github.benmanes.caffeine.cache.AsyncLoadingCache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.Ticker;
+import com.google.api.core.ApiFutureCallback;
+import com.google.api.core.ApiFutures;
+import com.google.cloud.pubsublite.Offset;
+import com.google.cloud.pubsublite.Partition;
+import com.google.cloud.pubsublite.TopicPath;
+import com.google.cloud.pubsublite.internal.TopicStatsClient;
+import com.google.cloud.pubsublite.proto.Cursor;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.MoreExecutors;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Rate limited HeadOffsetReader, utilizing a LoadingCache that refreshes all partitions head
+ * offsets for the topic at most once per minute.
+ */
+public class LimitingHeadOffsetReader implements PerTopicHeadOffsetReader {
+
+ private final TopicStatsClient topicStatsClient;
+ private final TopicPath topic;
+ private final long topicPartitionCount;
+ private final AsyncLoadingCache cachedHeadOffsets;
+
+ @VisibleForTesting
+ public LimitingHeadOffsetReader(
+ TopicStatsClient topicStatsClient, TopicPath topic, long topicPartitionCount, Ticker ticker) {
+ this.topicStatsClient = topicStatsClient;
+ this.topic = topic;
+ this.topicPartitionCount = topicPartitionCount;
+ this.cachedHeadOffsets =
+ Caffeine.newBuilder()
+ .ticker(ticker)
+ .expireAfterWrite(1, TimeUnit.MINUTES)
+ .buildAsync(this::loadHeadOffset);
+ }
+
+ private CompletableFuture loadHeadOffset(Partition partition, Executor executor) {
+
+ CompletableFuture result = new CompletableFuture<>();
+ ApiFutures.addCallback(
+ topicStatsClient.computeHeadCursor(topic, partition),
+ new ApiFutureCallback() {
+ @Override
+ public void onFailure(Throwable t) {
+ result.completeExceptionally(t);
+ }
+
+ @Override
+ public void onSuccess(Cursor c) {
+ result.complete(Offset.of(c.getOffset()));
+ }
+ },
+ MoreExecutors.directExecutor());
+ return result;
+ }
+
+ @Override
+ public PslSourceOffset getHeadOffset() {
+ Set keySet = new HashSet<>();
+ for (int i = 0; i < topicPartitionCount; i++) {
+ keySet.add(Partition.of(i));
+ }
+ CompletableFuture